Merge pull request #573 from status-im/feature/isolate-evms

Feature/isolate evms
This commit is contained in:
Jordan Hrycaj 2021-04-08 08:00:34 +01:00 committed by GitHub
commit 2eb46ca221
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 1008 additions and 530 deletions

View File

@ -12,7 +12,7 @@ import
chronos, eth/[keys, common, p2p, net/nat], chronicles, nimcrypto/hash,
eth/p2p/bootnodes, eth/p2p/rlpx_protocols/whisper_protocol,
./db/select_backend, eth/keys,
./vm/interpreter/vm_forks
./vm_types2
const
NimbusName* = "Nimbus"

View File

@ -194,7 +194,7 @@ clean-docs:
.SILENT: clean-bakfiles
clean-bakfiles:
for f in $(shell find . -type f \
-name '*~' -o -name '*.bak' -print); do \
\( -name '*~' -o -name '*.bak' \) -print); do \
(set -x; rm -f "$$f"); \
done

View File

@ -1,5 +1,5 @@
import ../db/db_chain, eth/common, chronicles, ../vm_state, ../vm_types,
../vm/[computation, message], ../vm/interpreter/vm_forks, stint, nimcrypto,
../vm_computation, ../vm_message, ../vm_types2, stint, nimcrypto,
../utils, eth/trie/db, ./executor, ../config, ../genesis, ../utils,
stew/endians2

View File

@ -3,10 +3,43 @@ import options, sets,
../db/[db_chain, accounts_cache],
../utils, ../constants, ../transaction,
../vm_state, ../vm_types, ../vm_state_transactions,
../vm/[computation, message, precompiles],
../vm/interpreter/vm_forks,
../vm_computation, ../vm_message, ../vm_precompiles,
../vm_types2,
./dao, ../config
proc validateTransaction*(vmState: BaseVMState, tx: Transaction,
sender: EthAddress, fork: Fork): bool =
let balance = vmState.readOnlyStateDB.getBalance(sender)
let nonce = vmState.readOnlyStateDB.getNonce(sender)
if vmState.cumulativeGasUsed + tx.gasLimit > vmState.blockHeader.gasLimit:
debug "invalid tx: block header gasLimit reached",
maxLimit=vmState.blockHeader.gasLimit,
gasUsed=vmState.cumulativeGasUsed,
addition=tx.gasLimit
return
let totalCost = tx.gasLimit.u256 * tx.gasPrice.u256 + tx.value
if totalCost > balance:
debug "invalid tx: not enough cash",
available=balance,
require=totalCost
return
if tx.gasLimit < tx.intrinsicGas(fork):
debug "invalid tx: not enough gas to perform calculation",
available=tx.gasLimit,
require=tx.intrinsicGas(fork)
return
if tx.accountNonce != nonce:
debug "invalid tx: account nonce mismatch",
txNonce=tx.accountnonce,
accountNonce=nonce
return
result = true
proc processTransaction*(tx: Transaction, sender: EthAddress, vmState: BaseVMState, fork: Fork): GasInt =
## Process the transaction, write the results to db.
## Returns amount of ETH to be rewarded to miner
@ -196,3 +229,67 @@ proc processBlock*(chainDB: BaseChainDB, header: BlockHeader, body: BlockBody, v
# rather than destroyed by trie pruning. But the current block will still get a pruned trie.
# If trie pruning deactivated, `applyDeletes` have no effects.
dbTx.commit(applyDeletes = false)
#[
method executeTransaction(vmState: BaseVMState, transaction: Transaction): (Computation, BlockHeader) {.base.}=
# Execute the transaction in the vm
# TODO: introduced here: https://github.com/ethereum/py-evm/commit/21c57f2d56ab91bb62723c3f9ebe291d0b132dde
# Refactored/Removed here: https://github.com/ethereum/py-evm/commit/cc991bf
# Deleted here: https://github.com/ethereum/py-evm/commit/746defb6f8e83cee2c352a0ab8690e1281c4227c
raise newException(ValueError, "Must be implemented by subclasses")
method addTransaction*(vmState: BaseVMState, transaction: Transaction, c: Computation, b: Block): (Block, Table[string, string]) =
# Add a transaction to the given block and
# return `trieData` to store the transaction data in chaindb in VM layer
# Update the bloomFilter, transaction trie and receipt trie roots, bloom_filter,
# bloom, and usedGas of the block
# transaction: the executed transaction
# computation: the Computation object with executed result
# block: the Block which the transaction is added in
# var receipt = vmState.makeReceipt(transaction, computation)
# vmState.add_receipt(receipt)
# block.transactions.append(transaction)
# # Get trie roots and changed key-values.
# tx_root_hash, tx_kv_nodes = make_trie_root_and_nodes(block.transactions)
# receipt_root_hash, receipt_kv_nodes = make_trie_root_and_nodes(self.receipts)
# trie_data = merge(tx_kv_nodes, receipt_kv_nodes)
# block.bloom_filter |= receipt.bloom
# block.header.transaction_root = tx_root_hash
# block.header.receipt_root = receipt_root_hash
# block.header.bloom = int(block.bloom_filter)
# block.header.gas_used = receipt.gas_used
# return block, trie_data
result = (b, initTable[string, string]())
method applyTransaction*(
vmState: BaseVMState,
transaction: Transaction,
b: Block,
isStateless: bool): (Computation, Block, Table[string, string]) =
# Apply transaction to the given block
# transaction: the transaction need to be applied
# b: the block which the transaction applies on
# isStateless: if isStateless, call vmState.addTransaction to set block
if isStateless:
var ourBlock = b # deepcopy
vmState.blockHeader = b.header
var (computation, blockHeader) = vmState.executeTransaction(transaction)
ourBlock.header = blockHeader
var trieData: Table[string, string]
(ourBlock, trieData) = vmState.addTransaction(transaction, computation, ourBlock)
result = (computation, ourBlock, trieData)
else:
var (computation, blockHeader) = vmState.executeTransaction(transaction)
return (computation, nil, initTable[string, string]())
]#

View File

@ -14,8 +14,8 @@ import
eth/p2p/rlpx_protocols/eth_protocol,
../transaction, ../config, ../vm_state, ../constants, ../vm_types,
../utils, ../db/[db_chain, state_db],
rpc_types, rpc_utils, ../vm/[message, computation],
../vm/interpreter/vm_forks
rpc_types, rpc_utils, ../vm_message, ../vm_computation,
../vm_types2
#[
Note:

View File

@ -10,8 +10,8 @@
import hexstrings, eth/[common, rlp, keys, trie/db], stew/byteutils, nimcrypto,
../db/[db_chain, accounts_cache], strutils, algorithm, options, times, json,
../constants, stint, hexstrings, rpc_types, ../config,
../vm_state_transactions, ../vm_state, ../vm_types, ../vm/interpreter/vm_forks,
../vm/computation, ../p2p/executor, ../utils, ../transaction
../vm_state_transactions, ../vm_state, ../vm_types, ../vm_types2,
../vm_computation, ../p2p/executor, ../utils, ../transaction
type
UnsignedTx* = object

