Merge pull request #3799 from peekpi/traceDB

[tracer][core] call tracer module when insert blocks
pull/4154/head
Leo Chen 3 years ago committed by GitHub
commit df0a56b775
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 19
      api/service/explorer/schema.go
  2. 10
      api/service/explorer/service.go
  3. 60
      api/service/explorer/storage.go
  4. 1
      cmd/harmony/default.go
  5. 12
      cmd/harmony/flags.go
  6. 2
      cmd/harmony/main.go
  7. 30
      core/blockchain.go
  8. 5
      core/events.go
  9. 12
      core/state/statedb.go
  10. 1
      core/vm/interface.go
  11. 2
      hmy/hmy.go
  12. 11
      hmy/tracer.go
  13. 253
      hmy/tracers/block_tracer.go
  14. 206
      hmy/tracers/block_tracer_storage.go
  15. 216
      hmy/tracers/block_tracer_test.go
  16. 2
      hmy/tracers/rosetta_block_tracer.go
  17. 1
      internal/configs/harmony/harmony.go
  18. 1
      internal/configs/node/config.go
  19. 1
      node/node.go
  20. 34
      node/node_explorer.go
  21. 3
      rpc/tracerParity.go

@ -16,6 +16,7 @@ import (
const (
LegAddressPrefix = "ad_"
CheckpointPrefix = "dc"
TracePrefix = "tr_"
oneAddrByteLen = 42 // byte size of string "one1..."
)
@ -36,6 +37,24 @@ func writeCheckpoint(db databaseWriter, bn uint64) error {
return db.Put(blockCheckpoint, []byte{})
}
func getTraceResultKey(key []byte) []byte {
return append([]byte(TracePrefix), key...)
}
func isTraceResultInDB(db databaseReader, key []byte) (bool, error) {
key = getTraceResultKey(key)
return db.Has(key)
}
func writeTraceResult(db databaseWriter, key []byte, data []byte) error {
key = getTraceResultKey(key)
return db.Put(key, data)
}
func getTraceResult(db databaseReader, key []byte) ([]byte, error) {
key = getTraceResultKey(key)
return db.Get(key)
}
// New schema
var (

@ -17,6 +17,7 @@ import (
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/hmy"
"github.com/harmony-one/harmony/hmy/tracers"
"github.com/harmony-one/harmony/internal/chain"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/utils"
@ -216,6 +217,15 @@ func (s *Service) GetStakingTxHashesByAccount(address string) ([]ethCommon.Hash,
return s.storage.GetStakingTxsByAddress(address)
}
func (s *Service) GetTraceResultByHash(hash ethCommon.Hash) (json.RawMessage, error) {
return s.storage.GetTraceResultByHash(hash)
}
// DumpTraceResult instruct the explorer storage to trace data in explorer DB
func (s *Service) DumpTraceResult(data *tracers.TraceBlockStorage) {
s.storage.DumpTraceResult(data)
}
// DumpNewBlock instruct the explorer storage to dump block data in explorer DB
func (s *Service) DumpNewBlock(b *types.Block) {
s.storage.DumpNewBlock(b)

@ -2,6 +2,7 @@ package explorer
import (
"bytes"
"encoding/json"
"errors"
"fmt"
"os"
@ -13,6 +14,7 @@ import (
"github.com/harmony-one/harmony/core"
core2 "github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/hmy/tracers"
common2 "github.com/harmony-one/harmony/internal/common"
"github.com/harmony-one/harmony/internal/utils"
staking "github.com/harmony-one/harmony/staking/types"
@ -35,6 +37,7 @@ type (
// TODO: optimize this with priority queue
tm *taskManager
resultC chan blockResult
resultT chan *traceResult
available *abool.AtomicBool
closeC chan struct{}
@ -45,6 +48,11 @@ type (
btc batch
bn uint64
}
traceResult struct {
btc batch
data *tracers.TraceBlockStorage
}
)
func newStorage(bc *core.BlockChain, dbPath string) (*storage, error) {
@ -59,6 +67,7 @@ func newStorage(bc *core.BlockChain, dbPath string) (*storage, error) {
bc: bc,
tm: newTaskManager(),
resultC: make(chan blockResult, numWorker),
resultT: make(chan *traceResult, numWorker),
available: abool.New(),
closeC: make(chan struct{}),
log: utils.Logger().With().Str("module", "explorer storage").Logger(),
@ -73,6 +82,10 @@ func (s *storage) Close() {
close(s.closeC)
}
func (s *storage) DumpTraceResult(data *tracers.TraceBlockStorage) {
s.tm.AddNewTraceTask(data)
}
func (s *storage) DumpNewBlock(b *types.Block) {
s.tm.AddNewTask(b)
}
@ -113,6 +126,22 @@ func (s *storage) GetStakingTxsByAddress(addr string) ([]common.Hash, []TxType,
return getStakingTxnHashesByAccount(s.db, oneAddress(addr))
}
func (s *storage) GetTraceResultByHash(hash common.Hash) (json.RawMessage, error) {
if !s.available.IsSet() {
return nil, ErrExplorerNotReady
}
traceStorage := &tracers.TraceBlockStorage{
Hash: hash,
}
err := traceStorage.FromDB(func(key []byte) ([]byte, error) {
return getTraceResult(s.db, key)
})
if err != nil {
return nil, err
}
return traceStorage.ToJson()
}
func (s *storage) run() {
if is, err := isVersionV100(s.db); !is || err != nil {
s.available.UnSet()
@ -136,6 +165,11 @@ func (s *storage) loop() {
if err := res.btc.Write(); err != nil {
s.log.Error().Err(err).Msg("explorer db failed to write")
}
case res := <-s.resultT:
s.log.Info().Str("block hash", res.data.Hash.Hex()).Msg("writing trace into explorer DB")
if err := res.btc.Write(); err != nil {
s.log.Error().Err(err).Msg("explorer db failed to write trace data")
}
case <-s.closeC:
return
@ -149,11 +183,13 @@ type taskManager struct {
lock sync.Mutex
C chan struct{}
T chan *traceResult
}
func newTaskManager() *taskManager {
return &taskManager{
C: make(chan struct{}, numWorker),
T: make(chan *traceResult, numWorker),
}
}
@ -168,6 +204,12 @@ func (tm *taskManager) AddNewTask(b *types.Block) {
}
}
func (tm *taskManager) AddNewTraceTask(data *tracers.TraceBlockStorage) {
tm.T <- &traceResult{
data: data,
}
}
func (tm *taskManager) AddCatchupTask(b *types.Block) {
tm.lock.Lock()
defer tm.lock.Unlock()
@ -211,6 +253,7 @@ func (s *storage) makeWorkersAndStart() {
db: s.db,
bc: s.bc,
resultC: s.resultC,
resultT: s.resultT,
closeC: s.closeC,
log: s.log.With().Int("worker", i).Logger(),
})
@ -225,6 +268,7 @@ type blockComputer struct {
db database
bc blockChainTxIndexer
resultC chan blockResult
resultT chan *traceResult
closeC chan struct{}
log zerolog.Logger
}
@ -255,7 +299,21 @@ LOOP:
return
}
}
case traceResult := <-bc.tm.T:
if exist, err := isTraceResultInDB(bc.db, traceResult.data.KeyDB()); exist || err != nil {
continue
}
traceResult.btc = bc.db.NewBatch()
traceResult.data.ToDB(func(key, value []byte) {
if exist, err := isTraceResultInDB(bc.db, key); !exist && err == nil {
_ = writeTraceResult(traceResult.btc, key, value)
}
})
select {
case bc.resultT <- traceResult:
case <-bc.closeC:
return
}
case <-bc.closeC:
return
}

@ -21,6 +21,7 @@ var defaultConfig = harmonyconfig.HarmonyConfig{
IsBeaconArchival: false,
IsOffline: false,
DataDir: "./",
TraceEnable: false,
},
Network: getDefaultNetworkConfig(defNetworkType),
P2P: harmonyconfig.P2pConfig{

@ -29,6 +29,8 @@ var (
legacyShardIDFlag,
legacyIsArchiveFlag,
legacyDataDirFlag,
taraceFlag,
}
dnsSyncFlags = []cli.Flag{
@ -296,6 +298,12 @@ var (
DefValue: defaultConfig.General.DataDir,
Deprecated: "use --datadir",
}
taraceFlag = cli.BoolFlag{
Name: "tracing",
Usage: "indicates if full transaction tracing should be enabled",
DefValue: defaultConfig.General.TraceEnable,
}
)
func getRootFlags() []cli.Flag {
@ -365,6 +373,10 @@ func applyGeneralFlags(cmd *cobra.Command, config *harmonyconfig.HarmonyConfig)
config.General.IsOffline = cli.GetBoolFlagValue(cmd, isOfflineFlag)
}
if cli.IsFlagChanged(cmd, taraceFlag) {
config.General.TraceEnable = cli.GetBoolFlagValue(cmd, taraceFlag)
}
if cli.IsFlagChanged(cmd, isBackupFlag) {
config.General.IsBackup = cli.GetBoolFlagValue(cmd, isBackupFlag)
}

@ -621,6 +621,8 @@ func createGlobalConfig(hc harmonyconfig.HarmonyConfig) (*nodeconfig.ConfigType,
nodeConfig.NtpServer = hc.Sys.NtpServer
nodeConfig.TraceEnable = hc.General.TraceEnable
return nodeConfig, nil
}

@ -49,6 +49,8 @@ import (
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/numeric"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/hmy/tracers"
"github.com/harmony-one/harmony/shard/committee"
"github.com/harmony-one/harmony/staking/apr"
"github.com/harmony-one/harmony/staking/effective"
@ -141,6 +143,8 @@ type BlockChain struct {
gcproc time.Duration // Accumulates canonical block processing for trie dumping
hc *HeaderChain
trace bool // atomic?
traceFeed event.Feed // send trace_block result to explorer
rmLogsFeed event.Feed
chainFeed event.Feed
chainSideFeed event.Feed
@ -1492,11 +1496,24 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifyHeaders bool) (int,
if err != nil {
return i, events, coalescedLogs, err
}
vmConfig := bc.vmConfig
if bc.trace {
ev := TraceEvent{
Tracer: &tracers.ParityBlockTracer{
Hash: block.Hash(),
Number: block.NumberU64(),
},
}
vmConfig = vm.Config{
Debug: true,
Tracer: ev.Tracer,
}
events = append(events, ev)
}
// Process block using the parent state as reference point.
substart := time.Now()
receipts, cxReceipts, stakeMsgs, logs, usedGas, payout, newState, err := bc.processor.Process(
block, state, bc.vmConfig, true,
block, state, vmConfig, true,
)
state = newState // update state in case the new state is cached.
if err != nil {
@ -1659,6 +1676,9 @@ func (bc *BlockChain) PostChainEvents(events []interface{}, logs []*types.Log) {
case ChainSideEvent:
bc.chainSideFeed.Send(ev)
case TraceEvent:
bc.traceFeed.Send(ev)
}
}
}
@ -1847,6 +1867,12 @@ func (bc *BlockChain) SubscribeRemovedLogsEvent(ch chan<- RemovedLogsEvent) even
return bc.scope.Track(bc.rmLogsFeed.Subscribe(ch))
}
// SubscribeChainEvent registers a subscription of ChainEvent.
func (bc *BlockChain) SubscribeTraceEvent(ch chan<- TraceEvent) event.Subscription {
bc.trace = true
return bc.scope.Track(bc.traceFeed.Subscribe(ch))
}
// SubscribeChainEvent registers a subscription of ChainEvent.
func (bc *BlockChain) SubscribeChainEvent(ch chan<- ChainEvent) event.Subscription {
return bc.scope.Track(bc.chainFeed.Subscribe(ch))

@ -19,6 +19,7 @@ package core
import (
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/hmy/tracers"
)
// NewTxsEvent is posted when a batch of transactions enter the transaction pool.
@ -42,6 +43,10 @@ type ChainEvent struct {
Logs []*types.Log
}
type TraceEvent struct {
Tracer *tracers.ParityBlockTracer
}
// ChainSideEvent is chain side event.
type ChainSideEvent struct {
Block *types.Block

@ -90,7 +90,8 @@ type DB struct {
// The refund counter, also used by state transitioning.
refund uint64
thash, bhash common.Hash
thash, bhash common.Hash // thash means hmy tx hash
ethTxHash common.Hash // ethTxHash is eth tx hash, use by tracer
txIndex int
logs map[common.Hash][]*types.Log
logSize uint
@ -158,6 +159,7 @@ func (db *DB) Reset(root common.Hash) error {
db.stateValidators = make(map[common.Address]*stk.ValidatorWrapper)
db.thash = common.Hash{}
db.bhash = common.Hash{}
db.ethTxHash = common.Hash{}
db.txIndex = 0
db.logs = make(map[common.Hash][]*types.Log)
db.logSize = 0
@ -264,6 +266,10 @@ func (s *DB) TxHash() common.Hash {
return s.thash
}
func (s *DB) TxHashETH() common.Hash {
return s.ethTxHash
}
// BlockHash returns the current block hash set by Prepare.
func (db *DB) BlockHash() common.Hash {
return db.bhash
@ -747,6 +753,10 @@ func (db *DB) Prepare(thash, bhash common.Hash, ti int) {
db.txIndex = ti
}
func (db *DB) SetTxHashETH(ethTxHash common.Hash) {
db.ethTxHash = ethTxHash
}
func (db *DB) clearJournalAndRefund() {
if len(db.journal.entries) > 0 {
db.journal = newJournal()

@ -80,6 +80,7 @@ type StateDB interface {
TxIndex() int
BlockHash() common.Hash
TxHash() common.Hash
TxHashETH() common.Hash // used by tracer
}
// CallContext provides a basic interface for the EVM calling conventions. The EVM

@ -2,6 +2,7 @@ package hmy
import (
"context"
"encoding/json"
"math/big"
"github.com/ethereum/go-ethereum/common"
@ -89,6 +90,7 @@ type NodeAPI interface {
GetStakingTransactionsHistory(address, txType, order string) ([]common.Hash, error)
GetTransactionsCount(address, txType string) (uint64, error)
GetStakingTransactionsCount(address, txType string) (uint64, error)
GetTraceResultByHash(hash common.Hash) (json.RawMessage, error)
IsCurrentlyLeader() bool
IsOutOfSync(shardID uint32) bool
SyncStatus(shardID uint32) (bool, uint64, uint64)

@ -381,9 +381,8 @@ traceLoop:
}
// Generate the next state snapshot fast without tracing
msg, _ := tx.AsMessage(signer)
ethTx := tx.ConvertToEth()
statedb.Prepare(ethTx.Hash(), blockHash, i)
statedb.Prepare(tx.Hash(), blockHash, i)
statedb.SetTxHashETH(tx.ConvertToEth().Hash())
vmctx := core.NewEVMContext(msg, block.Header(), hmy.BlockChain, nil)
res, err := hmy.TraceTx(ctx, msg, vmctx, statedb, config)
if err != nil {
@ -467,8 +466,9 @@ func (hmy *Harmony) TraceBlock(ctx context.Context, block *types.Block, config *
msg, _ := txs[task.index].AsMessage(signer)
vmctx := core.NewEVMContext(msg, block.Header(), hmy.BlockChain, nil)
ethTx := txs[task.index].ConvertToEth()
task.statedb.Prepare(ethTx.Hash(), blockHash, task.index)
tx := txs[task.index]
task.statedb.Prepare(tx.Hash(), blockHash, task.index)
task.statedb.SetTxHashETH(tx.ConvertToEth().Hash())
res, err := hmy.TraceTx(ctx, msg, vmctx, task.statedb, config)
if err != nil {
results[task.index] = &TxTraceResult{Error: err.Error()}
@ -491,6 +491,7 @@ func (hmy *Harmony) TraceBlock(ctx context.Context, block *types.Block, config *
// Generate the next state snapshot fast without tracing
msg, _ := tx.AsMessage(signer)
statedb.Prepare(tx.Hash(), block.Hash(), i)
statedb.SetTxHashETH(tx.ConvertToEth().Hash())
vmctx := core.NewEVMContext(msg, block.Header(), hmy.BlockChain, nil)
vmenv := vm.NewEVM(vmctx, statedb, hmy.BlockChain.Config(), vm.Config{})

@ -25,6 +25,7 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/harmony/core/vm"
)
@ -50,6 +51,110 @@ func (c *action) push(ac *action) {
c.subCalls = append(c.subCalls, ac)
}
func (c *action) fromStorage(blockStorage *TraceBlockStorage, acStorage *ActionStorage) {
c.op = vm.OpCode(acStorage.readByte())
errByte := acStorage.readByte()
if errByte != 0 {
revertIndex := int(acStorage.readNumber().Int64())
c.revert = blockStorage.getData(revertIndex)
c.err = errors.New("Reverted")
}
if c.op == vm.CREATE || c.op == vm.CREATE2 {
fromIndex := int(acStorage.readNumber().Int64())
toIndex := int(acStorage.readNumber().Int64())
c.value = acStorage.readNumber()
inputIndex := int(acStorage.readNumber().Int64())
outputIndex := int(acStorage.readNumber().Int64())
c.gas = acStorage.readNumber().Uint64()
c.gasUsed = acStorage.readNumber().Uint64()
c.from = blockStorage.getAddress(fromIndex)
c.to = blockStorage.getAddress(toIndex)
c.input = blockStorage.getData(inputIndex)
c.output = blockStorage.getData(outputIndex)
}
if c.op == vm.CALL || c.op == vm.CALLCODE || c.op == vm.DELEGATECALL || c.op == vm.STATICCALL {
fromIndex := int(acStorage.readNumber().Int64())
toIndex := int(acStorage.readNumber().Int64())
c.value = acStorage.readNumber()
inputIndex := int(acStorage.readNumber().Int64())
outputIndex := int(acStorage.readNumber().Int64())
c.gas = acStorage.readNumber().Uint64()
c.gasUsed = acStorage.readNumber().Uint64()
c.from = blockStorage.getAddress(fromIndex)
c.to = blockStorage.getAddress(toIndex)
c.input = blockStorage.getData(inputIndex)
c.output = blockStorage.getData(outputIndex)
}
if c.op == vm.SELFDESTRUCT {
fromIndex := int(acStorage.readNumber().Int64())
toIndex := int(acStorage.readNumber().Int64())
c.from = blockStorage.getAddress(fromIndex)
c.to = blockStorage.getAddress(toIndex)
c.value = acStorage.readNumber()
}
}
func (c action) toStorage(blockStorage *TraceBlockStorage) *ActionStorage {
acStorage := &ActionStorage{
TraceData: make([]byte, 0, 1024),
}
acStorage.appendByte(byte(c.op))
var errByte byte
if c.err != nil {
errByte = 1
}
acStorage.appendByte(errByte)
if errByte != 0 {
revertIndex := blockStorage.indexData(c.revert)
acStorage.appendNumber(big.NewInt(int64(revertIndex)))
}
if c.op == vm.CREATE || c.op == vm.CREATE2 {
fromIndex := blockStorage.indexAddress(c.from)
toIndex := blockStorage.indexAddress(c.to)
inputIndex := blockStorage.indexData(c.input)
outputIndex := blockStorage.indexData(c.output)
acStorage.appendNumber(big.NewInt(int64(fromIndex)))
acStorage.appendNumber(big.NewInt(int64(toIndex)))
acStorage.appendNumber(c.value)
acStorage.appendNumber(big.NewInt(int64(inputIndex)))
acStorage.appendNumber(big.NewInt(int64(outputIndex)))
acStorage.appendNumber((&big.Int{}).SetUint64(c.gas))
acStorage.appendNumber((&big.Int{}).SetUint64(c.gasUsed))
return acStorage
}
if c.op == vm.CALL || c.op == vm.CALLCODE || c.op == vm.DELEGATECALL || c.op == vm.STATICCALL {
if c.value == nil {
c.value = big.NewInt(0)
}
fromIndex := blockStorage.indexAddress(c.from)
toIndex := blockStorage.indexAddress(c.to)
inputIndex := blockStorage.indexData(c.input)
outputIndex := blockStorage.indexData(c.output)
acStorage.appendNumber(big.NewInt(int64(fromIndex)))
acStorage.appendNumber(big.NewInt(int64(toIndex)))
acStorage.appendNumber(c.value)
acStorage.appendNumber(big.NewInt(int64(inputIndex)))
acStorage.appendNumber(big.NewInt(int64(outputIndex)))
acStorage.appendNumber((&big.Int{}).SetUint64(c.gas))
acStorage.appendNumber((&big.Int{}).SetUint64(c.gasUsed))
return acStorage
}
if c.op == vm.SELFDESTRUCT {
fromIndex := blockStorage.indexAddress(c.from)
toIndex := blockStorage.indexAddress(c.to)
acStorage.appendNumber(big.NewInt(int64(fromIndex)))
acStorage.appendNumber(big.NewInt(int64(toIndex)))
acStorage.appendNumber(c.value)
return acStorage
}
return nil
}
func (c action) toJsonStr() (string, *string, *string) {
callType := strings.ToLower(c.op.String())
if c.op == vm.CREATE || c.op == vm.CREATE2 {
@ -94,7 +199,7 @@ func (c action) toJsonStr() (string, *string, *string) {
return "unkonw", nil, nil
}
type ParityBlockTracer struct {
type ParityTxTracer struct {
blockNumber uint64
blockHash common.Hash
transactionPosition uint64
@ -103,43 +208,50 @@ type ParityBlockTracer struct {
calls []*action
action
}
type ParityBlockTracer struct {
Hash common.Hash
Number uint64
cur *ParityTxTracer
tracers []*ParityTxTracer
}
func (jst *ParityBlockTracer) push(ac *action) {
jst.calls = append(jst.calls, ac)
func (ptt *ParityTxTracer) push(ac *action) {
ptt.calls = append(ptt.calls, ac)
}
func (jst *ParityBlockTracer) pop() *action {
popIndex := len(jst.calls) - 1
ac := jst.calls[popIndex]
jst.calls = jst.calls[:popIndex]
func (ptt *ParityTxTracer) pop() *action {
popIndex := len(ptt.calls) - 1
ac := ptt.calls[popIndex]
ptt.calls = ptt.calls[:popIndex]
return ac
}
func (jst *ParityBlockTracer) last() *action {
return jst.calls[len(jst.calls)-1]
func (ptt *ParityTxTracer) last() *action {
return ptt.calls[len(ptt.calls)-1]
}
func (jst *ParityBlockTracer) len() int {
return len(jst.calls)
func (ptt *ParityTxTracer) len() int {
return len(ptt.calls)
}
// CaptureStart implements the ParityBlockTracer interface to initialize the tracing operation.
func (jst *ParityBlockTracer) CaptureStart(env *vm.EVM, from common.Address, to common.Address, create bool, input []byte, gas uint64, value *big.Int) error {
jst.op = vm.CALL // vritual call
jst.cur = &ParityTxTracer{}
jst.cur.op = vm.CALL // vritual call
if create {
jst.op = vm.CREATE // virtual create
jst.cur.op = vm.CREATE // virtual create
}
jst.from = from
jst.to = to
jst.input = input
jst.gas = gas
jst.value = (&big.Int{}).Set(value)
jst.blockHash = env.StateDB.BlockHash()
jst.transactionPosition = uint64(env.StateDB.TxIndex())
jst.transactionHash = env.StateDB.TxHash()
jst.blockNumber = env.BlockNumber.Uint64()
jst.descended = false
jst.push(&jst.action)
jst.cur.from = from
jst.cur.to = to
jst.cur.input = input
jst.cur.gas = gas
jst.cur.value = (&big.Int{}).Set(value)
jst.cur.blockHash = env.StateDB.BlockHash()
jst.cur.transactionPosition = uint64(env.StateDB.TxIndex())
jst.cur.transactionHash = env.StateDB.TxHashETH()
jst.cur.blockNumber = env.BlockNumber.Uint64()
jst.cur.descended = false
jst.cur.push(&jst.cur.action)
return nil
}
@ -168,7 +280,7 @@ func (jst *ParityBlockTracer) CaptureState(env *vm.EVM, pc uint64, op vm.OpCode,
case vm.CREATE, vm.CREATE2:
inOff := stackPeek(1).Int64()
inSize := stackPeek(2).Int64()
jst.push(&action{
jst.cur.push(&action{
op: op,
from: contract.Address(),
input: memoryCopy(inOff, inSize),
@ -176,10 +288,10 @@ func (jst *ParityBlockTracer) CaptureState(env *vm.EVM, pc uint64, op vm.OpCode,
gasCost: cost,
value: (&big.Int{}).Set(stackPeek(0)),
})
jst.descended = true
jst.cur.descended = true
return nil, retErr
case vm.SELFDESTRUCT:
ac := jst.last()
ac := jst.cur.last()
ac.push(&action{
op: op,
from: contract.Address(),
@ -214,28 +326,27 @@ func (jst *ParityBlockTracer) CaptureState(env *vm.EVM, pc uint64, op vm.OpCode,
if op != vm.DELEGATECALL && op != vm.STATICCALL {
callObj.value = (&big.Int{}).Set(stackPeek(2))
}
jst.push(callObj)
jst.descended = true
jst.cur.push(callObj)
jst.cur.descended = true
return nil, retErr
}
if jst.descended {
jst.descended = false
if depth >= jst.len() { // >= to >
jst.last().gas = gas
if jst.cur.descended {
jst.cur.descended = false
if depth >= jst.cur.len() { // >= to >
jst.cur.last().gas = gas
}
}
if op == vm.REVERT {
last := jst.last()
last := jst.cur.last()
last.err = errors.New("execution reverted")
revertOff := stackPeek(0).Int64()
revertLen := stackPeek(1).Int64()
last.revert = memoryCopy(revertOff, revertLen)
return nil, retErr
}
if depth == jst.len()-1 { // depth == len - 1
call := jst.pop()
if depth == jst.cur.len()-1 { // depth == len - 1
call := jst.cur.pop()
if call.op == vm.CREATE || call.op == vm.CREATE2 {
call.gasUsed = call.gasIn - call.gasCost - gas
@ -257,7 +368,7 @@ func (jst *ParityBlockTracer) CaptureState(env *vm.EVM, pc uint64, op vm.OpCode,
call.err = errors.New("internal failure")
}
}
jst.last().push(call)
jst.cur.last().push(call)
}
return nil, retErr
}
@ -265,10 +376,10 @@ func (jst *ParityBlockTracer) CaptureState(env *vm.EVM, pc uint64, op vm.OpCode,
// CaptureFault implements the ParityBlockTracer interface to trace an execution fault
// while running an opcode.
func (jst *ParityBlockTracer) CaptureFault(env *vm.EVM, pc uint64, op vm.OpCode, gas, cost uint64, memory *vm.Memory, stack *vm.Stack, contract *vm.Contract, depth int, err error) error {
if jst.last().err != nil {
if jst.cur.last().err != nil {
return nil
}
call := jst.pop()
call := jst.cur.pop()
call.err = err
// Consume all available gas and clean any leftovers
if call.gas != 0 {
@ -277,34 +388,63 @@ func (jst *ParityBlockTracer) CaptureFault(env *vm.EVM, pc uint64, op vm.OpCode,
}
// Flatten the failed call into its parent
if jst.len() > 0 {
jst.last().push(call)
if jst.cur.len() > 0 {
jst.cur.last().push(call)
return nil
}
jst.push(call)
jst.cur.push(call)
return nil
}
// CaptureEnd is called after the call finishes to finalize the tracing.
func (jst *ParityBlockTracer) CaptureEnd(output []byte, gasUsed uint64, t time.Duration, err error) error {
jst.output = output
jst.gasUsed = gasUsed
jst.cur.output = output
jst.cur.gasUsed = gasUsed
if err != nil {
jst.err = err
jst.cur.err = err
}
jst.tracers = append(jst.tracers, jst.cur)
return nil
}
// get TraceBlockStorage from tracer, then store it to db
func (jst *ParityBlockTracer) GetStorage() *TraceBlockStorage {
blockStorage := &TraceBlockStorage{
Hash: jst.Hash,
Number: jst.Number,
addressIndex: make(map[common.Address]int),
dataIndex: make(map[common.Hash]int),
}
txStorage := &TxStorage{
Storages: make([]*ActionStorage, 0, 1024),
}
var finalize func(ac *action, traceAddress []uint)
finalize = func(ac *action, traceAddress []uint) {
acStorage := ac.toStorage(blockStorage)
acStorage.Subtraces = uint(len(ac.subCalls))
acStorage.TraceAddress = make([]uint, len(traceAddress))
copy(acStorage.TraceAddress, traceAddress)
txStorage.Storages = append(txStorage.Storages, acStorage)
for i, subAc := range ac.subCalls {
finalize(subAc, append(traceAddress[:], uint(i)))
}
}
for _, curTx := range jst.tracers {
root := &curTx.action
txStorage.Hash = curTx.transactionHash
txStorage.Storages = txStorage.Storages[:0]
finalize(root, make([]uint, 0))
b, _ := rlp.EncodeToBytes(txStorage)
blockStorage.TraceStorages = append(blockStorage.TraceStorages, b)
}
return blockStorage
}
// GetResult calls the Javascript 'result' function and returns its value, or any accumulated error
func (jst *ParityBlockTracer) GetResult() ([]json.RawMessage, error) {
root := &jst.action
headPiece := fmt.Sprintf(
`"blockNumber":%d,"blockHash":"%s","transactionHash":"%s","transactionPosition":%d`,
jst.blockNumber, jst.blockHash.Hex(), jst.transactionHash.Hex(), jst.transactionPosition,
)
var results []json.RawMessage
var err error
var headPiece string
var finalize func(ac *action, traceAddress []int)
finalize = func(ac *action, traceAddress []int) {
typStr, acStr, outStr := ac.toJsonStr()
@ -333,6 +473,13 @@ func (jst *ParityBlockTracer) GetResult() ([]json.RawMessage, error) {
finalize(subAc, append(traceAddress[:], i))
}
}
finalize(root, make([]int, 0))
for _, curTx := range jst.tracers {
root := &curTx.action
headPiece = fmt.Sprintf(
`"blockNumber":%d,"blockHash":"%s","transactionHash":"%s","transactionPosition":%d`,
curTx.blockNumber, curTx.blockHash.Hex(), curTx.transactionHash.Hex(), curTx.transactionPosition,
)
finalize(root, make([]int, 0))
}
return results, err
}

@ -0,0 +1,206 @@
// Copyright 2017 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package tracers
import (
"encoding/json"
"errors"
"fmt"
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/harmony/crypto/hash"
)
type ActionStorage struct {
Subtraces uint
TraceAddress []uint
TraceData []byte
}
func (storage *ActionStorage) appendByte(byt byte) {
storage.TraceData = append(storage.TraceData, byt)
}
func (storage *ActionStorage) appendFixed(data []byte) {
storage.TraceData = append(storage.TraceData, data...)
}
func (storage *ActionStorage) appendNumber(num *big.Int) {
bytes, _ := rlp.EncodeToBytes(num)
storage.appendByte(uint8(len(bytes)))
storage.appendFixed(bytes)
}
func (storage *ActionStorage) readByte() byte {
val := storage.TraceData[0]
storage.TraceData = storage.TraceData[1:]
return val
}
func (storage *ActionStorage) readFixedData(size uint) []byte {
fixedData := storage.TraceData[:size]
storage.TraceData = storage.TraceData[size:]
return fixedData
}
func (storage *ActionStorage) readNumber() *big.Int {
size := storage.readByte()
bytes := storage.readFixedData(uint(size))
var num big.Int
rlp.DecodeBytes(bytes, &num)
return &num
}
type TxStorage struct {
Hash common.Hash
Storages []*ActionStorage
}
type TraceBlockStorage struct {
Hash common.Hash
Number uint64
AddressTable []common.Address // address table
DataKeyTable []common.Hash // data key table
dataValueTable [][]byte // data, store in db, avoid RLPEncode
TraceStorages [][]byte // trace data, length equal the number of transaction in a block
addressIndex map[common.Address]int // address index in AddressTable
dataIndex map[common.Hash]int // data index in DataKeyTable
}
// get data of index i
func (ts *TraceBlockStorage) getData(i int) []byte {
return ts.dataValueTable[i]
}
// get address of index i
func (ts *TraceBlockStorage) getAddress(i int) common.Address {
return ts.AddressTable[i]
}
// store data and assign an index to it. if data existed, just return it's index
func (ts *TraceBlockStorage) indexData(data []byte) int {
key := hash.Keccak256Hash(data)
if index, exist := ts.dataIndex[key]; exist {
return index
}
index := len(ts.DataKeyTable)
ts.DataKeyTable = append(ts.DataKeyTable, key)
ts.dataValueTable = append(ts.dataValueTable, data)
ts.dataIndex[key] = index
return index
}
// store address and assign an index to it. if address existed, just return it's index
func (ts *TraceBlockStorage) indexAddress(address common.Address) int {
if index, exist := ts.addressIndex[address]; exist {
return index
}
index := len(ts.addressIndex)
ts.AddressTable = append(ts.AddressTable, address)
ts.addressIndex[address] = index
return index
}
// use this key as
func (ts *TraceBlockStorage) KeyDB() []byte {
return ts.Hash[:]
}
// store TraceBlockStorage to db
func (ts *TraceBlockStorage) ToDB(write func([]byte, []byte)) {
for index, key := range ts.DataKeyTable {
write(key[:], ts.dataValueTable[index])
}
bytes, _ := rlp.EncodeToBytes(ts)
write(ts.KeyDB(), bytes)
}
// load TraceBlockStorage from db
func (ts *TraceBlockStorage) FromDB(read func([]byte) ([]byte, error)) error {
bytes, err := read(ts.KeyDB())
if err != nil {
return err
}
err = rlp.DecodeBytes(bytes, ts)
if err != nil {
return err
}
for _, key := range ts.DataKeyTable {
data, err := read(key[:])
if err != nil {
return err
}
ts.dataValueTable = append(ts.dataValueTable, data)
}
return nil
}
// return trace result of a tx for giving index
func (ts *TraceBlockStorage) TxJson(index int) ([]json.RawMessage, error) {
var results []json.RawMessage
var txStorage TxStorage
var err error
b := ts.TraceStorages[index]
err = rlp.DecodeBytes(b, &txStorage)
if err != nil {
return nil, err
}
headPiece := fmt.Sprintf(
`"blockNumber":%d,"blockHash":"%s","transactionHash":"%s","transactionPosition":%d`,
ts.Number, ts.Hash.Hex(), txStorage.Hash.Hex(), index,
)
for _, acStorage := range txStorage.Storages {
ac := &action{}
ac.fromStorage(ts, acStorage)
typStr, acStr, outStr := ac.toJsonStr()
if acStr == nil {
err = errors.New("tracer internal failure")
return nil, err
}
traceStr, _ := json.Marshal(acStorage.TraceAddress)
bodyPiece := fmt.Sprintf(
`,"subtraces":%d,"traceAddress":%s,"type":"%s","action":%s`,
acStorage.Subtraces, string(traceStr), typStr, *acStr,
)
var resultPiece string
if ac.err != nil {
resultPiece = fmt.Sprintf(`,"error":"Reverted","revert":"0x%x"`, ac.revert)
} else if outStr != nil {
resultPiece = fmt.Sprintf(`,"result":%s`, *outStr)
} else {
resultPiece = `,"result":null`
}
jstr := "{" + headPiece + bodyPiece + resultPiece + "}"
results = append(results, json.RawMessage(jstr))
}
return results, nil
}
// return trace result of a block
func (ts *TraceBlockStorage) ToJson() (json.RawMessage, error) {
var results []json.RawMessage
for i := range ts.TraceStorages {
tx, err := ts.TxJson(i)
if err != nil {
return nil, err
}
results = append(results, tx...)
}
return json.Marshal(results)
}

@ -1,18 +1,234 @@
package tracers
import (
"bytes"
"encoding/json"
"errors"
"math/big"
"strconv"
"strings"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/harmony/core/vm"
"github.com/harmony-one/harmony/internal/utils"
)
var TestJsonsMock = []byte(`{"14054302":[{"blockNumber":14054302,"blockHash":"0x04d7a0d62d3211151db0dadcaebcb1686c4a3df0e551a00c023c651546293975","transactionHash":"0xce49e42e0fbd37a0cfd08c2da3f1acc371ddbc02c428afa123a43663e57953d7","transactionPosition":0,"subtraces":0,"traceAddress":[0],"type":"suicide","action":{"refundAddress":"0x12e49d93588e0056bd25530c3b1e8aac68f4b70a","balance":"0x0","address":"0x7006c42d6fa41844baa53b0388f9542e634cf55a"},"result":null}],"14833359":[{"blockNumber":14833359,"blockHash":"0x6d6660f3d042a145c7f95c408f28cbf036a18eaf603161c2c00ca3f6041d8b52","transactionHash":"0x9fd0daef346c72d51f7482ddc9a466caf52fa6a116ed13ee0c003e57e632b7c0","transactionPosition":0,"subtraces":0,"traceAddress":[],"type":"create","action":{"from":"0x8520021f89450394244cd4abda4cfe2f1b0ef61c","gas":"0x1017d","init":"0x608060405234801561001057600080fd5b50610149806100206000396000f3fe6080604052600436106100295760003560e01c80630c2ad69c1461002e57806315d55b281461007a575b600080fd5b6100646004803603604081101561004457600080fd5b810190808035906020019092919080359060200190929190505050610091565b6040518082815260200191505060405180910390f35b34801561008657600080fd5b5061008f6100a5565b005b600081838161009c57fe5b04905092915050565b6040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260058152602001807f68656c6c6f00000000000000000000000000000000000000000000000000000081525060200191505060405180910390fdfea26469706673582212202f9958b958267c4ed653e54dc0161cfb9b772209cbe086f4a9ac3d967f22f09564736f6c634300060c0033","value":"0x0"},"result":{"address":"0xf29fcf3a375ce5dd1c58f0e8a584ab5d782cc12b","code":"0x6080604052600436106100295760003560e01c80630c2ad69c1461002e57806315d55b281461007a575b600080fd5b6100646004803603604081101561004457600080fd5b810190808035906020019092919080359060200190929190505050610091565b6040518082815260200191505060405180910390f35b34801561008657600080fd5b5061008f6100a5565b005b600081838161009c57fe5b04905092915050565b6040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260058152602001807f68656c6c6f00000000000000000000000000000000000000000000000000000081525060200191505060405180910390fdfea26469706673582212202f9958b958267c4ed653e54dc0161cfb9b772209cbe086f4a9ac3d967f22f09564736f6c634300060c0033","gasUsed":"0x1017d"}},{"blockNumber":14833359,"blockHash":"0x6d6660f3d042a145c7f95c408f28cbf036a18eaf603161c2c00ca3f6041d8b52","transactionHash":"0xc3b81fa2f6786ffd11a588b9d951a39adb46b6e29abad819b0cb09ee32ea7072","transactionPosition":1,"subtraces":2,"traceAddress":[],"type":"call","action":{"callType":"call","value":"0x0","to":"0x4596817192fbbf0142c576ed3e7cfc0e8f40bbbe","gas":"0x2b71c","from":"0x87946ddc76a4c0a75c8ca1f63dffd0612ae6458c","input":"0x1801fbe5aebcf6e3d785238603dd88bb43cbdfcfeb51c95b570113ee65d2f9271d3b59510000000dcdf493a5e1610e23c037bc4c4e04ab9a6d8fe9d0d462ecd8d45643ac"},"result":{"output":"0x0000000000000000000000000000000000000000000000000000000000000001","gasUsed":"0x13c58"}}]}`)
type JsonCallAction struct {
CallType string `json:"callType"`
Value string `json:"value"`
From common.Address `json:"from"`
To common.Address `json:"to"`
Gas string `json:"gas"`
Input string `json:"input"`
}
type JsonCreateAction struct {
From common.Address `json:"from"`
Value string `json:"value"`
Gas string `json:"gas"`
Init string `json:"init"`
}
type JsonSuicideAction struct {
RefundAddress common.Address `json:"refundAddress"`
Balance string `json:"balance"`
Address common.Address `json:"address"`
}
type JsonCallOutput struct {
Output string `json:"output"`
GasUsed string `json:"gasUsed"`
}
type JsonCreateOutput struct {
Address common.Address `json:"address"`
Code string `json:"code"`
GasUsed string `json:"gasUsed"`
}
type JsonTrace struct {
BlockNumber uint64 `json:"blockNumber"`
BlockHash common.Hash `json:"blockHash"`
TransactionHash common.Hash `json:"transactionHash"`
TransactionPosition uint `json:"transactionPosition"`
Subtraces uint `json:"subtraces"`
TraceAddress []uint `json:"traceAddress"`
Typ string `json:"type"`
Action json.RawMessage `json:"action"`
Result json.RawMessage `json:"result"`
Err string `json:"error"`
Revert string `json:"revert"`
}
// this function only used by test
func initFromJson(ts *TraceBlockStorage, bytes []byte) {
var actionObjs []JsonTrace
var txs []*TxStorage
var tx *TxStorage
json.Unmarshal(bytes, &actionObjs)
for _, obj := range actionObjs {
ac := action{}
if len(obj.Err) > 0 {
ac.err = errors.New(obj.Err)
ac.revert = utils.FromHex(obj.Revert)
}
if obj.Typ == "call" {
callAc := &JsonCallAction{}
err := json.Unmarshal(obj.Action, callAc)
if err != nil {
panic(err)
}
switch callAc.CallType {
case "staticcall":
ac.op = vm.STATICCALL
case "call":
ac.op = vm.CALL
case "callcode":
ac.op = vm.CALLCODE
case "delegatecall":
ac.op = vm.DELEGATECALL
}
ac.from = callAc.From
ac.to = callAc.To
ac.value, _ = new(big.Int).SetString(callAc.Value[2:], 16)
ac.gas, _ = strconv.ParseUint(callAc.Gas, 0, 64)
ac.input = utils.FromHex(callAc.Input)
if ac.err == nil {
callOutput := &JsonCallOutput{}
err = json.Unmarshal(obj.Result, callOutput)
if err != nil {
panic(err)
}
ac.output = utils.FromHex(callOutput.Output)
ac.gasUsed, err = strconv.ParseUint(callOutput.GasUsed, 0, 64)
if err != nil {
panic(err)
}
}
}
if obj.Typ == "create" {
ac.op = vm.CREATE
callAc := &JsonCreateAction{}
err := json.Unmarshal(obj.Action, callAc)
if err != nil {
panic(err)
}
callOutput := &JsonCreateOutput{}
err = json.Unmarshal(obj.Result, callOutput)
if err != nil {
panic(err)
}
ac.from = callAc.From
ac.value, _ = new(big.Int).SetString(callAc.Value[2:], 16)
ac.gas, _ = strconv.ParseUint(callAc.Gas, 0, 64)
ac.input = utils.FromHex(callAc.Init)
if ac.err == nil {
ac.to = callOutput.Address
ac.output = utils.FromHex(callOutput.Code)
ac.gasUsed, _ = strconv.ParseUint(callOutput.GasUsed, 0, 64)
}
}
if obj.Typ == "suicide" {
ac.op = vm.SELFDESTRUCT
callAc := &JsonSuicideAction{}
err := json.Unmarshal(obj.Action, callAc)
if err != nil {
panic(err)
}
ac.from = callAc.Address
ac.to = callAc.RefundAddress
ac.value, _ = new(big.Int).SetString(callAc.Balance[2:], 16)
}
ts.Hash = obj.BlockHash
ts.Number = obj.BlockNumber
if tx == nil || tx.Hash != obj.TransactionHash {
tx = &TxStorage{
Hash: obj.TransactionHash,
}
txs = append(txs, tx)
}
acStorage := ac.toStorage(ts)
acStorage.Subtraces = obj.Subtraces
acStorage.TraceAddress = obj.TraceAddress
tx.Storages = append(tx.Storages, acStorage)
}
for _, tx := range txs {
b, err := rlp.EncodeToBytes(tx)
if err != nil {
panic(err)
}
ts.TraceStorages = append(ts.TraceStorages, b)
}
}
func TestStorage(t *testing.T) {
testJsons := make(map[string]json.RawMessage)
var sizeDB []byte
var memDB [][2][]byte
//TestJsonsMock, _ := os.ReadFile("/tmp/out.json")
json.Unmarshal(TestJsonsMock, &testJsons)
for _, testJson := range testJsons {
block := &TraceBlockStorage{
addressIndex: make(map[common.Address]int),
dataIndex: make(map[common.Hash]int),
}
initFromJson(block, testJson)
jsonRaw, err := block.ToJson()
if err != nil {
t.Error(err)
}
if !bytes.Equal(jsonRaw, testJson) {
t.Fatal("restroe failed!")
}
block.ToDB(func(key, data []byte) {
for _, kv := range memDB {
if bytes.Equal(kv[0], key) {
return
}
}
sizeDB = append(sizeDB, key...)
newKey := sizeDB[len(sizeDB)-len(key):]
sizeDB = append(sizeDB, data...)
newData := sizeDB[len(sizeDB)-len(data):]
memDB = append(memDB, [2][]byte{newKey, newData})
})
newBlock := &TraceBlockStorage{
Hash: block.Hash,
addressIndex: make(map[common.Address]int),
dataIndex: make(map[common.Hash]int),
}
newBlock.FromDB(func(key []byte) ([]byte, error) {
for _, kv := range memDB {
k, v := kv[0], kv[1]
if bytes.Equal(k, key) {
return v, nil
}
}
t.Fatalf("key not exist: %x", key)
return nil, nil
})
jsonRaw, err = newBlock.ToJson()
if err != nil {
t.Error(err)
}
if !bytes.Equal(jsonRaw, testJson) {
t.Fatal("restroe failed!")
}
//os.WriteFile("/tmp/trace.raw", sizeDB, os.ModePerm)
}
}
var TestActions = []string{
`{"callType":"call","value":"0x9f2d9ea5d38f03446","to":"0xf012702a5f0e54015362cbca26a26fc90aa832a3","gas":"0x177d97","from":"0x5a42560d64136caef1a92d4c0829c21385f9f182","input":"0xfb3bdb41000000000000000000000000000000000000000000000001a8a909dfcef4000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000005a42560d64136caef1a92d4c0829c21385f9f1820000000000000000000000000000000000000000000000000000000061cecc5c0000000000000000000000000000000000000000000000000000000000000002000000000000000000000000cf664087a5bb0237a0bad6742852ec6c8d69a27a000000000000000000000000a4e24f10712cec820dd04e35d52f8087a0699d19"}`,
`{"callType":"staticcall","to":"0x3fe7910191e07503a94237303c8c0497549b0b20","gas":"0x17120d","from":"0xf012702a5f0e54015362cbca26a26fc90aa832a3","input":"0x0902f1ac"}`,

@ -68,7 +68,7 @@ func (rbt *RosettaBlockTracer) AddRosettaLog(op vm.OpCode, from, to *vm.RosettaL
}
func (rbt *RosettaBlockTracer) GetResult() ([]*RosettaLogItem, error) {
root := &rbt.action
root := &rbt.cur.action
var results = make([]*RosettaLogItem, 0)
var err error

@ -63,6 +63,7 @@ type GeneralConfig struct {
IsBeaconArchival bool
IsOffline bool
DataDir string
TraceEnable bool
EnablePruneBeaconChain bool
}

@ -94,6 +94,7 @@ type ConfigType struct {
WebHooks struct {
Hooks *webhooks.Hooks
}
TraceEnable bool
}
// RPCServerConfig is the config for rpc listen addresses

@ -932,6 +932,7 @@ func (node *Node) StartPubSub() error {
}
}()
node.TraceLoopForExplorer()
return nil
}

@ -2,6 +2,7 @@ package node
import (
"context"
"encoding/json"
"sync"
"github.com/ethereum/go-ethereum/common"
@ -11,6 +12,7 @@ import (
"github.com/harmony-one/harmony/api/service/explorer"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/signature"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/internal/utils"
"github.com/pkg/errors"
@ -119,9 +121,32 @@ func (node *Node) explorerMessageHandler(ctx context.Context, msg *msg_pb.Messag
return nil
}
func (node *Node) TraceLoopForExplorer() {
if !node.HarmonyConfig.General.TraceEnable {
return
}
ch := make(chan core.TraceEvent)
subscribe := node.Blockchain().SubscribeTraceEvent(ch)
go func() {
loop:
select {
case ev := <-ch:
if exp, err := node.getExplorerService(); err == nil {
storage := ev.Tracer.GetStorage()
exp.DumpTraceResult(storage)
}
goto loop
case <-subscribe.Err():
//subscribe.Unsubscribe()
break
}
}()
}
// AddNewBlockForExplorer add new block for explorer.
func (node *Node) AddNewBlockForExplorer(block *types.Block) {
utils.Logger().Info().Uint64("blockHeight", block.NumberU64()).Msg("[Explorer] Adding new block for explorer node")
if _, err := node.Blockchain().InsertChain([]*types.Block{block}, false); err == nil {
if block.IsLastBlockInEpoch() {
node.Consensus.UpdateConsensusInformation()
@ -245,6 +270,15 @@ func (node *Node) GetStakingTransactionsCount(address, txType string) (uint64, e
return count, nil
}
// GetStakingTransactionsCount returns the number of staking transactions hashes of address for input type.
func (node *Node) GetTraceResultByHash(hash common.Hash) (json.RawMessage, error) {
exp, err := node.getExplorerService()
if err != nil {
return nil, err
}
return exp.GetTraceResultByHash(hash)
}
func (node *Node) getExplorerService() (*explorer.Service, error) {
rawService := node.serviceManager.GetService(service.SupportExplorer)
if rawService == nil {

@ -33,6 +33,9 @@ func (s *PublicParityTracerService) Block(ctx context.Context, number rpc.BlockN
if block == nil {
return nil, nil
}
if results, err := s.hmy.NodeAPI.GetTraceResultByHash(block.Hash()); err == nil {
return results, nil
}
results, err := s.hmy.TraceBlock(ctx, block, &hmy.TraceConfig{Tracer: &parityTraceGO})
if err != nil {
return results, err

Loading…
Cancel
Save