Fluffy State Bridge - Building state using state diffs. (#2486)

* Started state bridge.

* Implement call to fetch stateDiffs using trace_replayBlockTransactions.

* Convert JSON responses to stateDiff types.

* State updates working for first few blocks.

* Correctly building state for first 200K blocks.

* Add storage of code and cleanup.

* Start state bridge refactor.

* More cleanup and fixes.

* Use RocksDb as backend for state.

* Implement transactions.

* Build RocksDb dependency when building fluffy tools.

* Move code to world state helper.

* Implement producer and consumer queue.

* Cleanup exceptions.

* Improve logging.

* Add update caches to DatabaseRef backends.
This commit is contained in:
web3-developer 2024-07-18 17:01:40 +08:00 committed by GitHub
parent df4a21c910
commit f9956eba59
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 762 additions and 23 deletions

View File

@ -298,7 +298,7 @@ all_fluffy_tests: | build deps
fluffy-test: | all_fluffy_portal_spec_tests all_fluffy_tests fluffy-test: | all_fluffy_portal_spec_tests all_fluffy_tests
# builds the fluffy tools, wherever they are # builds the fluffy tools, wherever they are
$(FLUFFY_TOOLS): | build deps $(FLUFFY_TOOLS): | build deps rocksdb
for D in $(FLUFFY_TOOLS_DIRS); do [ -e "$${D}/$@.nim" ] && TOOL_DIR="$${D}" && break; done && \ for D in $(FLUFFY_TOOLS_DIRS); do [ -e "$${D}/$@.nim" ] && TOOL_DIR="$${D}" && break; done && \
echo -e $(BUILD_MSG) "build/$@" && \ echo -e $(BUILD_MSG) "build/$@" && \
$(ENV_SCRIPT) nim c $(NIM_PARAMS) -d:chronicles_log_level=TRACE -o:build/$@ "$${TOOL_DIR}/$@.nim" $(ENV_SCRIPT) nim c $(NIM_PARAMS) -d:chronicles_log_level=TRACE -o:build/$@ "$${TOOL_DIR}/$@.nim"

View File

@ -24,6 +24,10 @@ createRpcSigsFromNim(RpcClient):
blockId: BlockIdentifier, fullTransactions: bool blockId: BlockIdentifier, fullTransactions: bool
): Opt[BlockObject] ): Opt[BlockObject]
proc eth_getUncleByBlockNumberAndIndex(
blockId: BlockIdentifier, quantity: Quantity
): BlockObject
proc eth_getBlockTransactionCountByHash(data: BlockHash): Quantity proc eth_getBlockTransactionCountByHash(data: BlockHash): Quantity
proc eth_getTransactionReceipt(data: TxHash): Opt[ReceiptObject] proc eth_getTransactionReceipt(data: TxHash): Opt[ReceiptObject]
proc eth_getLogs(filterOptions: FilterOptions): seq[LogObject] proc eth_getLogs(filterOptions: FilterOptions): seq[LogObject]

View File

@ -0,0 +1,17 @@
# fluffy
# Copyright (c) 2023-2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import std/json, json_rpc/[client, jsonmarshal], web3/conversions, web3/eth_api_types
export eth_api_types, json
createRpcSigsFromNim(RpcClient):
proc trace_replayBlockTransactions(
blockId: BlockIdentifier, traceOpts: seq[string]
): JsonNode

View File

