Merge pull request #523 from status-im/implement_rpc

[WIP] RPC implementation and test
This commit is contained in:
andri lim 2020-07-31 11:42:37 +07:00 committed by GitHub
commit 66ade681b6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 1166 additions and 530 deletions

View File

@ -8,7 +8,7 @@
# those terms.
import
parseopt, strutils, macros, os, times, json, stew/[byteutils],
parseopt, strutils, macros, os, times, json, tables, stew/[byteutils],
chronos, eth/[keys, common, p2p, net/nat], chronicles, nimcrypto/hash,
eth/p2p/bootnodes, eth/p2p/rlpx_protocols/whisper_protocol,
./db/select_backend, eth/keys,
@ -33,7 +33,7 @@ const
NimbusIdent* = "$1/$2 ($3/$4)" % [NimbusName, NimbusVersion, hostCPU, hostOS]
## project ident name for networking services
GitRevision = staticExec("git rev-parse --short HEAD")
GitRevision = staticExec("git rev-parse --short HEAD").replace("\n") # remove CR
NimVersion = staticExec("nim --version")
@ -121,6 +121,11 @@ type
Full
Archive
NimbusAccount* = object
privateKey*: PrivateKey
keystore*: JsonNode
unlocked*: bool
ChainConfig* = object
chainId*: uint
homesteadBlock*: BlockNumber
@ -143,7 +148,7 @@ type
NimbusConfiguration* = ref object
## Main Nimbus configuration object
dataDir*: string
keyFile*: string
keyStore*: string
prune*: PruneMode
rpc*: RpcConfiguration ## JSON-RPC configuration
net*: NetConfiguration ## Network configuration
@ -153,6 +158,7 @@ type
# You should only create one instance of the RNG per application / library
# Ref is used so that it can be shared between components
rng*: ref BrHmacDrbgContext
accounts*: Table[EthAddress, NimbusAccount]
CustomGenesisConfig = object
chainId*: uint
@ -557,11 +563,8 @@ proc processEthArguments(key, value: string): ConfigStatus =
result = Success
let config = getConfiguration()
case key.toLowerAscii()
of "keyfile":
if fileExists(value):
config.keyFile = value
else:
result = ErrorIncorrectOption
of "keystore":
config.keyStore = value
of "datadir":
config.dataDir = value
of "prune":
@ -825,16 +828,20 @@ template processArgument(processor, key, value, msg: untyped) =
proc getDefaultDataDir*(): string =
when defined(windows):
"AppData" / "Roaming" / "Nimbus" / "DB"
"AppData" / "Roaming" / "Nimbus"
elif defined(macosx):
"Library" / "Application Support" / "Nimbus" / "DB"
"Library" / "Application Support" / "Nimbus"
else:
".cache" / "nimbus" / "db"
".cache" / "nimbus"
proc getDefaultKeystoreDir*(): string =
getDefaultDataDir() / "keystore"
proc initConfiguration(): NimbusConfiguration =
## Allocates and initializes `NimbusConfiguration` with default values
result = new NimbusConfiguration
result.rng = newRng()
result.accounts = initTable[EthAddress, NimbusAccount]()
## RPC defaults
result.rpc.flags = {}
@ -853,9 +860,12 @@ proc initConfiguration(): NimbusConfiguration =
result.net.protocols = defaultProtocols
result.net.nodekey = random(PrivateKey, result.rng[])
const dataDir = getDefaultDataDir()
const
dataDir = getDefaultDataDir()
keystore = getDefaultKeystoreDir()
result.dataDir = getHomeDir() / dataDir
result.keystore = getHomeDir() / keystore
result.prune = PruneMode.Full
## Whisper defaults
@ -897,7 +907,7 @@ USAGE:
nimbus [options]
ETHEREUM OPTIONS:
--keyfile:<value> Use keyfile storage file
--keystore:<value> Directory for the keystore (default = inside the datadir)
--datadir:<value> Base directory for all blockchain-related data
--prune:<value> Blockchain prune mode(full or archive)

View File