View File

@ -3,7 +3,7 @@ import
constants, vm_state, vm_types, transaction, p2p/executor,
eth/trie/db, nimcrypto, strutils,
chronicles, rpc/hexstrings, launcher,
vm/interpreter/vm_forks, ./config
./vm_types2, ./config
when defined(geth):
import db/geth_db

View File

@ -6,8 +6,8 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
constants, errors, eth/[common, keys], utils,
./vm/interpreter/[vm_forks, gas_costs], constants
./constants, ./errors, eth/[common, keys], ./utils,
./vm_types2, ./vm_gas_costs
import eth/common/transaction as common_transaction
export common_transaction

View File

@ -8,11 +8,12 @@
import
chronicles, strformat, macros, options, times,
sets, eth/[common, keys],
../constants, ../errors, ../vm_state, ../vm_types,
../constants, ../errors,
./interpreter/[opcode_values, gas_meter, gas_costs, vm_forks],
./code_stream, ./memory, ./message, ./stack, ../db/[accounts_cache, db_chain],
../utils/header, precompiles,
transaction_tracer, ../utils
./code_stream, ./memory, ./message, ./stack, ./types, ./state,
../db/[accounts_cache, db_chain],
../utils/header, ./precompiles,
./transaction_tracer, ../utils
when defined(chronicles_log_level):
import stew/byteutils

View File

@ -5,7 +5,7 @@
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import evmc/evmc, evmc_helpers, eth/common, ../constants
import evmc/evmc, ./evmc_helpers, eth/common, ../constants
type
# we are not using EVMC original signature here

View File

@ -1,20 +1,130 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
# see vm_opcode_value
import
./interpreter/opcode_values as vmo
export
vmo.Op
# see vm_forks
import
./interpreter/vm_forks as vmf
export
vmf.Fork
# see vm_message
import
./message as vmm
export
vmm.isCreate
# Used in vm_types. Beware of recursive dependencies
# see vm_computation
import
./computation as vmc
export
vmc.accountExists,
vmc.addLogEntry,
vmc.commit,
vmc.dispose,
vmc.execCall,
vmc.execCreate,
vmc.execSelfDestruct,
vmc.executeOpcodes,
vmc.fork,
vmc.getBalance,
vmc.getBlockHash,
vmc.getBlockNumber,
vmc.getChainId,
vmc.getCode,
vmc.getCodeHash,
vmc.getCodeSize,
vmc.getCoinbase,
vmc.getDifficulty,
vmc.getGasLimit,
vmc.getGasPrice,
vmc.getGasRefund,
vmc.getOrigin,
vmc.getStorage,
vmc.getTimestamp,
vmc.isError,
vmc.isOriginComputation,
vmc.isSuccess,
vmc.isSuicided,
vmc.merge,
vmc.newComputation,
vmc.prepareTracer,
vmc.refundSelfDestruct,
vmc.rollback,
vmc.selfDestruct,
vmc.setError,
vmc.shouldBurnGas,
vmc.snapshot,
vmc.traceError,
vmc.traceOpCodeEnded,
vmc.traceOpCodeStarted,
vmc.tracingEnabled,
vmc.writeContract
import
./interpreter/[opcode_values, gas_meter],
./interpreter/vm_forks
import # Used in vm_types. Beware of recursive dependencies
./code_stream, ./computation, ./stack, ./message
./interpreter/gas_meter as gmt
export
opcode_values, gas_meter,
vm_forks
gmt.consumeGas,
gmt.init,
gmt.refundGas,
gmt.returnGas
import
./code_stream as cst
export
code_stream, computation, stack, message
cst.CodeStream,
cst.`$`,
cst.newCodeStream,
cst.newCodeStreamFromUnescaped,
cst.read,
cst.readVmWord,
cst.len,
cst.next,
cst.items,
cst.`[]`,
cst.peek,
cst.updatePc,
cst.isValidOpcode,
cst.decompile,
cst.displayDecompiled,
cst.hasSStore,
cst.atEnd
import
./stack as stk
export
stk.Stack,
stk.`$`,
stk.`[]`,
stk.dup,
stk.len,
stk.newStack,
stk.peek,
stk.peekInt,
stk.popAddress,
stk.popInt,
stk.popTopic,
stk.push,
stk.swap,
stk.top
# End

View File

@ -7,7 +7,7 @@
import
chronicles, strformat, eth/common, # GasInt
../../errors, ../../vm_types
../../errors, ../types
logScope:
topics = "vm gas"

View File

@ -7,15 +7,15 @@
import
strformat, times, sets, sequtils, options,
chronicles, stint, nimcrypto, stew/ranges/[ptr_arith], eth/common,
chronicles, stint, nimcrypto, stew/ranges/ptr_arith, eth/common,
./utils/[macros_procs_opcodes, utils_numeric],
./gas_meter, ./gas_costs, ./opcode_values, ./vm_forks,
../memory, ../stack, ../code_stream, ../computation,
../../vm_state, ../../errors, ../../constants, ../../vm_types,
../memory, ../stack, ../code_stream, ../computation, ../state, ../types,
../../errors, ../../constants,
../../db/[db_chain, accounts_cache]
when defined(evmc_enabled):
import ../evmc_api, ../evmc_helpers, evmc/evmc
import ../evmc_api, ../evmc_helpers, evmc/evmc
logScope:
topics = "opcode impl"

View File

@ -10,10 +10,9 @@
import
macros, strformat, stint, eth/common,
../../computation, ../../stack, ../../code_stream,
../../../vm_types, ../../memory,
../../../errors, ../../interpreter/[gas_meter, opcode_values],
../../interpreter/utils/utils_numeric
../../computation, ../../stack, ../../code_stream, ../../memory,
../../types, ../../../errors, ../gas_meter, ../opcode_values,
./utils_numeric
when defined(evmc_enabled):
import ../../evmc_api, evmc/evmc

View File

@ -118,4 +118,4 @@ func toInt*(x: EthAddress): int =
type T = uint32
const len = sizeof(T)
fromBytesBE(T, makeOpenArray(x[x.len-len].unsafeAddr, len)).int

View File

@ -9,7 +9,7 @@ import
tables, macros,
chronicles,
./interpreter/[opcode_values, opcodes_impl, vm_forks, gas_costs, gas_meter, utils/macros_gen_opcodes],
./code_stream, ../vm_types, ../errors, precompiles, ./stack,
./code_stream, ./types, ../errors, ./precompiles, ./stack,
terminal # Those are only needed for logging
logScope:

View File

@ -5,7 +5,7 @@
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import ../vm_types
import ./types
proc isCreate*(message: Message): bool =
message.kind in {evmcCreate, evmcCreate2}

View File