@ -7,7 +7,14 @@
{.push raises: [].} {.push raises: [].}
import chronicles, json_rpc/rpcclient, ./portal_bridge_conf import
chronicles,
json_rpc/rpcclient,
web3/[eth_api, eth_api_types],
../../rpc/rpc_calls/rpc_trace_calls,
./portal_bridge_conf
export rpcclient
proc newRpcClientConnect*(url: JsonRpcUrl): RpcClient = proc newRpcClientConnect*(url: JsonRpcUrl): RpcClient =
## Instantiate a new JSON-RPC client and try to connect. Will quit on failure. ## Instantiate a new JSON-RPC client and try to connect. Will quit on failure.
@ -28,3 +35,33 @@ proc newRpcClientConnect*(url: JsonRpcUrl): RpcClient =
fatal "Failed to connect to JSON-RPC server", error = $e.msg, url = url.value fatal "Failed to connect to JSON-RPC server", error = $e.msg, url = url.value
quit QuitFailure quit QuitFailure
client client
proc getBlockByNumber*(
client: RpcClient, blockId: BlockIdentifier, fullTransactions: bool = true
): Future[Result[BlockObject, string]] {.async: (raises: []).} =
let blck =
try:
let res = await client.eth_getBlockByNumber(blockId, fullTransactions)
if res.isNil:
return err("EL failed to provide requested block")
res
except CatchableError as e:
return err("EL JSON-RPC eth_getBlockByNumber failed: " & e.msg)
return ok(blck)
proc getUncleByBlockNumberAndIndex*(
client: RpcClient, blockId: BlockIdentifier, index: Quantity
): Future[Result[BlockObject, string]] {.async: (raises: []).} =
let blck =
try:
let res = await client.eth_getUncleByBlockNumberAndIndex(blockId, index)
if res.isNil:
return err("EL failed to provide requested uncle block")
res
except CatchableError as e:
return err("EL JSON-RPC eth_getUncleByBlockNumberAndIndex failed: " & e.msg)
return ok(blck)

View File

@ -8,7 +8,11 @@
{.push raises: [].} {.push raises: [].}
import import
std/[strutils, os, uri], confutils, confutils/std/net, nimcrypto/hash, ../../logging std/[strutils, os, uri],
confutils,
confutils/std/net,
nimcrypto/hash,
../../[conf, logging]
export net export net
@ -26,6 +30,12 @@ proc defaultEthDataDir*(): string =
proc defaultEra1DataDir*(): string = proc defaultEra1DataDir*(): string =
defaultEthDataDir() / "era1" defaultEthDataDir() / "era1"
proc defaultPortalBridgeStateDir*(): string =
when defined(windows) or defined(macosx):
defaultDataDir() / "Bridge" / "State"
else:
defaultDataDir() / "bridge" / "state"
type type
TrustedDigest* = MDigest[32 * 8] TrustedDigest* = MDigest[32 * 8]
@ -121,6 +131,29 @@ type
web3UrlState* {.desc: "Execution layer JSON-RPC API URL", name: "web3-url".}: web3UrlState* {.desc: "Execution layer JSON-RPC API URL", name: "web3-url".}:
JsonRpcUrl JsonRpcUrl
stateDir* {.
desc: "The directory where the state data is stored",
defaultValue: defaultPortalBridgeStateDir(),
defaultValueDesc: defaultPortalBridgeStateDir(),
name: "state-dir"
.}: InputDir
# TODO: support starting from a specific block. Currently this is not possible using the existing HexaryTrie library.
# startBlockNumber* {.
# desc: "The block number to start from", defaultValue: 1, name: "start-block"
# .}: uint64
verifyState* {.
desc: "Verify the fetched state before gossiping it into the network",
defaultValue: true,
name: "verify-state"
.}: bool
backfillState* {.
desc: "Backfill pre-merge state data into the network",
defaultValue: true,
name: "backfill"
.}: bool
func parseCmdArg*(T: type TrustedDigest, input: string): T {.raises: [ValueError].} = func parseCmdArg*(T: type TrustedDigest, input: string): T {.raises: [ValueError].} =
TrustedDigest.fromHex(input) TrustedDigest.fromHex(input)

View File