@ -18,10 +18,12 @@ type
pruneTrie*: bool
config* : ChainConfig
#KeyType = enum
# blockNumberToHash
# blockHashToScore
#
# startingBlock, currentBlock, and highestBlock
# are progress indicator
startingBlock*: BlockNumber
currentBlock*: BlockNumber
highestBlock*: BlockNumber
TransactionKey = tuple
blockNumber: BlockNumber
index: int
@ -64,6 +66,15 @@ proc getCanonicalHead*(self: BaseChainDB): BlockHeader =
raise newException(CanonicalHeadNotFound,
"No canonical head set for this chain")
proc populateProgress*(self: BaseChainDB) =
try:
self.startingBlock = self.getCanonicalHead().blockNumber
except CanonicalHeadNotFound:
self.startingBlock = toBlockNumber(0)
self.currentBlock = self.startingBlock
self.highestBlock = self.startingBlock
proc getBlockHash*(self: BaseChainDB, n: BlockNumber, output: var Hash256): bool {.inline.} =
## Return the block hash for the given block number.
self.getHash(blockNumberToHashKey(n), output)
@ -117,7 +128,8 @@ proc addBlockNumberToHashLookup*(self: BaseChainDB; header: BlockHeader) =
self.db.put(blockNumberToHashKey(header.blockNumber).toOpenArray,
rlp.encode(header.hash))
proc persistTransactions*(self: BaseChainDB, blockNumber: BlockNumber, transactions: openArray[Transaction]) =
proc persistTransactions*(self: BaseChainDB, blockNumber:
BlockNumber, transactions: openArray[Transaction]): Hash256 =
var trie = initHexaryTrie(self.db)
for idx, tx in transactions:
let
@ -126,8 +138,16 @@ proc persistTransactions*(self: BaseChainDB, blockNumber: BlockNumber, transacti
txKey: TransactionKey = (blockNumber, idx)
trie.put(rlp.encode(idx), encodedTx)
self.db.put(transactionHashToBlockKey(txHash).toOpenArray, rlp.encode(txKey))
trie.rootHash
iterator getBlockTransactionData(self: BaseChainDB, transactionRoot: Hash256): seq[byte] =
proc getTransaction*(self: BaseChainDB, txRoot: Hash256, txIndex: int, res: var Transaction): bool =
var db = initHexaryTrie(self.db, txRoot)
let txData = db.get(rlp.encode(txIndex))
if txData.len > 0:
res = rlp.decode(txData, Transaction)
result = true
iterator getBlockTransactionData*(self: BaseChainDB, transactionRoot: Hash256): seq[byte] =
var transactionDb = initHexaryTrie(self.db, transactionRoot)
var transactionIdx = 0
while true:
@ -138,12 +158,38 @@ iterator getBlockTransactionData(self: BaseChainDB, transactionRoot: Hash256): s
break
inc transactionIdx
iterator getBlockTransactionHashes(self: BaseChainDB, blockHeader: BlockHeader): Hash256 =
iterator getBlockTransactions*(self: BaseChainDB, header: BlockHeader): Transaction =
for encodedTx in self.getBlockTransactionData(header.txRoot):
yield rlp.decode(encodedTx, Transaction)
iterator getBlockTransactionHashes*(self: BaseChainDB, blockHeader: BlockHeader): Hash256 =
## Returns an iterable of the transaction hashes from th block specified
## by the given block header.
for encodedTx in self.getBlockTransactionData(blockHeader.txRoot):
yield keccakHash(encodedTx)
proc getTransactionCount*(chain: BaseChainDB, txRoot: Hash256): int =
var trie = initHexaryTrie(chain.db, txRoot)
var txCount = 0
while true:
let txKey = rlp.encode(txCount)
if txKey notin trie:
break
inc txCount
proc getUnclesCount*(self: BaseChainDB, ommersHash: Hash256): int =
if ommersHash != EMPTY_UNCLE_HASH:
let encodedUncles = self.db.get(genericHashKey(ommersHash).toOpenArray)
if encodedUncles.len != 0:
let r = rlpFromBytes(encodedUncles)
result = r.listLen
proc getUncles*(self: BaseChainDB, ommersHash: Hash256): seq[BlockHeader] =
if ommersHash != EMPTY_UNCLE_HASH:
let encodedUncles = self.db.get(genericHashKey(ommersHash).toOpenArray)
if encodedUncles.len != 0:
result = rlp.decode(encodedUncles, seq[BlockHeader])
proc getBlockBody*(self: BaseChainDB, blockHash: Hash256, output: var BlockBody): bool =
var header: BlockHeader
if self.getBlockHeader(blockHash, header):
@ -170,11 +216,22 @@ proc getUncleHashes*(self: BaseChainDB, blockHashes: openArray[Hash256]): seq[Ha
for uncle in blockBody.uncles:
result.add uncle.hash
proc getUncleHashes*(self: BaseChainDB, header: BlockHeader): seq[Hash256] =
if header.ommersHash != EMPTY_UNCLE_HASH:
let encodedUncles = self.db.get(genericHashKey(header.ommersHash).toOpenArray)
if encodedUncles.len != 0:
let uncles = rlp.decode(encodedUncles, seq[BlockHeader])
for x in uncles:
result.add x.hash
proc getTransactionKey*(self: BaseChainDB, transactionHash: Hash256): tuple[blockNumber: BlockNumber, index: int] {.inline.} =
let
tx = self.db.get(transactionHashToBlockKey(transactionHash).toOpenArray)
key = rlp.decode(tx, TransactionKey)
return (key.blockNumber, key.index)
let tx = self.db.get(transactionHashToBlockKey(transactionHash).toOpenArray)
if tx.len > 0:
let key = rlp.decode(tx, TransactionKey)
result = (key.blockNumber, key.index)
else:
result = (0.toBlockNumber, -1)
proc removeTransactionFromCanonicalChain(self: BaseChainDB, transactionHash: Hash256) {.inline.} =
## Removes the transaction specified by the given hash from the canonical chain.
@ -214,10 +271,11 @@ proc headerExists*(self: BaseChainDB; blockHash: Hash256): bool =
## Returns True if the header with the given block hash is in our DB.
self.db.contains(genericHashKey(blockHash).toOpenArray)
proc persistReceipts*(self: BaseChainDB, receipts: openArray[Receipt]) =
proc persistReceipts*(self: BaseChainDB, receipts: openArray[Receipt]): Hash256 =
var trie = initHexaryTrie(self.db)
for idx, rec in receipts:
trie.put(rlp.encode(idx), rlp.encode(rec))
trie.rootHash
iterator getReceipts*(self: BaseChainDB; header: BlockHeader): Receipt =
var receiptDb = initHexaryTrie(self.db, header.receiptRoot)

View File

@ -15,7 +15,7 @@ import
eth/p2p/rlpx_protocols/[eth_protocol, les_protocol, whisper_protocol],
eth/p2p/blockchain_sync, eth/net/nat, eth/p2p/peer_pool,
config, genesis, rpc/[common, p2p, debug, whisper, key_storage], p2p/chain,
eth/trie/db, metrics, metrics/chronicles_support
eth/trie/db, metrics, metrics/chronicles_support, utils
## TODO:
## * No IPv6 support
@ -36,6 +36,10 @@ type
proc start(nimbus: NimbusNode) =
var conf = getConfiguration()
let res = conf.loadKeystoreFiles()
if res.isErr:
echo res.error()
quit(QuitFailure)
## logging
setLogLevel(conf.debug.logLevel)
@ -52,11 +56,6 @@ proc start(nimbus: NimbusNode) =
discard setTimer(Moment.fromNow(conf.debug.logMetricsInterval.seconds), logMetrics)
discard setTimer(Moment.fromNow(conf.debug.logMetricsInterval.seconds), logMetrics)
## Creating RPC Server
if RpcFlags.Enabled in conf.rpc.flags:
nimbus.rpcServer = newRpcHttpServer(conf.rpc.binds)
setupCommonRpc(nimbus.rpcServer)
## Creating P2P Server
let keypair = conf.net.nodekey.toKeyPair()
@ -88,6 +87,8 @@ proc start(nimbus: NimbusNode) =
conf.prune == PruneMode.Full,
conf.net.networkId.toPublicNetwork())
chainDB.populateProgress()
if canonicalHeadHashKey().toOpenArray notin trieDB:
initializeEmptyDb(chainDb)
doAssert(canonicalHeadHashKey().toOpenArray in trieDB)
@ -107,6 +108,11 @@ proc start(nimbus: NimbusNode) =
nimbus.ethNode.chain = newChain(chainDB)
## Creating RPC Server
if RpcFlags.Enabled in conf.rpc.flags:
nimbus.rpcServer = newRpcHttpServer(conf.rpc.binds)
setupCommonRpc(nimbus.ethNode, nimbus.rpcServer)
# Enable RPC APIs based on RPC flags and protocol flags
if RpcFlags.Eth in conf.rpc.flags and ProtocolFlags.Eth in conf.net.protocols:
setupEthRpc(nimbus.ethNode, chainDB, nimbus.rpcServer)

View File

@ -3,6 +3,9 @@ import ../db/db_chain, eth/common, chronicles, ../vm_state, ../vm_types,
../utils, eth/trie/db, ./executor, ../config, ../genesis, ../utils,
stew/endians2
when not defined(release):
import ../tracer
type
# Chain's forks not always equals to EVM's forks
ChainFork = enum
@ -125,6 +128,7 @@ method persistBlocks*(c: Chain, headers: openarray[BlockHeader], bodies: openarr
debug "Number of headers not matching number of bodies"
return ValidationResult.Error
c.db.highestBlock = headers[^1].blockNumber
let transaction = c.db.db.beginTransaction()
defer: transaction.dispose()
@ -148,8 +152,13 @@ method persistBlocks*(c: Chain, headers: openarray[BlockHeader], bodies: openarr
debug "Stored block header hash doesn't match declared hash"
return ValidationResult.Error
c.db.persistTransactions(headers[i].blockNumber, bodies[i].transactions)
c.db.persistReceipts(vmState.receipts)
discard c.db.persistTransactions(headers[i].blockNumber, bodies[i].transactions)
discard c.db.persistReceipts(vmState.receipts)
# update currentBlock *after* we persist it
# so the rpc return consistent result
# between eth_blockNumber and eth_syncing
c.db.currentBlock = headers[i].blockNumber
transaction.commit()

View File

@ -8,14 +8,28 @@
# those terms.
import
strutils,
strutils, tables,
nimcrypto, eth/common as eth_common, stint, json_rpc/server,
eth/p2p,
../config, hexstrings
proc setupCommonRPC*(server: RpcServer) =
proc setupCommonRPC*(node: EthereumNode, server: RpcServer) =
server.rpc("web3_clientVersion") do() -> string:
result = NimbusIdent
server.rpc("web3_sha3") do(data: HexDataStr) -> string:
var rawdata = nimcrypto.fromHex(data.string[2 .. ^1])
result = "0x" & $keccak_256.digest(rawdata)
server.rpc("net_version") do() -> string:
let conf = getConfiguration()
result = $conf.net.networkId
server.rpc("net_listening") do() -> bool:
let conf = getConfiguration()
let numPeers = node.peerPool.connectedNodes.len
result = numPeers < conf.net.maxPeers
server.rpc("net_peerCount") do() -> HexQuantityStr:
let peerCount = uint node.peerPool.connectedNodes.len
result = encodeQuantity(peerCount)

View File

@ -34,8 +34,8 @@ import
type
HexQuantityStr* = distinct string
HexDataStr* = distinct string
EthAddressStr* = distinct string # Same as HexDataStr but must be less <= 20 bytes
EthHashStr* = distinct string # Same as HexDataStr but must be exactly 32 bytes
EthAddressStr* = distinct string # Same as HexDataStr but must be less <= 20 bytes
EthHashStr* = distinct string # Same as HexDataStr but must be exactly 32 bytes
Identifier* = distinct string # 32 bytes, no 0x prefix!
HexStrings = HexQuantityStr | HexDataStr | EthAddressStr | EthHashStr |
Identifier
@ -51,9 +51,13 @@ template stripLeadingZeros(value: string): string =
cidx.inc
value[cidx .. ^1]
func encodeQuantity*(value: SomeUnsignedInt): string {.inline.} =
func encodeQuantity*(value: SomeUnsignedInt): HexQuantityStr {.inline.} =
var hValue = value.toHex.stripLeadingZeros
result = "0x" & hValue
result = HexQuantityStr("0x" & hValue)
func encodeQuantity*(value: UInt256): HexQuantityStr {.inline.} =
var hValue = value.toHex
result = HexQuantityStr("0x" & hValue)
template hasHexHeader(value: string): bool =
if value.len >= 2 and value[0] == '0' and value[1] in {'x', 'X'}: true
@ -65,6 +69,15 @@ template isHexChar(c: char): bool =
c notin {'A'..'F'}: false
else: true
func `==`*(a, b: HexQuantityStr): bool {.inline.} =
a.string == b.string
func `==`*(a, b: EthAddressStr): bool {.inline.} =
a.string == b.string
func `==`*(a, b: HexDataStr): bool {.inline.} =
a.string == b.string
func isValidHexQuantity*(value: string): bool =
if not value.hasHexHeader:
return false
@ -158,14 +171,26 @@ proc hexDataStr*(value: string): HexDataStr {.inline.} =
value.validateHexData
result = value.HexDataStr
proc hexDataStr*(value: openArray[byte]): HexDataStr {.inline.} =
result = HexDataStr("0x" & value.toHex)
proc hexDataStr*(value: Uint256): HexDataStr {.inline.} =
result = HexDataStr("0x" & toBytesBE(value).toHex)
proc ethAddressStr*(value: string): EthAddressStr {.inline.} =
value.validateHexAddressStr
result = value.EthAddressStr
func ethAddressStr*(x: EthAddress): EthAddressStr {.inline.} =
result = EthAddressStr("0x" & toHex(x))
proc ethHashStr*(value: string): EthHashStr {.inline.} =
value.validateHashStr
result = value.EthHashStr
func ethHashStr*(value: Hash256): EthHashStr {.inline.} =
result = EthHashStr("0x" & value.data.toHex)
# Converters for use in RPC
import json
@ -175,7 +200,6 @@ proc `%`*(value: HexStrings): JsonNode =
result = %(value.string)
# Overloads to support expected representation of hex data
proc `%`*(value: EthAddress): JsonNode =
result = %("0x" & value.toHex)
@ -207,7 +231,6 @@ proc `%`*(value: whisper_protocol.Topic): JsonNode =
proc `%`*(value: seq[byte]): JsonNode =
result = %("0x" & value.toHex)
# Helpers for the fromJson procs
proc toPublicKey*(key: string): PublicKey {.inline.} =
@ -247,6 +270,13 @@ proc fromJson*(n: JsonNode, argName: string, result: var EthAddressStr) =
raise newException(ValueError, invalidMsg(argName) & "\" as an Ethereum address \"" & hexStr & "\"")
result = hexStr.EthAddressStr
proc fromJson*(n: JsonNode, argName: string, result: var EthAddress) =
n.kind.expect(JString, argName)
let hexStr = n.getStr()
if not hexStr.isValidEthAddress:
raise newException(ValueError, invalidMsg(argName) & "\" as an Ethereum address \"" & hexStr & "\"")
hexToByteArray(hexStr, result)
proc fromJson*(n: JsonNode, argName: string, result: var EthHashStr) =
n.kind.expect(JString, argName)
let hexStr = n.getStr()
@ -312,3 +342,6 @@ proc fromJson*(n: JsonNode, argName: string, result: var Hash256) =
if not hexStr.isValidHash256:
raise newException(ValueError, invalidMsg(argName) & " as a Hash256 \"" & hexStr & "\"")
hexToByteArray(hexStr, result.data)
proc fromJson*(n: JsonNode, argName: string, result: var JsonNode) =
result = n

View File

@ -8,12 +8,12 @@
# those terms.
import
strutils, times, options,
strutils, times, options, tables,
json_rpc/rpcserver, hexstrings, stint, stew/byteutils,
eth/[common, keys, rlp, p2p], nimcrypto,
eth/p2p/rlpx_protocols/eth_protocol,
../transaction, ../config, ../vm_state, ../constants, ../vm_types,
../vm_state_transactions, ../utils,
../db/[db_chain, accounts_cache],
../utils, ../db/[db_chain, state_db],
rpc_types, rpc_utils, ../vm/[message, computation],
../vm/interpreter/vm_forks
@ -26,135 +26,78 @@ import
type cast to avoid extra processing.
]#
# Work around for https://github.com/nim-lang/Nim/issues/8645
proc `%`*(value: Time): JsonNode =
result = %value.toUnix
template balance(addressDb: ReadOnlyStateDb, address: EthAddress): GasInt =
# TODO: Account balance u256 but GasInt is int64?
addressDb.getBalance(address).truncate(int64)
proc binarySearchGas(vmState: var BaseVMState, transaction: Transaction, sender: EthAddress, gasPrice: GasInt, tolerance = 1): GasInt =
proc dummyComputation(vmState: var BaseVMState, transaction: Transaction, sender: EthAddress): Computation =
# Note that vmState may be altered
var chainDB = vmState.chainDB
let fork = chainDB.config.toFork(vmState.blockNumber)
setupComputation(
vmState,
transaction,
sender,
fork)
proc dummyTransaction(gasLimit, gasPrice: GasInt, destination: EthAddress, value: UInt256): Transaction =
Transaction(
accountNonce: 0.AccountNonce,
gasPrice: gasPrice,
gasLimit: gasLimit,
to: destination,
value: value
)
var
chainDB = vmState.chainDB
fork = chainDB.config.toFork(vmState.blockNumber)
hiGas = vmState.gasLimit
loGas = transaction.intrinsicGas(fork)
gasPrice = transaction.gasPrice # TODO: Or zero?
proc tryTransaction(vmState: var BaseVMState, gasLimit: GasInt): bool =
var
spoofTransaction = dummyTransaction(gasLimit, gasPrice, transaction.to, transaction.value)
computation = vmState.dummyComputation(spoofTransaction, sender)
computation.executeOpcodes
if not computation.isError:
return true
if vmState.tryTransaction(loGas):
return loGas
if not vmState.tryTransaction(hiGas):
return 0.GasInt # TODO: Reraise error from computation
var
minVal = vmState.gasLimit
maxVal = transaction.intrinsicGas(fork)
while loGas - hiGas > tolerance:
let midPoint = (loGas + hiGas) div 2
if vmState.tryTransaction(midPoint):
minVal = midPoint
else:
maxVal = midPoint
result = minVal
proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB , server: RpcServer) =
proc getAccountDb(header: BlockHeader): ReadOnlyStateDB =
## Retrieves the account db from canonical head
let ac = AccountsCache.init(chain.db, header.stateRoot, chain.pruneTrie)
# we don't use accounst_cache here because it's only read operations
let ac = newAccountStateDB(chain.db, header.stateRoot, chain.pruneTrie)
result = ReadOnlyStateDB(ac)
proc accountDbFromTag(tag: string, readOnly = true): ReadOnlyStateDB =
result = getAccountDb(chain.headerFromTag(tag))
proc getBlockBody(hash: KeccakHash): BlockBody =
if not chain.getBlockBody(hash, result):
raise newException(ValueError, "Cannot find hash")
server.rpc("eth_protocolVersion") do() -> string:
result = $eth_protocol.protocolVersion
rpcsrv.rpc("net_version") do() -> uint:
let conf = getConfiguration()
result = conf.net.networkId
rpcsrv.rpc("eth_syncing") do() -> JsonNode:
server.rpc("eth_syncing") do() -> JsonNode:
## Returns SyncObject or false when not syncing.
# TODO: Requires PeerPool to check sync state.
# TODO: Use variant objects
var
sync: SyncState
if true:
# TODO: Populate sync state, this is a placeholder
sync.startingBlock = GENESIS_BLOCK_NUMBER
sync.currentBlock = chain.getCanonicalHead().blockNumber
sync.highestBlock = chain.getCanonicalHead().blockNumber
# TODO: make sure we are not syncing
# when we reach the recent block
let numPeers = node.peerPool.connectedNodes.len
if numPeers > 0:
var sync = SyncState(
startingBlock: encodeQuantity chain.startingBlock,
currentBlock : encodeQuantity chain.currentBlock,
highestBlock : encodeQuantity chain.highestBlock
)
result = %sync
else:
result = newJBool(false)
rpcsrv.rpc("eth_coinbase") do() -> EthAddress:
server.rpc("eth_coinbase") do() -> EthAddress:
## Returns the current coinbase address.
result = chain.getCanonicalHead().coinbase
# currently we don't have miner
result = default(EthAddress)
rpcsrv.rpc("eth_mining") do() -> bool:
server.rpc("eth_mining") do() -> bool:
## Returns true if the client is mining, otherwise false.
discard
# currently we don't have miner
result = false
rpcsrv.rpc("eth_hashrate") do() -> int:
server.rpc("eth_hashrate") do() -> HexQuantityStr:
## Returns the number of hashes per second that the node is mining with.
discard
# currently we don't have miner
result = encodeQuantity(0.uint)
rpcsrv.rpc("eth_gasPrice") do() -> int64:
server.rpc("eth_gasPrice") do() -> HexQuantityStr:
## Returns an integer of the current gas price in wei.
discard
result = encodeQuantity(calculateMedianGasPrice(chain).uint64)
rpcsrv.rpc("eth_accounts") do() -> seq[EthAddressStr]:
server.rpc("eth_accounts") do() -> seq[EthAddressStr]:
## Returns a list of addresses owned by client.
result = @[]
let conf = getConfiguration()
result = newSeqOfCap[EthAddressStr](conf.accounts.len)
for k in keys(conf.accounts):
result.add ethAddressStr(k)
rpcsrv.rpc("eth_blockNumber") do() -> BlockNumber:
server.rpc("eth_blockNumber") do() -> HexQuantityStr:
## Returns integer of the current block number the client is on.
result = chain.getCanonicalHead().blockNumber
result = encodeQuantity(chain.getCanonicalHead().blockNumber)
rpcsrv.rpc("eth_getBalance") do(data: EthAddressStr, quantityTag: string) -> UInt256:
server.rpc("eth_getBalance") do(data: EthAddressStr, quantityTag: string) -> HexQuantityStr:
## Returns the balance of the account of given address.
##
## data: address to check for balance.
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns integer of the current balance in wei.
let
accountDb = accountDbFromTag(quantityTag)
addrBytes = data.toAddress
balance = accountDb.getBalance(addrBytes)
accDB = accountDbFromTag(quantityTag)
address = data.toAddress
balance = accDB.getBalance(address)
result = encodeQuantity(balance)
result = balance
rpcsrv.rpc("eth_getStorageAt") do(data: EthAddressStr, quantity: int, quantityTag: string) -> UInt256:
server.rpc("eth_getStorageAt") do(data: EthAddressStr, quantity: HexQuantityStr, quantityTag: string) -> HexDataStr:
## Returns the value from a storage position at a given address.
##
## data: address of the storage.
@ -162,72 +105,83 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns: the value at this storage position.
let
accountDb = accountDbFromTag(quantityTag)
addrBytes = data.toAddress
result = accountDb.getStorage(addrBytes, quantity.u256)
accDB = accountDbFromTag(quantityTag)
address = data.toAddress
key = fromHex(Uint256, quantity.string)
value = accDB.getStorage(address, key)[0]
result = hexDataStr(value)
rpcsrv.rpc("eth_getTransactionCount") do(data: EthAddressStr, quantityTag: string) -> AccountNonce:
server.rpc("eth_getTransactionCount") do(data: EthAddressStr, quantityTag: string) -> HexQuantityStr:
## Returns the number of transactions sent from an address.
##
## data: address.
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns integer of the number of transactions send from this address.
let
addrBytes = data.toAddress
accountDb = accountDbFromTag(quantityTag)
result = accountDb.getNonce(addrBytes)
address = data.toAddress
accDB = accountDbFromTag(quantityTag)
result = encodeQuantity(accDB.getNonce(address))
rpcsrv.rpc("eth_getBlockTransactionCountByHash") do(data: EthHashStr) -> int:
server.rpc("eth_getBlockTransactionCountByHash") do(data: EthHashStr) -> HexQuantityStr:
## Returns the number of transactions in a block from a block matching the given block hash.
##
## data: hash of a block
## Returns integer of the number of transactions in this block.
var hashData = data.toHash
result = getBlockBody(hashData).transactions.len
let
blockHash = data.toHash
header = chain.getBlockHeader(blockHash)
txCount = chain.getTransactionCount(header.txRoot)
result = encodeQuantity(txCount.uint)
rpcsrv.rpc("eth_getBlockTransactionCountByNumber") do(quantityTag: string) -> int:
server.rpc("eth_getBlockTransactionCountByNumber") do(quantityTag: string) -> HexQuantityStr:
## Returns the number of transactions in a block matching the given block number.
##
## data: integer of a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
## Returns integer of the number of transactions in this block.
let header = chain.headerFromTag(quantityTag)
result = getBlockBody(header.hash).transactions.len
let
header = chain.headerFromTag(quantityTag)
txCount = chain.getTransactionCount(header.txRoot)
result = encodeQuantity(txCount.uint)
rpcsrv.rpc("eth_getUncleCountByBlockHash") do(data: EthHashStr) -> int:
server.rpc("eth_getUncleCountByBlockHash") do(data: EthHashStr) -> HexQuantityStr:
## Returns the number of uncles in a block from a block matching the given block hash.
##
## data: hash of a block.
## Returns integer of the number of uncles in this block.
var hashData = data.toHash
result = getBlockBody(hashData).uncles.len
let
blockHash = data.toHash
header = chain.getBlockHeader(blockHash)
unclesCount = chain.getUnclesCount(header.ommersHash)
result = encodeQuantity(unclesCount.uint)
rpcsrv.rpc("eth_getUncleCountByBlockNumber") do(quantityTag: string) -> int:
server.rpc("eth_getUncleCountByBlockNumber") do(quantityTag: string) -> HexQuantityStr:
## Returns the number of uncles in a block from a block matching the given block number.
##
## quantityTag: integer of a block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns integer of uncles in this block.
let header = chain.headerFromTag(quantityTag)
result = getBlockBody(header.hash).uncles.len
let
header = chain.headerFromTag(quantityTag)
unclesCount = chain.getUnclesCount(header.ommersHash)
result = encodeQuantity(unclesCount.uint)
rpcsrv.rpc("eth_getCode") do(data: EthAddressStr, quantityTag: string) -> HexDataStr:
server.rpc("eth_getCode") do(data: EthAddressStr, quantityTag: string) -> HexDataStr:
## Returns code at a given address.
##
## data: address
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns the code from the given address.
let
accountDb = accountDbFromTag(quantityTag)
addrBytes = toAddress(data)
storage = accountDb.getCode(addrBytes)
# Easier to return the string manually here rather than expect ByteRange to be marshalled
result = byteutils.toHex(storage).HexDataStr
accDB = accountDbFromTag(quantityTag)
address = data.toAddress
storage = accDB.getCode(address)
result = hexDataStr(storage)
template sign(privateKey: PrivateKey, message: string): string =
# TODO: Is message length encoded as bytes or characters?
# message length encoded as ASCII representation of decimal
let msgData = "\x19Ethereum Signed Message:\n" & $message.len & message
$sign(privateKey, msgData.toBytes())
rpcsrv.rpc("eth_sign") do(data: EthAddressStr, message: HexDataStr) -> HexDataStr:
server.rpc("eth_sign") do(data: EthAddressStr, message: HexDataStr) -> HexDataStr:
## The sign method calculates an Ethereum specific signature with: sign(keccak256("\x19Ethereum Signed Message:\n" + len(message) + message))).
## By adding a prefix to the message makes the calculated signature recognisable as an Ethereum specific signature.
## This prevents misuse where a malicious DApp can sign arbitrary data (e.g. transaction) and use the signature to impersonate the victim.
@ -236,100 +190,80 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## data: address.
## message: message to sign.
## Returns signature.
var privateKey: PrivateKey # TODO: Get from key store
result = ("0x" & sign(privateKey, message.string)).HexDataStr
let
address = data.toAddress
conf = getConfiguration()
acc = conf.getAccount(address).tryGet()
msg = hexToSeqByte(message.string)
# proc setupTransaction(send: EthSend): Transaction =
# let
# source = send.source.toAddress
# destination = send.to.toAddress
# data = nimcrypto.utils.fromHex(send.data.string)
# contractCreation = false # TODO: Check if has code
# v = 0.byte # TODO
# r = 0.u256
# s = 0.u256
# result = initTransaction(send.nonce, send.gasPrice, send.gas, destination, send.value, data, v, r, s, contractCreation)
if not acc.unlocked:
raise newException(ValueError, "Account locked, please unlock it first")
result = ("0x" & sign(acc.privateKey, cast[string](msg))).HexDataStr
rpcsrv.rpc("eth_sendTransaction") do(obj: EthSend) -> HexDataStr:
server.rpc("eth_signTransaction") do(data: TxSend) -> HexDataStr:
## Signs a transaction that can be submitted to the network at a later time using with
## eth_sendRawTransaction
let
address = data.source.toAddress
conf = getConfiguration()
acc = conf.getAccount(address).tryGet()
if not acc.unlocked:
raise newException(ValueError, "Account locked, please unlock it first")
let
accDB = accountDbFromTag("latest")
tx = unsignedTx(data, chain, accDB.getNonce(address) + 1)
signedTx = signTransaction(tx, chain, acc.privateKey)
rlpTx = rlp.encode(signedTx)
result = hexDataStr(rlpTx)
server.rpc("eth_sendTransaction") do(data: TxSend) -> EthHashStr:
## Creates new message call transaction or a contract creation, if the data field contains code.
##
## obj: the transaction object.
## Returns the transaction hash, or the zero hash if the transaction is not yet available.
## Note: Use eth_getTransactionReceipt to get the contract address, after the transaction was mined, when you created a contract.
# TODO: Relies on pending pool implementation
discard
let
address = data.source.toAddress
conf = getConfiguration()
acc = conf.getAccount(address).tryGet()
rpcsrv.rpc("eth_sendRawTransaction") do(data: string, quantityTag: int) -> HexDataStr:
if not acc.unlocked:
raise newException(ValueError, "Account locked, please unlock it first")
let
accDB = accountDbFromTag("latest")
tx = unsignedTx(data, chain, accDB.getNonce(address) + 1)
signedTx = signTransaction(tx, chain, acc.privateKey)
rlpTx = rlp.encode(signedTx)
result = keccak_256.digest(rlpTx).ethHashStr
server.rpc("eth_sendRawTransaction") do(data: HexDataStr) -> EthHashStr:
## Creates new message call transaction or a contract creation for signed transactions.
##
## data: the signed transaction data.
## Returns the transaction hash, or the zero hash if the transaction is not yet available.
## Note: Use eth_getTransactionReceipt to get the contract address, after the transaction was mined, when you created a contract.
# TODO: Relies on pending pool implementation
discard
let rlpBytes = hexToSeqByte(data.string)
result = keccak_256.digest(rlpBytes).ethHashStr
proc setupComputation(vmState: BaseVMState,
value: UInt256, data: seq[byte],
sender, destination: EthAddress,
gasLimit, gasPrice: GasInt,
contractCreation: bool): Computation =
let
# Handle optional defaults.
message = Message(
kind: if contractCreation: evmcCreate else: evmcCall,
depth: 0,
gas: gasLimit,
sender: sender,
contractAddress: destination,
codeAddress: CREATE_CONTRACT_ADDRESS,
value: value,
data: data
)
vmState.setupTxContext(
origin = sender,
gasPrice = gasPrice
)
result = newComputation(vmState, message)
rpcsrv.rpc("eth_call") do(call: EthCall, quantityTag: string) -> HexDataStr:
server.rpc("eth_call") do(call: EthCall, quantityTag: string) -> HexDataStr:
## Executes a new message call immediately without creating a transaction on the block chain.
##
## call: the transaction call object.
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns the return value of executed contract.
let header = headerFromTag(chain, quantityTag)
var
# TODO: header.stateRoot to prevStateRoot
vmState = newBaseVMState(header.stateRoot, header, chain)
gasLimit =
if call.gas.isSome: call.gas.get
else: 0.GasInt
gasPrice =
if call.gasPrice.isSome: call.gasPrice.get
else: 0.GasInt
let
header = headerFromTag(chain, quantityTag)
callData = callData(call, true, chain)
result = doCall(callData, header, chain)
# Set defaults for gas limit if required
# Price remains zero by default
if gaslimit == 0.GasInt:
gasLimit = header.gasLimit
var
sender = if call.source.isSome: call.source.get.toAddress else: ZERO_ADDRESS
# Note that destination is a required parameter for call.
# In geth if it's zero they use the first wallet address,
# if no wallets, remains as ZERO_ADDRESS
# TODO: Wallets
destination = if call.to.isSome: call.to.get.toAddress else: ZERO_ADDRESS
data = if call.data.isSome: nimcrypto.utils.fromHex(call.data.get.string) else: @[]
value = if call.value.isSome: call.value.get else: 0.u256
comp = setupComputation(vmState, value, data, sender, destination, gasLimit, gasPrice, call.to.isNone)
comp.execComputation
result = ("0x" & nimcrypto.toHex(comp.output)).HexDataStr
rpcsrv.rpc("eth_estimateGas") do(call: EthCall, quantityTag: string) -> GasInt:
server.rpc("eth_estimateGas") do(call: EthCall, quantityTag: string) -> HexQuantityStr:
## Generates and returns an estimate of how much gas is necessary to allow the transaction to complete.
## The transaction will not be added to the blockchain. Note that the estimate may be significantly more than
## the amount of gas actually used by the transaction, for a variety of reasons including EVM mechanics and node performance.
@ -337,192 +271,96 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## call: the transaction call object.
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
## Returns the amount of gas used.
var
header = chain.headerFromTag(quantityTag)
# TODO: header.stateRoot to prevStateRoot?
vmState = newBaseVMState(header.stateRoot, header, chain)
let
gasLimit = if
call.gas.isSome and call.gas.get > 0.GasInt: call.gas.get
else: header.gasLimit
gasPrice = if
call.gasPrice.isSome and call.gasPrice.get > 0: call.gasPrice.get
else: 0.GasInt
sender = if
call.source.isSome: call.source.get.toAddress
else: ZERO_ADDRESS
destination = if
call.to.isSome: call.to.get.toAddress
else: ZERO_ADDRESS
curState = vmState.readOnlyStateDb()
nonce = curState.getNonce(sender)
value = if
call.value.isSome: call.value.get
else: 0.u256
header = chain.headerFromTag(quantityTag)
callData = callData(call, false, chain)
result = estimateGas(callData, header, chain, call.gas.isSome)
transaction = Transaction(
accountNonce: nonce,
gasPrice: gasPrice,
gasLimit: gasLimit,
to: destination,
value: value,
payload: @[]
)
result = vmState.binarySearchGas(transaction, sender, gasPrice)
func populateBlockObject(header: BlockHeader, blockBody: BlockBody): BlockObject =
result.number = some(header.blockNumber)
result.hash = some(header.hash)
result.parentHash = header.parentHash
result.nonce = header.nonce.toUint
# Calculate hash for all uncle headers
var
rawdata = newSeq[byte](blockBody.uncles.len * 32)
startIdx = 0
for i in 0 ..< blockBody.uncles.len:
rawData[startIdx .. startIdx + 32] = blockBody.uncles[i].hash.data
startIdx += 32
result.sha3Uncles = keccakHash(rawData)
result.logsBloom = some(header.bloom)
result.transactionsRoot = header.txRoot
result.stateRoot = header.stateRoot
result.receiptsRoot = header.receiptRoot
result.miner = ZERO_ADDRESS # TODO: Get miner address
result.difficulty = header.difficulty
result.totalDifficulty = header.difficulty # TODO: Calculate
result.extraData = header.extraData
result.size = 0 # TODO: Calculate block size
result.gasLimit = header.gasLimit
result.gasUsed = header.gasUsed
result.timestamp = header.timeStamp
result.transactions = blockBody.transactions
result.uncles = @[]
for i in 0 ..< blockBody.uncles.len:
result.uncles[i] = blockBody.uncles[i].hash
rpcsrv.rpc("eth_getBlockByHash") do(data: EthHashStr, fullTransactions: bool) -> Option[BlockObject]:
server.rpc("eth_getBlockByHash") do(data: EthHashStr, fullTransactions: bool) -> Option[BlockObject]:
## Returns information about a block by hash.
##
## data: Hash of a block.
## fullTransactions: If true it returns the full transaction objects, if false only the hashes of the transactions.
## Returns BlockObject or nil when no block was found.
let
h = data.toHash
header = chain.getBlockHeader(h)
result = some(populateBlockObject(header, getBlockBody(h)))
var
header: BlockHeader
hash = data.toHash
rpcsrv.rpc("eth_getBlockByNumber") do(quantityTag: string, fullTransactions: bool) -> Option[BlockObject]:
if chain.getBlockHeader(hash, header):
result = some(populateBlockObject(header, chain, fullTransactions))
server.rpc("eth_getBlockByNumber") do(quantityTag: string, fullTransactions: bool) -> Option[BlockObject]:
## Returns information about a block by block number.
##
## quantityTag: integer of a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
## fullTransactions: If true it returns the full transaction objects, if false only the hashes of the transactions.
## Returns BlockObject or nil when no block was found.
let
header = chain.headerFromTag(quantityTag)
try:
let header = chain.headerFromTag(quantityTag)
result = some(populateBlockObject(header, chain, fullTransactions))
except:
result = none(BlockObject)
result = some(populateBlockObject(header, getBlockBody(header.hash)))
proc populateTransactionObject(transaction: Transaction, txIndex: int64, blockHeader: BlockHeader, blockHash: Hash256): TransactionObject =
let
# TODO: header.stateRoot to prevStateRoot?
vmState = newBaseVMState(blockHeader.stateRoot, blockHeader, chain)
accountDb = vmState.readOnlyStateDB()
address = transaction.getSender()
txCount = accountDb.getNonce(address)
txHash = transaction.rlpHash
accountGas = accountDb.balance(address)
result.hash = txHash
result.nonce = txCount
result.blockHash = some(blockHash)
result.blockNumber = some(blockHeader.blockNumber)
result.transactionIndex = some(txIndex)
result.source = transaction.getSender()
result.to = some(transaction.to)
result.value = transaction.value
result.gasPrice = transaction.gasPrice
result.gas = accountGas
result.input = transaction.payload
rpcsrv.rpc("eth_getTransactionByHash") do(data: EthHashStr) -> TransactionObject:
server.rpc("eth_getTransactionByHash") do(data: EthHashStr) -> Option[TransactionObject]:
## Returns the information about a transaction requested by transaction hash.
##
## data: hash of a transaction.
## Returns requested transaction information.
let
h = data.toHash()
txDetails = chain.getTransactionKey(h)
header = chain.getBlockHeader(txDetails.blockNumber)
blockHash = chain.getBlockHash(txDetails.blockNumber)
transaction = getBlockBody(blockHash).transactions[txDetails.index]
result = populateTransactionObject(transaction, txDetails.index, header, blockHash)
let txDetails = chain.getTransactionKey(data.toHash())
if txDetails.index < 0:
return none(TransactionObject)
let header = chain.getBlockHeader(txDetails.blockNumber)
var tx: Transaction
if chain.getTransaction(header.txRoot, txDetails.index, tx):
result = some(populateTransactionObject(tx, header, txDetails.index))
# TODO: if the requested transaction not in blockchain
# try to look for pending transaction in txpool
rpcsrv.rpc("eth_getTransactionByBlockHashAndIndex") do(data: EthHashStr, quantity: int) -> TransactionObject:
server.rpc("eth_getTransactionByBlockHashAndIndex") do(data: EthHashStr, quantity: HexQuantityStr) -> Option[TransactionObject]:
## Returns information about a transaction by block hash and transaction index position.
##
## data: hash of a block.
## quantity: integer of the transaction index position.
## Returns requested transaction information.
let
blockHash = data.toHash()
header = chain.getBlockHeader(blockHash)
transaction = getBlockBody(blockHash).transactions[quantity]
result = populateTransactionObject(transaction, quantity, header, blockHash)
let index = hexToInt(quantity.string, int)
var header: BlockHeader
if not chain.getBlockHeader(data.toHash(), header):
return none(TransactionObject)
rpcsrv.rpc("eth_getTransactionByBlockNumberAndIndex") do(quantityTag: string, quantity: int) -> TransactionObject:
var tx: Transaction
if chain.getTransaction(header.txRoot, index, tx):
result = some(populateTransactionObject(tx, header, index))
server.rpc("eth_getTransactionByBlockNumberAndIndex") do(quantityTag: string, quantity: HexQuantityStr) -> Option[TransactionObject]:
## Returns information about a transaction by block number and transaction index position.
##
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
## quantity: the transaction index position.
let
header = chain.headerFromTag(quantityTag)
blockHash = header.hash
transaction = getBlockBody(blockHash).transactions[quantity]
result = populateTransactionObject(transaction, quantity, header, blockHash)
index = hexToInt(quantity.string, int)
proc populateReceipt(receipt: Receipt, gasUsed: GasInt, tx: Transaction, txIndex: int, blockHeader: BlockHeader): ReceiptObject =
result.transactionHash = tx.rlpHash
result.transactionIndex = txIndex
result.blockHash = blockHeader.hash
result.blockNumber = blockHeader.blockNumber
result.sender = tx.getSender()
result.to = some(tx.to)
result.cumulativeGasUsed = receipt.cumulativeGasUsed
result.gasUsed = gasUsed
var tx: Transaction
if chain.getTransaction(header.txRoot, index, tx):
result = some(populateTransactionObject(tx, header, index))
if tx.isContractCreation:
var sender: EthAddress
if tx.getSender(sender):
let contractAddress = generateAddress(sender, tx.accountNonce)
result.contractAddress = some(contractAddress)
else:
doAssert(false)
else:
result.contractAddress = none(EthAddress)
result.logs = receipt.logs
result.logsBloom = receipt.bloom
# post-transaction stateroot (pre Byzantium).
if receipt.hasStateRoot:
result.root = some(receipt.stateRoot)
else:
# 1 = success, 0 = failure.
result.status = some(receipt.status)
rpcsrv.rpc("eth_getTransactionReceipt") do(data: EthHashStr) -> ReceiptObject:
server.rpc("eth_getTransactionReceipt") do(data: EthHashStr) -> Option[ReceiptObject]:
## Returns the receipt of a transaction by transaction hash.
##
## data: hash of a transaction.
## Returns transaction receipt.
let
h = data.toHash
txDetails = chain.getTransactionKey(h)
header = chain.getBlockHeader(txDetails.blockNumber)
body = getBlockBody(header.hash)
let txDetails = chain.getTransactionKey(data.toHash())
if txDetails.index < 0:
return none(ReceiptObject)
let header = chain.getBlockHeader(txDetails.blockNumber)
var tx: Transaction
if not chain.getTransaction(header.txRoot, txDetails.index, tx):
return none(ReceiptObject)
var
idx = 0
prevGasUsed = GasInt(0)
@ -531,38 +369,48 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
let gasUsed = receipt.cumulativeGasUsed - prevGasUsed
prevGasUsed = receipt.cumulativeGasUsed
if idx == txDetails.index:
return populateReceipt(receipt, gasUsed, body.transactions[txDetails.index], txDetails.index, header)
return some(populateReceipt(receipt, gasUsed, tx, txDetails.index, header))
idx.inc
rpcsrv.rpc("eth_getUncleByBlockHashAndIndex") do(data: EthHashStr, quantity: int) -> Option[BlockObject]:
server.rpc("eth_getUncleByBlockHashAndIndex") do(data: EthHashStr, quantity: HexQuantityStr) -> Option[BlockObject]:
## Returns information about a uncle of a block by hash and uncle index position.
##
## data: hash of block.
## quantity: the uncle's index position.
## Returns BlockObject or nil when no block was found.
let
blockHash = data.toHash()
body = getBlockBody(blockHash)
if quantity < 0 or quantity >= body.uncles.len:
raise newException(ValueError, "Uncle index out of range")
let uncle = body.uncles[quantity]
result = some(populateBlockObject(uncle, body))
let index = hexToInt(quantity.string, int)
var header: BlockHeader
if not chain.getBlockHeader(data.toHash(), header):
return none(BlockObject)
rpcsrv.rpc("eth_getUncleByBlockNumberAndIndex") do(quantityTag: string, quantity: int) -> Option[BlockObject]:
let uncles = chain.getUncles(header.ommersHash)
if index < 0 or index >= uncles.len:
return none(BlockObject)
var uncle = populateBlockObject(uncles[index], chain, false, true)
uncle.totalDifficulty = encodeQuantity(chain.getScore(header.hash))
result = some(uncle)
server.rpc("eth_getUncleByBlockNumberAndIndex") do(quantityTag: string, quantity: HexQuantityStr) -> Option[BlockObject]:
# Returns information about a uncle of a block by number and uncle index position.
##
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
## quantity: the uncle's index position.
## Returns BlockObject or nil when no block was found.
let
index = hexToInt(quantity.string, int)
header = chain.headerFromTag(quantityTag)
body = getBlockBody(header.hash)
if quantity < 0 or quantity >= body.uncles.len:
raise newException(ValueError, "Uncle index out of range")
let uncle = body.uncles[quantity]
result = some(populateBlockObject(uncle, body))
uncles = chain.getUncles(header.ommersHash)
rpcsrv.rpc("eth_newFilter") do(filterOptions: FilterOptions) -> int:
if index < 0 or index >= uncles.len:
return none(BlockObject)
var uncle = populateBlockObject(uncles[index], chain, false, true)
uncle.totalDifficulty = encodeQuantity(chain.getScore(header.hash))
result = some(uncle)
#[
server.rpc("eth_newFilter") do(filterOptions: FilterOptions) -> int:
## Creates a filter object, based on filter options, to notify when the state changes (logs).
## To check if the state has changed, call eth_getFilterChanges.
## Topics are order-dependent. A transaction with a log with topics [A, B] will be matched by the following topic filters:
@ -576,21 +424,21 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## Returns integer filter id.
discard
rpcsrv.rpc("eth_newBlockFilter") do() -> int:
server.rpc("eth_newBlockFilter") do() -> int:
## Creates a filter in the node, to notify when a new block arrives.
## To check if the state has changed, call eth_getFilterChanges.
##
## Returns integer filter id.
discard
rpcsrv.rpc("eth_newPendingTransactionFilter") do() -> int:
server.rpc("eth_newPendingTransactionFilter") do() -> int:
## Creates a filter in the node, to notify when a new block arrives.
## To check if the state has changed, call eth_getFilterChanges.
##
## Returns integer filter id.
discard
rpcsrv.rpc("eth_uninstallFilter") do(filterId: int) -> bool:
server.rpc("eth_uninstallFilter") do(filterId: int) -> bool:
## Uninstalls a filter with given id. Should always be called when watch is no longer needed.
## Additonally Filters timeout when they aren't requested with eth_getFilterChanges for a period of time.
##
@ -598,23 +446,23 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## Returns true if the filter was successfully uninstalled, otherwise false.
discard
rpcsrv.rpc("eth_getFilterChanges") do(filterId: int) -> seq[FilterLog]:
server.rpc("eth_getFilterChanges") do(filterId: int) -> seq[FilterLog]:
## Polling method for a filter, which returns an list of logs which occurred since last poll.
##
## filterId: the filter id.
result = @[]
rpcsrv.rpc("eth_getFilterLogs") do(filterId: int) -> seq[FilterLog]:
server.rpc("eth_getFilterLogs") do(filterId: int) -> seq[FilterLog]:
## filterId: the filter id.
## Returns a list of all logs matching filter with given id.
result = @[]
rpcsrv.rpc("eth_getLogs") do(filterOptions: FilterOptions) -> seq[FilterLog]:
server.rpc("eth_getLogs") do(filterOptions: FilterOptions) -> seq[FilterLog]:
## filterOptions: settings for this filter.
## Returns a list of all logs matching a given filter object.
result = @[]
rpcsrv.rpc("eth_getWork") do() -> array[3, UInt256]:
server.rpc("eth_getWork") do() -> array[3, UInt256]:
## Returns the hash of the current block, the seedHash, and the boundary condition to be met ("target").
## Returned list has the following properties:
## DATA, 32 Bytes - current block header pow-hash.
@ -622,7 +470,7 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## DATA, 32 Bytes - the boundary condition ("target"), 2^256 / difficulty.
discard
rpcsrv.rpc("eth_submitWork") do(nonce: int64, powHash: HexDataStr, mixDigest: HexDataStr) -> bool:
server.rpc("eth_submitWork") do(nonce: int64, powHash: HexDataStr, mixDigest: HexDataStr) -> bool:
## Used for submitting a proof-of-work solution.
##
## nonce: the nonce found.
@ -631,12 +479,10 @@ proc setupEthRpc*(node: EthereumNode, chain: BaseChainDB, rpcsrv: RpcServer) =
## Returns true if the provided solution is valid, otherwise false.
discard
rpcsrv.rpc("eth_submitHashrate") do(hashRate: HexDataStr, id: HexDataStr) -> bool:
server.rpc("eth_submitHashrate") do(hashRate: HexDataStr, id: HexDataStr) -> bool:
## Used for submitting mining hashrate.
##
## hashRate: a hexadecimal string representation (32 bytes) of the hash rate.
## id: a random hexadecimal(32 bytes) ID identifying the client.
## Returns true if submitting went through succesfully and false otherwise.
discard
discard]#