@ -1,7 +1,8 @@
import
../vm_types, interpreter/[gas_meter, gas_costs, utils/utils_numeric, vm_forks],
./types,
./interpreter/[gas_meter, gas_costs, utils/utils_numeric, vm_forks],
../errors, stint, eth/[keys, common], chronicles, tables, macros,
math, nimcrypto, bncurve/[fields, groups], blake2b_f, ./blscurve
math, nimcrypto, bncurve/[fields, groups], ./blake2b_f, ./blscurve
type
PrecompileAddresses* = enum

225
nimbus/vm/state.nim Normal file
View File

@ -0,0 +1,225 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
macros, strformat, tables, sets, options,
eth/[common, keys, rlp], nimcrypto/keccak,
./interpreter/[vm_forks, gas_costs], ../errors,
../constants, ../db/[db_chain, accounts_cache],
../utils, json, ./transaction_tracer, ./types,
../config, ../../stateless/[witness_from_tree, witness_types]
proc newAccessLogs*: AccessLogs =
AccessLogs(reads: initTable[string, string](), writes: initTable[string, string]())
proc update*[K, V](t: var Table[K, V], elements: Table[K, V]) =
for k, v in elements:
t[k] = v
proc `$`*(vmState: BaseVMState): string =
if vmState.isNil:
result = "nil"
else:
result = &"VMState {vmState.name}:\n header: {vmState.blockHeader}\n chaindb: {vmState.chaindb}"
proc getMinerAddress(vmState: BaseVMState): EthAddress
proc init*(self: BaseVMState, prevStateRoot: Hash256, header: BlockHeader,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}) =
self.prevHeaders = @[]
self.name = "BaseVM"
self.accessLogs = newAccessLogs()
self.blockHeader = header
self.chaindb = chainDB
self.tracer.initTracer(tracerFlags)
self.logEntries = @[]
self.accountDb = AccountsCache.init(chainDB.db, prevStateRoot, chainDB.pruneTrie)
self.touchedAccounts = initHashSet[EthAddress]()
{.gcsafe.}:
self.minerAddress = self.getMinerAddress()
proc newBaseVMState*(prevStateRoot: Hash256, header: BlockHeader,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}): BaseVMState =
new result
result.init(prevStateRoot, header, chainDB, tracerFlags)
proc newBaseVMState*(prevStateRoot: Hash256,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}): BaseVMState =
new result
var header: BlockHeader
result.init(prevStateRoot, header, chainDB, tracerFlags)
proc setupTxContext*(vmState: BaseVMState, origin: EthAddress, gasPrice: GasInt, forkOverride=none(Fork)) =
## this proc will be called each time a new transaction
## is going to be executed
vmState.txOrigin = origin
vmState.txGasPrice = gasPrice
vmState.fork =
if forkOverride.isSome:
forkOverride.get
else:
vmState.chainDB.config.toFork(vmState.blockHeader.blockNumber)
vmState.gasCosts = vmState.fork.forkToSchedule
proc consensusEnginePoA*(vmState: BaseVMState): bool =
let chainId = PublicNetwork(vmState.chainDB.config.chainId)
# PoA consensus engine have no reward for miner
result = chainId in {GoerliNet, RinkebyNet, KovanNet}
proc getSignature(bytes: openArray[byte], output: var Signature): bool =
let sig = Signature.fromRaw(bytes)
if sig.isOk:
output = sig[]
return true
return false
proc headerHashOriExtraData(vmState: BaseVMState): Hash256 =
var tmp = vmState.blockHeader
tmp.extraData.setLen(tmp.extraData.len-65)
result = keccak256.digest(rlp.encode(tmp))
proc calcMinerAddress(sigRaw: openArray[byte], vmState: BaseVMState, output: var EthAddress): bool =
var sig: Signature
if sigRaw.getSignature(sig):
let headerHash = headerHashOriExtraData(vmState)
let pubkey = recover(sig, SKMessage(headerHash.data))
if pubkey.isOk:
output = pubkey[].toCanonicalAddress()
result = true
proc getMinerAddress(vmState: BaseVMState): EthAddress =
if not vmState.consensusEnginePoA:
return vmState.blockHeader.coinbase
template data: untyped =
vmState.blockHeader.extraData
let len = data.len
doAssert(len >= 65)
var miner: EthAddress
if calcMinerAddress(data.toOpenArray(len - 65, len-1), vmState, miner):
result = miner
else:
raise newException(ValidationError, "Could not derive miner address from header extradata")
proc updateBlockHeader*(vmState: BaseVMState, header: BlockHeader) =
vmState.blockHeader = header
vmState.touchedAccounts.clear()
vmState.suicides.clear()
if EnableTracing in vmState.tracer.flags:
vmState.tracer.initTracer(vmState.tracer.flags)
vmState.logEntries = @[]
vmState.receipts = @[]
vmState.minerAddress = vmState.getMinerAddress()
method blockhash*(vmState: BaseVMState): Hash256 {.base, gcsafe.} =
vmState.blockHeader.hash
method coinbase*(vmState: BaseVMState): EthAddress {.base, gcsafe.} =
vmState.minerAddress
method timestamp*(vmState: BaseVMState): EthTime {.base, gcsafe.} =
vmState.blockHeader.timestamp
method blockNumber*(vmState: BaseVMState): BlockNumber {.base, gcsafe.} =
# it should return current block number
# and not head.blockNumber
vmState.blockHeader.blockNumber
method difficulty*(vmState: BaseVMState): UInt256 {.base, gcsafe.} =
vmState.blockHeader.difficulty
method gasLimit*(vmState: BaseVMState): GasInt {.base, gcsafe.} =
vmState.blockHeader.gasLimit
when defined(geth):
import db/geth_db
method getAncestorHash*(vmState: BaseVMState, blockNumber: BlockNumber): Hash256 {.base, gcsafe.} =
var ancestorDepth = vmState.blockHeader.blockNumber - blockNumber - 1
if ancestorDepth >= constants.MAX_PREV_HEADER_DEPTH:
return
if blockNumber >= vmState.blockHeader.blockNumber:
return
when defined(geth):
result = vmState.chainDB.headerHash(blockNumber.truncate(uint64))
else:
result = vmState.chainDB.getBlockHash(blockNumber)
#TODO: should we use deque here?
# someday we may revive this code when
# we already have working miner
when false:
let idx = ancestorDepth.toInt
if idx >= vmState.prevHeaders.len:
return
var header = vmState.prevHeaders[idx]
result = header.hash
proc readOnlyStateDB*(vmState: BaseVMState): ReadOnlyStateDB {.inline.} =
ReadOnlyStateDB(vmState.accountDb)
template mutateStateDB*(vmState: BaseVMState, body: untyped) =
block:
var db {.inject.} = vmState.accountDb
body
proc getTracingResult*(vmState: BaseVMState): JsonNode {.inline.} =
doAssert(EnableTracing in vmState.tracer.flags)
vmState.tracer.trace
proc getAndClearLogEntries*(vmState: BaseVMState): seq[Log] =
shallowCopy(result, vmState.logEntries)
vmState.logEntries = @[]
proc enableTracing*(vmState: BaseVMState) {.inline.} =
vmState.tracer.flags.incl EnableTracing
proc disableTracing*(vmState: BaseVMState) {.inline.} =
vmState.tracer.flags.excl EnableTracing
iterator tracedAccounts*(vmState: BaseVMState): EthAddress =
for acc in vmState.tracer.accounts:
yield acc
iterator tracedAccountsPairs*(vmState: BaseVMState): (int, EthAddress) =
var idx = 0
for acc in vmState.tracer.accounts:
yield (idx, acc)
inc idx
proc removeTracedAccounts*(vmState: BaseVMState, accounts: varargs[EthAddress]) =
for acc in accounts:
vmState.tracer.accounts.excl acc
proc status*(vmState: BaseVMState): bool {.inline.} =
ExecutionOK in vmState.flags
proc `status=`*(vmState: BaseVMState, status: bool) =
if status: vmState.flags.incl ExecutionOK
else: vmState.flags.excl ExecutionOK
proc generateWitness*(vmState: BaseVMState): bool {.inline.} =
GenerateWitness in vmState.flags
proc `generateWitness=`*(vmState: BaseVMState, status: bool) =
if status: vmState.flags.incl GenerateWitness
else: vmState.flags.excl GenerateWitness
proc buildWitness*(vmState: BaseVMState): seq[byte] =
let rootHash = vmState.accountDb.rootHash
let mkeys = vmState.accountDb.makeMultiKeys()
let flags = if vmState.fork >= FKSpurious: {wfEIP170} else: {}
# build witness from tree
var wb = initWitnessBuilder(vmState.chainDB.db, rootHash, flags)
result = wb.buildWitness(mkeys)