@ -121,21 +121,6 @@ func asReceipts(receiptObjects: seq[ReceiptObject]): Result[seq[Receipt], string
## EL JSON-RPC API helper calls for requesting block and receipts ## EL JSON-RPC API helper calls for requesting block and receipts
proc getBlockByNumber(
client: RpcClient, blockTag: RtBlockIdentifier, fullTransactions: bool = true
): Future[Result[BlockObject, string]] {.async: (raises: []).} =
let blck =
try:
let res = await client.eth_getBlockByNumber(blockTag, fullTransactions)
if res.isNil:
return err("EL failed to provide requested block")
res
except CatchableError as e:
return err("EL JSON-RPC eth_getBlockByNumber failed: " & e.msg)
return ok(blck)
proc getBlockReceipts( proc getBlockReceipts(
client: RpcClient, blockNumber: uint64 client: RpcClient, blockNumber: uint64
): Future[Result[seq[ReceiptObject], string]] {.async: (raises: []).} = ): Future[Result[seq[ReceiptObject], string]] {.async: (raises: []).} =

View File

@ -7,12 +7,139 @@
{.push raises: [].} {.push raises: [].}
import chronicles, ./[portal_bridge_conf, portal_bridge_common] import
std/sequtils,
chronicles,
chronos,
stint,
web3/[eth_api, eth_api_types],
results,
eth/common/[eth_types, eth_types_rlp],
../../../nimbus/common/chain_config,
../../rpc/rpc_calls/rpc_trace_calls,
./state_bridge/[database, state_diff, world_state_helper],
./[portal_bridge_conf, portal_bridge_common]
type BlockData = object
blockNumber: uint64
blockObject: BlockObject
stateDiffs: seq[StateDiffRef]
uncleBlocks: seq[BlockObject]
proc runBackfillCollectBlockDataLoop(
blockDataQueue: AsyncQueue[BlockData],
web3Client: RpcClient,
startBlockNumber: uint64,
) {.async: (raises: [CancelledError]).} =
debug "Starting state backfill collect block data loop"
var currentBlockNumber = startBlockNumber
while true:
if currentBlockNumber mod 10000 == 0:
info "Collecting block data for block number: ", blockNumber = currentBlockNumber
let
blockId = blockId(currentBlockNumber)
blockRequest = web3Client.getBlockByNumber(blockId, false)
stateDiffsRequest = web3Client.getStateDiffsByBlockNumber(blockId)
blockObject = (await blockRequest).valueOr:
error "Failed to get block", error
await sleepAsync(1.seconds)
continue
var uncleBlockRequests: seq[Future[Result[BlockObject, string]]]
for i in 0 .. blockObject.uncles.high:
uncleBlockRequests.add(
web3Client.getUncleByBlockNumberAndIndex(blockId, i.Quantity)
)
let stateDiffs = (await stateDiffsRequest).valueOr:
error "Failed to get state diffs", error
await sleepAsync(1.seconds)
continue
var uncleBlocks: seq[BlockObject]
for uncleBlockRequest in uncleBlockRequests:
try:
let uncleBlock = (await uncleBlockRequest).valueOr:
error "Failed to get uncle blocks", error
await sleepAsync(1.seconds)
break
uncleBlocks.add(uncleBlock)
except CatchableError as e:
error "Failed to get uncleBlockRequest", error = e.msg
break
if uncleBlocks.len() < uncleBlockRequests.len():
continue
let blockData = BlockData(
blockNumber: currentBlockNumber,
blockObject: blockObject,
stateDiffs: stateDiffs,
uncleBlocks: uncleBlocks,
)
await blockDataQueue.addLast(blockData)
inc currentBlockNumber
proc runBackfillBuildStateLoop(
blockDataQueue: AsyncQueue[BlockData], stateDir: string
) {.async: (raises: [CancelledError]).} =
debug "Starting state backfill build state loop"
let db = DatabaseRef.init(stateDir).get()
defer:
db.close()
let worldState = db.withTransaction:
let
# Requires an active transaction because it writes an emptyRlp node
# to the accounts HexaryTrie on initialization
ws = WorldStateRef.init(db)
genesisAccounts =
try:
genesisBlockForNetwork(MainNet).alloc
except CatchableError as e:
raiseAssert(e.msg) # Should never happen
ws.applyGenesisAccounts(genesisAccounts)
ws
while true:
let blockData = await blockDataQueue.popFirst()
if blockData.blockNumber mod 10000 == 0:
info "Building state for block number: ", blockNumber = blockData.blockNumber
db.withTransaction:
for stateDiff in blockData.stateDiffs:
worldState.applyStateDiff(stateDiff)
let
minerData =
(EthAddress(blockData.blockObject.miner), blockData.blockObject.number.uint64)
uncleMinersData =
blockData.uncleBlocks.mapIt((EthAddress(it.miner), it.number.uint64))
worldState.applyBlockRewards(minerData, uncleMinersData)
doAssert(blockData.blockObject.stateRoot.bytes() == worldState.stateRoot.data)
trace "State diffs successfully applied to block number:",
blockNumber = blockData.blockNumber
proc runBackfillMetricsLoop(
blockDataQueue: AsyncQueue[BlockData]
) {.async: (raises: [CancelledError]).} =
debug "Starting state backfill metrics loop"
while true:
await sleepAsync(10.seconds)
info "Block data queue length: ", queueLen = blockDataQueue.len()
proc runState*(config: PortalBridgeConf) = proc runState*(config: PortalBridgeConf) =
let let
portalClient = newRpcClientConnect(config.portalRpcUrl) #portalClient = newRpcClientConnect(config.portalRpcUrl)
web3Client = newRpcClientConnect(config.web3Url) web3Client = newRpcClientConnect(config.web3UrlState)
# TODO: # TODO:
# Here we'd want to implement initially a loop that backfills the state # Here we'd want to implement initially a loop that backfills the state
@ -25,5 +152,22 @@ proc runState*(config: PortalBridgeConf) =
# method from nimbus-eth1. # method from nimbus-eth1.
# It could also be implemented by having the whole state execution happening # It could also be implemented by having the whole state execution happening
# inside the bridge, and getting the blocks from era1 files. # inside the bridge, and getting the blocks from era1 files.
notice "State bridge functionality not yet implemented"
quit QuitSuccess if config.backfillState:
const startBlockNumber = 1
# This will become a parameter in the config once we can support it
info "Starting state backfill from block number: ", startBlockNumber
const bufferSize = 1000 # Should we make this configurable?
let blockDataQueue = newAsyncQueue[BlockData](bufferSize)
asyncSpawn runBackfillCollectBlockDataLoop(
blockDataQueue, web3Client, startBlockNumber
)
asyncSpawn runBackfillBuildStateLoop(blockDataQueue, config.stateDir.string)
asyncSpawn runBackfillMetricsLoop(blockDataQueue)
while true:
poll()

View File

@ -0,0 +1,181 @@
# Fluffy
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import std/[os, tables, sequtils], results, eth/trie/db, rocksdb
export results, db
const COL_FAMILY_NAME_ACCOUNTS = "A"
const COL_FAMILY_NAME_STORAGE = "S"
const COL_FAMILY_NAME_BYTECODE = "B"
const COL_FAMILY_NAMES =
[COL_FAMILY_NAME_ACCOUNTS, COL_FAMILY_NAME_STORAGE, COL_FAMILY_NAME_BYTECODE]
type
AccountsBackendRef = ref object of RootObj
cfHandle: ColFamilyHandleRef
tx: TransactionRef
updatedCache: TableRef[seq[byte], seq[byte]]
StorageBackendRef = ref object of RootObj
cfHandle: ColFamilyHandleRef
tx: TransactionRef
updatedCache: TableRef[seq[byte], seq[byte]]
BytecodeBackendRef = ref object of RootObj
cfHandle: ColFamilyHandleRef
tx: TransactionRef
updatedCache: TableRef[seq[byte], seq[byte]]
DatabaseBackendRef = AccountsBackendRef | StorageBackendRef | BytecodeBackendRef
DatabaseRef* = ref object
rocksDb: OptimisticTxDbRef
pendingTransaction: TransactionRef
accountsBackend: AccountsBackendRef
storageBackend: StorageBackendRef
bytecodeBackend: BytecodeBackendRef
proc init*(T: type DatabaseRef, baseDir: string): Result[T, string] =
let dbPath = baseDir / "db"
try:
createDir(dbPath)
except OSError, IOError:
return err("DatabaseRef: cannot create database directory")
let cfOpts = defaultColFamilyOptions(autoClose = true)
cfOpts.`compression=` Compression.lz4Compression
cfOpts.`bottommostCompression=` Compression.zstdCompression
let
db =
?openOptimisticTxDb(
dbPath,
columnFamilies = COL_FAMILY_NAMES.mapIt(initColFamilyDescriptor(it, cfOpts)),
)
accountsBackend = AccountsBackendRef(
cfHandle: db.getColFamilyHandle(COL_FAMILY_NAME_ACCOUNTS).get()
)
storageBackend =
StorageBackendRef(cfHandle: db.getColFamilyHandle(COL_FAMILY_NAME_STORAGE).get())
bytecodeBackend = BytecodeBackendRef(
cfHandle: db.getColFamilyHandle(COL_FAMILY_NAME_BYTECODE).get()
)
ok(
T(
rocksDb: db,
pendingTransaction: nil,
accountsBackend: accountsBackend,
storageBackend: storageBackend,
bytecodeBackend: bytecodeBackend,
)
)
proc onData(data: openArray[byte]) {.gcsafe, raises: [].} =
discard # noop used to check if key exists
proc contains(
dbBackend: DatabaseBackendRef, key: openArray[byte]
): bool {.gcsafe, raises: [].} =
dbBackend.tx.get(key, onData, dbBackend.cfHandle).get()
proc put(
dbBackend: DatabaseBackendRef, key, val: openArray[byte]
) {.gcsafe, raises: [].} =
doAssert dbBackend.tx.put(key, val, dbBackend.cfHandle).isOk()
dbBackend.updatedCache[@key] = @val
proc get(
dbBackend: DatabaseBackendRef, key: openArray[byte]
): seq[byte] {.gcsafe, raises: [].} =
if dbBackend.contains(key):
dbBackend.tx.get(key, dbBackend.cfHandle).get()
else:
@[]
proc del(
dbBackend: DatabaseBackendRef, key: openArray[byte]
): bool {.gcsafe, raises: [].} =
if dbBackend.contains(key):
doAssert dbBackend.tx.delete(key, dbBackend.cfHandle).isOk()
true
else:
false
proc getAccountsBackend*(db: DatabaseRef): TrieDatabaseRef =
trieDB(db.accountsBackend)
proc getStorageBackend*(db: DatabaseRef): TrieDatabaseRef =
trieDB(db.storageBackend)
proc getBytecodeBackend*(db: DatabaseRef): TrieDatabaseRef =
trieDB(db.bytecodeBackend)
proc beginTransaction*(db: DatabaseRef): Result[void, string] =
if not db.pendingTransaction.isNil():
return err("DatabaseRef: Pending transaction already in progress")
let tx = db.rocksDb.beginTransaction()
db.pendingTransaction = tx
db.accountsBackend.tx = tx
db.storageBackend.tx = tx
db.bytecodeBackend.tx = tx
db.accountsBackend.updatedCache = newTable[seq[byte], seq[byte]]()
db.storageBackend.updatedCache = newTable[seq[byte], seq[byte]]()
db.bytecodeBackend.updatedCache = newTable[seq[byte], seq[byte]]()
ok()
proc commitTransaction*(db: DatabaseRef): Result[void, string] =
if db.pendingTransaction.isNil():
return err("DatabaseRef: No pending transaction")
?db.pendingTransaction.commit()
db.pendingTransaction.close()
db.pendingTransaction = nil
ok()
proc rollbackTransaction*(db: DatabaseRef): Result[void, string] =
if db.pendingTransaction.isNil():
return err("DatabaseRef: No pending transaction")
?db.pendingTransaction.rollback()
db.pendingTransaction.close()
db.pendingTransaction = nil
ok()
template withTransaction*(db: DatabaseRef, body: untyped): auto =
db.beginTransaction().expect("Transaction should be started")
try:
body
finally:
db.commitTransaction().expect("Transaction should be commited")
template accountsBackendUpdatedCache*(db: DatabaseRef): TableRef[seq[byte], seq[byte]] =
db.accountsBackend.updatedCache
template storageBackendUpdatedCache*(db: DatabaseRef): TableRef[seq[byte], seq[byte]] =
db.storageBackend.updatedCache
template bytecodeBackendUpdatedCache*(db: DatabaseRef): TableRef[seq[byte], seq[byte]] =
db.bytecodeBackend.updatedCache
proc close*(db: DatabaseRef) =
if not db.pendingTransaction.isNil():
discard db.rollbackTransaction()
db.rocksDb.close()

View File

@ -0,0 +1,112 @@
# Fluffy
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
chronicles,
stew/byteutils,
stint,
eth/common/[eth_types, eth_types_rlp],
../../../rpc/rpc_calls/rpc_trace_calls,
../portal_bridge_common
type
DiffType* = enum
unchanged
create
update
delete
Code = seq[byte]
StateValue* = UInt256 | AccountNonce | Code
StateValueDiff*[StateValue] = object
kind*: DiffType
before*: StateValue
after*: StateValue
StateDiffRef* = ref object
balances*: Table[EthAddress, StateValueDiff[UInt256]]
nonces*: Table[EthAddress, StateValueDiff[AccountNonce]]
storage*: Table[EthAddress, Table[UInt256, StateValueDiff[UInt256]]]
code*: Table[EthAddress, StateValueDiff[Code]]
proc toStateValue(T: type UInt256, hex: string): T {.raises: [ValueError].} =
UInt256.fromHex(hex)
proc toStateValue(T: type AccountNonce, hex: string): T {.raises: [ValueError].} =
UInt256.fromHex(hex).truncate(uint64)
proc toStateValue(T: type Code, hex: string): T {.raises: [ValueError].} =
hexToSeqByte(hex)
proc toStateValueDiff(
diffJson: JsonNode, T: type StateValue
): StateValueDiff[T] {.raises: [ValueError].} =
if diffJson.kind == JString and diffJson.getStr() == "=":
return StateValueDiff[T](kind: unchanged)
elif diffJson.kind == JObject:
if diffJson{"+"} != nil:
return
StateValueDiff[T](kind: create, after: T.toStateValue(diffJson{"+"}.getStr()))
elif diffJson{"-"} != nil:
return
StateValueDiff[T](kind: delete, before: T.toStateValue(diffJson{"-"}.getStr()))
elif diffJson{"*"} != nil:
return StateValueDiff[T](
kind: update,
before: T.toStateValue(diffJson{"*"}{"from"}.getStr()),
after: T.toStateValue(diffJson{"*"}{"to"}.getStr()),
)
else:
doAssert false # unreachable
else:
doAssert false # unreachable
proc toStateDiff(stateDiffJson: JsonNode): StateDiffRef {.raises: [ValueError].} =
let stateDiff = StateDiffRef()
for addrJson, accJson in stateDiffJson.pairs:
let address = EthAddress.fromHex(addrJson)
stateDiff.balances[address] = toStateValueDiff(accJson["balance"], UInt256)
stateDiff.nonces[address] = toStateValueDiff(accJson["nonce"], AccountNonce)
stateDiff.code[address] = toStateValueDiff(accJson["code"], Code)
let storageDiff = accJson["storage"]
var accountStorage: Table[UInt256, StateValueDiff[UInt256]]
for slotKeyJson, slotValueJson in storageDiff.pairs:
let slotKey = UInt256.fromHex(slotKeyJson)
accountStorage[slotKey] = toStateValueDiff(slotValueJson, UInt256)
stateDiff.storage[address] = ensureMove(accountStorage)
stateDiff
proc toStateDiffs(
blockTraceJson: JsonNode
): seq[StateDiffRef] {.raises: [ValueError].} =
var stateDiffs = newSeqOfCap[StateDiffRef](blockTraceJson.len())
for blockTrace in blockTraceJson:
stateDiffs.add(blockTrace["stateDiff"].toStateDiff())
stateDiffs
proc getStateDiffsByBlockNumber*(
client: RpcClient, blockId: BlockIdentifier
): Future[Result[seq[StateDiffRef], string]] {.async: (raises: []).} =
const traceOpts = @["stateDiff"]
try:
let blockTraceJson = await client.trace_replayBlockTransactions(blockId, traceOpts)
if blockTraceJson.isNil:
return err("EL failed to provide requested state diff")
ok(blockTraceJson.toStateDiffs())
except CatchableError as e:
return err("EL JSON-RPC trace_replayBlockTransactions failed: " & e.msg)

View File

@ -0,0 +1,125 @@
# Fluffy
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
stint,
eth/[common, trie, trie/db],
eth/common/[eth_types, eth_types_rlp],
../../../common/common_types,
./database
# Account State definition
type AccountState* = ref object
account: Account
storageUpdates: Table[UInt256, UInt256]
code: seq[byte]
codeUpdated: bool
proc init(T: type AccountState, account = newAccount()): T =
T(account: account, codeUpdated: false)
proc setBalance*(accState: var AccountState, balance: UInt256) =
accState.account.balance = balance
proc addBalance*(accState: var AccountState, balance: UInt256) =
accState.account.balance += balance
proc setNonce*(accState: var AccountState, nonce: AccountNonce) =
accState.account.nonce = nonce
proc setStorage*(accState: var AccountState, slotKey: UInt256, slotValue: UInt256) =
accState.storageUpdates[slotKey] = slotValue
proc deleteStorage*(accState: var AccountState, slotKey: UInt256) =
# setting to zero has the effect of deleting the slot
accState.setStorage(slotKey, 0.u256)
proc setCode*(accState: var AccountState, code: seq[byte]) =
accState.code = code
accState.codeUpdated = true
# World State definition
type
AccountHash = KeccakHash
SlotKeyHash = KeccakHash
WorldStateRef* = ref object
accountsTrie: HexaryTrie
storageTries: TableRef[AccountHash, HexaryTrie]
storageDb: TrieDatabaseRef
bytecodeDb: TrieDatabaseRef # maps AccountHash -> seq[byte]
proc init*(T: type WorldStateRef, db: DatabaseRef): T =
WorldStateRef(
accountsTrie: initHexaryTrie(db.getAccountsBackend(), isPruning = false),
storageTries: newTable[AccountHash, HexaryTrie](),
storageDb: db.getStorageBackend(),
bytecodeDb: db.getBytecodeBackend(),
)
template stateRoot*(state: WorldStateRef): KeccakHash =
state.accountsTrie.rootHash()
template toAccountKey(address: EthAddress): AccountHash =
keccakHash(address)
template toStorageKey(slotKey: UInt256): SlotKeyHash =
keccakHash(toBytesBE(slotKey))
proc getAccount*(state: WorldStateRef, address: EthAddress): AccountState =
let accountKey = toAccountKey(address)
try:
if state.accountsTrie.contains(accountKey.data):
let accountBytes = state.accountsTrie.get(accountKey.data)
AccountState.init(rlp.decode(accountBytes, Account))
else:
AccountState.init()
except RlpError as e:
raiseAssert(e.msg) # should never happen unless the database is corrupted
proc setAccount*(state: WorldStateRef, address: EthAddress, accState: AccountState) =
let accountKey = toAccountKey(address)
try:
if not state.storageTries.contains(accountKey):
state.storageTries[accountKey] =
initHexaryTrie(state.storageDb, isPruning = false)
var storageTrie = state.storageTries.getOrDefault(accountKey)
for k, v in accState.storageUpdates:
if v == 0.u256:
storageTrie.del(toStorageKey(k).data)
else:
storageTrie.put(toStorageKey(k).data, rlp.encode(v))
state.storageTries[accountKey] = storageTrie
var accountToSave = accState.account
accountToSave.storageRoot = storageTrie.rootHash()
if accState.codeUpdated:
state.bytecodeDb.put(accountKey.data, accState.code)
accountToSave.codeHash = keccakHash(accState.code)
state.accountsTrie.put(accountKey.data, rlp.encode(accountToSave))
except RlpError as e:
raiseAssert(e.msg) # should never happen unless the database is corrupted
proc deleteAccount*(state: WorldStateRef, address: EthAddress) =
let accountKey = toAccountKey(address)
try:
state.accountsTrie.del(accountKey.data)
state.storageTries.del(accountKey)
state.bytecodeDb.del(accountKey.data)
except RlpError as e:
raiseAssert(e.msg) # should never happen unless the database is corrupted

View File

@ -0,0 +1,101 @@
# Fluffy
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
chronicles,
stint,
results,
eth/common/[eth_types, eth_types_rlp],
../../../../nimbus/common/chain_config,
./[state_diff, world_state]
export chain_config, state_diff, world_state
proc applyGenesisAccounts*(worldState: WorldStateRef, alloc: GenesisAlloc) =
for address, genAccount in alloc:
var accState = worldState.getAccount(address)
accState.setBalance(genAccount.balance)
accState.setNonce(genAccount.nonce)
if genAccount.code.len() > 0:
for slotKey, slotValue in genAccount.storage:
accState.setStorage(slotKey, slotValue)
accState.setCode(genAccount.code)
worldState.setAccount(address, accState)
proc applyStateDiff*(worldState: WorldStateRef, stateDiff: StateDiffRef) =
for address, balanceDiff in stateDiff.balances:
let
nonceDiff = stateDiff.nonces.getOrDefault(address)
codeDiff = stateDiff.code.getOrDefault(address)
storageDiff = stateDiff.storage.getOrDefault(address)
var
deleteAccount = false
accState = worldState.getAccount(address)
if balanceDiff.kind == create or balanceDiff.kind == update:
accState.setBalance(balanceDiff.after)
elif balanceDiff.kind == delete:
deleteAccount = true
if nonceDiff.kind == create or nonceDiff.kind == update:
accState.setNonce(nonceDiff.after)
elif nonceDiff.kind == delete:
doAssert deleteAccount == true
if codeDiff.kind == create and codeDiff.after.len() > 0:
accState.setCode(codeDiff.after)
elif codeDiff.kind == update:
accState.setCode(codeDiff.after)
elif codeDiff.kind == delete:
doAssert deleteAccount == true
for slotKey, slotDiff in storageDiff:
if slotDiff.kind == create or slotDiff.kind == update:
if slotDiff.after == 0:
accState.deleteStorage(slotKey)
else:
accState.setStorage(slotKey, slotDiff.after)
elif slotDiff.kind == delete:
accState.deleteStorage(slotKey)
if deleteAccount:
worldState.deleteAccount(address)
else:
worldState.setAccount(address, accState)
proc applyBlockRewards*(
worldState: WorldStateRef,
minerData: tuple[miner: EthAddress, number: uint64],
uncleMinersData: openArray[tuple[miner: EthAddress, number: uint64]],
) =
const baseReward = u256(5) * pow(u256(10), 18)
block:
# calculate block miner reward
let
minerAddress = EthAddress(minerData.miner)
uncleInclusionReward = (baseReward shr 5) * u256(uncleMinersData.len())
var accState = worldState.getAccount(minerAddress)
accState.addBalance(baseReward + uncleInclusionReward)
worldState.setAccount(minerAddress, accState)
# calculate uncle miners rewards
for i, uncleMinerData in uncleMinersData:
let
uncleMinerAddress = EthAddress(uncleMinerData.miner)
uncleReward =
(u256(8 + uncleMinerData.number - minerData.number) * baseReward) shr 3
var accState = worldState.getAccount(uncleMinerAddress)
accState.addBalance(uncleReward)
worldState.setAccount(uncleMinerAddress, accState)