View File

@ -1,5 +1,5 @@
import
hexstrings, options, eth/[common, keys, rlp],
hexstrings, options, eth/[common, keys, rlp], json,
eth/p2p/rlpx_protocols/whisper_protocol
#[
@ -16,66 +16,69 @@ import
type
SyncState* = object
# Returned to user
startingBlock*: BlockNumber
currentBlock*: BlockNumber
highestBlock*: BlockNumber
startingBlock*: HexQuantityStr # BlockNumber
currentBlock* : HexQuantityStr # BlockNumber
highestBlock* : HexQuantityStr # BlockNumber
EthSend* = object
TxSend* = object
# Parameter from user
source*: EthAddressStr # the address the transaction is send from.
to*: EthAddressStr # (optional when creating new contract) the address the transaction is directed to.
gas*: GasInt # (optional, default: 90000) integer of the gas provided for the transaction execution. It will return unused gas.
gasPrice*: GasInt # (optional, default: To-Be-Determined) integer of the gasPrice used for each paid gas.
value*: UInt256 # (optional) integer of the value sent with this transaction.
data*: EthHashStr # TODO: Support more data. The compiled code of a contract OR the hash of the invoked method signature and encoded parameters. For details see Ethereum Contract ABI.
nonce*: AccountNonce # (optional) integer of a nonce. This allows to overwrite your own pending transactions that use the same nonce
source*: EthAddressStr # 20 bytes, the address the transaction is send from.
to*: Option[EthAddressStr] # (optional when creating new contract) 20 bytes, the address the transaction is directed to.
gas*: Option[HexQuantityStr] # (optional, default: 90000) integer of the gas provided for the transaction execution. It will return unused gas.
gasPrice*: Option[HexQuantityStr] # (optional, default: To-Be-Determined) integer of the gasPrice used for each paid gas.
value*: Option[HexQuantityStr] # (optional) integer of the value sent with this transaction.
data*: HexDataStr # TODO: Support more data. The compiled code of a contract OR the hash of the invoked method signature and encoded parameters. For details see Ethereum Contract ABI.
nonce*: Option[HexQuantityStr] # (optional) integer of a nonce. This allows to overwrite your own pending transactions that use the same nonce
EthCall* = object
# Parameter from user
source*: Option[EthAddressStr] # (optional) The address the transaction is send from.
to*: Option[EthAddressStr] # (optional in eth_estimateGas, not in eth_call) The address the transaction is directed to.
gas*: Option[GasInt] # (optional) Integer of the gas provided for the transaction execution. eth_call consumes zero gas, but this parameter may be needed by some executions.
gasPrice*: Option[GasInt] # (optional) Integer of the gasPrice used for each paid gas.
value*: Option[UInt256] # (optional) Integer of the value sent with this transaction.
data*: Option[EthHashStr] # (optional) Hash of the method signature and encoded parameters. For details see Ethereum Contract ABI.
source*: Option[EthAddressStr] # (optional) The address the transaction is send from.
to*: Option[EthAddressStr] # (optional in eth_estimateGas, not in eth_call) The address the transaction is directed to.
gas*: Option[HexQuantityStr]# (optional) Integer of the gas provided for the transaction execution. eth_call consumes zero gas, but this parameter may be needed by some executions.
gasPrice*: Option[HexQuantityStr]# (optional) Integer of the gasPrice used for each paid gas.
value*: Option[HexQuantityStr] # (optional) Integer of the value sent with this transaction.
data*: Option[EthHashStr] # (optional) Hash of the method signature and encoded parameters. For details see Ethereum Contract ABI.
## A block object, or null when no block was found
## Note that this includes slightly different information from eth/common.BlockHeader
BlockObject* = object
# Returned to user
number*: Option[BlockNumber] # the block number. null when its pending block.
number*: Option[HexQuantityStr] # the block number. null when its pending block.
hash*: Option[Hash256] # hash of the block. null when its pending block.
parentHash*: Hash256 # hash of the parent block.
nonce*: uint64 # hash of the generated proof-of-work. null when its pending block.
nonce*: Option[HexDataStr] # hash of the generated proof-of-work. null when its pending block.
sha3Uncles*: Hash256 # SHA3 of the uncles data in the block.
logsBloom*: Option[BloomFilter] # the bloom filter for the logs of the block. null when its pending block.
transactionsRoot*: Hash256 # the root of the transaction trie of the block.
stateRoot*: Hash256 # the root of the final state trie of the block.
receiptsRoot*: Hash256 # the root of the receipts trie of the block.
miner*: EthAddress # the address of the beneficiary to whom the mining rewards were given.
difficulty*: UInt256 # integer of the difficulty for this block.
totalDifficulty*: UInt256 # integer of the total difficulty of the chain until this block.
extraData*: Blob # the "extra data" field of this block.
size*: int # integer the size of this block in bytes.
gasLimit*: GasInt # the maximum gas allowed in this block.
gasUsed*: GasInt # the total used gas by all transactions in this block.
timestamp*: EthTime # the unix timestamp for when the block was collated.
transactions*: seq[Transaction] # list of transaction objects, or 32 Bytes transaction hashes depending on the last given parameter.
difficulty*: HexQuantityStr # integer of the difficulty for this block.
totalDifficulty*: HexQuantityStr# integer of the total difficulty of the chain until this block.
extraData*: HexDataStr # the "extra data" field of this block.
size*: HexQuantityStr # integer the size of this block in bytes.
gasLimit*: HexQuantityStr # the maximum gas allowed in this block.
gasUsed*: HexQuantityStr # the total used gas by all transactions in this block.
timestamp*: HexQuantityStr # the unix timestamp for when the block was collated.
transactions*: seq[JsonNode] # list of transaction objects, or 32 Bytes transaction hashes depending on the last given parameter.
uncles*: seq[Hash256] # list of uncle hashes.
TransactionObject* = object # A transaction object, or null when no transaction was found:
# Returned to user
hash*: Hash256 # hash of the transaction.
nonce*: AccountNonce # the number of transactions made by the sender prior to this one.
blockHash*: Option[Hash256] # hash of the block where this transaction was in. null when its pending.
blockNumber*: Option[BlockNumber] # block number where this transaction was in. null when its pending.
transactionIndex*: Option[int64] # integer of the transactions index position in the block. null when its pending.
source*: EthAddress # address of the sender.
to*: Option[EthAddress] # address of the receiver. null when its a contract creation transaction.
value*: UInt256 # value transferred in Wei.
gasPrice*: GasInt # gas price provided by the sender in Wei.
gas*: GasInt # gas provided by the sender.
blockNumber*: Option[HexQuantityStr] # block number where this transaction was in. null when its pending.
`from`*: EthAddress # address of the sender.
gas*: HexQuantityStr # gas provided by the sender.
gasPrice*: HexQuantityStr # gas price provided by the sender in Wei.
hash*: Hash256 # hash of the transaction.
input*: Blob # the data send along with the transaction.
nonce*: HexQuantityStr # the number of transactions made by the sender prior to this one.
to*: Option[EthAddress] # address of the receiver. null when its a contract creation transaction.
transactionIndex*: Option[HexQuantityStr] # integer of the transactions index position in the block. null when its pending.
value*: HexQuantityStr # value transferred in Wei.
v*: HexQuantityStr # ECDSA recovery id
r*: HexQuantityStr # 32 Bytes - ECDSA signature r
s*: HexQuantityStr # 32 Bytes - ECDSA signature s
FilterLog* = object
# Returned to user
@ -94,13 +97,13 @@ type
ReceiptObject* = object
# A transaction receipt object, or null when no receipt was found:
transactionHash*: Hash256 # hash of the transaction.
transactionIndex*: int # integer of the transactions index position in the block.
transactionIndex*: HexQuantityStr # integer of the transactions index position in the block.
blockHash*: Hash256 # hash of the block where this transaction was in.
blockNumber*: BlockNumber # block number where this transaction was in.
sender*: EthAddress # address of the sender.
blockNumber*: HexQuantityStr # block number where this transaction was in.
`from`*: EthAddress # address of the sender.
to*: Option[EthAddress] # address of the receiver. null when its a contract creation transaction.
cumulativeGasUsed*: GasInt # the total amount of gas used when this transaction was executed in the block.
gasUsed*: GasInt # the amount of gas used by this specific transaction alone.
cumulativeGasUsed*: HexQuantityStr # the total amount of gas used when this transaction was executed in the block.
gasUsed*: HexQuantityStr # the amount of gas used by this specific transaction alone.
contractAddress*: Option[EthAddress] # the contract address created, if the transaction was a contract creation, otherwise null.
logs*: seq[Log] # list of log objects which this transaction generated.
logsBloom*: BloomFilter # bloom filter for light clients to quickly retrieve related logs.