View File

@ -0,0 +1,58 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
options, sets,
eth/common, chronicles, ../db/accounts_cache,
../transaction,
./computation, ./interpreter, ./state, ./types
proc setupComputation*(vmState: BaseVMState, tx: Transaction, sender: EthAddress, fork: Fork) : Computation =
var gas = tx.gasLimit - tx.intrinsicGas(fork)
assert gas >= 0
vmState.setupTxContext(
origin = sender,
gasPrice = tx.gasPrice,
forkOverride = some(fork)
)
let msg = Message(
kind: if tx.isContractCreation: evmcCreate else: evmcCall,
depth: 0,
gas: gas,
sender: sender,
contractAddress: tx.getRecipient(),
codeAddress: tx.to,
value: tx.value,
data: tx.payload
)
result = newComputation(vmState, msg)
doAssert result.isOriginComputation
proc execComputation*(c: Computation) =
if c.msg.isCreate:
c.execCreate()
else:
c.vmState.mutateStateDB:
db.incNonce(c.msg.sender)
c.execCall()
if c.isSuccess:
c.refundSelfDestruct()
shallowCopy(c.vmState.suicides, c.suicides)
shallowCopy(c.vmState.logEntries, c.logEntries)
c.vmState.touchedAccounts.incl c.touchedAccounts
c.vmstate.status = c.isSuccess
proc refundGas*(c: Computation, tx: Transaction, sender: EthAddress) =
let maxRefund = (tx.gasLimit - c.gasMeter.gasRemaining) div 2
c.gasMeter.returnGas min(c.getGasRefund(), maxRefund)
c.vmState.mutateStateDB:
db.addBalance(sender, c.gasMeter.gasRemaining.u256 * tx.gasPrice.u256)

View File

@ -1,7 +1,7 @@
import
json, strutils, sets, hashes,
chronicles, nimcrypto, eth/common, stint,
../vm_types, memory, stack, ../db/accounts_cache,
./types, ./memory, ./stack, ../db/accounts_cache,
eth/trie/hexary,
./interpreter/opcode_values

117
nimbus/vm/types.nim Normal file
View File

@ -0,0 +1,117 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
tables, eth/common,
options, json, sets,
./memory, ./stack, ./code_stream,
./interpreter/[gas_costs, opcode_values, vm_forks],
# TODO - will be hidden at a lower layer
../db/[db_chain, accounts_cache]
when defined(evmc_enabled):
import
./evmc_api
type
VMFlag* = enum
ExecutionOK
GenerateWitness
ClearCache
BaseVMState* = ref object of RootObj
prevHeaders* : seq[BlockHeader]
chaindb* : BaseChainDB
accessLogs* : AccessLogs
blockHeader* : BlockHeader
name* : string
flags* : set[VMFlag]
tracer* : TransactionTracer
logEntries* : seq[Log]
receipts* : seq[Receipt]
accountDb* : AccountsCache
cumulativeGasUsed*: GasInt
touchedAccounts*: HashSet[EthAddress]
suicides* : HashSet[EthAddress]
txOrigin* : EthAddress
txGasPrice* : GasInt
gasCosts* : GasCosts
fork* : Fork
minerAddress* : EthAddress
AccessLogs* = ref object
reads*: Table[string, string]
writes*: Table[string, string]
TracerFlags* {.pure.} = enum
EnableTracing
DisableStorage
DisableMemory
DisableStack
DisableState
DisableStateDiff
EnableAccount
TransactionTracer* = object
trace*: JsonNode
flags*: set[TracerFlags]
accounts*: HashSet[EthAddress]
storageKeys*: seq[HashSet[Uint256]]
Computation* = ref object
# The execution computation
vmState*: BaseVMState
when defined(evmc_enabled):
host*: HostContext
msg*: Message
memory*: Memory
stack*: Stack
returnStack*: seq[int]
gasMeter*: GasMeter
code*: CodeStream
output*: seq[byte]
returnData*: seq[byte]
error*: Error
touchedAccounts*: HashSet[EthAddress]
suicides*: HashSet[EthAddress]
logEntries*: seq[Log]
savePoint*: SavePoint
instr*: Op
opIndex*: int
Error* = ref object
info*: string
burnsGas*: bool
GasMeter* = object
gasRefunded*: GasInt
gasRemaining*: GasInt
CallKind* = enum
evmcCall = 0, # CALL
evmcDelegateCall = 1, # DELEGATECALL
evmcCallCode = 2, # CALLCODE
evmcCreate = 3, # CREATE
evmcCreate2 = 4 # CREATE2
MsgFlags* = enum
emvcNoFlags = 0
emvcStatic = 1
Message* = ref object
kind*: CallKind
depth*: int
gas*: GasInt
sender*: EthAddress
contractAddress*: EthAddress
codeAddress*: EthAddress
value*: UInt256
data*: seq[byte]
flags*: MsgFlags

61
nimbus/vm_computation.nim Normal file
View File

