Skip to content

Commit

Permalink
add verify execution result cmd
Browse files Browse the repository at this point in the history
  • Loading branch information
zhangchiqing committed Nov 22, 2024
1 parent 5be1172 commit e424cd7
Show file tree
Hide file tree
Showing 7 changed files with 421 additions and 89 deletions.
2 changes: 2 additions & 0 deletions cmd/util/cmd/root.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/onflow/flow-go/cmd/util/cmd/snapshot"
system_addresses "github.com/onflow/flow-go/cmd/util/cmd/system-addresses"
truncate_database "github.com/onflow/flow-go/cmd/util/cmd/truncate-database"
verify_execution_result "github.com/onflow/flow-go/cmd/util/cmd/verify_execution_result"
"github.com/onflow/flow-go/cmd/util/cmd/version"
"github.com/onflow/flow-go/module/profiler"
)
Expand Down Expand Up @@ -118,6 +119,7 @@ func addCommands() {
rootCmd.AddCommand(run_script.Cmd)
rootCmd.AddCommand(system_addresses.Cmd)
rootCmd.AddCommand(check_storage.Cmd)
rootCmd.AddCommand(verify_execution_result.Cmd)
}

func initConfig() {
Expand Down
98 changes: 98 additions & 0 deletions cmd/util/cmd/verify_execution_result/cmd.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
package verify

import (
"fmt"
"strconv"
"strings"

"github.com/rs/zerolog/log"
"github.com/spf13/cobra"

"github.com/onflow/flow-go/engine/verification/verifier"
"github.com/onflow/flow-go/model/flow"
)

var (
flagLastK uint64
flagDatadir string
flagChunkDataPackDir string
flagChain string
flagFromTo string
)

// # verify the last 100 sealed blocks
// ./util verify_execution_result --chain flow-testnet --datadir /var/flow/data/protocol --chunk_data_pack_dir /var/flow/data/chunk_data_pack --lastk 100
// # verify the blocks from height 2000 to 3000
// ./util verify_execution_result --chain flow-testnet --datadir /var/flow/data/protocol --chunk_data_pack_dir /var/flow/data/chunk_data_pack --from_to 2000-3000
var Cmd = &cobra.Command{
Use: "verify-execution-result",
Short: "verify block execution by verifying all chunks in the result",
Run: run,
}

func init() {
Cmd.Flags().StringVar(&flagChain, "chain", "", "Chain name")
_ = Cmd.MarkFlagRequired("chain")

Cmd.Flags().StringVar(&flagDatadir, "datadir", "/var/flow/data/protocol",
"directory that stores the protocol state")

Cmd.Flags().StringVar(&flagChunkDataPackDir, "chunk_data_pack_dir", "/var/flow/data/chunk_data_pack",
"directory that stores the protocol state")

Cmd.Flags().Uint64Var(&flagLastK, "lastk", 1,
"last k sealed blocks to verify")

Cmd.Flags().StringVar(&flagFromTo, "from_to", "",
"the height range to verify blocks, i.e, 1-1000, 1000-2000, 2000-3000, etc.")
}

func run(*cobra.Command, []string) {
_ = flow.ChainID(flagChain).Chain()

if flagFromTo != "" {
from, to, err := parseFromTo(flagFromTo)
if err != nil {
log.Fatal().Err(err).Msg("could not parse from_to")
}

log.Info().Msgf("verifying range from %d to %d", from, to)
err = verifier.VerifyRange(from, to, flow.Testnet, flagDatadir, flagChunkDataPackDir)
if err != nil {
log.Fatal().Err(err).Msg("could not verify last k height")
}
log.Info().Msgf("successfully verified range from %d to %d", from, to)

} else {
log.Info().Msgf("verifying last %d sealed blocks", flagLastK)
err := verifier.VerifyLastKHeight(flagLastK, flow.Testnet, flagDatadir, flagChunkDataPackDir)
if err != nil {
log.Fatal().Err(err).Msg("could not verify last k height")
}

log.Info().Msgf("successfully verified last %d sealed blocks", flagLastK)
}
}

func parseFromTo(fromTo string) (from, to uint64, err error) {
parts := strings.Split(fromTo, "-")
if len(parts) != 2 {
return 0, 0, fmt.Errorf("invalid format: expected 'from-to', got '%s'", fromTo)
}

from, err = strconv.ParseUint(strings.TrimSpace(parts[0]), 10, 64)
if err != nil {
return 0, 0, fmt.Errorf("invalid 'from' value: %w", err)
}

to, err = strconv.ParseUint(strings.TrimSpace(parts[1]), 10, 64)
if err != nil {
return 0, 0, fmt.Errorf("invalid 'to' value: %w", err)
}

if from > to {
return 0, 0, fmt.Errorf("'from' value (%d) must be less than or equal to 'to' value (%d)", from, to)
}

return from, to, nil
}
28 changes: 14 additions & 14 deletions engine/execution/computation/execution_verification_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/onflow/flow-go/engine/execution/testutil"
"github.com/onflow/flow-go/engine/execution/utils"
"github.com/onflow/flow-go/engine/testutil/mocklocal"
"github.com/onflow/flow-go/engine/verification/fetcher"
"github.com/onflow/flow-go/fvm"
"github.com/onflow/flow-go/fvm/blueprints"
"github.com/onflow/flow-go/fvm/environment"
Expand All @@ -36,6 +35,7 @@ import (
"github.com/onflow/flow-go/ledger/complete/wal/fixtures"
"github.com/onflow/flow-go/model/flow"
"github.com/onflow/flow-go/model/verification"
"github.com/onflow/flow-go/model/verification/convert"
"github.com/onflow/flow-go/module/chunks"
"github.com/onflow/flow-go/module/executiondatasync/execution_data"
exedataprovider "github.com/onflow/flow-go/module/executiondatasync/provider"
Expand Down Expand Up @@ -69,7 +69,7 @@ func Test_ExecutionMatchesVerification(t *testing.T) {
`access(all) contract Foo {
access(all) event FooEvent(x: Int, y: Int)
access(all) fun emitEvent() {
access(all) fun emitEvent() {
emit FooEvent(x: 2, y: 1)
}
}`), "Foo")
Expand Down Expand Up @@ -113,7 +113,7 @@ func Test_ExecutionMatchesVerification(t *testing.T) {
`access(all) contract Foo {
access(all) event FooEvent(x: Int, y: Int)
access(all) fun emitEvent() {
access(all) fun emitEvent() {
emit FooEvent(x: 2, y: 1)
}
}`), "Foo")
Expand Down Expand Up @@ -585,34 +585,34 @@ func TestTransactionFeeDeduction(t *testing.T) {
//
// The withdraw amount and the account from getAccount
// would be the parameters to the transaction
import FungibleToken from 0x%s
import FlowToken from 0x%s
transaction(amount: UFix64, to: Address) {
// The Vault resource that holds the tokens that are being transferred
let sentVault: @{FungibleToken.Vault}
prepare(signer: auth(BorrowValue) &Account) {
// Get a reference to the signer's stored vault
let vaultRef = signer.storage.borrow<auth(FungibleToken.Withdraw) &FlowToken.Vault>(from: /storage/flowTokenVault)
?? panic("Could not borrow reference to the owner's Vault!")
// Withdraw tokens from the signer's stored vault
self.sentVault <- vaultRef.withdraw(amount: amount)
}
execute {
// Get the recipient's public account object
let recipient = getAccount(to)
// Get a reference to the recipient's Receiver
let receiverRef = recipient.capabilities.borrow<&{FungibleToken.Receiver}>(/public/flowTokenReceiver)
?? panic("Could not borrow receiver reference to the recipient's Vault")
// Deposit the withdrawn tokens in the recipient's receiver
receiverRef.deposit(from: <-self.sentVault)
}
Expand Down Expand Up @@ -840,7 +840,7 @@ func executeBlockAndVerifyWithParameters(t *testing.T,

for i, chunk := range er.Chunks {
isSystemChunk := i == er.Chunks.Len()-1
offsetForChunk, err := fetcher.TransactionOffsetForChunk(er.Chunks, chunk.Index)
offsetForChunk, err := convert.TransactionOffsetForChunk(er.Chunks, chunk.Index)
require.NoError(t, err)

vcds[i] = &verification.VerifiableChunkData{
Expand Down
74 changes: 10 additions & 64 deletions engine/verification/fetcher/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"github.com/onflow/flow-go/model/flow"
"github.com/onflow/flow-go/model/flow/filter"
"github.com/onflow/flow-go/model/verification"
"github.com/onflow/flow-go/model/verification/convert"
"github.com/onflow/flow-go/module"
"github.com/onflow/flow-go/module/mempool"
"github.com/onflow/flow-go/module/trace"
Expand Down Expand Up @@ -259,7 +260,7 @@ func (e *Engine) HandleChunkDataPack(originID flow.Identifier, response *verific
Uint64("block_height", status.BlockHeight).
Hex("result_id", logging.ID(resultID)).
Uint64("chunk_index", status.ChunkIndex).
Bool("system_chunk", IsSystemChunk(status.ChunkIndex, status.ExecutionResult)).
Bool("system_chunk", convert.IsSystemChunk(status.ChunkIndex, status.ExecutionResult)).
Logger()

span, ctx := e.tracer.StartBlockSpan(context.Background(), status.ExecutionResult.BlockID, trace.VERFetcherHandleChunkDataPack)
Expand Down Expand Up @@ -413,7 +414,7 @@ func (e Engine) validateCollectionID(
result *flow.ExecutionResult,
chunk *flow.Chunk) error {

if IsSystemChunk(chunk.Index, result) {
if convert.IsSystemChunk(chunk.Index, result) {
return e.validateSystemChunkCollection(chunkDataPack)
}

Expand Down Expand Up @@ -550,29 +551,13 @@ func (e *Engine) makeVerifiableChunkData(chunk *flow.Chunk,
chunkDataPack *flow.ChunkDataPack,
) (*verification.VerifiableChunkData, error) {

// system chunk is the last chunk
isSystemChunk := IsSystemChunk(chunk.Index, result)

endState, err := EndStateCommitment(result, chunk.Index, isSystemChunk)
if err != nil {
return nil, fmt.Errorf("could not compute end state of chunk: %w", err)
}

transactionOffset, err := TransactionOffsetForChunk(result.Chunks, chunk.Index)
if err != nil {
return nil, fmt.Errorf("cannot compute transaction offset for chunk: %w", err)
}

return &verification.VerifiableChunkData{
IsSystemChunk: isSystemChunk,
Chunk: chunk,
Header: header,
Snapshot: snapshot,
Result: result,
ChunkDataPack: chunkDataPack,
EndState: endState,
TransactionOffset: transactionOffset,
}, nil
return convert.FromChunkDataPack(
chunk,
chunkDataPack,
header,
snapshot,
result,
)
}

// requestChunkDataPack creates and dispatches a chunk data pack request to the requester engine.
Expand Down Expand Up @@ -661,42 +646,3 @@ func executorsOf(receipts []*flow.ExecutionReceipt, resultID flow.Identifier) (f

return agrees, disagrees
}

// EndStateCommitment computes the end state of the given chunk.
func EndStateCommitment(result *flow.ExecutionResult, chunkIndex uint64, systemChunk bool) (flow.StateCommitment, error) {
var endState flow.StateCommitment
if systemChunk {
var err error
// last chunk in a result is the system chunk and takes final state commitment
endState, err = result.FinalStateCommitment()
if err != nil {
return flow.DummyStateCommitment, fmt.Errorf("can not read final state commitment, likely a bug:%w", err)
}
} else {
// any chunk except last takes the subsequent chunk's start state
endState = result.Chunks[chunkIndex+1].StartState
}

return endState, nil
}

// TransactionOffsetForChunk calculates transaction offset for a given chunk which is the index of the first
// transaction of this chunk within the whole block
func TransactionOffsetForChunk(chunks flow.ChunkList, chunkIndex uint64) (uint32, error) {
if int(chunkIndex) > len(chunks)-1 {
return 0, fmt.Errorf("chunk list out of bounds, len %d asked for chunk %d", len(chunks), chunkIndex)
}
var offset uint32 = 0
for i := 0; i < int(chunkIndex); i++ {
offset += uint32(chunks[i].NumberOfTransactions)
}
return offset, nil
}

// IsSystemChunk returns true if `chunkIndex` points to a system chunk in `result`.
// Otherwise, it returns false.
// In the current version, a chunk is a system chunk if it is the last chunk of the
// execution result.
func IsSystemChunk(chunkIndex uint64, result *flow.ExecutionResult) bool {
return chunkIndex == uint64(len(result.Chunks)-1)
}
23 changes: 12 additions & 11 deletions engine/verification/fetcher/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/onflow/flow-go/model/chunks"
"github.com/onflow/flow-go/model/flow"
"github.com/onflow/flow-go/model/verification"
"github.com/onflow/flow-go/model/verification/convert"
mempool "github.com/onflow/flow-go/module/mempool/mock"
module "github.com/onflow/flow-go/module/mock"
"github.com/onflow/flow-go/module/trace"
Expand Down Expand Up @@ -757,10 +758,10 @@ func mockVerifierEngine(t *testing.T,
require.Equal(t, expected.Result.ID(), vc.Result.ID())
require.Equal(t, expected.Header.ID(), vc.Header.ID())

isSystemChunk := fetcher.IsSystemChunk(vc.Chunk.Index, vc.Result)
isSystemChunk := convert.IsSystemChunk(vc.Chunk.Index, vc.Result)
require.Equal(t, isSystemChunk, vc.IsSystemChunk)

endState, err := fetcher.EndStateCommitment(vc.Result, vc.Chunk.Index, isSystemChunk)
endState, err := convert.EndStateCommitment(vc.Result, vc.Chunk.Index, isSystemChunk)
require.NoError(t, err)

require.Equal(t, endState, vc.EndState)
Expand Down Expand Up @@ -872,7 +873,7 @@ func chunkDataPackResponseFixture(t *testing.T,
collection *flow.Collection,
result *flow.ExecutionResult) *verification.ChunkDataPackResponse {

require.Equal(t, collection != nil, !fetcher.IsSystemChunk(chunk.Index, result), "only non-system chunks must have a collection")
require.Equal(t, collection != nil, !convert.IsSystemChunk(chunk.Index, result), "only non-system chunks must have a collection")

return &verification.ChunkDataPackResponse{
Locator: chunks.Locator{
Expand Down Expand Up @@ -917,7 +918,7 @@ func verifiableChunkFixture(t *testing.T,
result *flow.ExecutionResult,
chunkDataPack *flow.ChunkDataPack) *verification.VerifiableChunkData {

offsetForChunk, err := fetcher.TransactionOffsetForChunk(result.Chunks, chunk.Index)
offsetForChunk, err := convert.TransactionOffsetForChunk(result.Chunks, chunk.Index)
require.NoError(t, err)

// TODO: add end state
Expand Down Expand Up @@ -1000,7 +1001,7 @@ func completeChunkStatusListFixture(t *testing.T, chunkCount int, statusCount in
locators := unittest.ChunkStatusListToChunkLocatorFixture(statuses)

for _, status := range statuses {
if fetcher.IsSystemChunk(status.ChunkIndex, result) {
if convert.IsSystemChunk(status.ChunkIndex, result) {
// system-chunk should have a nil collection
continue
}
Expand All @@ -1012,7 +1013,7 @@ func completeChunkStatusListFixture(t *testing.T, chunkCount int, statusCount in

func TestTransactionOffsetForChunk(t *testing.T) {
t.Run("first chunk index always returns zero offset", func(t *testing.T) {
offsetForChunk, err := fetcher.TransactionOffsetForChunk([]*flow.Chunk{nil}, 0)
offsetForChunk, err := convert.TransactionOffsetForChunk([]*flow.Chunk{nil}, 0)
require.NoError(t, err)
assert.Equal(t, uint32(0), offsetForChunk)
})
Expand Down Expand Up @@ -1042,19 +1043,19 @@ func TestTransactionOffsetForChunk(t *testing.T) {
},
}

offsetForChunk, err := fetcher.TransactionOffsetForChunk(chunksList, 0)
offsetForChunk, err := convert.TransactionOffsetForChunk(chunksList, 0)
require.NoError(t, err)
assert.Equal(t, uint32(0), offsetForChunk)

offsetForChunk, err = fetcher.TransactionOffsetForChunk(chunksList, 1)
offsetForChunk, err = convert.TransactionOffsetForChunk(chunksList, 1)
require.NoError(t, err)
assert.Equal(t, uint32(1), offsetForChunk)

offsetForChunk, err = fetcher.TransactionOffsetForChunk(chunksList, 2)
offsetForChunk, err = convert.TransactionOffsetForChunk(chunksList, 2)
require.NoError(t, err)
assert.Equal(t, uint32(3), offsetForChunk)

offsetForChunk, err = fetcher.TransactionOffsetForChunk(chunksList, 3)
offsetForChunk, err = convert.TransactionOffsetForChunk(chunksList, 3)
require.NoError(t, err)
assert.Equal(t, uint32(6), offsetForChunk)
})
Expand All @@ -1063,7 +1064,7 @@ func TestTransactionOffsetForChunk(t *testing.T) {

chunksList := make([]*flow.Chunk, 2)

_, err := fetcher.TransactionOffsetForChunk(chunksList, 2)
_, err := convert.TransactionOffsetForChunk(chunksList, 2)
require.Error(t, err)
})
}
Loading

0 comments on commit e424cd7

Please sign in to comment.