View File

@ -7,9 +7,42 @@
# This file may not be copied, modified, or distributed except according to
# those terms.
import hexstrings, eth/common, stew/byteutils,
../db/[db_chain], strutils,
../constants, stint
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
type
UnsignedTx* = object
nonce* : AccountNonce
gasPrice*: GasInt
gasLimit*: GasInt
to* {.rlpCustomSerialization.}: EthAddress
value * : UInt256
payload* : Blob
contractCreation* {.rlpIgnore.}: bool
CallData* = object
source: EthAddress
to: EthAddress
gas: GasInt
gasPrice: GasInt
value: UInt256
data: seq[byte]
contractCreation: bool
proc read(rlp: var Rlp, t: var UnsignedTx, _: type EthAddress): EthAddress {.inline.} =
if rlp.blobLen != 0:
result = rlp.read(EthAddress)
else:
t.contractCreation = true
proc append(rlpWriter: var RlpWriter, t: UnsignedTx, a: EthAddress) {.inline.} =
if t.contractCreation:
rlpWriter.append("")
else:
rlpWriter.append(a)
func toAddress*(value: EthAddressStr): EthAddress = hexToPaddedByteArray[20](value.string)
@ -19,6 +52,15 @@ func toHash*(value: array[32, byte]): Hash256 {.inline.} =
func toHash*(value: EthHashStr): Hash256 {.inline.} =
result = hexToPaddedByteArray[32](value.string).toHash
func hexToInt*(s: string, T: typedesc[SomeInteger]): T =
var i = 0
if s[i] == '0' and (s[i+1] in {'x', 'X'}): inc(i, 2)
if s.len - i > sizeof(T) * 2:
raise newException(ValueError, "input hex too big for destination int")
while i < s.len:
result = result shl 4 or readHexChar(s[i]).T
inc(i)
proc headerFromTag*(chain: BaseChainDB, blockTag: string): BlockHeader =
let tag = blockTag.toLowerAscii
case tag
@ -32,3 +74,255 @@ proc headerFromTag*(chain: BaseChainDB, blockTag: string): BlockHeader =
tag.validateHexQuantity
let blockNum = stint.fromHex(UInt256, tag)
result = chain.getBlockHeader(blockNum.toBlockNumber)
proc calculateMedianGasPrice*(chain: BaseChainDB): GasInt =
var prices = newSeqOfCap[GasInt](64)
let header = chain.getCanonicalHead()
for encodedTx in chain.getBlockTransactionData(header.txRoot):
let tx = rlp.decode(encodedTx, Transaction)
prices.add(tx.gasPrice)
if prices.len > 0:
sort(prices)
let middle = prices.len div 2
if prices.len mod 2 == 0:
# prevent overflow
let price = prices[middle].uint64 + prices[middle - 1].uint64
result = (price div 2).GasInt
else:
result = prices[middle]
proc unsignedTx*(tx: TxSend, chain: BaseChainDB, defaultNonce: AccountNonce): UnsignedTx =
if tx.to.isSome:
result.to = toAddress(tx.to.get())
result.contractCreation = false
else:
result.contractCreation = true
if tx.gas.isSome:
result.gasLimit = hexToInt(tx.gas.get().string, GasInt)
else:
result.gasLimit = 90000.GasInt
if tx.gasPrice.isSome:
result.gasPrice = hexToInt(tx.gasPrice.get().string, GasInt)
else:
result.gasPrice = calculateMedianGasPrice(chain)
if tx.value.isSome:
result.value = UInt256.fromHex(tx.value.get().string)
else:
result.value = 0.u256
if tx.nonce.isSome:
result.nonce = hexToInt(tx.nonce.get().string, AccountNonce)
else:
result.nonce = defaultNonce
result.payload = hexToSeqByte(tx.data.string)
func rlpEncode(tx: UnsignedTx, chainId: uint): auto =
rlp.encode(Transaction(
accountNonce: tx.nonce,
gasPrice: tx.gasPrice,
gasLimit: tx.gasLimit,
to: tx.to,
value: tx.value,
payload: tx.payload,
isContractCreation: tx.contractCreation,
V: chainId.byte,
R: 0.u256,
S: 0.u256
))
proc signTransaction*(tx: UnsignedTx, chain: BaseChainDB, privateKey: PrivateKey): Transaction =
let eip155 = chain.currentBlock >= chain.config.eip155Block
let rlpTx = if eip155:
rlpEncode(tx, chain.config.chainId)
else:
rlp.encode(tx)
let sig = sign(privateKey, rlpTx).toRaw
let v = if eip155:
byte(sig[64].uint + chain.config.chainId * 2'u + 35'u)
else:
sig[64] + 27.byte
result = Transaction(
accountNonce: tx.nonce,
gasPrice: tx.gasPrice,
gasLimit: tx.gasLimit,
to: tx.to,
value: tx.value,
payload: tx.payload,
isContractCreation: tx.contractCreation,
V: v,
R: Uint256.fromBytesBE(sig[0..31]),
S: Uint256.fromBytesBE(sig[32..63])
)
proc callData*(call: EthCall, callMode: bool = true, chain: BaseChainDB): CallData =
if call.source.isSome:
result.source = toAddress(call.source.get)
if call.to.isSome:
result.to = toAddress(call.to.get)
else:
if callMode:
raise newException(ValueError, "call.to required for eth_call operation")
else:
result.contractCreation = true
if call.gas.isSome:
result.gas = hexToInt(call.gas.get.string, GasInt)
if call.gasPrice.isSome:
result.gasPrice = hexToInt(call.gasPrice.get.string, GasInt)
else:
if not callMode:
result.gasPrice = calculateMedianGasPrice(chain)
if call.value.isSome:
result.value = UInt256.fromHex(call.value.get.string)
if call.data.isSome:
result.data = hexToSeqByte(call.data.get.string)
proc setupComputation(vmState: BaseVMState, call: CallData, fork: Fork) : Computation =
vmState.setupTxContext(
origin = call.source,
gasPrice = call.gasPrice,
forkOverride = some(fork)
)
let msg = Message(
kind: evmcCall,
depth: 0,
gas: call.gas,
sender: call.source,
contractAddress: call.to,
codeAddress: call.to,
value: call.value,
data: call.data
)
result = newComputation(vmState, msg)
proc doCall*(call: CallData, header: BlockHeader, chain: BaseChainDB): HexDataStr =
var
# we use current header stateRoot, unlike block validation
# which use previous block stateRoot
vmState = newBaseVMState(header.stateRoot, header, chain)
fork = toFork(chain.config, header.blockNumber)
comp = setupComputation(vmState, call, fork)
comp.execComputation()
result = hexDataStr(comp.output)
# TODO: handle revert and error
# TODO: handle contract ABI
proc estimateGas*(call: CallData, header: BlockHeader, chain: BaseChainDB, haveGasLimit: bool): HexQuantityStr =
var
# we use current header stateRoot, unlike block validation
# which use previous block stateRoot
vmState = newBaseVMState(header.stateRoot, header, chain)
fork = toFork(chain.config, header.blockNumber)
tx = Transaction(
accountNonce: vmState.accountdb.getNonce(call.source),
gasPrice: call.gasPrice,
gasLimit: if haveGasLimit: call.gas else: header.gasLimit - vmState.cumulativeGasUsed,
to : call.to,
value : call.value,
payload : call.data,
isContractCreation: call.contractCreation
)
var dbTx = chain.db.beginTransaction()
defer: dbTx.dispose()
let gasUsed = processTransaction(tx, call.source, vmState, fork)
result = encodeQuantity(gasUsed.uint64)
dbTx.dispose()
# TODO: handle revert and error
proc populateTransactionObject*(tx: Transaction, header: BlockHeader, txIndex: int): TransactionObject =
result.blockHash = some(header.hash)
result.blockNumber = some(encodeQuantity(header.blockNumber))
result.`from` = tx.getSender()
result.gas = encodeQuantity(tx.gasLimit.uint64)
result.gasPrice = encodeQuantity(tx.gasPrice.uint64)
result.hash = tx.rlpHash
result.input = tx.payLoad
result.nonce = encodeQuantity(tx.accountNonce.uint64)
if not tx.isContractCreation:
result.to = some(tx.to)
result.transactionIndex = some(encodeQuantity(txIndex.uint64))
result.value = encodeQuantity(tx.value)
result.v = encodeQuantity(tx.V.uint)
result.r = encodeQuantity(tx.R)
result.s = encodeQuantity(tx.S)
proc populateBlockObject*(header: BlockHeader, chain: BaseChainDB, fullTx: bool, isUncle = false): BlockObject =
let blockHash = header.blockHash
result.number = some(encodeQuantity(header.blockNumber))
result.hash = some(blockHash)
result.parentHash = header.parentHash
result.nonce = some(hexDataStr(header.nonce))
result.sha3Uncles = header.ommersHash
result.logsBloom = some(header.bloom)
result.transactionsRoot = header.txRoot
result.stateRoot = header.stateRoot
result.receiptsRoot = header.receiptRoot
result.miner = header.coinbase
result.difficulty = encodeQuantity(header.difficulty)
result.extraData = hexDataStr(header.extraData)
# discard sizeof(seq[byte]) of extraData and use actual length
let size = sizeof(BlockHeader) - sizeof(Blob) + header.extraData.len
result.size = encodeQuantity(size.uint)
result.gasLimit = encodeQuantity(header.gasLimit.uint64)
result.gasUsed = encodeQuantity(header.gasUsed.uint64)
result.timestamp = encodeQuantity(header.timeStamp.toUnix.uint64)
if not isUncle:
result.totalDifficulty = encodeQuantity(chain.getScore(blockHash))
result.uncles = chain.getUncleHashes(header)
if fullTx:
var i = 0
for tx in chain.getBlockTransactions(header):
result.transactions.add %(populateTransactionObject(tx, header, i))
inc i
else:
for x in chain.getBlockTransactionHashes(header):
result.transactions.add %(x)
proc populateReceipt*(receipt: Receipt, gasUsed: GasInt, tx: Transaction, txIndex: int, header: BlockHeader): ReceiptObject =
result.transactionHash = tx.rlpHash
result.transactionIndex = encodeQuantity(txIndex.uint)
result.blockHash = header.hash
result.blockNumber = encodeQuantity(header.blockNumber)
result.`from` = tx.getSender()
if tx.isContractCreation:
result.to = some(tx.to)
result.cumulativeGasUsed = encodeQuantity(receipt.cumulativeGasUsed.uint64)
result.gasUsed = encodeQuantity(gasUsed.uint64)
if tx.isContractCreation:
var sender: EthAddress
if tx.getSender(sender):
let contractAddress = generateAddress(sender, tx.accountNonce)
result.contractAddress = some(contractAddress)
result.logs = receipt.logs
result.logsBloom = receipt.bloom
# post-transaction stateroot (pre Byzantium).
if receipt.hasStateRoot:
result.root = some(receipt.stateRoot)
else:
# 1 = success, 0 = failure.
result.status = some(receipt.status)