@ -0,0 +1,61 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
# The computation module suffers from a circular include/import dependency.
# After fixing this wrapper should be re-factored.
import
./vm/computation as vmc
export
vmc.accountExists,
vmc.addLogEntry,
vmc.commit,
vmc.dispose,
vmc.execCall,
vmc.execCreate,
vmc.execSelfDestruct,
vmc.executeOpcodes,
vmc.fork,
vmc.getBalance,
vmc.getBlockHash,
vmc.getBlockNumber,
vmc.getChainId,
vmc.getCode,
vmc.getCodeHash,
vmc.getCodeSize,
vmc.getCoinbase,
vmc.getDifficulty,
vmc.getGasLimit,
vmc.getGasPrice,
vmc.getGasRefund,
vmc.getOrigin,
vmc.getStorage,
vmc.getTimestamp,
vmc.isError,
vmc.isOriginComputation,
vmc.isSuccess,
vmc.isSuicided,
vmc.merge,
vmc.newComputation,
vmc.prepareTracer,
vmc.refundSelfDestruct,
vmc.rollback,
vmc.selfDestruct,
vmc.setError,
vmc.shouldBurnGas,
vmc.snapshot,
vmc.traceError,
vmc.traceOpCodeEnded,
vmc.traceOpCodeStarted,
vmc.tracingEnabled,
vmc.writeContract
# End

49
nimbus/vm_gas_costs.nim Normal file
View File

@ -0,0 +1,49 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
./vm/interpreter/gas_costs as vmg
export
vmg.Bls12381G1AddGas,
vmg.Bls12381G1MulGas,
vmg.Bls12381G2AddGas,
vmg.Bls12381G2MulGas,
vmg.Bls12381MapG1Gas,
vmg.Bls12381MapG2Gas,
vmg.Bls12381PairingBaseGas,
vmg.Bls12381PairingPerPairGas,
vmg.ColdAccountAccessCost,
vmg.ColdSloadCost,
vmg.GasCostKind,
vmg.GasCosts,
vmg.GasECAdd,
vmg.GasECAddIstanbul,
vmg.GasECMul,
vmg.GasECMulIstanbul,
vmg.GasECPairingBase,
vmg.GasECPairingBaseIstanbul,
vmg.GasECPairingPerPoint,
vmg.GasECPairingPerPointIstanbul,
vmg.GasECRecover,
vmg.GasFeeKind,
vmg.GasIdentity,
vmg.GasIdentityWord,
vmg.GasParams,
vmg.GasQuadDivisorEIP2565,
vmg.GasRIPEMD160,
vmg.GasRIPEMD160Word,
vmg.GasSHA256,
vmg.GasSHA256Word,
vmg.WarmStorageReadCost,
vmg.forkToSchedule,
vmg.gasFees
# End

51
nimbus/vm_internals.nim Normal file
View File

@ -0,0 +1,51 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
# At the moment, this header file interface is only used for testing.
import
./vm/memory as vmm
export
vmm.Memory,
vmm.extend,
vmm.len,
vmm.newMemory,
vmm.read,
vmm.write
when defined(evmc_enabled):
export
vmm.readPtr
import
./vm/interpreter/utils/utils_numeric as vmn
export
vmn.GasNatural,
vmn.calcMemSize,
vmn.ceil32,
vmn.cleanMemRef,
vmn.log2,
vmn.log256,
vmn.rangeToPadded,
vmn.rangeToPadded2,
vmn.safeInt,
vmn.setSign,
vmn.toInt,
vmn.wordCount
# Wrapping the wrapper -- lol
import
./vm/interpreter as vmi
export
vmi
# End

20
nimbus/vm_message.nim Normal file
View File

@ -0,0 +1,20 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
# Should be considered part of another header file (e.g. vm_misc) once the
# circular computation.nim include/import dependency is solved. The problem is
# with vm_types.nim (included by message.nim) which includes computation.nim.
import
./vm/message as vmm
export
vmm.isCreate
# End

37
nimbus/vm_precompiles.nim Normal file
View File

@ -0,0 +1,37 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
./vm/precompiles as vmp
export
vmp.PrecompileAddresses,
vmp.activePrecompiles,
vmp.blake2bf,
vmp.blsG1Add,
vmp.blsG1Mul,
vmp.blsG1MultiExp,
vmp.blsG2Add,
vmp.blsG2Mul,
vmp.blsG2MultiExp,
vmp.blsMapG1,
vmp.blsMapG2,
vmp.blsPairing,
vmp.bn256ecAdd,
vmp.bn256ecMul,
vmp.ecRecover,
vmp.execPrecompiles,
vmp.identity,
vmp.modExp,
vmp.ripemd160,
vmp.sha256,
vmp.simpleDecode
# End

View File

@ -1,222 +1,45 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
macros, strformat, tables, sets, options,
eth/[common, keys, rlp], nimcrypto/keccak,
vm/interpreter/[vm_forks, gas_costs], ./errors,
./constants, ./db/[db_chain, accounts_cache],
./utils, json, vm_types, vm/transaction_tracer,
./config, ../stateless/[witness_from_tree, witness_types]
./vm/state as vms
proc newAccessLogs*: AccessLogs =
AccessLogs(reads: initTable[string, string](), writes: initTable[string, string]())
export
vms.`$`,
vms.blockNumber,
vms.blockhash,
vms.buildWitness,
vms.coinbase,
vms.consensusEnginePoA,
vms.difficulty,
vms.disableTracing,
vms.enableTracing,
vms.gasLimit,
vms.generateWitness,
vms.`generateWitness=`,
vms.getAncestorHash,
vms.getAndClearLogEntries,
vms.getTracingResult,
vms.init,
vms.mutateStateDB,
vms.newAccessLogs,
vms.newBaseVMState,
vms.readOnlyStateDB,
vms.removeTracedAccounts,
vms.setupTxContext,
vms.status,
vms.`status=`,
vms.timestamp,
vms.tracedAccounts,
vms.tracedAccountsPairs,
vms.update,
vms.updateBlockHeader
proc update*[K, V](t: var Table[K, V], elements: Table[K, V]) =
for k, v in elements:
t[k] = v
proc `$`*(vmState: BaseVMState): string =
if vmState.isNil:
result = "nil"
else:
result = &"VMState {vmState.name}:\n header: {vmState.blockHeader}\n chaindb: {vmState.chaindb}"
proc getMinerAddress(vmState: BaseVMState): EthAddress
proc init*(self: BaseVMState, prevStateRoot: Hash256, header: BlockHeader,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}) =
self.prevHeaders = @[]
self.name = "BaseVM"
self.accessLogs = newAccessLogs()
self.blockHeader = header
self.chaindb = chainDB
self.tracer.initTracer(tracerFlags)
self.logEntries = @[]
self.accountDb = AccountsCache.init(chainDB.db, prevStateRoot, chainDB.pruneTrie)
self.touchedAccounts = initHashSet[EthAddress]()
{.gcsafe.}:
self.minerAddress = self.getMinerAddress()
proc newBaseVMState*(prevStateRoot: Hash256, header: BlockHeader,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}): BaseVMState =
new result
result.init(prevStateRoot, header, chainDB, tracerFlags)
proc newBaseVMState*(prevStateRoot: Hash256,
chainDB: BaseChainDB, tracerFlags: set[TracerFlags] = {}): BaseVMState =
new result
var header: BlockHeader
result.init(prevStateRoot, header, chainDB, tracerFlags)
proc setupTxContext*(vmState: BaseVMState, origin: EthAddress, gasPrice: GasInt, forkOverride=none(Fork)) =
## this proc will be called each time a new transaction
## is going to be executed
vmState.txOrigin = origin
vmState.txGasPrice = gasPrice
vmState.fork =
if forkOverride.isSome:
forkOverride.get
else:
vmState.chainDB.config.toFork(vmState.blockHeader.blockNumber)
vmState.gasCosts = vmState.fork.forkToSchedule
proc consensusEnginePoA*(vmState: BaseVMState): bool =
let chainId = PublicNetwork(vmState.chainDB.config.chainId)
# PoA consensus engine have no reward for miner
result = chainId in {GoerliNet, RinkebyNet, KovanNet}
proc getSignature(bytes: openArray[byte], output: var Signature): bool =
let sig = Signature.fromRaw(bytes)
if sig.isOk:
output = sig[]
return true
return false
proc headerHashOriExtraData(vmState: BaseVMState): Hash256 =
var tmp = vmState.blockHeader
tmp.extraData.setLen(tmp.extraData.len-65)
result = keccak256.digest(rlp.encode(tmp))
proc calcMinerAddress(sigRaw: openArray[byte], vmState: BaseVMState, output: var EthAddress): bool =
var sig: Signature
if sigRaw.getSignature(sig):
let headerHash = headerHashOriExtraData(vmState)
let pubkey = recover(sig, SKMessage(headerHash.data))
if pubkey.isOk:
output = pubkey[].toCanonicalAddress()
result = true
proc getMinerAddress(vmState: BaseVMState): EthAddress =
if not vmState.consensusEnginePoA:
return vmState.blockHeader.coinbase
template data: untyped =
vmState.blockHeader.extraData
let len = data.len
doAssert(len >= 65)
var miner: EthAddress
if calcMinerAddress(data.toOpenArray(len - 65, len-1), vmState, miner):
result = miner
else:
raise newException(ValidationError, "Could not derive miner address from header extradata")
proc updateBlockHeader*(vmState: BaseVMState, header: BlockHeader) =
vmState.blockHeader = header
vmState.touchedAccounts.clear()
vmState.suicides.clear()
if EnableTracing in vmState.tracer.flags:
vmState.tracer.initTracer(vmState.tracer.flags)
vmState.logEntries = @[]
vmState.receipts = @[]
vmState.minerAddress = vmState.getMinerAddress()
method blockhash*(vmState: BaseVMState): Hash256 {.base, gcsafe.} =
vmState.blockHeader.hash
method coinbase*(vmState: BaseVMState): EthAddress {.base, gcsafe.} =
vmState.minerAddress
method timestamp*(vmState: BaseVMState): EthTime {.base, gcsafe.} =
vmState.blockHeader.timestamp
method blockNumber*(vmState: BaseVMState): BlockNumber {.base, gcsafe.} =
# it should return current block number
# and not head.blockNumber
vmState.blockHeader.blockNumber
method difficulty*(vmState: BaseVMState): UInt256 {.base, gcsafe.} =
vmState.blockHeader.difficulty
method gasLimit*(vmState: BaseVMState): GasInt {.base, gcsafe.} =
vmState.blockHeader.gasLimit
when defined(geth):
import db/geth_db
method getAncestorHash*(vmState: BaseVMState, blockNumber: BlockNumber): Hash256 {.base, gcsafe.} =
var ancestorDepth = vmState.blockHeader.blockNumber - blockNumber - 1
if ancestorDepth >= constants.MAX_PREV_HEADER_DEPTH:
return
if blockNumber >= vmState.blockHeader.blockNumber:
return
when defined(geth):
result = vmState.chainDB.headerHash(blockNumber.truncate(uint64))
else:
result = vmState.chainDB.getBlockHash(blockNumber)
#TODO: should we use deque here?
# someday we may revive this code when
# we already have working miner
when false:
let idx = ancestorDepth.toInt
if idx >= vmState.prevHeaders.len:
return
var header = vmState.prevHeaders[idx]
result = header.hash
proc readOnlyStateDB*(vmState: BaseVMState): ReadOnlyStateDB {.inline.} =
ReadOnlyStateDB(vmState.accountDb)
template mutateStateDB*(vmState: BaseVMState, body: untyped) =
block:
var db {.inject.} = vmState.accountDb
body
proc getTracingResult*(vmState: BaseVMState): JsonNode {.inline.} =
doAssert(EnableTracing in vmState.tracer.flags)
vmState.tracer.trace
proc getAndClearLogEntries*(vmState: BaseVMState): seq[Log] =
shallowCopy(result, vmState.logEntries)
vmState.logEntries = @[]
proc enableTracing*(vmState: BaseVMState) {.inline.} =
vmState.tracer.flags.incl EnableTracing
proc disableTracing*(vmState: BaseVMState) {.inline.} =
vmState.tracer.flags.excl EnableTracing
iterator tracedAccounts*(vmState: BaseVMState): EthAddress =
for acc in vmState.tracer.accounts:
yield acc
iterator tracedAccountsPairs*(vmState: BaseVMState): (int, EthAddress) =
var idx = 0
for acc in vmState.tracer.accounts:
yield (idx, acc)
inc idx
proc removeTracedAccounts*(vmState: BaseVMState, accounts: varargs[EthAddress]) =
for acc in accounts:
vmState.tracer.accounts.excl acc
proc status*(vmState: BaseVMState): bool {.inline.} =
ExecutionOK in vmState.flags
proc `status=`*(vmState: BaseVMState, status: bool) =
if status: vmState.flags.incl ExecutionOK
else: vmState.flags.excl ExecutionOK
proc generateWitness*(vmState: BaseVMState): bool {.inline.} =
GenerateWitness in vmState.flags
proc `generateWitness=`*(vmState: BaseVMState, status: bool) =
if status: vmState.flags.incl GenerateWitness
else: vmState.flags.excl GenerateWitness
proc buildWitness*(vmState: BaseVMState): seq[byte] =
let rootHash = vmState.accountDb.rootHash
let mkeys = vmState.accountDb.makeMultiKeys()
let flags = if vmState.fork >= FKSpurious: {wfEIP170} else: {}
# build witness from tree
var wb = initWitnessBuilder(vmState.chainDB.db, rootHash, flags)
result = wb.buildWitness(mkeys)
# End

View File