View File

@ -50,7 +50,7 @@ proc captureAccount(n: JsonNode, db: AccountsCache, address: EthAddress, name: s
let codeHash = db.getCodeHash(address)
let storageRoot = db.getStorageRoot(address)
jaccount["nonce"] = %(encodeQuantity(nonce).toLowerAscii)
jaccount["nonce"] = %(encodeQuantity(nonce).string.toLowerAscii)
jaccount["balance"] = %("0x" & balance.toHex)
let code = db.getCode(address)

View File

@ -1,4 +1,6 @@
import eth/trie/db, eth/[trie, rlp, common], nimcrypto
import
os, tables, json, ./config, stew/[results, byteutils],
eth/trie/db, eth/[trie, rlp, common, keyfile], nimcrypto
export nimcrypto.`$`
@ -50,3 +52,40 @@ proc crc32*(crc: uint32, buf: openArray[byte]): uint32 =
crcu32 = (crcu32 shr 4) xor kcrc32[int((crcu32 and 0xF) xor (uint32(b) shr 4'u32))]
result = not crcu32
proc loadKeystoreFiles*(conf: NimbusConfiguration): Result[void, string] =
try:
createDir(conf.keyStore)
except OSError, IOError:
return err("keystore: cannot create directory")
for filename in walkDirRec(conf.keyStore):
try:
var data = json.parseFile(filename)
let address: EthAddress = hexToByteArray[20](data["address"].getStr())
conf.accounts[address] = NimbusAccount(keystore: data, unlocked: false)
except JsonParsingError:
return err("keystore: json parsing error " & filename)
except ValueError:
return err("keystore: data parsing error")
except Exception: # json raises Exception
return err("keystore: " & getCurrentExceptionMsg())
result = ok()
proc getAccount*(conf: NimbusConfiguration, address: EthAddress): Result[NimbusAccount, string] =
conf.accounts.withValue(address, val) do:
result = ok(val[])
do:
result = err("getAccount: not available " & address.toHex)
proc unlockAccount*(conf: NimbusConfiguration, address: EthAddress, password: string): Result[void, string] =
var acc = conf.getAccount(address).tryGet()
let res = decodeKeyFileJson(acc.keystore, password)
if res.isOk:
acc.privateKey = res.get()
acc.unlocked = true
conf.accounts[address] = acc
result = ok()
else:
result = err($res.error)

View File

@ -136,6 +136,8 @@ proc generateContractAddress(c: Computation, salt: Uint256): EthAddress =
else:
result = generateSafeAddress(c.msg.sender, salt, c.msg.data)
import stew/byteutils
proc newComputation*(vmState: BaseVMState, message: Message, salt= 0.u256): Computation =
new result
result.vmState = vmState

View File

@ -3,8 +3,8 @@ import
eth/[rlp, common], httputils, nimcrypto, chronicles,
stint, stew/byteutils
import
../nimbus/[transaction, rpc/hexstrings]
import ../nimbus/transaction
from ../nimbus/rpc/hexstrings import encodeQuantity
func hexToInt*(s: string, T: typedesc[SomeInteger]): T =
var i = 0
@ -19,7 +19,7 @@ proc prefixHex*(x: Hash256): string =
"0x" & toLowerAscii($x)
proc prefixHex*(x: int64 | uint64 | byte | int): string =
encodeQuantity(x.uint64).toLowerAscii
toLowerAscii(encodeQuantity(x.uint64).string)
proc prefixHex*(x: openArray[byte]): string =
"0x" & toHex(x, true)

View File

@ -13,7 +13,7 @@ proc generatePrestate*(nimbus, geth: JsonNode, blockNumber: Uint256, parent, hea
chainDB = newBaseChainDB(memoryDB, false)
chainDB.setHead(parent, true)
chainDB.persistTransactions(blockNumber, body.transactions)
discard chainDB.persistTransactions(blockNumber, body.transactions)
discard chainDB.persistUncles(body.uncles)
memoryDB.put(genericHashKey(headerHash).toOpenArray, rlp.encode(header))

View File

@ -0,0 +1 @@
{"password":"applebanana","address":"0e69cde81b1aa07a45c32c6cd85d67229d36bb1b","crypto":{"cipher":"aes-128-ctr","cipherparams":{"iv":"7ead94c4f16f3bf31e1cdffd44227403"},"ciphertext":"f023cbdd0ebb26bea0342e2c5719da6a0499e9f675fff3d3c4beb84e1071174f","kdf":"pbkdf2","kdfparams":{"dklen":32,"c":1000000,"prf":"hmac-sha256","salt":"91edacb4f74698e6516062b230064385"},"mac":"f37290005290f56c71cce329f61945d9b2a1c719bc84e966d30c8c395477d625"},"id":"29014ab9-38c0-42fa-b5cf-f049d8568a41","version":3}

View File

@ -0,0 +1 @@
{"password":"bananamonkey","address":"597176e9a64aad0845d83afdaf698fbeff77703b","crypto":{"cipher":"aes-128-ctr","cipherparams":{"iv":"5ad7a73aa1b46b336bd9640a8cf19436"},"ciphertext":"068c556204f86b9e0d6670e1e80f8d34dfb55416b605c0b9f9d833ba962cc64c","kdf":"pbkdf2","kdfparams":{"dklen":32,"c":1000000,"prf":"hmac-sha256","salt":"aea9c4e3dd598d41926268ed2358965e"},"mac":"30cd4c38da212a85aa277c60ee8be7ae4e9cceea75cedeae516827d339d90550"},"id":"7667da26-1e5b-4281-98a3-c9aad03e8491","version":3}

View File

@ -0,0 +1 @@
{"password":"monkeyelephant","address":"a3b2222afa5c987da6ef773fde8d01b9f23d481f","crypto":{"cipher":"aes-128-ctr","cipherparams":{"iv":"3ca1d5c5151fba8281f8880ece920740"},"ciphertext":"ec8b7626f3494605e4b66a889c3dbf9bae4d8fe249718d25f010441af3cf6c61","kdf":"pbkdf2","kdfparams":{"dklen":32,"c":1000000,"prf":"hmac-sha256","salt":"364a67e8bc0a782d715832946662850c"},"mac":"7001a9fb0f63db2b7538bab2a34d2103c67035f4754a3bcf4760359f530d5c21"},"id":"5098841b-9060-4ce7-baf7-c748b12d79c9","version":3}

View File

@ -377,3 +377,8 @@ macro assembler*(list: untyped): untyped =
of "fork": boa.fork = parseFork(body)
else: error("unknown section '" & label & "'", callSection[0])
result = boa.generateVMProxy()
macro evmByteCode*(list: untyped): untyped =
list.expectKind nnkStmtList
var code = parseCode(list)
result = newLitFixed(code)

View File

@ -14,42 +14,42 @@ import
../../nimbus/rpc/hexstrings, ../../nimbus/rpc/rpc_types
proc web3_clientVersion(): string
proc web3_sha3(data: string): string
proc web3_sha3(data: HexDataStr): string
proc net_version(): string
proc net_peerCount(): int
proc net_peerCount(): HexQuantityStr
proc net_listening(): bool
proc eth_protocolVersion(): string
proc eth_syncing(): JsonNode
proc eth_coinbase(): EthAddressStr
proc eth_mining(): bool
proc eth_hashrate(): int
proc eth_gasPrice(): GasInt
proc eth_hashrate(): HexQuantityStr
proc eth_gasPrice(): HexQuantityStr
proc eth_accounts(): seq[EthAddressStr]
proc eth_blockNumber(): BlockNumber
proc eth_getBalance(data: EthAddressStr, quantityTag: string): UInt256
proc eth_getStorageAt(data: EthAddressStr, quantity: int, quantityTag: string): seq[byte]
proc eth_getTransactionCount(data: EthAddressStr, quantityTag: string)
proc eth_getBlockTransactionCountByHash(data: array[32, byte])
proc eth_getBlockTransactionCountByNumber(quantityTag: string)
proc eth_getUncleCountByBlockHash(data: array[32, byte])
proc eth_getUncleCountByBlockNumber(quantityTag: string)
proc eth_blockNumber(): HexQuantityStr
proc eth_getBalance(data: EthAddressStr, quantityTag: string): HexQuantityStr
proc eth_getStorageAt(data: EthAddressStr, quantity: HexQuantityStr, quantityTag: string): seq[byte]
proc eth_getTransactionCount(data: EthAddressStr, quantityTag: string): HexQuantityStr
proc eth_getBlockTransactionCountByHash(data: Hash256): HexQuantityStr
proc eth_getBlockTransactionCountByNumber(quantityTag: string): HexQuantityStr
proc eth_getUncleCountByBlockHash(data: Hash256): HexQuantityStr
proc eth_getUncleCountByBlockNumber(quantityTag: string): HexQuantityStr
proc eth_getCode(data: EthAddressStr, quantityTag: string): HexDataStr
proc eth_sign(data:EthAddressStr, message: HexDataStr): HexDataStr
#proc eth_sendRawTransaction(data: string, quantityTag: int): UInt256
proc eth_call(call: EthCall, quantityTag: string): string
proc eth_estimateGas(call: EthCall, quantityTag: string): GasInt
proc eth_sign(data: EthAddressStr, message: HexDataStr): HexDataStr
proc eth_signTransaction(data: TxSend): HexDataStr
proc eth_sendTransaction(data: TxSend): EthHashStr
proc eth_sendRawTransaction(data: HexDataStr): EthHashStr
proc eth_call(call: EthCall, quantityTag: string): HexDataStr
proc eth_estimateGas(call: EthCall, quantityTag: string): HexQuantityStr
proc eth_getBlockByHash(data: Hash256, fullTransactions: bool): Option[BlockObject]
proc eth_getBlockByNumber(quantityTag: string, fullTransactions: bool): Option[BlockObject]
proc eth_getTransactionByHash(data: Hash256): Option[TransactionObject]
proc eth_getTransactionByBlockHashAndIndex(data: Hash256, quantity: HexQuantityStr): Option[TransactionObject]
proc eth_getTransactionByBlockNumberAndIndex(quantityTag: string, quantity: HexQuantityStr): Option[TransactionObject]
proc eth_getTransactionReceipt(data: Hash256): Option[ReceiptObject]
proc eth_getUncleByBlockHashAndIndex(data: Hash256, quantity: HexQuantityStr): Option[BlockObject]
proc eth_getUncleByBlockNumberAndIndex(quantityTag: string, quantity: HexQuantityStr): Option[BlockObject]
# TODO: Use eth/common types
#[proc eth_sendTransaction(obj: EthSend): UInt256
proc eth_getBlockByHash(data: array[32, byte], fullTransactions: bool): BlockObject
proc eth_getBlockByNumber(quantityTag: string, fullTransactions: bool): BlockObject
proc eth_getTransactionByHash(data: Uint256): TransactionObject
proc eth_getTransactionByBlockHashAndIndex(data: UInt256, quantity: int): TransactionObject
proc eth_getTransactionByBlockNumberAndIndex(quantityTag: string, quantity: int): TransactionObject
proc eth_getTransactionReceipt(data: UInt256): ReceiptObject
proc eth_getUncleByBlockHashAndIndex(data: UInt256, quantity: int64): BlockObject
proc eth_getUncleByBlockNumberAndIndex(quantityTag: string, quantity: int64): BlockObject
#[
proc eth_getCompilers(): seq[string]
proc eth_compileLLL(): seq[byte]
proc eth_compileSolidity(): seq[byte]

View File

@ -6,19 +6,20 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
unittest, json, strformat, options, nimcrypto, stew/byteutils,
unittest, json, strformat, strutils, options, tables, os,
nimcrypto, stew/byteutils, times,
json_rpc/[rpcserver, rpcclient], eth/common as eth_common,
eth/[rlp, keys], eth/trie/db, eth/p2p/rlpx_protocols/eth_protocol,
../nimbus/rpc/[common, p2p, hexstrings, rpc_types],
../nimbus/[constants, vm_state, config, genesis],
../nimbus/db/[accounts_cache, db_chain, storage_types],
../nimbus/p2p/chain,
./rpcclient/test_hexstrings, ./test_helpers
../nimbus/rpc/[common, p2p, hexstrings, rpc_types, rpc_utils],
../nimbus/[constants, vm_state, config, genesis, utils, transaction],
../nimbus/db/[accounts_cache, db_chain, storage_types, state_db],
../nimbus/p2p/[chain, executor], ../nimbus/utils/difficulty,
./rpcclient/test_hexstrings, ./test_helpers, ./macro_assembler
from eth/p2p/rlpx_protocols/whisper_protocol import SymKey
# Perform checks for hex string validation
doHexStrTests()
#doHexStrTests()
from os import getCurrentDir, DirSep
from strutils import rsplit
@ -29,35 +30,127 @@ template sourceDir: string = currentSourcePath.rsplit(DirSep, 1)[0]
const sigPath = &"{sourceDir}{DirSep}rpcclient{DirSep}ethcallsigs.nim"
createRpcSigs(RpcSocketClient, sigPath)
proc toEthAddressStr(address: EthAddress): EthAddressStr =
result = ("0x" & address.toHex).ethAddressStr
type
TestEnv = object
txHash: Hash256
blockHash: HAsh256
proc setupEnv(chain: BaseChainDB, signer, ks2: EthAddress, conf: NimbusConfiguration): TestEnv =
var
parent = chain.getCanonicalHead()
ac = newAccountStateDB(chain.db, parent.stateRoot, chain.pruneTrie)
acc = conf.getAccount(signer).tryGet()
blockNumber = 1.toBlockNumber
parentHash = parent.blockHash
fork = chain.config.toFork(blockNumber)
const code = evmByteCode:
PUSH4 "0xDEADBEEF" # PUSH
PUSH1 "0x00" # MSTORE AT 0x00
MSTORE
PUSH1 "0x04" # RETURN LEN
PUSH1 "0x1C" # RETURN OFFSET at 28
RETURN
ac.setCode(ks2, code)
ac.addBalance(signer, 9_000_000_000.u256)
var vmState = newBaseVMState(ac.rootHash, BlockHeader(parentHash: parentHash), chain)
let
unsignedTx1 = UnsignedTx(
nonce : 0,
gasPrice: 1_100,
gasLimit: 70_000,
value : 1.u256,
contractCreation: false
)
unsignedTx2 = UnsignedTx(
nonce : 0,
gasPrice: 1_200,
gasLimit: 70_000,
value : 2.u256,
contractCreation: false
)
signedTx1 = signTransaction(unsignedTx1, chain, acc.privateKey)
signedTx2 = signTransaction(unsignedTx2, chain, acc.privateKey)
txs = [signedTx1, signedTx2]
txRoot = chain.persistTransactions(blockNumber, txs)
vmState.receipts = newSeq[Receipt](txs.len)
vmState.cumulativeGasUsed = 0
for txIndex, tx in txs:
let sender = tx.getSender()
discard processTransaction(tx, sender, vmState, fork)
vmState.receipts[txIndex] = makeReceipt(vmState, fork)
let
receiptRoot = chain.persistReceipts(vmState.receipts)
date = initDateTime(30, mMar, 2017, 00, 00, 00, 00, utc())
timeStamp = date.toTime
difficulty = calcDifficulty(chain.config, timeStamp, parent)
var header = BlockHeader(
parentHash : parentHash,
#coinbase*: EthAddress
stateRoot : vmState.accountDb.rootHash,
txRoot : txRoot,
receiptRoot : receiptRoot,
bloom : createBloom(vmState.receipts),
difficulty : difficulty,
blockNumber : blockNumber,
gasLimit : vmState.cumulativeGasUsed + 1_000_000,
gasUsed : vmState.cumulativeGasUsed,
timestamp : timeStamp
#extraData: Blob
#mixDigest: Hash256
#nonce: BlockNonce
)
let uncles = [header]
header.ommersHash = chain.persistUncles(uncles)
discard chain.persistHeaderToDb(header)
result = TestEnv(
txHash: signedTx1.rlpHash,
blockHash: header.hash
)
proc doTests {.async.} =
# TODO: Include other transports such as Http
var ethNode = setupEthNode(eth)
let
emptyRlpHash = keccak256.digest(rlp.encode(""))
header = BlockHeader(stateRoot: emptyRlpHash)
var
ethNode = setupEthNode(eth)
chain = newBaseChainDB(newMemoryDb())
state = newBaseVMState(emptyRlpHash, header, chain)
ethNode.chain = newChain(chain)
let
balance = 100.u256
address: EthAddress = hexToByteArray[20]("0x0f572e5295c57f15886f9b263e2f6d2d6c7b5ec6")
signer: EthAddress = hexToByteArray[20]("0x0e69cde81b1aa07a45c32c6cd85d67229d36bb1b")
ks2: EthAddress = hexToByteArray[20]("0xa3b2222afa5c987da6ef773fde8d01b9f23d481f")
ks3: EthAddress = hexToByteArray[20]("0x597176e9a64aad0845d83afdaf698fbeff77703b")
conf = getConfiguration()
ethNode.chain = newChain(chain)
conf.keyStore = "tests" / "keystore"
let res = conf.loadKeystoreFiles()
if res.isErr:
debugEcho res.error
doAssert(res.isOk)
let acc1 = conf.getAccount(signer).tryGet()
let unlock = conf.unlockAccount(signer, acc1.keystore["password"].getStr())
if unlock.isErr:
debugEcho unlock.error
doAssert(unlock.isOk)
defaultGenesisBlockForNetwork(conf.net.networkId.toPublicNetwork()).commit(chain)
state.mutateStateDB:
db.setBalance(address, balance)
doAssert(canonicalHeadHashKey().toOpenArray in state.chainDb.db)
doAssert(canonicalHeadHashKey().toOpenArray in chain.db)
let env = setupEnv(chain, signer, ks2, conf)
# Create Ethereum RPCs
let RPC_PORT = 8545
var
rpcServer = newRpcSocketServer(["localhost:" & $RPC_PORT])
client = newRpcSocketClient()
setupCommonRpc(rpcServer)
setupCommonRpc(ethNode, rpcServer)
setupEthRpc(ethNode, chain, rpcServer)
# Begin tests
@ -66,25 +159,246 @@ proc doTests {.async.} =
# TODO: add more tests here
suite "Remote Procedure Calls":
test "eth_call":
let
blockNum = state.blockheader.blockNumber
callParams = EthCall(value: some(100.u256))
r1 = await client.eth_call(callParams, "0x" & blockNum.toHex)
check r1 == "0x"
test "eth_getBalance":
let r2 = await client.eth_getBalance(ZERO_ADDRESS.toEthAddressStr, "0x0")
check r2 == 0
test "web3_clientVersion":
let res = await client.web3_clientVersion()
check res == NimbusIdent
test "web3_sha3":
expect ValueError:
discard await client.web3_sha3(NimbusName.HexDataStr)
let data = "0x" & byteutils.toHex(NimbusName.toOpenArrayByte(0, NimbusName.len-1))
let res = await client.web3_sha3(data.hexDataStr)
let rawdata = nimcrypto.fromHex(data[2 .. ^1])
let hash = "0x" & $keccak_256.digest(rawdata)
check hash == res
test "net_version":
let res = await client.net_version()
check res == $conf.net.networkId
test "net_listening":
let res = await client.net_listening()
let listening = ethNode.peerPool.connectedNodes.len < conf.net.maxPeers
check res == listening
test "net_peerCount":
let res = await client.net_peerCount()
let peerCount = ethNode.peerPool.connectedNodes.len
check isValidHexQuantity res.string
check res == encodeQuantity(peerCount.uint)
test "eth_protocolVersion":
let res = await client.eth_protocolVersion()
check res == $eth_protocol.protocolVersion
test "eth_syncing":
let res = await client.eth_syncing()
if res.kind == JBool:
let syncing = ethNode.peerPool.connectedNodes.len > 0
check res.getBool() == syncing
else:
check res.kind == JObject
check chain.startingBlock == UInt256.fromHex(res["startingBlock"].getStr())
check chain.currentBlock == UInt256.fromHex(res["currentBlock"].getStr())
check chain.highestBlock == UInt256.fromHex(res["highestBlock"].getStr())
test "eth_coinbase":
let res = await client.eth_coinbase()
# currently we don't have miner
check isValidEthAddress(res.string)
check res == ethAddressStr(EthAddress.default)
test "eth_mining":
let res = await client.eth_mining()
# currently we don't have miner
check res == false
test "eth_hashrate":
let res = await client.eth_hashrate()
# currently we don't have miner
check res == encodeQuantity(0.uint)
test "eth_gasPrice":
let res = await client.eth_gasPrice()
check res.string == "0x47E"
test "eth_accounts":
let res = await client.eth_accounts()
check signer.ethAddressStr in res
check ks2.ethAddressStr in res
check ks3.ethAddressStr in res
test "eth_blockNumber":
let res = await client.eth_blockNumber()
check res.string == "0x1"
test "eth_getBalance":
let a = await client.eth_getBalance(ethAddressStr("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), "0x0")
check a.string == "0x1b1ae4d6e2ef5000000"
let b = await client.eth_getBalance(ethAddressStr("0xfff4bad596633479a2a29f9a8b3f78eefd07e6ee"), "0x0")
check b.string == "0x56bc75e2d63100000"
let c = await client.eth_getBalance(ethAddressStr("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), "0x0")
check c.string == "0x3635c9adc5dea00000"
test "eth_getStorageAt":
let res = await client.eth_getStorageAt(ethAddressStr("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), hexQuantityStr "0x0", "0x0")
check hexDataStr(0.u256).string == hexDataStr(res).string
test "eth_getTransactionCount":
let res = await client.eth_getTransactionCount(ethAddressStr("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), "0x0")
check res.string == "0x0"
test "eth_getBlockTransactionCountByHash":
let hash = chain.getBlockHash(0.toBlockNumber)
let res = await client.eth_getBlockTransactionCountByHash(hash)
check res.string == "0x0"
test "eth_getBlockTransactionCountByNumber":
let res = await client.eth_getBlockTransactionCountByNumber("0x0")
check res.string == "0x0"
test "eth_getUncleCountByBlockHash":
let hash = chain.getBlockHash(0.toBlockNumber)
let res = await client.eth_getUncleCountByBlockHash(hash)
check res.string == "0x0"
test "eth_getUncleCountByBlockNumber":
let res = await client.eth_getUncleCountByBlockNumber("0x0")
check res.string == "0x0"
test "eth_getCode":
let res = await client.eth_getCode(ethAddressStr("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), "0x0")
check res.string == "0x"
test "eth_sign":
let msg = "hello world"
let msgHex = hexDataStr(msg.toOpenArrayByte(0, msg.len-1))
expect ValueError:
discard await client.eth_sign(ethAddressStr(ks2), msgHex)
let res = await client.eth_sign(ethAddressStr(signer), msgHex)
let sig = Signature.fromHex(res.string).tryGet()
# now let us try to verify signature
let msgData = "\x19Ethereum Signed Message:\n" & $msg.len & msg
let msgDataHex = hexDataStr(msgData.toOpenArrayByte(0, msgData.len-1))
let sha3Data = await client.web3_sha3(msgDataHex)
let msgHash = hexToByteArray[32](sha3Data)
let pubkey = recover(sig, SkMessage(msgHash)).tryGet()
let recoveredAddr = pubkey.toCanonicalAddress()
check recoveredAddr == signer # verified
test "eth_signTransaction, eth_sendTransaction, eth_sendRawTransaction":
var unsignedTx = TxSend(
source: ethAddressStr(signer),
to: ethAddressStr(ks2).some,
gas: encodeQuantity(100000'u).some,
gasPrice: none(HexQuantityStr),
value: encodeQuantity(100'u).some,
data: HexDataStr("0x"),
nonce: none(HexQuantityStr)
)
let signedTxHex = await client.eth_signTransaction(unsignedTx)
let signedTx = rlp.decode(hexToSeqByte(signedTxHex.string), Transaction)
check signer == signedTx.getSender() # verified
let hashAhex = await client.eth_sendTransaction(unsignedTx)
let hashBhex = await client.eth_sendRawTransaction(signedTxHex)
check hashAhex.string == hashBhex.string
test "eth_call":
var ec = EthCall(
source: ethAddressStr(signer).some,
to: ethAddressStr(ks2).some,
gas: encodeQuantity(100000'u).some,
gasPrice: none(HexQuantityStr),
value: encodeQuantity(100'u).some
)
let res = await client.eth_call(ec, "latest")
check hexToByteArray[4](res.string) == hexToByteArray[4]("deadbeef")
let blockNum = state.blockheader.blockNumber
let r3 = await client.eth_getBalance(address.toEthAddressStr, "0x" & blockNum.toHex)
check r3 == 0
test "eth_estimateGas":
let
call = EthCall()
blockNum = state.blockheader.blockNumber
r4 = await client.eth_estimateGas(call, "0x" & blockNum.toHex)
check r4 == 21_000
var ec = EthCall(
source: ethAddressStr(signer).some,
to: ethAddressStr(ks3).some,
gas: encodeQuantity(42000'u).some,
gasPrice: encodeQuantity(100'u).some,
value: encodeQuantity(100'u).some
)
let res = await client.eth_estimateGas(ec, "latest")
check hexToInt(res.string, int) == 21000
test "eth_getBlockByHash":
let res = await client.eth_getBlockByHash(env.blockHash, true)
check res.isSome
check res.get().hash.get() == env.blockHash
let res2 = await client.eth_getBlockByHash(env.txHash, true)
check res2.isNone
test "eth_getBlockByNumber":
let res = await client.eth_getBlockByNumber("latest", true)
check res.isSome
check res.get().hash.get() == env.blockHash
let res2 = await client.eth_getBlockByNumber($1, true)
check res2.isNone
test "eth_getTransactionByHash":
let res = await client.eth_getTransactionByHash(env.txHash)
check res.isSome
check res.get().blockNumber.get().string.hexToInt(int) == 1
let res2 = await client.eth_getTransactionByHash(env.blockHash)
check res2.isNone
test "eth_getTransactionByBlockHashAndIndex":
let res = await client.eth_getTransactionByBlockHashAndIndex(env.blockHash, encodeQuantity(0))
check res.isSome
check res.get().blockNumber.get().string.hexToInt(int) == 1
let res2 = await client.eth_getTransactionByBlockHashAndIndex(env.blockHash, encodeQuantity(3))
check res2.isNone
let res3 = await client.eth_getTransactionByBlockHashAndIndex(env.txHash, encodeQuantity(3))
check res3.isNone
test "eth_getTransactionByBlockNumberAndIndex":
let res = await client.eth_getTransactionByBlockNumberAndIndex("latest", encodeQuantity(1))
check res.isSome
check res.get().blockNumber.get().string.hexToInt(int) == 1
let res2 = await client.eth_getTransactionByBlockNumberAndIndex("latest", encodeQuantity(3))
check res2.isNone
test "eth_getTransactionReceipt":
let res = await client.eth_getTransactionReceipt(env.txHash)
check res.isSome
check res.get().blockNumber.string.hexToInt(int) == 1
let res2 = await client.eth_getTransactionReceipt(env.blockHash)
check res2.isNone
test "eth_getUncleByBlockHashAndIndex":
let res = await client.eth_getUncleByBlockHashAndIndex(env.blockHash, encodeQuantity(0))
check res.isSome
check res.get().number.get().string.hexToInt(int) == 1
let res2 = await client.eth_getUncleByBlockHashAndIndex(env.blockHash, encodeQuantity(1))
check res2.isNone
let res3 = await client.eth_getUncleByBlockHashAndIndex(env.txHash, encodeQuantity(0))
check res3.isNone
test "eth_getUncleByBlockNumberAndIndex":
let res = await client.eth_getUncleByBlockNumberAndIndex("latest", encodeQuantity(0))
check res.isSome
check res.get().number.get().string.hexToInt(int) == 1
let res2 = await client.eth_getUncleByBlockNumberAndIndex("latest", encodeQuantity(1))
check res2.isNone
rpcServer.stop()
rpcServer.close()