@ -1,153 +1,19 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
options, sets,
eth/common, chronicles, ./db/accounts_cache,
transaction, vm_types, vm_state,
./vm/[computation, interpreter]
vm/state_transactions as vmx
proc validateTransaction*(vmState: BaseVMState, tx: Transaction, sender: EthAddress, fork: Fork): bool =
let balance = vmState.readOnlyStateDB.getBalance(sender)
let nonce = vmState.readOnlyStateDB.getNonce(sender)
export
vmx.execComputation,
vmx.refundGas,
vmx.setupComputation
if vmState.cumulativeGasUsed + tx.gasLimit > vmState.blockHeader.gasLimit:
debug "invalid tx: block header gasLimit reached",
maxLimit=vmState.blockHeader.gasLimit,
gasUsed=vmState.cumulativeGasUsed,
addition=tx.gasLimit
return
let totalCost = tx.gasLimit.u256 * tx.gasPrice.u256 + tx.value
if totalCost > balance:
debug "invalid tx: not enough cash",
available=balance,
require=totalCost
return
if tx.gasLimit < tx.intrinsicGas(fork):
debug "invalid tx: not enough gas to perform calculation",
available=tx.gasLimit,
require=tx.intrinsicGas(fork)
return
if tx.accountNonce != nonce:
debug "invalid tx: account nonce mismatch",
txNonce=tx.accountnonce,
accountNonce=nonce
return
result = true
proc setupComputation*(vmState: BaseVMState, tx: Transaction, sender: EthAddress, fork: Fork) : Computation =
var gas = tx.gasLimit - tx.intrinsicGas(fork)
assert gas >= 0
vmState.setupTxContext(
origin = sender,
gasPrice = tx.gasPrice,
forkOverride = some(fork)
)
let msg = Message(
kind: if tx.isContractCreation: evmcCreate else: evmcCall,
depth: 0,
gas: gas,
sender: sender,
contractAddress: tx.getRecipient(),
codeAddress: tx.to,
value: tx.value,
data: tx.payload
)
result = newComputation(vmState, msg)
doAssert result.isOriginComputation
proc execComputation*(c: Computation) =
if c.msg.isCreate:
c.execCreate()
else:
c.vmState.mutateStateDB:
db.incNonce(c.msg.sender)
c.execCall()
if c.isSuccess:
c.refundSelfDestruct()
shallowCopy(c.vmState.suicides, c.suicides)
shallowCopy(c.vmState.logEntries, c.logEntries)
c.vmState.touchedAccounts.incl c.touchedAccounts
c.vmstate.status = c.isSuccess
proc refundGas*(c: Computation, tx: Transaction, sender: EthAddress) =
let maxRefund = (tx.gasLimit - c.gasMeter.gasRemaining) div 2
c.gasMeter.returnGas min(c.getGasRefund(), maxRefund)
c.vmState.mutateStateDB:
db.addBalance(sender, c.gasMeter.gasRemaining.u256 * tx.gasPrice.u256)
#[
method executeTransaction(vmState: BaseVMState, transaction: Transaction): (Computation, BlockHeader) {.base.}=
# Execute the transaction in the vm
# TODO: introduced here: https://github.com/ethereum/py-evm/commit/21c57f2d56ab91bb62723c3f9ebe291d0b132dde
# Refactored/Removed here: https://github.com/ethereum/py-evm/commit/cc991bf
# Deleted here: https://github.com/ethereum/py-evm/commit/746defb6f8e83cee2c352a0ab8690e1281c4227c
raise newException(ValueError, "Must be implemented by subclasses")
method addTransaction*(vmState: BaseVMState, transaction: Transaction, c: Computation, b: Block): (Block, Table[string, string]) =
# Add a transaction to the given block and
# return `trieData` to store the transaction data in chaindb in VM layer
# Update the bloomFilter, transaction trie and receipt trie roots, bloom_filter,
# bloom, and usedGas of the block
# transaction: the executed transaction
# computation: the Computation object with executed result
# block: the Block which the transaction is added in
# var receipt = vmState.makeReceipt(transaction, computation)
# vmState.add_receipt(receipt)
# block.transactions.append(transaction)
# # Get trie roots and changed key-values.
# tx_root_hash, tx_kv_nodes = make_trie_root_and_nodes(block.transactions)
# receipt_root_hash, receipt_kv_nodes = make_trie_root_and_nodes(self.receipts)
# trie_data = merge(tx_kv_nodes, receipt_kv_nodes)
# block.bloom_filter |= receipt.bloom
# block.header.transaction_root = tx_root_hash
# block.header.receipt_root = receipt_root_hash
# block.header.bloom = int(block.bloom_filter)
# block.header.gas_used = receipt.gas_used
# return block, trie_data
result = (b, initTable[string, string]())
method applyTransaction*(
vmState: BaseVMState,
transaction: Transaction,
b: Block,
isStateless: bool): (Computation, Block, Table[string, string]) =
# Apply transaction to the given block
# transaction: the transaction need to be applied
# b: the block which the transaction applies on
# isStateless: if isStateless, call vmState.addTransaction to set block
if isStateless:
var ourBlock = b # deepcopy
vmState.blockHeader = b.header
var (computation, blockHeader) = vmState.executeTransaction(transaction)
ourBlock.header = blockHeader
var trieData: Table[string, string]
(ourBlock, trieData) = vmState.addTransaction(transaction, computation, ourBlock)
result = (computation, ourBlock, trieData)
else:
var (computation, blockHeader) = vmState.executeTransaction(transaction)
return (computation, nil, initTable[string, string]())
]#
# End

View File

@ -1,112 +1,54 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
import
tables, eth/common,
options, json, sets,
./vm/[memory, stack, code_stream],
./vm/interpreter/[gas_costs, opcode_values, vm_forks], # TODO - will be hidden at a lower layer
./db/[db_chain, accounts_cache]
./vm/types as vmt
export
vmt.AccessLogs,
vmt.BaseVMState,
vmt.CallKind,
vmt.Computation,
vmt.Error,
vmt.GasMeter,
vmt.Message,
vmt.MsgFlags,
vmt.TracerFlags,
vmt.TransactionTracer,
vmt.VMFlag
when defined(evmc_enabled):
import ./vm/evmc_api
import
./vm/evmc_api as evmc
export
evmc.HostContext,
evmc.accountExists,
evmc.call,
evmc.copyCode,
evmc.emitLog,
evmc.getBalance,
evmc.getBlockHash,
evmc.getCodeHash,
evmc.getCodeSize,
evmc.getStorage,
evmc.getTxContext,
evmc.init,
evmc.nim_create_nimbus_vm,
evmc.nim_host_create_context,
evmc.nim_host_destroy_context,
evmc.nim_host_get_interface,
evmc.nimbus_host_interface,
evmc.nimbus_message,
evmc.nimbus_result,
evmc.nimbus_tx_context,
evmc.selfDestruct,
evmc.setStorage
type
VMFlag* = enum
ExecutionOK
GenerateWitness
ClearCache
BaseVMState* = ref object of RootObj
prevHeaders* : seq[BlockHeader]
chaindb* : BaseChainDB
accessLogs* : AccessLogs
blockHeader* : BlockHeader
name* : string
flags* : set[VMFlag]
tracer* : TransactionTracer
logEntries* : seq[Log]
receipts* : seq[Receipt]
accountDb* : AccountsCache
cumulativeGasUsed*: GasInt
touchedAccounts*: HashSet[EthAddress]
suicides* : HashSet[EthAddress]
txOrigin* : EthAddress
txGasPrice* : GasInt
gasCosts* : GasCosts
fork* : Fork
minerAddress* : EthAddress
AccessLogs* = ref object
reads*: Table[string, string]
writes*: Table[string, string]
TracerFlags* {.pure.} = enum
EnableTracing
DisableStorage
DisableMemory
DisableStack
DisableState
DisableStateDiff
EnableAccount
TransactionTracer* = object
trace*: JsonNode
flags*: set[TracerFlags]
accounts*: HashSet[EthAddress]
storageKeys*: seq[HashSet[Uint256]]
Computation* = ref object
# The execution computation
vmState*: BaseVMState
when defined(evmc_enabled):
host*: HostContext
msg*: Message
memory*: Memory
stack*: Stack
returnStack*: seq[int]
gasMeter*: GasMeter
code*: CodeStream
output*: seq[byte]
returnData*: seq[byte]
error*: Error
touchedAccounts*: HashSet[EthAddress]
suicides*: HashSet[EthAddress]
logEntries*: seq[Log]
savePoint*: SavePoint
instr*: Op
opIndex*: int
Error* = ref object
info*: string
burnsGas*: bool
GasMeter* = object
gasRefunded*: GasInt
gasRemaining*: GasInt
CallKind* = enum
evmcCall = 0, # CALL
evmcDelegateCall = 1, # DELEGATECALL
evmcCallCode = 2, # CALLCODE
evmcCreate = 3, # CREATE
evmcCreate2 = 4 # CREATE2
MsgFlags* = enum
emvcNoFlags = 0
emvcStatic = 1
Message* = ref object
kind*: CallKind
depth*: int
gas*: GasInt
sender*: EthAddress
contractAddress*: EthAddress
codeAddress*: EthAddress
value*: UInt256
data*: seq[byte]
flags*: MsgFlags
# End

24
nimbus/vm_types2.nim Normal file
View File

@ -0,0 +1,24 @@
# Nimbus
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or
# http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or
# http://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except
# according to those terms.
# The following should really go into vm_types once the circular computation.nim
# include/import dependency is solved. The problem is with vm_types.nim which
# includes computation.nim.
import
./vm/interpreter/vm_forks as vmf
export
vmf.Fork
import
./vm/interpreter/opcode_values as vmo
export
vmo.Op
# End

View File

@ -1,18 +1,16 @@
import
macrocache, strutils, unittest2,
stew/byteutils, chronicles, eth/common,
../nimbus/vm/interpreter/opcode_values,
stew/shims/macros, ../nimbus/config
import
options, json, os, eth/trie/[db, hexary],
../nimbus/[vm_state, vm_types, transaction, utils],
../nimbus/[transaction, utils],
../nimbus/db/[db_chain, accounts_cache],
../nimbus/vm_state_transactions,
../nimbus/vm/interpreter/vm_forks,
../nimbus/vm/[message, computation, memory]
../nimbus/[vm_state_transactions, vm_types2,
vm_message, vm_internals, vm_state, vm_types]
export opcode_values, byteutils
export vm_types2, byteutils
{.experimental: "dynamicBindSym".}
# backported from Nim 0.19.9

View File

@ -12,8 +12,7 @@ import
ethash, stew/endians2, nimcrypto,
./test_helpers, ./test_allowed_to_fail,
../premix/parser, test_config,
../nimbus/vm/interpreter/vm_forks,
../nimbus/[vm_state, utils, vm_types, errors, transaction, constants],
../nimbus/[vm_state, utils, vm_types, errors, transaction, constants, vm_types2],
../nimbus/db/[db_chain, accounts_cache],
../nimbus/utils/header,
../nimbus/p2p/[executor, dao],

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import unittest2, sequtils,
../nimbus/vm/interpreter
../nimbus/vm_internals
proc codeStreamMain*() =
suite "parse bytecode":

View File

@ -1,6 +1,6 @@
import
parseopt, strutils,
../nimbus/vm/interpreter/vm_forks
../nimbus/vm_types2
type
ConfigStatus* = enum

View File

@ -1,7 +1,7 @@
import unittest2, strutils, tables, os, json,
../nimbus/utils/difficulty, stint, times,
eth/common, test_helpers, stew/byteutils,
../nimbus/constants, ../nimbus/vm/interpreter/vm_forks,
../nimbus/constants, ../nimbus/vm_types2,
../nimbus/config
type

View File

@ -8,7 +8,7 @@
import
unittest2, macros, strformat,
eth/common/eth_types,
../nimbus/[vm_types, errors, vm/interpreter]
../nimbus/[vm_types, errors, vm_internals]
# TODO: quicktest
# PS: parametrize can be easily immitated, but still quicktests would be even more useful

View File

@ -13,7 +13,7 @@ import
../nimbus/p2p/executor, test_config,
../nimbus/transaction,
../nimbus/[vm_state, vm_types, utils],
../nimbus/vm/interpreter,
../nimbus/vm_internals,
../nimbus/db/[db_chain, accounts_cache]
type

View File

@ -10,7 +10,7 @@ import
stew/byteutils, net, eth/[common, keys, rlp, p2p], unittest2,
testutils/markdown_reports,
../nimbus/[config, transaction, utils, errors],
../nimbus/vm/interpreter/vm_forks,
../nimbus/vm_types2,
../nimbus/db/accounts_cache,
../nimbus/random_keys

View File

@ -8,7 +8,7 @@
import
unittest2, sequtils,
eth/common/eth_types,
../nimbus/[errors, vm/memory]
../nimbus/[errors, vm_internals]
proc memory32: Memory =
result = newMemory()

View File

@ -1,7 +1,7 @@
import
unittest2, stew/byteutils,
eth/common/eth_types,
../nimbus/vm/interpreter/utils/utils_numeric
../nimbus/vm_internals
func toAddress(n: int): EthAddress =
result[19] = n.byte

View File

@ -6,10 +6,10 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
unittest2, ../nimbus/vm/precompiles, json, stew/byteutils, test_helpers, os, tables,
unittest2, ../nimbus/vm_precompiles, json, stew/byteutils, test_helpers, os, tables,
strformat, strutils, eth/trie/db, eth/common, ../nimbus/db/db_chain,
../nimbus/[vm_types, vm_state], ../nimbus/vm/computation, macros,
../nimbus/vm/interpreter/vm_forks, test_allowed_to_fail
../nimbus/[vm_computation, vm_types, vm_state, vm_types2], macros,
test_allowed_to_fail
proc initAddress(i: byte): EthAddress = result[19] = i

View File

@ -8,7 +8,7 @@
import
unittest2,
eth/common/eth_types,
../nimbus/[constants, errors, vm/interpreter]
../nimbus/[constants, errors, vm_internals]
template testPush(value: untyped, expected: untyped): untyped =

View File

@ -8,7 +8,7 @@
import
unittest2, strformat, strutils, tables, json, os, times, sequtils,
stew/byteutils, eth/[rlp, common], eth/trie/db,
./test_helpers, ./test_allowed_to_fail, ../nimbus/vm/interpreter,
./test_helpers, ./test_allowed_to_fail, ../nimbus/vm_internals,
../nimbus/[constants, vm_state, vm_types, utils],
../nimbus/db/[db_chain]