diff --git a/Makefile b/Makefile index f60c96dc490..cc56ad7bc09 100644 --- a/Makefile +++ b/Makefile @@ -19,7 +19,7 @@ ifeq (${IMAGE_TAG},) IMAGE_TAG := ${SHORT_COMMIT} endif -IMAGE_TAG_NO_NETGO := $(IMAGE_TAG)-without-netgo +IMAGE_TAG_NO_NETGO := $(IMAGE_TAG)+without-netgo # Name of the cover profile COVER_PROFILE := coverage.txt @@ -207,11 +207,14 @@ generate-mocks: install-mock-generators mockery --name '.*' --dir=./cmd/util/ledger/reporters --case=underscore --output="./cmd/util/ledger/reporters/mock" --outpkg="mock" mockery --name 'Storage' --dir=module/executiondatasync/tracker --case=underscore --output="module/executiondatasync/tracker/mock" --outpkg="mocktracker" mockery --name 'ScriptExecutor' --dir=module/execution --case=underscore --output="module/execution/mock" --outpkg="mock" + mockery --name 'StorageSnapshot' --dir=fvm/storage/snapshot --case=underscore --output="fvm/storage/snapshot/mock" --outpkg="mock" #temporarily make insecure/ a non-module to allow mockery to create mocks mv insecure/go.mod insecure/go2.mod + if [ -f go.work ]; then mv go.work go2.work; fi mockery --name '.*' --dir=insecure/ --case=underscore --output="./insecure/mock" --outpkg="mockinsecure" mv insecure/go2.mod insecure/go.mod + if [ -f go2.work ]; then mv go2.work go.work; fi # this ensures there is no unused dependency being added by accident .PHONY: tidy diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 07398c02ba6..6d48a429f57 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -182,7 +182,8 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { MaxFailures: 5, MaxRequests: 1, }, - ScriptExecutionMode: backend.ScriptExecutionModeExecutionNodesOnly.String(), // default to ENs only for now + ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly.String(), // default to ENs only for now + EventQueryMode: backend.IndexQueryModeExecutionNodesOnly.String(), // default to ENs only for now }, RestConfig: rest.Config{ ListenAddress: "", @@ -273,6 +274,7 @@ type FlowAccessNodeBuilder struct { ExecutionIndexerCore *indexer.IndexerCore ScriptExecutor *backend.ScriptExecutor RegistersAsyncStore *execution.RegistersAsyncStore + IndexerDependencies *cmd.DependencyList // The sync engine participants provider is the libp2p peer store for the access node // which is not available until after the network has started. @@ -488,8 +490,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess // setup dependency chain to ensure indexer starts after the requester requesterDependable := module.NewProxiedReadyDoneAware() - indexerDependencies := cmd.NewDependencyList() - indexerDependencies.Add(requesterDependable) + builder.IndexerDependencies.Add(requesterDependable) builder. AdminCommand("read-execution-data", func(config *cmd.NodeConfig) commands.AdminCommand { @@ -669,10 +670,6 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess indexedBlockHeight = bstorage.NewConsumerProgress(builder.DB, module.ConsumeProgressExecutionDataIndexerBlockHeight) return nil }). - Module("events storage", func(node *cmd.NodeConfig) error { - builder.Storage.Events = bstorage.NewEvents(node.Metrics.Cache, node.DB) - return nil - }). Module("transaction results storage", func(node *cmd.NodeConfig) error { builder.Storage.LightTransactionResults = bstorage.NewLightTransactionResults(node.Metrics.Cache, node.DB, bstorage.DefaultCacheSize) return nil @@ -716,14 +713,20 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess checkpointHeight := builder.SealedRootBlock.Header.Height - buutstrap, err := pStorage.NewRegisterBootstrap(pdb, checkpointFile, checkpointHeight, builder.Logger) + if builder.SealedRootBlock.ID() != builder.RootSeal.BlockID { + return nil, fmt.Errorf("mismatching sealed root block and root seal: %v != %v", + builder.SealedRootBlock.ID(), builder.RootSeal.BlockID) + } + + rootHash := ledger.RootHash(builder.RootSeal.FinalState) + bootstrap, err := pStorage.NewRegisterBootstrap(pdb, checkpointFile, checkpointHeight, rootHash, builder.Logger) if err != nil { - return nil, fmt.Errorf("could not create registers bootstrapper: %w", err) + return nil, fmt.Errorf("could not create registers bootstrap: %w", err) } // TODO: find a way to hook a context up to this to allow a graceful shutdown workerCount := 10 - err = buutstrap.IndexCheckpointFile(context.Background(), workerCount) + err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) if err != nil { return nil, fmt.Errorf("could not load checkpoint file: %w", err) } @@ -744,6 +747,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.Storage.Headers, builder.Storage.Events, builder.Storage.LightTransactionResults, + builder.IngestEng.OnCollection, ) if err != nil { return nil, err @@ -790,7 +794,7 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess builder.ScriptExecutor.InitReporter(builder.ExecutionIndexer, scripts) return builder.ExecutionIndexer, nil - }, indexerDependencies) + }, builder.IndexerDependencies) } if builder.stateStreamConf.ListenAddr != "" { @@ -813,11 +817,22 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess } broadcaster := engine.NewBroadcaster() + eventQueryMode, err := backend.ParseIndexQueryMode(builder.rpcConf.BackendConfig.EventQueryMode) + if err != nil { + return nil, fmt.Errorf("could not parse event query mode: %w", err) + } + + // use the events index for events if enabled and the node is configured to use it for + // regular event queries + useIndex := builder.executionDataIndexingEnabled && + eventQueryMode != backend.IndexQueryModeExecutionNodesOnly + builder.stateStreamBackend, err = statestreambackend.New( node.Logger, builder.stateStreamConf, node.State, node.Storage.Headers, + node.Storage.Events, node.Storage.Seals, node.Storage.Results, builder.ExecutionDataStore, @@ -825,7 +840,9 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess broadcaster, builder.executionDataConfig.InitialBlockHeight, highestAvailableHeight, - builder.RegistersAsyncStore) + builder.RegistersAsyncStore, + useIndex, + ) if err != nil { return nil, fmt.Errorf("could not create state stream backend: %w", err) } @@ -861,6 +878,7 @@ func FlowAccessNode(nodeBuilder *cmd.FlowNodeBuilder) *FlowAccessNodeBuilder { AccessNodeConfig: DefaultAccessNodeConfig(), FlowNodeBuilder: nodeBuilder, FollowerDistributor: dist, + IndexerDependencies: cmd.NewDependencyList(), } } @@ -1063,6 +1081,11 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { flags.StringVar(&builder.registersDBPath, "execution-state-dir", defaultConfig.registersDBPath, "directory to use for execution-state database") flags.StringVar(&builder.checkpointFile, "execution-state-checkpoint", defaultConfig.checkpointFile, "execution-state checkpoint file") + flags.StringVar(&builder.rpcConf.BackendConfig.EventQueryMode, + "event-query-mode", + defaultConfig.rpcConf.BackendConfig.EventQueryMode, + "mode to use when querying events. one of [local-only, execution-nodes-only(default), failover]") + // Script Execution flags.StringVar(&builder.rpcConf.BackendConfig.ScriptExecutionMode, "script-execution-mode", @@ -1259,6 +1282,9 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.BuildExecutionSyncComponents() } + ingestionDependable := module.NewProxiedReadyDoneAware() + builder.IndexerDependencies.Add(ingestionDependable) + builder. BuildConsensusFollower(). Module("collection node client", func(node *cmd.NodeConfig) error { @@ -1402,6 +1428,10 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { builder.RegistersAsyncStore = execution.NewRegistersAsyncStore() return nil }). + Module("events storage", func(node *cmd.NodeConfig) error { + builder.Storage.Events = bstorage.NewEvents(node.Metrics.Cache, node.DB) + return nil + }). Component("RPC engine", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { config := builder.rpcConf backendConfig := config.BackendConfig @@ -1434,17 +1464,26 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { ), } - scriptExecMode, err := backend.ParseScriptExecutionMode(config.BackendConfig.ScriptExecutionMode) + scriptExecMode, err := backend.ParseIndexQueryMode(config.BackendConfig.ScriptExecutionMode) if err != nil { return nil, fmt.Errorf("could not parse script execution mode: %w", err) } + eventQueryMode, err := backend.ParseIndexQueryMode(config.BackendConfig.EventQueryMode) + if err != nil { + return nil, fmt.Errorf("could not parse script execution mode: %w", err) + } + if eventQueryMode == backend.IndexQueryModeCompare { + return nil, fmt.Errorf("event query mode 'compare' is not supported") + } + nodeBackend, err := backend.New(backend.Params{ State: node.State, CollectionRPC: builder.CollectionRPC, HistoricalAccessNodes: builder.HistoricalAccessRPCs, Blocks: node.Storage.Blocks, Headers: node.Storage.Headers, + Events: node.Storage.Events, Collections: node.Storage.Collections, Transactions: node.Storage.Transactions, ExecutionReceipts: node.Storage.Receipts, @@ -1463,6 +1502,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { TxErrorMessagesCacheSize: builder.TxErrorMessagesCacheSize, ScriptExecutor: builder.ScriptExecutor, ScriptExecutionMode: scriptExecMode, + EventQueryMode: eventQueryMode, }) if err != nil { return nil, fmt.Errorf("could not initialize backend: %w", err) @@ -1535,6 +1575,7 @@ func (builder *FlowAccessNodeBuilder) Build() (cmd.Node, error) { if err != nil { return nil, err } + ingestionDependable.Init(builder.IngestEng) builder.RequestEng.WithHandle(builder.IngestEng.OnCollection) builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.IngestEng.OnFinalizedBlock) diff --git a/cmd/bootstrap/cmd/finalize.go b/cmd/bootstrap/cmd/finalize.go index 9cfee15c9c2..35ed1e23beb 100644 --- a/cmd/bootstrap/cmd/finalize.go +++ b/cmd/bootstrap/cmd/finalize.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/model/dkg" "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/badger" @@ -156,7 +155,7 @@ func finalize(cmd *cobra.Command, args []string) { log.Info().Msg("") // create flow.IdentityList representation of participant set - participants := model.ToIdentityList(stakingNodes).Sort(order.Canonical) + participants := model.ToIdentityList(stakingNodes).Sort(flow.Canonical) log.Info().Msg("reading root block data") block := readRootBlock() @@ -492,7 +491,7 @@ func mergeNodeInfos(internalNodes, partnerNodes []model.NodeInfo) []model.NodeIn } // sort nodes using the canonical ordering - nodes = model.Sort(nodes, order.Canonical) + nodes = model.Sort(nodes, flow.Canonical) return nodes } diff --git a/cmd/bootstrap/cmd/keys.go b/cmd/bootstrap/cmd/keys.go index 9624ade3a1a..91885b454bb 100644 --- a/cmd/bootstrap/cmd/keys.go +++ b/cmd/bootstrap/cmd/keys.go @@ -6,7 +6,6 @@ import ( "github.com/onflow/flow-go/crypto/hash" "github.com/onflow/flow-go/cmd/bootstrap/utils" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/crypto" @@ -49,7 +48,7 @@ func genNetworkAndStakingKeys() []model.NodeInfo { internalNodes = append(internalNodes, nodeInfo) } - return model.Sort(internalNodes, order.Canonical) + return model.Sort(internalNodes, flow.Canonical) } func assembleNodeInfo(nodeConfig model.NodeConfig, networkKey, stakingKey crypto.PrivateKey) model.NodeInfo { diff --git a/cmd/bootstrap/cmd/seal.go b/cmd/bootstrap/cmd/seal.go index 1a34c394e13..05f1ab293b3 100644 --- a/cmd/bootstrap/cmd/seal.go +++ b/cmd/bootstrap/cmd/seal.go @@ -6,7 +6,6 @@ import ( "github.com/onflow/flow-go/cmd/bootstrap/run" "github.com/onflow/flow-go/model/dkg" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/signature" ) @@ -39,7 +38,7 @@ func constructRootResultAndSeal( DKGPhase1FinalView: firstView + flagNumViewsInStakingAuction + flagNumViewsInDKGPhase - 1, DKGPhase2FinalView: firstView + flagNumViewsInStakingAuction + flagNumViewsInDKGPhase*2 - 1, DKGPhase3FinalView: firstView + flagNumViewsInStakingAuction + flagNumViewsInDKGPhase*3 - 1, - Participants: participants.Sort(order.Canonical), + Participants: participants.Sort(flow.Canonical), Assignments: assignments, RandomSource: GenerateRandomSeed(flow.EpochSetupRandomSourceLength), } diff --git a/cmd/bootstrap/run/cluster_qc.go b/cmd/bootstrap/run/cluster_qc.go index fa91e5cc4f8..7d3e41ed8c8 100644 --- a/cmd/bootstrap/run/cluster_qc.go +++ b/cmd/bootstrap/run/cluster_qc.go @@ -14,7 +14,6 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/cluster" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/local" ) @@ -29,7 +28,7 @@ func GenerateClusterRootQC(signers []bootstrap.NodeInfo, allCommitteeMembers flo } // STEP 2: create VoteProcessor - ordered := allCommitteeMembers.Sort(order.Canonical) + ordered := allCommitteeMembers.Sort(flow.Canonical) committee, err := committees.NewStaticCommittee(ordered, flow.Identifier{}, nil, nil) if err != nil { return nil, err diff --git a/cmd/bootstrap/run/qc_test.go b/cmd/bootstrap/run/qc_test.go index 4f925a5e793..d6cb294aa5b 100644 --- a/cmd/bootstrap/run/qc_test.go +++ b/cmd/bootstrap/run/qc_test.go @@ -9,7 +9,6 @@ import ( "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/utils/unittest" ) @@ -45,7 +44,7 @@ func TestGenerateRootQCWithSomeInvalidVotes(t *testing.T) { } func createSignerData(t *testing.T, n int) *ParticipantData { - identities := unittest.IdentityListFixture(n).Sort(order.Canonical) + identities := unittest.IdentityListFixture(n).Sort(flow.Canonical) networkingKeys := unittest.NetworkingKeys(n) stakingKeys := unittest.StakingKeys(n) diff --git a/cmd/consensus/main.go b/cmd/consensus/main.go index d54e460f4d1..401272ec338 100644 --- a/cmd/consensus/main.go +++ b/cmd/consensus/main.go @@ -89,7 +89,6 @@ func main() { requiredApprovalsForSealVerification uint requiredApprovalsForSealConstruction uint emergencySealing bool - dkgControllerConfig dkgmodule.ControllerConfig dkgMessagingEngineConfig = dkgeng.DefaultMessagingEngineConfig() cruiseCtlConfig = cruisectl.DefaultConfig() cruiseCtlTargetTransitionTimeFlag = cruiseCtlConfig.TargetTransition.String() @@ -161,9 +160,6 @@ func main() { flags.BoolVar(&emergencySealing, "emergency-sealing-active", flow.DefaultEmergencySealingActive, "(de)activation of emergency sealing") flags.BoolVar(&insecureAccessAPI, "insecure-access-api", false, "required if insecure GRPC connection should be used") flags.StringSliceVar(&accessNodeIDS, "access-node-ids", []string{}, fmt.Sprintf("array of access node IDs sorted in priority order where the first ID in this array will get the first connection attempt and each subsequent ID after serves as a fallback. Minimum length %d. Use '*' for all IDs in protocol state.", common.DefaultAccessNodeIDSMinimum)) - flags.DurationVar(&dkgControllerConfig.BaseStartDelay, "dkg-controller-base-start-delay", dkgmodule.DefaultBaseStartDelay, "used to define the range for jitter prior to DKG start (eg. 500µs) - the base value is scaled quadratically with the # of DKG participants") - flags.DurationVar(&dkgControllerConfig.BaseHandleFirstBroadcastDelay, "dkg-controller-base-handle-first-broadcast-delay", dkgmodule.DefaultBaseHandleFirstBroadcastDelay, "used to define the range for jitter prior to DKG handling the first broadcast messages (eg. 50ms) - the base value is scaled quadratically with the # of DKG participants") - flags.DurationVar(&dkgControllerConfig.HandleSubsequentBroadcastDelay, "dkg-controller-handle-subsequent-broadcast-delay", dkgmodule.DefaultHandleSubsequentBroadcastDelay, "used to define the constant delay introduced prior to DKG handling subsequent broadcast messages (eg. 2s)") flags.DurationVar(&dkgMessagingEngineConfig.RetryBaseWait, "dkg-messaging-engine-retry-base-wait", dkgMessagingEngineConfig.RetryBaseWait, "the inter-attempt wait time for the first attempt (base of exponential retry)") flags.Uint64Var(&dkgMessagingEngineConfig.RetryMax, "dkg-messaging-engine-retry-max", dkgMessagingEngineConfig.RetryMax, "the maximum number of retry attempts for an outbound DKG message") flags.Uint64Var(&dkgMessagingEngineConfig.RetryJitterPercent, "dkg-messaging-engine-retry-jitter-percent", dkgMessagingEngineConfig.RetryJitterPercent, "the percentage of jitter to apply to each inter-attempt wait time") @@ -915,7 +911,6 @@ func main() { node.Me, dkgContractClients, dkgBrokerTunnel, - dkgControllerConfig, ), viewsObserver, ) diff --git a/cmd/execution_builder.go b/cmd/execution_builder.go index dd8953af4c7..847a38d45e1 100644 --- a/cmd/execution_builder.go +++ b/cmd/execution_builder.go @@ -801,14 +801,24 @@ func (exeNode *ExecutionNode) LoadRegisterStore( if !bootstrapped { checkpointFile := path.Join(exeNode.exeConf.triedir, modelbootstrap.FilenameWALRootCheckpoint) - root, err := exeNode.builder.RootSnapshot.Head() + sealedRoot, err := node.State.Params().SealedRoot() if err != nil { - return fmt.Errorf("could not get root snapshot head: %w", err) + return fmt.Errorf("could not get sealed root: %w", err) } - checkpointHeight := root.Height + rootSeal, err := node.State.Params().Seal() + if err != nil { + return fmt.Errorf("could not get root seal: %w", err) + } + + if sealedRoot.ID() != rootSeal.BlockID { + return fmt.Errorf("mismatching root seal and sealed root: %v != %v", sealedRoot.ID(), rootSeal.BlockID) + } + + checkpointHeight := sealedRoot.Height + rootHash := ledgerpkg.RootHash(rootSeal.FinalState) - err = bootstrap.ImportRegistersFromCheckpoint(node.Logger, checkpointFile, checkpointHeight, pebbledb, exeNode.exeConf.importCheckpointWorkerCount) + err = bootstrap.ImportRegistersFromCheckpoint(node.Logger, checkpointFile, checkpointHeight, rootHash, pebbledb, exeNode.exeConf.importCheckpointWorkerCount) if err != nil { return fmt.Errorf("could not import registers from checkpoint: %w", err) } @@ -820,12 +830,17 @@ func (exeNode *ExecutionNode) LoadRegisterStore( reader := finalizedreader.NewFinalizedReader(node.Storage.Headers, node.LastFinalizedHeader.Height) node.ProtocolEvents.AddConsumer(reader) + notifier := storehouse.NewRegisterStoreMetrics(exeNode.collector) + + // report latest finalized and executed height as metrics + notifier.OnFinalizedAndExecutedHeightUpdated(diskStore.LatestHeight()) registerStore, err := storehouse.NewRegisterStore( diskStore, nil, // TODO: replace with real WAL reader, node.Logger, + notifier, ) if err != nil { return err @@ -926,12 +941,19 @@ func (exeNode *ExecutionNode) LoadCheckerEngine( module.ReadyDoneAware, error, ) { - exeNode.checkerEng = checker.New( + if !exeNode.exeConf.enableChecker { + node.Logger.Warn().Msgf("checker engine is disabled") + return &module.NoopReadyDoneAware{}, nil + } + + node.Logger.Info().Msgf("checker engine is enabled") + + core := checker.NewCore( node.Logger, node.State, exeNode.executionState, - node.Storage.Seals, ) + exeNode.checkerEng = checker.NewEngine(core) return exeNode.checkerEng, nil } @@ -969,7 +991,6 @@ func (exeNode *ExecutionNode) LoadIngestionEngine( exeNode.ingestionUnit, node.Logger, node.EngineRegistry, - node.Me, fetcher, node.Storage.Headers, node.Storage.Blocks, @@ -1041,8 +1062,6 @@ func (exeNode *ExecutionNode) LoadFollowerCore( return nil, fmt.Errorf("could not find latest finalized block and pending blocks to recover consensus follower: %w", err) } - exeNode.followerDistributor.AddFinalizationConsumer(exeNode.checkerEng) - // creates a consensus follower with ingestEngine as the notifier // so that it gets notified upon each new finalized block exeNode.followerCore, err = consensus.NewFollower( diff --git a/cmd/execution_config.go b/cmd/execution_config.go index 033bf2c4d87..3a16e5e6a3a 100644 --- a/cmd/execution_config.go +++ b/cmd/execution_config.go @@ -65,6 +65,7 @@ type ExecutionConfig struct { // file descriptors causing connection failures. onflowOnlyLNs bool enableStorehouse bool + enableChecker bool } func (exeConf *ExecutionConfig) SetupFlags(flags *pflag.FlagSet) { @@ -118,6 +119,7 @@ func (exeConf *ExecutionConfig) SetupFlags(flags *pflag.FlagSet) { flags.BoolVar(&exeConf.onflowOnlyLNs, "temp-onflow-only-lns", false, "do not use unless required. forces node to only request collections from onflow collection nodes") flags.BoolVar(&exeConf.enableStorehouse, "enable-storehouse", false, "enable storehouse to store registers on disk, default is false") + flags.BoolVar(&exeConf.enableChecker, "enable-checker", true, "enable checker to check the correctness of the execution result, default is true") } diff --git a/cmd/util/cmd/common/flow_client.go b/cmd/util/cmd/common/flow_client.go index e16438da9f6..4f7fe6a704e 100644 --- a/cmd/util/cmd/common/flow_client.go +++ b/cmd/util/cmd/common/flow_client.go @@ -11,7 +11,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/utils/grpcutils" ) @@ -97,7 +96,7 @@ func FlowClientConfigs(accessNodeIDS []flow.Identifier, insecureAccessAPI bool, if err != nil { return nil, fmt.Errorf("failed get identities access node identities (ids=%v) from snapshot: %w", accessNodeIDS, err) } - identities = identities.Sort(order.ByReferenceOrder(accessNodeIDS)) + identities = identities.Sort(flow.ByReferenceOrder(accessNodeIDS)) // make sure we have identities for all the access node IDs provided if len(identities) != len(accessNodeIDS) { diff --git a/cmd/util/ledger/migrations/account_based_migration.go b/cmd/util/ledger/migrations/account_based_migration.go index 7a5331ce503..8a78728b7b6 100644 --- a/cmd/util/ledger/migrations/account_based_migration.go +++ b/cmd/util/ledger/migrations/account_based_migration.go @@ -1,206 +1,384 @@ package migrations import ( + "container/heap" + "context" "fmt" + "io" + "sync" + "time" - "github.com/rs/zerolog/log" + "github.com/rs/zerolog" + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/cmd/util/ledger/util" "github.com/onflow/flow-go/ledger" - "github.com/onflow/flow-go/ledger/common/convert" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/util" + moduleUtil "github.com/onflow/flow-go/module/util" ) -// PayloadToAccount takes a payload and return: -// - (address, true, nil) if the payload is for an account, the account address is returned -// - ("", false, nil) if the payload is not for an account -// - ("", false, err) if running into any exception -func PayloadToAccount(p ledger.Payload) (string, bool, error) { - k, err := p.Key() - if err != nil { - return "", false, fmt.Errorf("could not find key for payload: %w", err) - } - id, err := convert.LedgerKeyToRegisterID(k) - if err != nil { - return "", false, fmt.Errorf("error converting key to register ID") - } - if len([]byte(id.Owner)) != flow.AddressLength { - return "", false, nil - } - return id.Owner, true, nil +// logTopNDurations is the number of longest migrations to log at the end of the migration +const logTopNDurations = 20 + +// AccountBasedMigration is an interface for migrations that migrate account by account +// concurrently getting all the payloads for each account at a time. +type AccountBasedMigration interface { + InitMigration( + log zerolog.Logger, + allPayloads []*ledger.Payload, + nWorkers int, + ) error + MigrateAccount( + ctx context.Context, + address common.Address, + payloads []*ledger.Payload, + ) ([]*ledger.Payload, error) + io.Closer } -// PayloadGroup groups payloads by account. -// For global payloads, it's stored under NonAccountPayloads field -type PayloadGroup struct { - NonAccountPayloads []ledger.Payload - Accounts map[string][]ledger.Payload +// CreateAccountBasedMigration creates a migration function that migrates the payloads +// account by account using the given migrations +// accounts are processed concurrently using the given number of workers +// but each account is processed sequentially by the given migrations in order. +// The migrations InitMigration function is called once before the migration starts +// And the Close function is called once after the migration finishes if the migration +// is a finisher. +func CreateAccountBasedMigration( + log zerolog.Logger, + nWorker int, + migrations []AccountBasedMigration, +) func(payloads []*ledger.Payload) ([]*ledger.Payload, error) { + return func(payloads []*ledger.Payload) ([]*ledger.Payload, error) { + return MigrateByAccount( + log, + nWorker, + payloads, + migrations, + ) + } } -// PayloadGrouping is a reducer function that adds the given payload to the corresponding -// group under its account -func PayloadGrouping(groups *PayloadGroup, payload ledger.Payload) (*PayloadGroup, error) { - address, isAccount, err := PayloadToAccount(payload) - if err != nil { - return nil, err +// MigrateByAccount takes migrations and all the Payloads, +// and returns the migrated Payloads. +func MigrateByAccount( + log zerolog.Logger, + nWorker int, + allPayloads []*ledger.Payload, + migrations []AccountBasedMigration, +) ( + []*ledger.Payload, + error, +) { + if len(allPayloads) == 0 { + return allPayloads, nil } - if isAccount { - groups.Accounts[address] = append(groups.Accounts[address], payload) - } else { - groups.NonAccountPayloads = append(groups.NonAccountPayloads, payload) + for i, migrator := range migrations { + if err := migrator.InitMigration( + log.With(). + Int("migration_index", i). + Logger(), + allPayloads, + nWorker, + ); err != nil { + return nil, fmt.Errorf("could not init migration: %w", err) + } } - return groups, nil -} + log.Info(). + Int("inner_migrations", len(migrations)). + Int("nWorker", nWorker). + Msgf("created account migrations") + + defer func() { + for i, migrator := range migrations { + log.Info(). + Int("migration_index", i). + Type("migration", migrator). + Msg("closing migration") + if err := migrator.Close(); err != nil { + log.Error().Err(err).Msg("error closing migration") + } + } + }() -// AccountMigrator takes all the payloads that belong to the given account -// and return the migrated payloads -type AccountMigrator interface { - MigratePayloads(account string, payloads []ledger.Payload) ([]ledger.Payload, error) -} + // group the Payloads by account + accountGroups := util.GroupPayloadsByAccount(log, allPayloads, nWorker) + + // migrate the Payloads under accounts + migrated, err := MigrateGroupConcurrently(log, migrations, accountGroups, nWorker) -// MigrateByAccount teaks a migrator function and all the payloads, and return the migrated payloads -func MigrateByAccount(migrator AccountMigrator, allPayloads []ledger.Payload, nWorker int) ( - []ledger.Payload, error) { - groups := &PayloadGroup{ - NonAccountPayloads: make([]ledger.Payload, 0), - Accounts: make(map[string][]ledger.Payload), + if err != nil { + return nil, fmt.Errorf("could not migrate accounts: %w", err) } - log.Info().Msgf("start grouping for a total of %v payloads", len(allPayloads)) + log.Info(). + Int("account_count", accountGroups.Len()). + Int("payload_count", len(allPayloads)). + Msgf("finished migrating Payloads") - var err error - logGrouping := util.LogProgress( - log.Logger, - util.DefaultLogProgressConfig( - "grouping payload", - len(allPayloads), - ), - ) - for _, payload := range allPayloads { - groups, err = PayloadGrouping(groups, payload) - if err != nil { - return nil, err - } - logGrouping(1) - } + return migrated, nil +} - log.Info().Msgf("finish grouping for payloads by account: %v groups in total, %v NonAccountPayloads", - len(groups.Accounts), len(groups.NonAccountPayloads)) +// MigrateGroupConcurrently migrate the Payloads in the given account groups. +// It uses nWorker to process the Payloads concurrently. The Payloads in each account +// are processed sequentially by the given migrations in order. +func MigrateGroupConcurrently( + log zerolog.Logger, + migrations []AccountBasedMigration, + accountGroups *util.PayloadAccountGrouping, + nWorker int, +) ([]*ledger.Payload, error) { - // migrate the payloads under accounts - migrated, err := MigrateGroupConcurrently(migrator, groups.Accounts, nWorker) + ctx := context.Background() + ctx, cancel := context.WithCancelCause(ctx) + defer cancel(nil) - if err != nil { - return nil, fmt.Errorf("could not migrate group: %w", err) - } + jobs := make(chan jobMigrateAccountGroup, accountGroups.Len()) - log.Info().Msgf("finished migrating payloads for %v account", len(groups.Accounts)) + wg := sync.WaitGroup{} + wg.Add(nWorker) + resultCh := make(chan *migrationResult, accountGroups.Len()) + for i := 0; i < nWorker; i++ { + go func() { + defer wg.Done() + + for { + select { + case <-ctx.Done(): + return + case job, ok := <-jobs: + if !ok { + return + } + start := time.Now() + + // This is not an account, but service level keys. + if util.IsServiceLevelAddress(job.Address) { + resultCh <- &migrationResult{ + migrationDuration: migrationDuration{ + Address: job.Address, + Duration: time.Since(start), + PayloadCount: len(job.Payloads), + }, + Migrated: job.Payloads, + } + continue + } + + if _, ok := knownProblematicAccounts[job.Address]; ok { + log.Info(). + Hex("address", job.Address[:]). + Int("payload_count", len(job.Payloads)). + Msg("skipping problematic account") + resultCh <- &migrationResult{ + migrationDuration: migrationDuration{ + Address: job.Address, + Duration: time.Since(start), + PayloadCount: len(job.Payloads), + }, + Migrated: job.Payloads, + } + continue + } + + var err error + accountMigrated := job.Payloads + for m, migrator := range migrations { + + select { + case <-ctx.Done(): + return + default: + } + + accountMigrated, err = migrator.MigrateAccount(ctx, job.Address, accountMigrated) + if err != nil { + log.Error(). + Err(err). + Int("migration_index", m). + Type("migration", migrator). + Hex("address", job.Address[:]). + Msg("could not migrate account") + cancel(fmt.Errorf("could not migrate account: %w", err)) + return + } + } + + resultCh <- &migrationResult{ + migrationDuration: migrationDuration{ + Address: job.Address, + Duration: time.Since(start), + PayloadCount: len(job.Payloads), + }, + Migrated: accountMigrated, + } + } + } + }() + } - // add the non accounts which don't need to be migrated - migrated = append(migrated, groups.NonAccountPayloads...) + go func() { + defer close(jobs) + for { + g, err := accountGroups.Next() + if err != nil { + cancel(fmt.Errorf("could not get next account group: %w", err)) + return + } - log.Info().Msgf("finished migrating all account based payloads, total migrated payloads: %v", len(migrated)) + if g == nil { + break + } - return migrated, nil -} + job := jobMigrateAccountGroup{ + Address: g.Address, + Payloads: g.Payloads, + } -// MigrateGroupSequentially migrate the payloads in the given payloadsByAccount map which -// using the migrator -func MigrateGroupSequentially( - migrator AccountMigrator, - payloadsByAccount map[string][]ledger.Payload, -) ( - []ledger.Payload, error) { + select { + case <-ctx.Done(): + return + case jobs <- job: + } + } + }() - logAccount := util.LogProgress( - log.Logger, - util.DefaultLogProgressConfig( + // read job results + logAccount := moduleUtil.LogProgress( + log, + moduleUtil.DefaultLogProgressConfig( "processing account group", - len(payloadsByAccount), + accountGroups.Len(), ), ) - migrated := make([]ledger.Payload, 0) - for address, payloads := range payloadsByAccount { - accountMigrated, err := migrator.MigratePayloads(address, payloads) - if err != nil { - return nil, fmt.Errorf("could not migrate for account address %v: %w", address, err) + migrated := make([]*ledger.Payload, 0, accountGroups.AllPayloadsCount()) + durations := newMigrationDurations(logTopNDurations) + contextDone := false + for i := 0; i < accountGroups.Len(); i++ { + select { + case <-ctx.Done(): + contextDone = true + break + case result := <-resultCh: + durations.Add(result) + + accountMigrated := result.Migrated + migrated = append(migrated, accountMigrated...) + logAccount(1) } + if contextDone { + break + } + } + + // make sure to exit all workers before returning from this function + // so that the migrator can be closed properly + log.Info().Msg("waiting for migration workers to finish") + wg.Wait() - migrated = append(migrated, accountMigrated...) - logAccount(1) + log.Info(). + Array("top_longest_migrations", durations.Array()). + Msgf("Top longest migrations") + + if ctx.Err() != nil { + return nil, fmt.Errorf("fail to migrate payload: %w", ctx.Err()) } return migrated, nil } +var knownProblematicAccounts = map[common.Address]string{ + // Testnet accounts with broken contracts + mustHexToAddress("434a1f199a7ae3ba"): "Broken contract FanTopPermission", + mustHexToAddress("454c9991c2b8d947"): "Broken contract Test", + mustHexToAddress("48602d8056ff9d93"): "Broken contract FanTopPermission", + mustHexToAddress("5d63c34d7f05e5a4"): "Broken contract FanTopPermission", + mustHexToAddress("5e3448b3cffb97f2"): "Broken contract FanTopPermission", + mustHexToAddress("7d8c7e050c694eaa"): "Broken contract Test", + mustHexToAddress("ba53f16ede01972d"): "Broken contract FanTopPermission", + mustHexToAddress("c843c1f5a4805c3a"): "Broken contract FanTopPermission", + mustHexToAddress("48d3be92e6e4a973"): "Broken contract FanTopPermission", + // Mainnet account +} + +func mustHexToAddress(hex string) common.Address { + address, err := common.HexToAddress(hex) + if err != nil { + panic(err) + } + return address +} + type jobMigrateAccountGroup struct { - Account string - Payloads []ledger.Payload + Address common.Address + Payloads []*ledger.Payload } type migrationResult struct { - Migrated []ledger.Payload - Err error + migrationDuration + + Migrated []*ledger.Payload } -// MigrateGroupConcurrently migrate the payloads in the given payloadsByAccount map which -// using the migrator -// It's similar to MigrateGroupSequentially, except it will migrate different groups concurrently -func MigrateGroupConcurrently( - migrator AccountMigrator, - payloadsByAccount map[string][]ledger.Payload, - nWorker int, -) ( - []ledger.Payload, error) { +type migrationDuration struct { + Address common.Address + Duration time.Duration + PayloadCount int +} - jobs := make(chan jobMigrateAccountGroup, len(payloadsByAccount)) - go func() { - for account, payloads := range payloadsByAccount { - jobs <- jobMigrateAccountGroup{ - Account: account, - Payloads: payloads, - } - } - close(jobs) - }() +// migrationDurations implements heap methods for the timer results +type migrationDurations struct { + v []migrationDuration - resultCh := make(chan *migrationResult) - for i := 0; i < int(nWorker); i++ { - go func() { - for job := range jobs { - accountMigrated, err := migrator.MigratePayloads(job.Account, job.Payloads) - resultCh <- &migrationResult{ - Migrated: accountMigrated, - Err: err, - } - } - }() + KeepTopN int +} + +// newMigrationDurations creates a new migrationDurations which are used to track the +// accounts that took the longest time to migrate. +func newMigrationDurations(keepTopN int) *migrationDurations { + return &migrationDurations{ + v: make([]migrationDuration, 0, keepTopN), + KeepTopN: keepTopN, } +} - // read job results - logAccount := util.LogProgress( - log.Logger, - util.DefaultLogProgressConfig( - "processing account group", - len(payloadsByAccount), - ), - ) +func (h *migrationDurations) Len() int { return len(h.v) } +func (h *migrationDurations) Less(i, j int) bool { + return h.v[i].Duration < h.v[j].Duration +} +func (h *migrationDurations) Swap(i, j int) { + h.v[i], h.v[j] = h.v[j], h.v[i] +} +func (h *migrationDurations) Push(x interface{}) { + h.v = append(h.v, x.(migrationDuration)) +} +func (h *migrationDurations) Pop() interface{} { + old := h.v + n := len(old) + x := old[n-1] + h.v = old[0 : n-1] + return x +} - migrated := make([]ledger.Payload, 0) +func (h *migrationDurations) Array() zerolog.LogArrayMarshaler { + array := zerolog.Arr() + for _, result := range h.v { + array = array.Str(fmt.Sprintf("%s [payloads: %d]: %s", + result.Address.Hex(), + result.PayloadCount, + result.Duration.String(), + )) + } + return array +} - for i := 0; i < len(payloadsByAccount); i++ { - result := <-resultCh - if result.Err != nil { - return nil, fmt.Errorf("fail to migrate payload: %w", result.Err) +func (h *migrationDurations) Add(result *migrationResult) { + if h.Len() < h.KeepTopN || result.Duration > h.v[0].Duration { + if h.Len() == h.KeepTopN { + heap.Pop(h) // remove the element with the smallest duration } - - accountMigrated := result.Migrated - migrated = append(migrated, accountMigrated...) - logAccount(1) + heap.Push(h, result.migrationDuration) } - - return migrated, nil } diff --git a/cmd/util/ledger/migrations/account_migration.go b/cmd/util/ledger/migrations/account_migration.go deleted file mode 100644 index e0bd7c029dd..00000000000 --- a/cmd/util/ledger/migrations/account_migration.go +++ /dev/null @@ -1,100 +0,0 @@ -package migrations - -import ( - "fmt" - - "github.com/rs/zerolog/log" - - "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/ledger" - "github.com/onflow/flow-go/ledger/common/convert" - "github.com/onflow/flow-go/model/flow" -) - -func MigrateAccountUsage(payloads []ledger.Payload, nWorker int) ([]ledger.Payload, error) { - return MigrateByAccount(AccountUsageMigrator{}, payloads, nWorker) -} - -func payloadSize(key ledger.Key, payload ledger.Payload) (uint64, error) { - id, err := convert.LedgerKeyToRegisterID(key) - if err != nil { - return 0, err - } - - return uint64(registerSize(id, payload)), nil -} - -func isAccountKey(key ledger.Key) bool { - return string(key.KeyParts[1].Value) == flow.AccountStatusKey -} - -type AccountUsageMigrator struct{} - -// AccountUsageMigrator iterate through each payload, and calculate the storage usage -// and update the accoutns status with the updated storage usage -func (m AccountUsageMigrator) MigratePayloads(account string, payloads []ledger.Payload) ([]ledger.Payload, error) { - var status *environment.AccountStatus - var statusIndex int - totalSize := uint64(0) - for i, payload := range payloads { - key, err := payload.Key() - if err != nil { - return nil, err - } - if isAccountKey(key) { - statusIndex = i - status, err = environment.AccountStatusFromBytes(payload.Value()) - if err != nil { - return nil, fmt.Errorf("could not parse account status: %w", err) - } - - } - - size, err := payloadSize(key, payload) - if err != nil { - return nil, err - } - totalSize += size - } - - err := compareUsage(status, totalSize) - if err != nil { - log.Error().Msgf("%v", err) - } - - if status == nil { - return nil, fmt.Errorf("could not find account status for account %v", account) - } - - // update storage used - status.SetStorageUsed(totalSize) - - newValue := status.ToBytes() - newPayload, err := newPayloadWithValue(payloads[statusIndex], newValue) - if err != nil { - return nil, fmt.Errorf("cannot create new payload with value: %w", err) - } - - payloads[statusIndex] = newPayload - - return payloads, nil -} - -func compareUsage(status *environment.AccountStatus, totalSize uint64) error { - oldSize := status.StorageUsed() - if oldSize != totalSize { - return fmt.Errorf("old size: %v, new size: %v", oldSize, totalSize) - } - return nil -} - -// newPayloadWithValue returns a new payload with the key from the given payload, and -// the value from the argument -func newPayloadWithValue(payload ledger.Payload, value ledger.Value) (ledger.Payload, error) { - key, err := payload.Key() - if err != nil { - return ledger.Payload{}, err - } - newPayload := ledger.NewPayload(key, payload.Value()) - return *newPayload, nil -} diff --git a/cmd/util/ledger/migrations/change_contract_code_migration.go b/cmd/util/ledger/migrations/change_contract_code_migration.go new file mode 100644 index 00000000000..6ec29f936db --- /dev/null +++ b/cmd/util/ledger/migrations/change_contract_code_migration.go @@ -0,0 +1,348 @@ +package migrations + +import ( + "context" + "fmt" + "sync" + + coreContracts "github.com/onflow/flow-core-contracts/lib/go/contracts" + ftContracts "github.com/onflow/flow-ft/lib/go/contracts" + nftContracts "github.com/onflow/flow-nft/lib/go/contracts" + "github.com/rs/zerolog" + + sdk "github.com/onflow/flow-go-sdk" + + "github.com/onflow/cadence/runtime/common" + + evm "github.com/onflow/flow-go/fvm/evm/stdlib" + "github.com/onflow/flow-go/fvm/systemcontracts" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +type ChangeContractCodeMigration struct { + log zerolog.Logger + mutex sync.RWMutex + contracts map[common.Address]map[flow.RegisterID]string +} + +var _ AccountBasedMigration = (*ChangeContractCodeMigration)(nil) + +func (d *ChangeContractCodeMigration) Close() error { + d.mutex.RLock() + defer d.mutex.RUnlock() + + if len(d.contracts) > 0 { + return fmt.Errorf("failed to find all contract registers that need to be changed") + } + + return nil +} + +func (d *ChangeContractCodeMigration) InitMigration( + log zerolog.Logger, + _ []*ledger.Payload, + _ int, +) error { + d.log = log. + With(). + Str("migration", "ChangeContractCodeMigration"). + Logger() + + return nil +} + +func (d *ChangeContractCodeMigration) MigrateAccount( + _ context.Context, + address common.Address, + payloads []*ledger.Payload, +) ([]*ledger.Payload, error) { + + contracts, ok := (func() (map[flow.RegisterID]string, bool) { + d.mutex.Lock() + defer d.mutex.Unlock() + + contracts, ok := d.contracts[address] + + // remove address from set of addresses + // to keep track of which addresses are left to change + delete(d.contracts, address) + + return contracts, ok + })() + + if !ok { + // no contracts to change on this address + return payloads, nil + } + + for payloadIndex, payload := range payloads { + key, err := payload.Key() + if err != nil { + return nil, err + } + + registerID, err := convert.LedgerKeyToRegisterID(key) + if err != nil { + return nil, err + } + + newContract, ok := contracts[registerID] + if !ok { + // not a contract register, or + // not interested in this contract + continue + } + + // change contract code + payloads[payloadIndex] = ledger.NewPayload( + key, + []byte(newContract), + ) + + // TODO: maybe log diff between old and new + + // remove contract from list of contracts to change + // to keep track of which contracts are left to change + delete(contracts, registerID) + } + + if len(contracts) > 0 { + return nil, fmt.Errorf("failed to find all contract registers that need to be changed") + } + + return payloads, nil +} + +func (d *ChangeContractCodeMigration) RegisterContractChange( + address common.Address, + contractName string, + newContractCode string, +) ( + previousNewContractCode string, +) { + d.mutex.Lock() + defer d.mutex.Unlock() + + if d.contracts == nil { + d.contracts = map[common.Address]map[flow.RegisterID]string{} + } + + if _, ok := d.contracts[address]; !ok { + d.contracts[address] = map[flow.RegisterID]string{} + } + + registerID := flow.ContractRegisterID(flow.ConvertAddress(address), contractName) + + previousNewContractCode = d.contracts[address][registerID] + + d.contracts[address][registerID] = newContractCode + + return +} + +type SystemContractChange struct { + Address common.Address + ContractName string + NewContractCode string +} + +func NewSystemContractChange( + systemContract systemcontracts.SystemContract, + newContractCode []byte, +) SystemContractChange { + return SystemContractChange{ + Address: common.Address(systemContract.Address), + ContractName: systemContract.Name, + NewContractCode: string(newContractCode), + } +} + +func SystemContractChanges(chainID flow.ChainID) []SystemContractChange { + systemContracts := systemcontracts.SystemContractsForChain(chainID) + + var stakingCollectionAddress, stakingProxyAddress common.Address + + switch chainID { + case flow.Mainnet: + stakingCollectionAddress = mustHexAddress("0x8d0e87b65159ae63") + stakingProxyAddress = mustHexAddress("0x62430cf28c26d095") + + case flow.Testnet: + stakingCollectionAddress = mustHexAddress("0x95e019a17d0e23d7") + stakingProxyAddress = mustHexAddress("0x7aad92e5a0715d21") + + default: + panic(fmt.Errorf("unsupported chain ID: %s", chainID)) + } + + lockedTokensAddress := stakingCollectionAddress + fungibleTokenMetadataViewsAddress := common.Address(systemContracts.FungibleToken.Address) + fungibleTokenSwitchboardAddress := common.Address(systemContracts.FungibleToken.Address) + + return []SystemContractChange{ + // epoch related contracts + NewSystemContractChange( + systemContracts.Epoch, + coreContracts.FlowEpoch( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.IDTableStaking.Address.HexWithPrefix(), + systemContracts.ClusterQC.Address.HexWithPrefix(), + systemContracts.DKG.Address.HexWithPrefix(), + systemContracts.FlowFees.Address.HexWithPrefix(), + ), + ), + NewSystemContractChange( + systemContracts.IDTableStaking, + coreContracts.FlowIDTableStaking( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.FlowFees.Address.HexWithPrefix(), + true, + ), + ), + NewSystemContractChange( + systemContracts.ClusterQC, + coreContracts.FlowQC(), + ), + NewSystemContractChange( + systemContracts.DKG, + coreContracts.FlowDKG(), + ), + + // service account related contracts + NewSystemContractChange( + systemContracts.FlowServiceAccount, + coreContracts.FlowServiceAccount( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.FlowFees.Address.HexWithPrefix(), + systemContracts.FlowStorageFees.Address.HexWithPrefix(), + ), + ), + NewSystemContractChange( + systemContracts.NodeVersionBeacon, + coreContracts.NodeVersionBeacon(), + ), + NewSystemContractChange( + systemContracts.RandomBeaconHistory, + coreContracts.RandomBeaconHistory(), + ), + NewSystemContractChange( + systemContracts.FlowStorageFees, + coreContracts.FlowStorageFees( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + ), + ), + { + Address: stakingCollectionAddress, + ContractName: "FlowStakingCollection", + NewContractCode: string(coreContracts.FlowStakingCollection( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.IDTableStaking.Address.HexWithPrefix(), + stakingProxyAddress.HexWithPrefix(), + lockedTokensAddress.HexWithPrefix(), + systemContracts.FlowStorageFees.Address.HexWithPrefix(), + systemContracts.ClusterQC.Address.HexWithPrefix(), + systemContracts.DKG.Address.HexWithPrefix(), + systemContracts.Epoch.Address.HexWithPrefix(), + )), + }, + { + Address: stakingProxyAddress, + ContractName: "StakingProxy", + NewContractCode: string(coreContracts.FlowStakingProxy()), + }, + { + Address: lockedTokensAddress, + ContractName: "LockedTokens", + NewContractCode: string(coreContracts.FlowLockedTokens( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.IDTableStaking.Address.HexWithPrefix(), + stakingProxyAddress.HexWithPrefix(), + systemContracts.FlowStorageFees.Address.HexWithPrefix(), + )), + }, + + // token related contracts + NewSystemContractChange( + systemContracts.FlowFees, + coreContracts.FlowFees( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.FlowToken.Address.HexWithPrefix(), + systemContracts.FlowStorageFees.Address.HexWithPrefix(), + ), + ), + NewSystemContractChange( + systemContracts.FlowToken, + coreContracts.FlowToken( + systemContracts.FungibleToken.Address.HexWithPrefix(), + fungibleTokenMetadataViewsAddress.HexWithPrefix(), + systemContracts.MetadataViews.Address.HexWithPrefix(), + systemContracts.ViewResolver.Address.HexWithPrefix(), + ), + ), + NewSystemContractChange( + systemContracts.FungibleToken, + ftContracts.FungibleToken(), + ), + { + Address: fungibleTokenMetadataViewsAddress, + ContractName: "FungibleTokenMetadataViews", + NewContractCode: string(ftContracts.FungibleTokenMetadataViews( + systemContracts.FungibleToken.Address.HexWithPrefix(), + systemContracts.MetadataViews.Address.HexWithPrefix(), + systemContracts.ViewResolver.Address.HexWithPrefix(), + )), + }, + { + Address: fungibleTokenSwitchboardAddress, + ContractName: "FungibleTokenSwitchboard", + NewContractCode: string(ftContracts.FungibleTokenSwitchboard( + systemContracts.FungibleToken.Address.HexWithPrefix(), + )), + }, + + // NFT related contracts + NewSystemContractChange( + systemContracts.NonFungibleToken, + nftContracts.NonFungibleToken( + sdk.Address(systemContracts.ViewResolver.Address), + ), + ), + NewSystemContractChange( + systemContracts.MetadataViews, + nftContracts.MetadataViews( + sdk.Address(systemContracts.FungibleToken.Address), + sdk.Address(systemContracts.NonFungibleToken.Address), + sdk.Address(systemContracts.ViewResolver.Address), + ), + ), + NewSystemContractChange( + systemContracts.ViewResolver, + nftContracts.Resolver(), + ), + + // EVM related contracts + NewSystemContractChange( + systemContracts.EVM, + evm.ContractCode( + systemContracts.FlowToken.Address, + true, + ), + ), + } +} + +func mustHexAddress(hexAddress string) common.Address { + address, err := common.HexToAddress(hexAddress) + if err != nil { + panic(err) + } + return address +} diff --git a/cmd/util/ledger/migrations/change_contract_code_migration_test.go b/cmd/util/ledger/migrations/change_contract_code_migration_test.go new file mode 100644 index 00000000000..1f499115dda --- /dev/null +++ b/cmd/util/ledger/migrations/change_contract_code_migration_test.go @@ -0,0 +1,221 @@ +package migrations_test + +import ( + "context" + "testing" + + "github.com/onflow/cadence/runtime/common" + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/cmd/util/ledger/migrations" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +func newContractPayload(address common.Address, contractName string, contract []byte) *ledger.Payload { + return ledger.NewPayload( + convert.RegisterIDToLedgerKey( + flow.ContractRegisterID(flow.ConvertAddress(address), contractName), + ), + contract, + ) +} + +func TestChangeContractCodeMigration(t *testing.T) { + t.Parallel() + + address1, err := common.HexToAddress("0x1") + require.NoError(t, err) + + address2, err := common.HexToAddress("0x2") + require.NoError(t, err) + + ctx := context.Background() + + t.Run("no contracts", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + _, err = migration.MigrateAccount(ctx, address1, + []*ledger.Payload{}, + ) + + require.NoError(t, err) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("1 contract - dont migrate", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + }, + ) + + require.NoError(t, err) + require.Len(t, payloads, 1) + require.Equal(t, []byte("A"), []byte(payloads[0].Value())) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("1 contract - migrate", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address1, "A", "B") + + payloads, err := migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + }, + ) + + require.NoError(t, err) + require.Len(t, payloads, 1) + require.Equal(t, []byte("B"), []byte(payloads[0].Value())) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("2 contracts - migrate 1", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address1, "A", "B") + + payloads, err := migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + newContractPayload(address1, "B", []byte("A")), + }, + ) + + require.NoError(t, err) + require.Len(t, payloads, 2) + require.Equal(t, []byte("B"), []byte(payloads[0].Value())) + require.Equal(t, []byte("A"), []byte(payloads[1].Value())) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("2 contracts - migrate 2", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address1, "A", "B") + migration.RegisterContractChange(address1, "B", "B") + + payloads, err := migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + newContractPayload(address1, "B", []byte("A")), + }, + ) + + require.NoError(t, err) + require.Len(t, payloads, 2) + require.Equal(t, []byte("B"), []byte(payloads[0].Value())) + require.Equal(t, []byte("B"), []byte(payloads[1].Value())) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("2 contracts on different accounts - migrate 1", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address1, "A", "B") + + payloads, err := migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + newContractPayload(address2, "A", []byte("A")), + }, + ) + + require.NoError(t, err) + require.Len(t, payloads, 2) + require.Equal(t, []byte("B"), []byte(payloads[0].Value())) + require.Equal(t, []byte("A"), []byte(payloads[1].Value())) + + err = migration.Close() + require.NoError(t, err) + }) + + t.Run("not all contracts on one account migrated", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address1, "A", "B") + migration.RegisterContractChange(address1, "B", "B") + + _, err = migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + }, + ) + + require.Error(t, err) + }) + + t.Run("not all accounts migrated", func(t *testing.T) { + t.Parallel() + + migration := migrations.ChangeContractCodeMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + migration.RegisterContractChange(address2, "A", "B") + + _, err = migration.MigrateAccount(ctx, address1, + []*ledger.Payload{ + newContractPayload(address1, "A", []byte("A")), + }, + ) + + require.NoError(t, err) + + err = migration.Close() + require.Error(t, err) + }) +} diff --git a/cmd/util/ledger/migrations/deduplicate_contract_names_migration.go b/cmd/util/ledger/migrations/deduplicate_contract_names_migration.go new file mode 100644 index 00000000000..ab35e04d8a3 --- /dev/null +++ b/cmd/util/ledger/migrations/deduplicate_contract_names_migration.go @@ -0,0 +1,133 @@ +package migrations + +import ( + "context" + "fmt" + + "github.com/fxamacker/cbor/v2" + "github.com/rs/zerolog" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +// DeduplicateContractNamesMigration checks if the contract names have been duplicated and +// removes the duplicate ones. +// +// This migration de-syncs storage used, so it should be run before the StorageUsedMigration. +type DeduplicateContractNamesMigration struct { + log zerolog.Logger +} + +func (d *DeduplicateContractNamesMigration) Close() error { + return nil +} + +func (d *DeduplicateContractNamesMigration) InitMigration( + log zerolog.Logger, + _ []*ledger.Payload, + _ int, +) error { + d.log = log. + With(). + Str("migration", "DeduplicateContractNamesMigration"). + Logger() + + return nil +} + +func (d *DeduplicateContractNamesMigration) MigrateAccount( + ctx context.Context, + address common.Address, + payloads []*ledger.Payload, +) ([]*ledger.Payload, error) { + flowAddress := flow.ConvertAddress(address) + contractNamesID := flow.ContractNamesRegisterID(flowAddress) + + var contractNamesPayload *ledger.Payload + contractNamesPayloadIndex := 0 + for i, payload := range payloads { + key, err := payload.Key() + if err != nil { + return nil, err + } + id, err := convert.LedgerKeyToRegisterID(key) + if err != nil { + return nil, err + } + if id == contractNamesID { + contractNamesPayload = payload + contractNamesPayloadIndex = i + break + } + } + if contractNamesPayload == nil { + return payloads, nil + } + + value := contractNamesPayload.Value() + if len(value) == 0 { + // Remove the empty payload + copy(payloads[contractNamesPayloadIndex:], payloads[contractNamesPayloadIndex+1:]) + payloads = payloads[:len(payloads)-1] + + return payloads, nil + } + + var contractNames []string + err := cbor.Unmarshal(value, &contractNames) + if err != nil { + return nil, fmt.Errorf("failed to get contract names: %w", err) + } + + var foundDuplicate bool + i := 1 + for i < len(contractNames) { + if contractNames[i-1] != contractNames[i] { + + if contractNames[i-1] > contractNames[i] { + // this is not a valid state and we should fail. + // Contract names must be sorted by definition. + return nil, fmt.Errorf( + "contract names for account %s are not sorted: %s", + address.Hex(), + contractNames, + ) + } + + i++ + continue + } + // Found duplicate (contactNames[i-1] == contactNames[i]) + // Remove contractNames[i] + copy(contractNames[i:], contractNames[i+1:]) + contractNames = contractNames[:len(contractNames)-1] + foundDuplicate = true + } + + if !foundDuplicate { + return payloads, nil + } + + d.log.Info(). + Str("address", address.Hex()). + Strs("contract_names", contractNames). + Msg("removing duplicate contract names") + + newContractNames, err := cbor.Marshal(contractNames) + if err != nil { + return nil, fmt.Errorf( + "cannot encode contract names: %s", + contractNames, + ) + } + + payloads[contractNamesPayloadIndex] = ledger.NewPayload(convert.RegisterIDToLedgerKey(contractNamesID), newContractNames) + return payloads, nil + +} + +var _ AccountBasedMigration = &DeduplicateContractNamesMigration{} diff --git a/cmd/util/ledger/migrations/deduplicate_contract_names_migration_test.go b/cmd/util/ledger/migrations/deduplicate_contract_names_migration_test.go new file mode 100644 index 00000000000..ba81bc826cd --- /dev/null +++ b/cmd/util/ledger/migrations/deduplicate_contract_names_migration_test.go @@ -0,0 +1,221 @@ +package migrations_test + +import ( + "context" + "fmt" + "math/rand" + "sort" + "testing" + + "github.com/fxamacker/cbor/v2" + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/cmd/util/ledger/migrations" + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +func TestDeduplicateContractNamesMigration(t *testing.T) { + migration := migrations.DeduplicateContractNamesMigration{} + log := zerolog.New(zerolog.NewTestWriter(t)) + err := migration.InitMigration(log, nil, 0) + require.NoError(t, err) + + address, err := common.HexToAddress("0x1") + require.NoError(t, err) + + ctx := context.Background() + + accountStatus := environment.NewAccountStatus() + accountStatus.SetStorageUsed(1000) + accountStatusPayload := ledger.NewPayload( + convert.RegisterIDToLedgerKey( + flow.AccountStatusRegisterID(flow.ConvertAddress(address)), + ), + accountStatus.ToBytes(), + ) + + contractNamesPayload := func(contractNames []byte) *ledger.Payload { + return ledger.NewPayload( + convert.RegisterIDToLedgerKey( + flow.RegisterID{ + Owner: string(address.Bytes()), + Key: flow.ContractNamesKey, + }, + ), + contractNames, + ) + } + + requireContractNames := func(payloads []*ledger.Payload, f func([]string)) { + for _, payload := range payloads { + key, err := payload.Key() + require.NoError(t, err) + id, err := convert.LedgerKeyToRegisterID(key) + require.NoError(t, err) + + if id.Key != flow.ContractNamesKey { + continue + } + + contracts := make([]string, 0) + err = cbor.Unmarshal(payload.Value(), &contracts) + require.NoError(t, err) + + f(contracts) + + } + } + + t.Run("no contract names", func(t *testing.T) { + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + }, + ) + + require.NoError(t, err) + require.Equal(t, 1, len(payloads)) + }) + + t.Run("one contract", func(t *testing.T) { + contractNames := []string{"test"} + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.NoError(t, err) + require.Equal(t, 2, len(payloads)) + + requireContractNames(payloads, func(contracts []string) { + require.Equal(t, 1, len(contracts)) + require.Equal(t, "test", contracts[0]) + }) + }) + + t.Run("two unique contracts", func(t *testing.T) { + contractNames := []string{"test", "test2"} + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.NoError(t, err) + require.Equal(t, 2, len(payloads)) + + requireContractNames(payloads, func(contracts []string) { + require.Equal(t, 2, len(contracts)) + require.Equal(t, "test", contracts[0]) + require.Equal(t, "test2", contracts[1]) + }) + }) + + t.Run("two contracts", func(t *testing.T) { + contractNames := []string{"test", "test"} + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.NoError(t, err) + require.Equal(t, 2, len(payloads)) + + requireContractNames(payloads, func(contracts []string) { + require.Equal(t, 1, len(contracts)) + require.Equal(t, "test", contracts[0]) + }) + }) + + t.Run("not sorted contracts", func(t *testing.T) { + contractNames := []string{"test2", "test"} + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + _, err = migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.Error(t, err) + }) + + t.Run("duplicate contracts", func(t *testing.T) { + contractNames := []string{"test", "test", "test2", "test3", "test3"} + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.NoError(t, err) + require.Equal(t, 2, len(payloads)) + + requireContractNames(payloads, func(contracts []string) { + require.Equal(t, 3, len(contracts)) + require.Equal(t, "test", contracts[0]) + require.Equal(t, "test2", contracts[1]) + require.Equal(t, "test3", contracts[2]) + }) + }) + + t.Run("random contracts", func(t *testing.T) { + contractNames := make([]string, 1000) + uniqueContracts := 1 + for i := 0; i < 1000; i++ { + // i > 0 so it's easier to know how many unique contracts there are + if i > 0 && rand.Float32() < 0.5 { + uniqueContracts++ + } + contractNames[i] = fmt.Sprintf("test%d", uniqueContracts) + } + + // sort contractNames alphabetically, because they are not sorted + sort.Slice(contractNames, func(i, j int) bool { + return contractNames[i] < contractNames[j] + }) + + newContractNames, err := cbor.Marshal(contractNames) + require.NoError(t, err) + + payloads, err := migration.MigrateAccount(ctx, address, + []*ledger.Payload{ + accountStatusPayload, + contractNamesPayload(newContractNames), + }, + ) + + require.NoError(t, err) + require.Equal(t, 2, len(payloads)) + + requireContractNames(payloads, func(contracts []string) { + require.Equal(t, uniqueContracts, len(contracts)) + }) + }) +} diff --git a/cmd/util/ledger/migrations/storage_fees_migration.go b/cmd/util/ledger/migrations/storage_fees_migration.go deleted file mode 100644 index b7dd49a1ec6..00000000000 --- a/cmd/util/ledger/migrations/storage_fees_migration.go +++ /dev/null @@ -1,63 +0,0 @@ -package migrations - -import ( - fvm "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/ledger" - "github.com/onflow/flow-go/ledger/common/convert" - "github.com/onflow/flow-go/ledger/common/utils" - "github.com/onflow/flow-go/model/flow" -) - -// iterates through registers keeping a map of register sizes -// after it has reached the end it add storage used and storage capacity for each address -func StorageFeesMigration(payload []ledger.Payload) ([]ledger.Payload, error) { - storageUsed := make(map[string]uint64) - newPayload := make([]ledger.Payload, len(payload)) - - for i, p := range payload { - err := incrementStorageUsed(p, storageUsed) - if err != nil { - return nil, err - } - newPayload[i] = p - } - - for s, u := range storageUsed { - // this is the storage used by the storage_used register we are about to add - id := flow.NewRegisterID( - flow.BytesToAddress([]byte(s)), - "storage_used") - storageUsedByStorageUsed := fvm.RegisterSize(id, make([]byte, 8)) - u = u + uint64(storageUsedByStorageUsed) - - newPayload = append(newPayload, *ledger.NewPayload( - convert.RegisterIDToLedgerKey(id), - utils.Uint64ToBinary(u), - )) - } - return newPayload, nil -} - -func incrementStorageUsed(p ledger.Payload, used map[string]uint64) error { - k, err := p.Key() - if err != nil { - return err - } - id, err := convert.LedgerKeyToRegisterID(k) - if err != nil { - return err - } - if len([]byte(id.Owner)) != flow.AddressLength { - // not an address - return nil - } - if _, ok := used[id.Owner]; !ok { - used[id.Owner] = 0 - } - used[id.Owner] = used[id.Owner] + uint64(registerSize(id, p)) - return nil -} - -func registerSize(id flow.RegisterID, p ledger.Payload) int { - return fvm.RegisterSize(id, p.Value()) -} diff --git a/cmd/util/ledger/migrations/storage_used_migration.go b/cmd/util/ledger/migrations/storage_used_migration.go new file mode 100644 index 00000000000..3e79d67fb22 --- /dev/null +++ b/cmd/util/ledger/migrations/storage_used_migration.go @@ -0,0 +1,151 @@ +package migrations + +import ( + "context" + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/fvm/environment" + fvm "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +// AccountUsageMigrator iterates through each payload, and calculate the storage usage +// and update the accounts status with the updated storage usage +type AccountUsageMigrator struct { + log zerolog.Logger +} + +var _ AccountBasedMigration = &AccountUsageMigrator{} + +func (m *AccountUsageMigrator) InitMigration( + log zerolog.Logger, + _ []*ledger.Payload, + _ int, +) error { + m.log = log.With().Str("component", "AccountUsageMigrator").Logger() + return nil +} + +const oldAccountStatusSize = 25 + +func (m *AccountUsageMigrator) Close() error { + return nil +} + +func (m *AccountUsageMigrator) MigrateAccount( + _ context.Context, + address common.Address, + payloads []*ledger.Payload, +) ([]*ledger.Payload, error) { + + var status *environment.AccountStatus + var statusIndex int + actualSize := uint64(0) + for i, payload := range payloads { + key, err := payload.Key() + if err != nil { + return nil, err + } + if isAccountKey(key) { + statusIndex = i + status, err = environment.AccountStatusFromBytes(payload.Value()) + if err != nil { + return nil, fmt.Errorf("could not parse account status: %w", err) + } + + } + + size, err := payloadSize(key, payload) + if err != nil { + return nil, err + } + actualSize += size + } + + if status == nil { + return nil, fmt.Errorf("could not find account status for account %v", address.Hex()) + } + + isOldVersionOfStatusRegister := len(payloads[statusIndex].Value()) == oldAccountStatusSize + + same := m.compareUsage(isOldVersionOfStatusRegister, status, actualSize) + if same { + // there is no problem with the usage, return + return payloads, nil + } + + if isOldVersionOfStatusRegister { + // size will grow by 8 bytes because of the on-the-fly + // migration of account status in AccountStatusFromBytes + actualSize += 8 + } + + // update storage used + status.SetStorageUsed(actualSize) + + newValue := status.ToBytes() + newPayload, err := newPayloadWithValue(payloads[statusIndex], newValue) + if err != nil { + return nil, fmt.Errorf("cannot create new payload with value: %w", err) + } + + payloads[statusIndex] = newPayload + + return payloads, nil +} + +func (m *AccountUsageMigrator) compareUsage( + isOldVersionOfStatusRegister bool, + status *environment.AccountStatus, + actualSize uint64, +) bool { + oldSize := status.StorageUsed() + if isOldVersionOfStatusRegister { + // size will be reported as 8 bytes larger than the actual size due to on-the-fly + // migration of account status in AccountStatusFromBytes + oldSize -= 8 + } + + if oldSize != actualSize { + m.log.Warn(). + Uint64("old_size", oldSize). + Uint64("new_size", actualSize). + Msg("account storage used usage mismatch") + return false + } + return true +} + +// newPayloadWithValue returns a new payload with the key from the given payload, and +// the value from the argument +func newPayloadWithValue(payload *ledger.Payload, value ledger.Value) (*ledger.Payload, error) { + key, err := payload.Key() + if err != nil { + return nil, err + } + newPayload := ledger.NewPayload(key, value) + return newPayload, nil +} + +func registerSize(id flow.RegisterID, p *ledger.Payload) int { + return fvm.RegisterSize(id, p.Value()) +} + +func payloadSize(key ledger.Key, payload *ledger.Payload) (uint64, error) { + id, err := convert.LedgerKeyToRegisterID(key) + if err != nil { + return 0, err + } + + return uint64(registerSize(id, payload)), nil +} + +func isAccountKey(key ledger.Key) bool { + return string(key.KeyParts[1].Value) == flow.AccountStatusKey +} diff --git a/cmd/util/ledger/reporters/fungible_token_tracker.go b/cmd/util/ledger/reporters/fungible_token_tracker.go index 24a2c09ac56..a84d8282b7a 100644 --- a/cmd/util/ledger/reporters/fungible_token_tracker.go +++ b/cmd/util/ledger/reporters/fungible_token_tracker.go @@ -13,7 +13,7 @@ import ( "github.com/onflow/cadence/runtime/common" "github.com/onflow/cadence/runtime/interpreter" - "github.com/onflow/flow-go/cmd/util/ledger/migrations" + "github.com/onflow/flow-go/cmd/util/ledger/util" "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/storage/state" "github.com/onflow/flow-go/fvm/systemcontracts" @@ -147,7 +147,7 @@ func (r *FungibleTokenTracker) worker( state.DefaultParameters()) accounts := environment.NewAccounts(txnState) storage := cadenceRuntime.NewStorage( - &migrations.AccountsAtreeLedger{Accounts: accounts}, + &util.AccountsAtreeLedger{Accounts: accounts}, nil, ) diff --git a/cmd/util/ledger/util/migration_runtime_interface.go b/cmd/util/ledger/util/migration_runtime_interface.go new file mode 100644 index 00000000000..c72d8493095 --- /dev/null +++ b/cmd/util/ledger/util/migration_runtime_interface.go @@ -0,0 +1,295 @@ +package util + +import ( + "fmt" + "time" + + "go.opentelemetry.io/otel/attribute" + + "github.com/onflow/atree" + "github.com/onflow/cadence" + "github.com/onflow/cadence/runtime" + "github.com/onflow/cadence/runtime/common" + "github.com/onflow/cadence/runtime/interpreter" + + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/model/flow" +) + +// MigrationRuntimeInterface is a runtime interface that can be used in migrations. +type MigrationRuntimeInterface struct { + Accounts environment.Accounts + Programs *environment.Programs + + // GetOrLoadProgramFunc allows for injecting extra logic + GetOrLoadProgramFunc func(location runtime.Location, load func() (*interpreter.Program, error)) (*interpreter.Program, error) +} + +func (m MigrationRuntimeInterface) ResolveLocation( + identifiers []runtime.Identifier, + location runtime.Location, +) ([]runtime.ResolvedLocation, error) { + + addressLocation, isAddress := location.(common.AddressLocation) + + // if the location is not an address location, e.g. an identifier location (`import Crypto`), + // then return a single resolved location which declares all identifiers. + if !isAddress { + return []runtime.ResolvedLocation{ + { + Location: location, + Identifiers: identifiers, + }, + }, nil + } + + // if the location is an address, + // and no specific identifiers where requested in the import statement, + // then fetch all identifiers at this address + if len(identifiers) == 0 { + address := flow.Address(addressLocation.Address) + + contractNames, err := m.Accounts.GetContractNames(address) + if err != nil { + return nil, fmt.Errorf("ResolveLocation failed: %w", err) + } + + // if there are no contractNames deployed, + // then return no resolved locations + if len(contractNames) == 0 { + return nil, nil + } + + identifiers = make([]runtime.Identifier, len(contractNames)) + + for i := range identifiers { + identifiers[i] = runtime.Identifier{ + Identifier: contractNames[i], + } + } + } + + // return one resolved location per identifier. + // each resolved location is an address contract location + resolvedLocations := make([]runtime.ResolvedLocation, len(identifiers)) + for i := range resolvedLocations { + identifier := identifiers[i] + resolvedLocations[i] = runtime.ResolvedLocation{ + Location: common.AddressLocation{ + Address: addressLocation.Address, + Name: identifier.Identifier, + }, + Identifiers: []runtime.Identifier{identifier}, + } + } + + return resolvedLocations, nil +} + +func (m MigrationRuntimeInterface) GetCode(location runtime.Location) ([]byte, error) { + contractLocation, ok := location.(common.AddressLocation) + if !ok { + return nil, fmt.Errorf("GetCode failed: expected AddressLocation") + } + + add, err := m.Accounts.GetContract(contractLocation.Name, flow.Address(contractLocation.Address)) + if err != nil { + return nil, fmt.Errorf("GetCode failed: %w", err) + } + + return add, nil +} + +func (m MigrationRuntimeInterface) GetAccountContractCode( + l common.AddressLocation, +) (code []byte, err error) { + return m.Accounts.GetContract(l.Name, flow.Address(l.Address)) +} + +func (m MigrationRuntimeInterface) GetOrLoadProgram(location runtime.Location, load func() (*interpreter.Program, error)) (*interpreter.Program, error) { + if m.GetOrLoadProgramFunc != nil { + return m.GetOrLoadProgramFunc(location, load) + } + + return m.Programs.GetOrLoadProgram(location, load) +} + +func (m MigrationRuntimeInterface) MeterMemory(_ common.MemoryUsage) error { + return nil +} + +func (m MigrationRuntimeInterface) MeterComputation(_ common.ComputationKind, _ uint) error { + return nil +} + +func (m MigrationRuntimeInterface) GetValue(_, _ []byte) (value []byte, err error) { + panic("unexpected GetValue call") +} + +func (m MigrationRuntimeInterface) SetValue(_, _, _ []byte) (err error) { + panic("unexpected SetValue call") +} + +func (m MigrationRuntimeInterface) CreateAccount(_ runtime.Address) (address runtime.Address, err error) { + panic("unexpected CreateAccount call") +} + +func (m MigrationRuntimeInterface) AddEncodedAccountKey(_ runtime.Address, _ []byte) error { + panic("unexpected AddEncodedAccountKey call") +} + +func (m MigrationRuntimeInterface) RevokeEncodedAccountKey(_ runtime.Address, _ int) (publicKey []byte, err error) { + panic("unexpected RevokeEncodedAccountKey call") +} + +func (m MigrationRuntimeInterface) AddAccountKey(_ runtime.Address, _ *runtime.PublicKey, _ runtime.HashAlgorithm, _ int) (*runtime.AccountKey, error) { + panic("unexpected AddAccountKey call") +} + +func (m MigrationRuntimeInterface) GetAccountKey(_ runtime.Address, _ int) (*runtime.AccountKey, error) { + panic("unexpected GetAccountKey call") +} + +func (m MigrationRuntimeInterface) RevokeAccountKey(_ runtime.Address, _ int) (*runtime.AccountKey, error) { + panic("unexpected RevokeAccountKey call") +} + +func (m MigrationRuntimeInterface) UpdateAccountContractCode(_ common.AddressLocation, _ []byte) (err error) { + panic("unexpected UpdateAccountContractCode call") +} + +func (m MigrationRuntimeInterface) RemoveAccountContractCode(common.AddressLocation) (err error) { + panic("unexpected RemoveAccountContractCode call") +} + +func (m MigrationRuntimeInterface) GetSigningAccounts() ([]runtime.Address, error) { + panic("unexpected GetSigningAccounts call") +} + +func (m MigrationRuntimeInterface) ProgramLog(_ string) error { + panic("unexpected ProgramLog call") +} + +func (m MigrationRuntimeInterface) EmitEvent(_ cadence.Event) error { + panic("unexpected EmitEvent call") +} + +func (m MigrationRuntimeInterface) ValueExists(_, _ []byte) (exists bool, err error) { + panic("unexpected ValueExists call") +} + +func (m MigrationRuntimeInterface) GenerateUUID() (uint64, error) { + panic("unexpected GenerateUUID call") +} + +func (m MigrationRuntimeInterface) GetComputationLimit() uint64 { + panic("unexpected GetComputationLimit call") +} + +func (m MigrationRuntimeInterface) SetComputationUsed(_ uint64) error { + panic("unexpected SetComputationUsed call") +} + +func (m MigrationRuntimeInterface) DecodeArgument(_ []byte, _ cadence.Type) (cadence.Value, error) { + panic("unexpected DecodeArgument call") +} + +func (m MigrationRuntimeInterface) GetCurrentBlockHeight() (uint64, error) { + panic("unexpected GetCurrentBlockHeight call") +} + +func (m MigrationRuntimeInterface) GetBlockAtHeight(_ uint64) (block runtime.Block, exists bool, err error) { + panic("unexpected GetBlockAtHeight call") +} + +func (m MigrationRuntimeInterface) ReadRandom([]byte) error { + panic("unexpected ReadRandom call") +} + +func (m MigrationRuntimeInterface) VerifySignature(_ []byte, _ string, _ []byte, _ []byte, _ runtime.SignatureAlgorithm, _ runtime.HashAlgorithm) (bool, error) { + panic("unexpected VerifySignature call") +} + +func (m MigrationRuntimeInterface) Hash(_ []byte, _ string, _ runtime.HashAlgorithm) ([]byte, error) { + panic("unexpected Hash call") +} + +func (m MigrationRuntimeInterface) GetAccountBalance(_ common.Address) (value uint64, err error) { + panic("unexpected GetAccountBalance call") +} + +func (m MigrationRuntimeInterface) GetAccountAvailableBalance(_ common.Address) (value uint64, err error) { + panic("unexpected GetAccountAvailableBalance call") +} + +func (m MigrationRuntimeInterface) GetStorageUsed(_ runtime.Address) (value uint64, err error) { + panic("unexpected GetStorageUsed call") +} + +func (m MigrationRuntimeInterface) GetStorageCapacity(_ runtime.Address) (value uint64, err error) { + panic("unexpected GetStorageCapacity call") +} + +func (m MigrationRuntimeInterface) ImplementationDebugLog(_ string) error { + panic("unexpected ImplementationDebugLog call") +} + +func (m MigrationRuntimeInterface) ValidatePublicKey(_ *runtime.PublicKey) error { + panic("unexpected ValidatePublicKey call") +} + +func (m MigrationRuntimeInterface) GetAccountContractNames(_ runtime.Address) ([]string, error) { + panic("unexpected GetAccountContractNames call") +} + +func (m MigrationRuntimeInterface) AllocateStorageIndex(_ []byte) (atree.StorageIndex, error) { + panic("unexpected AllocateStorageIndex call") +} + +func (m MigrationRuntimeInterface) ComputationUsed() (uint64, error) { + panic("unexpected ComputationUsed call") +} + +func (m MigrationRuntimeInterface) MemoryUsed() (uint64, error) { + panic("unexpected MemoryUsed call") +} + +func (m MigrationRuntimeInterface) InteractionUsed() (uint64, error) { + panic("unexpected InteractionUsed call") +} + +func (m MigrationRuntimeInterface) SetInterpreterSharedState(_ *interpreter.SharedState) { + panic("unexpected SetInterpreterSharedState call") +} + +func (m MigrationRuntimeInterface) GetInterpreterSharedState() *interpreter.SharedState { + panic("unexpected GetInterpreterSharedState call") +} + +func (m MigrationRuntimeInterface) AccountKeysCount(_ runtime.Address) (uint64, error) { + panic("unexpected AccountKeysCount call") +} + +func (m MigrationRuntimeInterface) BLSVerifyPOP(_ *runtime.PublicKey, _ []byte) (bool, error) { + panic("unexpected BLSVerifyPOP call") +} + +func (m MigrationRuntimeInterface) BLSAggregateSignatures(_ [][]byte) ([]byte, error) { + panic("unexpected BLSAggregateSignatures call") +} + +func (m MigrationRuntimeInterface) BLSAggregatePublicKeys(_ []*runtime.PublicKey) (*runtime.PublicKey, error) { + panic("unexpected BLSAggregatePublicKeys call") +} + +func (m MigrationRuntimeInterface) ResourceOwnerChanged(_ *interpreter.Interpreter, _ *interpreter.CompositeValue, _ common.Address, _ common.Address) { + panic("unexpected ResourceOwnerChanged call") +} + +func (m MigrationRuntimeInterface) GenerateAccountID(_ common.Address) (uint64, error) { + panic("unexpected GenerateAccountID call") +} + +func (m MigrationRuntimeInterface) RecordTrace(_ string, _ runtime.Location, _ time.Duration, _ []attribute.KeyValue) { + panic("unexpected RecordTrace call") +} diff --git a/cmd/util/ledger/util/nop_meter.go b/cmd/util/ledger/util/nop_meter.go new file mode 100644 index 00000000000..dc8a9c2ac6b --- /dev/null +++ b/cmd/util/ledger/util/nop_meter.go @@ -0,0 +1,49 @@ +package util + +import ( + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/meter" +) + +// NopMeter is a meter that does nothing. It can be used in migrations. +type NopMeter struct{} + +func (n NopMeter) ComputationAvailable(_ common.ComputationKind, _ uint) bool { + return false +} + +func (n NopMeter) MeterComputation(_ common.ComputationKind, _ uint) error { + return nil +} + +func (n NopMeter) ComputationUsed() (uint64, error) { + return 0, nil +} + +func (n NopMeter) ComputationIntensities() meter.MeteredComputationIntensities { + return meter.MeteredComputationIntensities{} +} + +func (n NopMeter) MeterMemory(_ common.MemoryUsage) error { + return nil +} + +func (n NopMeter) MemoryUsed() (uint64, error) { + return 0, nil +} + +func (n NopMeter) MeterEmittedEvent(_ uint64) error { + return nil +} + +func (n NopMeter) TotalEmittedEventBytes() uint64 { + return 0 +} + +func (n NopMeter) InteractionUsed() (uint64, error) { + return 0, nil +} + +var _ environment.Meter = NopMeter{} diff --git a/cmd/util/ledger/util/payload_grouping.go b/cmd/util/ledger/util/payload_grouping.go new file mode 100644 index 00000000000..9aec5d76efa --- /dev/null +++ b/cmd/util/ledger/util/payload_grouping.go @@ -0,0 +1,275 @@ +package util + +import ( + "bytes" + "fmt" + "sort" + "sync" + "time" + + "github.com/rs/zerolog" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +// encodedKeyAddressPrefixLength is the length of the address prefix in the encoded key +// 2 for uint16 of number of key parts +// 4 for uint32 of the length of the first key part +// 2 for uint16 of the key part type +// 8 for the address which is the actual length of the first key part +const encodedKeyAddressPrefixLength = 2 + 4 + 2 + flow.AddressLength + +// minSizeForSplitSortingIntoGoroutines below this size, no need to split +// the sorting into goroutines +const minSizeForSplitSortingIntoGoroutines = 100_000 + +const estimatedNumOfAccount = 30_000_000 + +// PayloadAccountGroup is a grouping of payloads by account +type PayloadAccountGroup struct { + Address common.Address + Payloads []*ledger.Payload +} + +// PayloadAccountGrouping is a grouping of payloads by account. +type PayloadAccountGrouping struct { + payloads sortablePayloads + indexes []int + + current int +} + +// Next returns the next account group. If there is no more account group, it returns nil. +// The zero address is used for global Payloads and is not an actual account. +func (g *PayloadAccountGrouping) Next() (*PayloadAccountGroup, error) { + if g.current == len(g.indexes) { + // reached the end + return nil, nil + } + + accountStartIndex := g.indexes[g.current] + accountEndIndex := len(g.payloads) + if g.current != len(g.indexes)-1 { + accountEndIndex = g.indexes[g.current+1] + } + g.current++ + + address, err := payloadToAddress(g.payloads[accountStartIndex]) + if err != nil { + return nil, fmt.Errorf("failed to get address from payload: %w", err) + } + + return &PayloadAccountGroup{ + Address: address, + Payloads: g.payloads[accountStartIndex:accountEndIndex], + }, nil +} + +// Len returns the number of accounts +func (g *PayloadAccountGrouping) Len() int { + return len(g.indexes) +} + +// AllPayloadsCount the number of payloads +func (g *PayloadAccountGrouping) AllPayloadsCount() int { + return len(g.payloads) +} + +// GroupPayloadsByAccount takes a list of payloads and groups them by account. +// it uses nWorkers to sort the payloads by address and find the start and end indexes of +// each account. +func GroupPayloadsByAccount( + log zerolog.Logger, + payloads []*ledger.Payload, + nWorkers int, +) *PayloadAccountGrouping { + if len(payloads) == 0 { + return &PayloadAccountGrouping{} + } + p := sortablePayloads(payloads) + + start := time.Now() + log.Info(). + Int("payloads", len(payloads)). + Int("workers", nWorkers). + Msg("Sorting payloads by address") + + // sort the payloads by address + sortPayloads(0, len(p), p, make(sortablePayloads, len(p)), nWorkers) + end := time.Now() + + log.Info(). + Int("payloads", len(payloads)). + Str("duration", end.Sub(start).Round(1*time.Second).String()). + Msg("Sorted. Finding account boundaries in sorted payloads") + + start = time.Now() + // find the indexes of the payloads that start a new account + indexes := make([]int, 0, estimatedNumOfAccount) + for i := 0; i < len(p); { + indexes = append(indexes, i) + i = p.FindNextKeyIndex(i) + } + end = time.Now() + + log.Info(). + Int("accounts", len(indexes)). + Str("duration", end.Sub(start).Round(1*time.Second).String()). + Msg("Done grouping payloads by account") + + return &PayloadAccountGrouping{ + payloads: p, + indexes: indexes, + } +} + +// payloadToAddress takes a payload and return: +// - (address, nil) if the payload is for an account, the account address is returned +// - (common.ZeroAddress, nil) if the payload is not for an account +// - (common.ZeroAddress, err) if running into any exception +// The zero address is used for global Payloads and is not an actual account +func payloadToAddress(p *ledger.Payload) (common.Address, error) { + k, err := p.Key() + if err != nil { + return common.ZeroAddress, fmt.Errorf("could not find key for payload: %w", err) + } + + id, err := convert.LedgerKeyToRegisterID(k) + if err != nil { + return common.ZeroAddress, fmt.Errorf("error converting key to register ID") + } + + if len([]byte(id.Owner)) != flow.AddressLength { + return common.ZeroAddress, nil + } + + address, err := common.BytesToAddress([]byte(id.Owner)) + if err != nil { + return common.ZeroAddress, fmt.Errorf("invalid account address: %w", err) + } + + return address, nil +} + +type sortablePayloads []*ledger.Payload + +func (s sortablePayloads) Len() int { + return len(s) +} + +func (s sortablePayloads) Less(i, j int) bool { + return s.Compare(i, j) < 0 +} + +func (s sortablePayloads) Compare(i, j int) int { + // sort descending to force one of the big accounts to be more at the beginning + return bytes.Compare( + s[j].EncodedKey()[:encodedKeyAddressPrefixLength], + s[i].EncodedKey()[:encodedKeyAddressPrefixLength], + ) +} + +func (s sortablePayloads) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +func (s sortablePayloads) FindNextKeyIndex(i int) int { + low := i + step := 1 + for low+step < len(s) && s.Compare(low+step, i) == 0 { + low += step + step *= 2 + } + + high := low + step + if high > len(s) { + high = len(s) + } + + for low < high { + mid := (low + high) / 2 + if s.Compare(mid, i) == 0 { + low = mid + 1 + } else { + high = mid + } + } + + return low +} + +// sortPayloads sorts the payloads in the range [i, j) using goroutines and merges +// the results using the intermediate buffer. The goroutine allowance is the number +// of goroutines that can be used for sorting. If the allowance is less than 2, +// the payloads are sorted using the built-in sort. +// The buffer must be of the same length as the source and can be disposed after. +func sortPayloads(i, j int, source, buffer sortablePayloads, goroutineAllowance int) { + // if the length is less than 2, no need to sort + if j-i <= 1 { + return + } + + // if we are out of goroutine allowance, sort with built-in sort + // if the length is less than minSizeForSplit, sort with built-in sort + if goroutineAllowance < 2 || j-i < minSizeForSplitSortingIntoGoroutines { + sort.Sort(source[i:j]) + return + } + + goroutineAllowance -= 2 + allowance1 := goroutineAllowance / 2 + allowance2 := goroutineAllowance - allowance1 + mid := (i + j) / 2 + + wg := sync.WaitGroup{} + wg.Add(2) + go func() { + sortPayloads(i, mid, source, buffer, allowance1) + wg.Done() + }() + go func() { + sortPayloads(mid, j, source, buffer, allowance2) + wg.Done() + }() + wg.Wait() + + mergeInto(source, buffer, i, mid, j) +} + +func mergeInto(source, buffer sortablePayloads, i int, mid int, j int) { + left := i + right := mid + k := i + for left < mid && right < j { + // More elements in the both partitions to process. + if source.Compare(left, right) <= 0 { + // Move left partition elements with the same address to buffer. + nextLeft := source.FindNextKeyIndex(left) + n := copy(buffer[k:], source[left:nextLeft]) + left = nextLeft + k += n + } else { + // Move right partition elements with the same address to buffer. + nextRight := source.FindNextKeyIndex(right) + n := copy(buffer[k:], source[right:nextRight]) + right = nextRight + k += n + } + } + // At this point: + // - one partition is exhausted. + // - remaining elements in the other partition (already sorted) can be copied over. + if left < mid { + // Copy remaining elements in the left partition. + copy(buffer[k:], source[left:mid]) + } else { + // Copy remaining elements in the right partition. + copy(buffer[k:], source[right:j]) + } + // Copy merged buffer back to source. + copy(source[i:j], buffer[i:j]) +} diff --git a/cmd/util/ledger/util/payload_grouping_test.go b/cmd/util/ledger/util/payload_grouping_test.go new file mode 100644 index 00000000000..96b50bd4e5b --- /dev/null +++ b/cmd/util/ledger/util/payload_grouping_test.go @@ -0,0 +1,155 @@ +package util_test + +import ( + "crypto/rand" + "encoding/hex" + rand2 "math/rand" + "runtime" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/cmd/util/ledger/util" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +func TestGroupPayloadsByAccount(t *testing.T) { + log := zerolog.New(zerolog.NewTestWriter(t)) + payloads := generateRandomPayloads(1000000) + tmp := make([]*ledger.Payload, len(payloads)) + copy(tmp, payloads) + + groups := util.GroupPayloadsByAccount(log, payloads, 0) + + require.Greater(t, groups.Len(), 1) +} + +func TestGroupPayloadsByAccountCompareResults(t *testing.T) { + log := zerolog.Nop() + payloads := generateRandomPayloads(1000000) + tmp1 := make([]*ledger.Payload, len(payloads)) + tmp2 := make([]*ledger.Payload, len(payloads)) + copy(tmp1, payloads) + copy(tmp2, payloads) + + groups1 := util.GroupPayloadsByAccount(log, tmp1, 0) + groups2 := util.GroupPayloadsByAccount(log, tmp2, runtime.NumCPU()) + + groups3 := map[common.Address][]*ledger.Payload{} + for _, payload := range payloads { + key, err := payload.Key() + require.NoError(t, err) + registerID, err := convert.LedgerKeyToRegisterID(key) + require.NoError(t, err) + address, err := common.BytesToAddress([]byte(registerID.Owner)) + require.NoError(t, err) + if _, ok := groups3[address]; !ok { + groups3[address] = []*ledger.Payload{} + } + groups3[address] = append(groups3[address], payload) + } + + require.Equal(t, groups1.Len(), groups2.Len()) + require.Equal(t, groups1.Len(), len(groups3)) + for { + group1, err1 := groups1.Next() + group2, err2 := groups2.Next() + + require.NoError(t, err1) + require.NoError(t, err2) + + if group1 == nil { + require.Nil(t, group2) + break + } + + require.Equal(t, group1.Address, group2.Address) + require.Equal(t, len(group1.Payloads), len(group2.Payloads)) + require.ElementsMatch(t, group1.Payloads, group2.Payloads) + require.Equal(t, len(group1.Payloads), len(groups3[group1.Address])) + require.ElementsMatch(t, group1.Payloads, groups3[group1.Address]) + } +} + +func BenchmarkGroupPayloadsByAccount(b *testing.B) { + log := zerolog.Nop() + payloads := generateRandomPayloads(10000000) + tmp := make([]*ledger.Payload, len(payloads)) + + bench := func(b *testing.B, nWorker int) func(b *testing.B) { + return func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + copy(tmp, payloads) + b.StartTimer() + util.GroupPayloadsByAccount(log, tmp, nWorker) + } + } + } + + b.Run("1 worker", bench(b, 1)) + b.Run("2 worker", bench(b, 2)) + b.Run("4 worker", bench(b, 4)) + b.Run("8 worker", bench(b, 8)) + b.Run("max worker", bench(b, runtime.NumCPU())) +} + +// GeneratePayloads generates n random payloads +// with a random number of payloads per account (exponentially distributed) +func generateRandomPayloads(n int) []*ledger.Payload { + const meanPayloadsPerAccount = 100 + const minPayloadsPerAccount = 1 + + payloads := make([]*ledger.Payload, 0, n) + + for i := 0; i < n; { + + registersForAccount := minPayloadsPerAccount + int(rand2.ExpFloat64()*(meanPayloadsPerAccount-minPayloadsPerAccount)) + if registersForAccount > n-i { + registersForAccount = n - i + } + i += registersForAccount + + accountKey := generateRandomAccountKey() + for j := 0; j < registersForAccount; j++ { + payloads = append(payloads, + ledger.NewPayload( + accountKey, + []byte(generateRandomString(10)), + )) + } + } + + return payloads +} + +func generateRandomAccountKey() ledger.Key { + return convert.RegisterIDToLedgerKey(flow.RegisterID{ + Owner: generateRandomAddress(), + Key: generateRandomString(10), + }) +} + +func generateRandomString(i int) string { + buf := make([]byte, i) + _, err := rand.Read(buf) + if err != nil { + panic(err) + } + return hex.EncodeToString(buf) +} + +func generateRandomAddress() string { + buf := make([]byte, flow.AddressLength) + _, err := rand.Read(buf) + if err != nil { + panic(err) + } + return string(buf) +} diff --git a/cmd/util/ledger/util/util.go b/cmd/util/ledger/util/util.go new file mode 100644 index 00000000000..46cc54e6850 --- /dev/null +++ b/cmd/util/ledger/util/util.go @@ -0,0 +1,154 @@ +package util + +import ( + "fmt" + + "github.com/onflow/atree" + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/storage/snapshot" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/ledger/common/convert" + "github.com/onflow/flow-go/model/flow" +) + +type AccountsAtreeLedger struct { + Accounts environment.Accounts +} + +func NewAccountsAtreeLedger(accounts environment.Accounts) *AccountsAtreeLedger { + return &AccountsAtreeLedger{Accounts: accounts} +} + +var _ atree.Ledger = &AccountsAtreeLedger{} + +func (a *AccountsAtreeLedger) GetValue(owner, key []byte) ([]byte, error) { + v, err := a.Accounts.GetValue( + flow.NewRegisterID( + flow.BytesToAddress(owner), + string(key))) + if err != nil { + return nil, fmt.Errorf("getting value failed: %w", err) + } + return v, nil +} + +func (a *AccountsAtreeLedger) SetValue(owner, key, value []byte) error { + err := a.Accounts.SetValue( + flow.NewRegisterID( + flow.BytesToAddress(owner), + string(key)), + value) + if err != nil { + return fmt.Errorf("setting value failed: %w", err) + } + return nil +} + +func (a *AccountsAtreeLedger) ValueExists(owner, key []byte) (exists bool, err error) { + v, err := a.GetValue(owner, key) + if err != nil { + return false, fmt.Errorf("checking value existence failed: %w", err) + } + + return len(v) > 0, nil +} + +// AllocateStorageIndex allocates new storage index under the owner accounts to store a new register +func (a *AccountsAtreeLedger) AllocateStorageIndex(owner []byte) (atree.StorageIndex, error) { + v, err := a.Accounts.AllocateStorageIndex(flow.BytesToAddress(owner)) + if err != nil { + return atree.StorageIndex{}, fmt.Errorf("storage index allocation failed: %w", err) + } + return v, nil +} + +type PayloadSnapshot struct { + Payloads map[flow.RegisterID]*ledger.Payload +} + +var _ snapshot.StorageSnapshot = (*PayloadSnapshot)(nil) + +func NewPayloadSnapshot(payloads []*ledger.Payload) (*PayloadSnapshot, error) { + l := &PayloadSnapshot{ + Payloads: make(map[flow.RegisterID]*ledger.Payload, len(payloads)), + } + for _, payload := range payloads { + key, err := payload.Key() + if err != nil { + return nil, err + } + id, err := convert.LedgerKeyToRegisterID(key) + if err != nil { + return nil, err + } + l.Payloads[id] = payload + } + return l, nil +} + +func (p PayloadSnapshot) Get(id flow.RegisterID) (flow.RegisterValue, error) { + value, exists := p.Payloads[id] + if !exists { + return nil, nil + } + return value.Value(), nil +} + +// NopMemoryGauge is a no-op implementation of the MemoryGauge interface +type NopMemoryGauge struct{} + +func (n NopMemoryGauge) MeterMemory(common.MemoryUsage) error { + return nil +} + +var _ common.MemoryGauge = (*NopMemoryGauge)(nil) + +type PayloadsReadonlyLedger struct { + Snapshot *PayloadSnapshot + + AllocateStorageIndexFunc func(owner []byte) (atree.StorageIndex, error) + SetValueFunc func(owner, key, value []byte) (err error) +} + +func (p *PayloadsReadonlyLedger) GetValue(owner, key []byte) (value []byte, err error) { + v, err := p.Snapshot.Get(flow.NewRegisterID(flow.BytesToAddress(owner), string(key))) + if err != nil { + return nil, fmt.Errorf("getting value failed: %w", err) + } + return v, nil +} + +func (p *PayloadsReadonlyLedger) SetValue(owner, key, value []byte) (err error) { + if p.SetValueFunc != nil { + return p.SetValueFunc(owner, key, value) + } + + panic("SetValue not expected to be called") +} + +func (p *PayloadsReadonlyLedger) ValueExists(owner, key []byte) (exists bool, err error) { + _, ok := p.Snapshot.Payloads[flow.NewRegisterID(flow.BytesToAddress(owner), string(key))] + return ok, nil +} + +func (p *PayloadsReadonlyLedger) AllocateStorageIndex(owner []byte) (atree.StorageIndex, error) { + if p.AllocateStorageIndexFunc != nil { + return p.AllocateStorageIndexFunc(owner) + } + + panic("AllocateStorageIndex not expected to be called") +} + +func NewPayloadsReadonlyLedger(snapshot *PayloadSnapshot) *PayloadsReadonlyLedger { + return &PayloadsReadonlyLedger{Snapshot: snapshot} +} + +// IsServiceLevelAddress returns true if the given address is the service level address. +// Which means it's not an actual account but instead holds service lever registers. +func IsServiceLevelAddress(address common.Address) bool { + return address == common.ZeroAddress +} + +var _ atree.Ledger = &PayloadsReadonlyLedger{} diff --git a/config/default-config.yml b/config/default-config.yml index 5c783bc1d63..620b3c8361c 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -1,4 +1,8 @@ config-file: "./default-config.yml" +# WARNING: Only modify the network configurations below if you fully understand their implications. +# Incorrect settings may lead to system instability, security vulnerabilities, or degraded performance. +# Make changes with caution and refer to the documentation for guidance. +# Network configuration. network-config: # Network Configuration # Connection pruning determines whether connections to nodes @@ -176,12 +180,6 @@ network-config: tracker-cache-decay: 0.99 # The upper bound on the amount of cluster prefixed control messages that will be processed hard-threshold: 100 - metrics: - # RPC metrics observer inspector configs - # The number of metrics inspector pool workers - workers: 1 - # The size of the queue used by worker pool for the control message metrics inspector - cache-size: 100 rpc-tracer: # The default interval at which the mesh tracer logs the mesh topology. This is used for debugging and forensics purposes. # Note that we purposefully choose this logging interval high enough to avoid spamming the logs. Moreover, the @@ -201,38 +199,331 @@ network-config: # Peer scoring is the default value for enabling peer scoring peer-scoring-enabled: true scoring-parameters: - app-specific-score: - # number of workers that asynchronously update the app specific score requests when they are expired. - score-update-worker-num: 5 - # size of the queue used by the worker pool for the app specific score update requests. The queue is used to buffer the app specific score update requests - # before they are processed by the worker pool. The queue size must be larger than total number of peers in the network. - # The queue is deduplicated based on the peer ids ensuring that there is only one app specific score update request per peer in the queue. - score-update-request-queue-size: 10_000 - # score ttl is the time to live for the app specific score. Once the score is expired; a new request will be sent to the app specific score provider to update the score. - # until the score is updated, the previous score will be used. - score-ttl: 1m - spam-record-cache: - # size of cache used to track spam records at gossipsub. Each peer id is mapped to a spam record that keeps track of the spam score for that peer. - # cache should be big enough to keep track of the entire network's size. Otherwise, the local node's view of the network will be incomplete due to cache eviction. - cache-size: 10_000 - # Threshold level for spam record penalty. - # At each evaluation period, when a node's penalty is below this value, the decay rate slows down, ensuring longer decay periods for malicious nodes and quicker decay for honest ones. - penalty-decay-slowdown-threshold: -99 - # This setting adjusts the decay rate when a node's penalty falls below the threshold. - # The decay rate, ranging between 0 and 1, dictates how quickly penalties decrease: a higher rate results in slower decay. - # The decay calculation is multiplicative (newPenalty = decayRate * oldPenalty). - # The reduction factor increases the decay rate, thus decelerating the penalty reduction. For instance, with a 0.01 reduction factor, - # the decay rate increases by 0.01 at each evaluation interval when the penalty is below the threshold. - # Consequently, a decay rate of `x` diminishes the penalty to zero more rapidly than a rate of `x+0.01`. - penalty-decay-rate-reduction-factor: 0.01 - # Defines the frequency for evaluating and potentially adjusting the decay process of a spam record. - # At each interval, the system assesses the current penalty of a node. - # If this penalty is below the defined threshold, the decay rate is modified according to the reduction factor, slowing down the penalty reduction process. - # This reassessment at regular intervals ensures that the decay rate is dynamically adjusted to reflect the node's ongoing behavior, - # maintaining a balance between penalizing malicious activity and allowing recovery for honest nodes. - penalty-decay-evaluation-period: 10m - # the intervals at which counters associated with a peer behavior in gossipsub system are decayed. - decay-interval: 1m + peer-scoring: + internal: + # The weight for app-specific scores. + # It is used to scale the app-specific scores to the same range as the other scores. + # At the current version, we don't distinguish between the app-specific scores + # and the other scores, so we set it to 1. + app-specific-score-weight: 1 + # The default decay interval for the overall score of a peer at the GossipSub scoring + # system. We set it to 1 minute so that it is not too short so that a malicious node can recover from a penalty + # and is not too long so that a well-behaved node can't recover from a penalty. + decay-interval: 1m + # The default decay to zero for the overall score of a peer at the GossipSub scoring system. + # It defines the maximum value below which a peer scoring counter is reset to zero. + # This is to prevent the counter from decaying to a very small value. + # The default value is 0.01, which means that a counter will be reset to zero if it decays to 0.01. + # When a counter hits the DecayToZero threshold, it means that the peer did not exhibit the behavior + # for a long time, and we can reset the counter. + decay-to-zero: 0.01 + topic: + # This is the default value for the skip atomic validation flag for topics. + # We set it to true, which means gossipsub parameter validation will not fail if we leave some of the + # topic parameters at their default values, i.e., zero. This is because we are not setting all + # topic parameters at the current implementation. + skip-atomic-validation: true + # This value is applied to the square of the number of invalid message deliveries on a topic. + # It is used to penalize peers that send invalid messages. By an invalid message, we mean a message that is not signed by the + # publisher, or a message that is not signed by the peer that sent it. We set it to -1.0, which means that with around 14 invalid + # message deliveries within a gossipsub heartbeat interval, the peer will be disconnected. + # The supporting math is as follows: + # - each staked (i.e., authorized) peer is rewarded by the fixed reward of 100 (i.e., DefaultStakedIdentityReward). + # - x invalid message deliveries will result in a penalty of x^2 * DefaultTopicInvalidMessageDeliveriesWeight, i.e., -x^2. + # - the peer will be disconnected when its penalty reaches -100 (i.e., MaxAppSpecificPenalty). + # - so, the maximum number of invalid message deliveries that a peer can have before being disconnected is sqrt(200/DefaultTopicInvalidMessageDeliveriesWeight) ~ 14. + invalid-message-deliveries-weight: -1.0 + # The decay factor used to decay the number of invalid message deliveries. + # The total number of invalid message deliveries is multiplied by this factor at each heartbeat interval to + # decay the number of invalid message deliveries, and prevent the peer from being disconnected if it stops + # sending invalid messages. We set it to 0.99, which means that the number of invalid message deliveries will + # decay by 1% at each heartbeat interval. + # The decay heartbeats are defined by the heartbeat interval of the gossipsub scoring system, which is 1 Minute (defaultDecayInterval). + invalid-message-deliveries-decay: 0.99 + # The default time in mesh quantum for the GossipSub scoring system. It is used to gauge + # a discrete time interval for the time in mesh counter. We set it to 1 hour, which means that every one complete hour a peer is + # in a topic mesh, the time in mesh counter will be incremented by 1 and is counted towards the availability score of the peer in that topic mesh. + # The reason for setting it to 1 hour is that we want to reward peers that are in a topic mesh for a long time, and we want to avoid rewarding peers that + # are churners, i.e., peers that join and leave a topic mesh frequently. + time-in-mesh-quantum: 1h + # The default weight of a topic in the GossipSub scoring system. + # The overall score of a peer in a topic mesh is multiplied by the weight of the topic when calculating the overall score of the peer. + # We set it to 1.0, which means that the overall score of a peer in a topic mesh is not affected by the weight of the topic. + topic-weight: 1.0 + # This is applied to the number of actual message deliveries in a topic mesh + # at each decay interval (i.e., defaultDecayInterval). + # It is used to decay the number of actual message deliveries, and prevents past message + # deliveries from affecting the current score of the peer. + # As the decay interval is 1 minute, we set it to 0.5, which means that the number of actual message + # deliveries will decay by 50% at each decay interval. + mesh-message-deliveries-decay: 0.5 + # The maximum number of actual message deliveries in a topic + # mesh that is used to calculate the score of a peer in that topic mesh. + # We set it to 1000, which means that the maximum number of actual message deliveries in a + # topic mesh that is used to calculate the score of a peer in that topic mesh is 1000. + # This is to prevent the score of a peer in a topic mesh from being affected by a large number of actual + # message deliveries and also affect the score of the peer in other topic meshes. + # When the total delivered messages in a topic mesh exceeds this value, the score of the peer in that topic + # mesh will not be affected by the actual message deliveries in that topic mesh. + # Moreover, this does not allow the peer to accumulate a large number of actual message deliveries in a topic mesh + # and then start under-performing in that topic mesh without being penalized. + mesh-message-deliveries-cap: 1000 + # The threshold for the number of actual message deliveries in a + # topic mesh that is used to calculate the score of a peer in that topic mesh. + # If the number of actual message deliveries in a topic mesh is less than this value, + # the peer will be penalized by square of the difference between the actual message deliveries and the threshold, + # i.e., -w * (actual - threshold)^2 where `actual` and `threshold` are the actual message deliveries and the + # threshold, respectively, and `w` is the weight (i.e., defaultTopicMeshMessageDeliveriesWeight). + # We set it to 0.1 * defaultTopicMeshMessageDeliveriesCap, which means that if a peer delivers less tha 10% of the + # maximum number of actual message deliveries in a topic mesh, it will be considered as an under-performing peer + # in that topic mesh. + mesh-message-deliveries-threshold: 100 + # The weight for applying penalty when a peer is under-performing in a topic mesh. + # Upon every decay interval, if the number of actual message deliveries is less than the topic mesh message deliveries threshold + # (i.e., defaultTopicMeshMessageDeliveriesThreshold), the peer will be penalized by square of the difference between the actual + # message deliveries and the threshold, multiplied by this weight, i.e., -w * (actual - threshold)^2 where w is the weight, and + # `actual` and `threshold` are the actual message deliveries and the threshold, respectively. + # We set this value to be - 0.05 MaxAppSpecificReward / (defaultTopicMeshMessageDeliveriesThreshold^2). This guarantees that even if a peer + # is not delivering any message in a topic mesh, it will not be disconnected. + # Rather, looses part of the MaxAppSpecificReward that is awarded by our app-specific scoring function to all staked + # nodes by default will be withdrawn, and the peer will be slightly penalized. In other words, under-performing in a topic mesh + # will drop the overall score of a peer by 5% of the MaxAppSpecificReward that is awarded by our app-specific scoring function. + # It means that under-performing in a topic mesh will not cause a peer to be disconnected, but it will cause the peer to lose + # its MaxAppSpecificReward that is awarded by our app-specific scoring function. + # At this point, we do not want to disconnect a peer only because it is under-performing in a topic mesh as it might be + # causing a false positive network partition. + mesh-deliveries-weight: -0.0005 + # The window size is time interval that we count a delivery of an already + # seen message towards the score of a peer in a topic mesh. The delivery is counted + # by GossipSub only if the previous sender of the message is different from the current sender. + # We set it to the decay interval of the GossipSub scoring system, which is 1 minute. + # It means that if a peer delivers a message that it has already seen less than one minute ago, + # the delivery will be counted towards the score of the peer in a topic mesh only if the previous sender of the message. + # This also prevents replay attacks of messages that are older than one minute. As replayed messages will not + # be counted towards the actual message deliveries of a peer in a topic mesh. + mesh-message-deliveries-window: 1m + # The time interval that we wait for a new peer that joins a topic mesh + # till start counting the number of actual message deliveries of that peer in that topic mesh. + # We set it to 2 * defaultDecayInterval, which means that we wait for 2 decay intervals before start counting + # the number of actual message deliveries of a peer in a topic mesh. + # With a default decay interval of 1 minute, it means that we wait for 2 minutes before start counting the + # number of actual message deliveries of a peer in a topic mesh. This is to account for + # the time that it takes for a peer to start up and receive messages from other peers in the topic mesh. + mesh-message-delivery-activation: 2m + thresholds: + # This is the threshold when a peer's penalty drops below this threshold, no gossip + # is emitted towards that peer and gossip from that peer is ignored. + # Validation Constraint: GossipThreshold >= PublishThreshold && GossipThreshold < 0 + # How we use it: As the current max penalty is -100, we set the threshold to -99 + # so that all gossips to and from peers with penalty -100 are ignored. + gossip: -99 + # This is the threshold when a peer's penalty drops below this threshold, + # self-published messages are not propagated towards this peer. + # Validation Constraint: + # PublishThreshold >= GraylistThreshold && PublishThreshold <= GossipThreshold && PublishThreshold < 0. + # How we use it: As the current max penalty is -100, we set the threshold to -99 + # so that all penalized peers are deprived of receiving any published messages. + publish: -99 + # This is the threshold when a peer's penalty drops below this threshold, + # the peer is graylisted, i.e., incoming RPCs from the peer are ignored. + # Validation Constraint: + # GraylistThreshold =< PublishThreshold && GraylistThreshold =< GossipThreshold && GraylistThreshold < 0 + # How we use it: As the current max penalty is -100, we set the threshold to -99 + # so that all penalized peers are graylisted. + graylist: -99 + # This is the threshold when a peer sends us PX information with a prune, + # we only accept it and connect to the supplied peers if the originating peer's + # penalty exceeds this threshold. + # Validation Constraint: must be non-negative. + # How we use it: As the current max reward is 100, we set the threshold to 99 + # so that we only receive supplied peers from well-behaved peers. + accept-px: 99 + # This is the threshold when the median peer penalty in the mesh drops + # below this value, the peer may select more peers with penalty above the median + # to opportunistically graft on the mesh. + # Validation Constraint: must be non-negative. + # How we use it: We set it to the -100 + 1 so that we only + # opportunistically graft peers that are not access nodes (i.e., with -1), + # or penalized peers (i.e., with -100). + opportunistic-graft: 101 + behaviour: + # The threshold when the behavior of a peer is considered as bad by GossipSub. + # Currently, the misbehavior is defined as advertising an iHave without responding to the iWants (iHave broken promises), as well as attempting + # on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh + # for a while, and the remote peer keep attempting on GRAFT (aka GRAFT flood). + # When the misbehavior counter of a peer goes beyond this threshold, the peer is penalized by defaultBehaviorPenaltyWeight (see below) for the excess misbehavior. + # + # An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + # For iHave broken promises, the gossipsub scoring works as follows: + # It samples ONLY A SINGLE iHave out of the entire RPC. + # If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + # + # We set it to 10, meaning that we at most tolerate 10 of such RPCs containing iHave broken promises. After that, the peer is penalized for every + # excess RPC containing iHave broken promises. + # The counter is also decayed by (0.99) every decay interval (defaultDecayInterval) i.e., every minute. + # Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through + # the ALSP system). + penalty-threshold: 1000 + # The weight for applying penalty when a peer misbehavior goes beyond the threshold. + # Misbehavior of a peer at gossipsub layer is defined as advertising an iHave without responding to the iWants (broken promises), as well as attempting + # on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh + # This is detected by the GossipSub scoring system, and the peer is penalized by defaultBehaviorPenaltyWeight. + # + # An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + # For iHave broken promises, the gossipsub scoring works as follows: + # It samples ONLY A SINGLE iHave out of the entire RPC. + # If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + # + # The penalty is applied to the square of the difference between the misbehavior counter and the threshold, i.e., -|w| * (misbehavior counter - threshold)^2. + # We set it to 0.01 * MaxAppSpecificPenalty, which means that misbehaving 10 times more than the threshold (i.e., 10 + 10) will cause the peer to lose + # its entire AppSpecificReward that is awarded by our app-specific scoring function to all staked (i.e., authorized) nodes by default. + # Moreover, as the MaxAppSpecificPenalty is -MaxAppSpecificReward, misbehaving sqrt(2) * 10 times more than the threshold will cause the peer score + # to be dropped below the MaxAppSpecificPenalty, which is also below the GraylistThreshold, and the peer will be graylisted (i.e., disconnected). + # + # The math is as follows: -|w| * (misbehavior - threshold)^2 = 0.01 * MaxAppSpecificPenalty * (misbehavior - threshold)^2 < 2 * MaxAppSpecificPenalty + # if misbehavior > threshold + sqrt(2) * 10. + # As shown above, with this choice of defaultBehaviorPenaltyWeight, misbehaving sqrt(2) * 10 times more than the threshold will cause the peer score + # to be dropped below the MaxAppSpecificPenalty, which is also below the GraylistThreshold, and the peer will be graylisted (i.e., disconnected). This weight + # is chosen in a way that with almost a few misbehaviors more than the threshold, the peer will be graylisted. The rationale relies on the fact that + # the misbehavior counter is incremented by 1 for each RPC containing one or more broken promises. Hence, it is per RPC, and not per broken promise. + # Having sqrt(2) * 10 broken promises RPC is a blatant misbehavior, and the peer should be graylisted. With decay interval of 1 minute, and decay value of + # 0.99 we expect a graylisted node due to borken promises to get back in about 527 minutes, i.e., (0.99)^x * (sqrt(2) * 10)^2 * MaxAppSpecificPenalty > GraylistThreshold + # where x is the number of decay intervals that the peer is graylisted. As MaxAppSpecificPenalty and GraylistThresholds are close, we can simplify the inequality + # to (0.99)^x * (sqrt(2) * 10)^2 > 1 --> (0.99)^x * 200 > 1 --> (0.99)^x > 1/200 --> x > log(1/200) / log(0.99) --> x > 527.17 decay intervals, i.e., 527 minutes. + # Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through + # the ALSP system that are reported by the engines). + penalty-weight: -0.01 + # The decay interval for the misbehavior counter of a peer. The misbehavior counter is + # incremented by GossipSub for iHave broken promises or the GRAFT flooding attacks (i.e., each GRAFT received from a remote peer while that peer is on a PRUNE backoff). + # + # An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + # For iHave broken promises, the gossipsub scoring works as follows: + # It samples ONLY A SINGLE iHave out of the entire RPC. + # If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + # This means that regardless of how many iHave broken promises an RPC contains, the misbehavior counter is incremented by 1. + # That is why we decay the misbehavior counter very slow, as this counter indicates a severe misbehavior. + # + # The misbehavior counter is decayed per decay interval (i.e., defaultDecayInterval = 1 minute) by GossipSub. + # We set it to 0.99, which means that the misbehavior counter is decayed by 1% per decay interval. + # With the generous threshold that we set (i.e., defaultBehaviourPenaltyThreshold = 10), we take the peers going beyond the threshold as persistent misbehaviors, + # We expect honest peers never to go beyond the threshold, and if they do, we expect them to go back below the threshold quickly. + # + # Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through + # the ALSP system that is based on the engines report). + penalty-decay: 0.5 + protocol: + # The max number of debug/trace log events per second. + # Logs emitted above this threshold are dropped. + max-debug-logs: 50 + application-specific: + # This is the maximum penalty for severe offenses that we apply + # to a remote node score. The score mechanism of GossipSub in Flow is designed + # in a way that all other infractions are penalized with a fraction of this value. + # We have also set the other parameters such as GraylistThreshold, + # GossipThreshold, and PublishThreshold to be a bit higher than this, + # i.e., -100 + 1. This ensures that a node with a score of + # -100 will be graylisted (i.e., all incoming and outgoing RPCs + # are rejected) and will not be able to publish or gossip any messages. + max-app-specific-penalty: -100 + min-app-specific-penalty: -1 + # This is the penalty for unknown identity. It is + # applied to the peer's score when the peer is not in the identity list. + unknown-identity-penalty: -100 + # This is the penalty for invalid subscription. + # It is applied to the peer's score when the peer subscribes to a topic that it is + # not authorized to subscribe to. + invalid-subscription-penalty: -100 + # This is the reward for well-behaving staked peers. + # If a peer does not have any misbehavior record, e.g., invalid subscription, + # invalid message, etc., it will be rewarded with this score. + max-app-specific-reward: 100 + # This is the reward for staking peers. It is applied + # to the peer's score when the peer does not have any misbehavior record, e.g., + # invalid subscription, invalid message, etc. The purpose is to reward the staking + # peers for their contribution to the network and prioritize them in neighbor selection. + staked-identity-reward: 100 + scoring-registry: + # Defines the duration of time, after the node startup, + # during which the scoring registry remains inactive before penalizing nodes. + # Throughout this startup silence period, the application-specific penalty + # returned for all nodes will be 0, and any invalid control message notifications + # will be ignored. This configuration allows nodes to stabilize and initialize before + # applying penalties or processing invalid control message notifications. + startup-silence-duration: 1h + app-specific-score: + # number of workers that asynchronously update the app specific score requests when they are expired. + score-update-worker-num: 5 + # size of the queue used by the worker pool for the app specific score update requests. The queue is used to buffer the app specific score update requests + # before they are processed by the worker pool. The queue size must be larger than total number of peers in the network. + # The queue is deduplicated based on the peer ids ensuring that there is only one app specific score update request per peer in the queue. + score-update-request-queue-size: 10_000 + # score ttl is the time to live for the app specific score. Once the score is expired; a new request will be sent to the app specific score provider to update the score. + # until the score is updated, the previous score will be used. + score-ttl: 1m + spam-record-cache: + # size of cache used to track spam records at gossipsub. Each peer id is mapped to a spam record that keeps track of the spam score for that peer. + # cache should be big enough to keep track of the entire network's size. Otherwise, the local node's view of the network will be incomplete due to cache eviction. + cache-size: 10_000 + decay: + # Threshold level for spam record penalty. + # At each evaluation period, when a node's penalty is below this value, the decay rate slows down, ensuring longer decay periods for malicious nodes and quicker decay for honest ones. + penalty-decay-slowdown-threshold: -99 + # This setting adjusts the decay rate when a node's penalty falls below the threshold. + # The decay rate, ranging between 0 and 1, dictates how quickly penalties decrease: a higher rate results in slower decay. + # The decay calculation is multiplicative (newPenalty = decayRate * oldPenalty). + # The reduction factor increases the decay rate, thus decelerating the penalty reduction. For instance, with a 0.01 reduction factor, + # the decay rate increases by 0.01 at each evaluation interval when the penalty is below the threshold. + # Consequently, a decay rate of `x` diminishes the penalty to zero more rapidly than a rate of `x+0.01`. + penalty-decay-rate-reduction-factor: 0.01 + # Defines the frequency for evaluating and potentially adjusting the decay process of a spam record. + # At each interval, the system assesses the current penalty of a node. + # If this penalty is below the defined threshold, the decay rate is modified according to the reduction factor, slowing down the penalty reduction process. + # This reassessment at regular intervals ensures that the decay rate is dynamically adjusted to reflect the node's ongoing behavior, + # maintaining a balance between penalizing malicious activity and allowing recovery for honest nodes. + penalty-decay-evaluation-period: 10m + # The minimum speed at which the spam penalty value of a peer is decayed. + # Spam record will be initialized with a decay value between .5 , .7 and this value will then be decayed up to .99 on consecutive misbehavior's, + # The maximum decay value decays the penalty by 1% every second. The decay is applied geometrically, i.e., `newPenalty = oldPenalty * decay`, hence, the higher decay value + # indicates a lower decay speed, i.e., it takes more heartbeat intervals to decay a penalty back to zero when the decay value is high. + # assume: + # penalty = -100 (the maximum application specific penalty is -100) + # skipDecayThreshold = -0.1 + # it takes around 459 seconds for the penalty to decay to reach greater than -0.1 and turn into 0. + # x * 0.99 ^ n > -0.1 (assuming negative x). + # 0.99 ^ n > -0.1 / x + # Now we can take the logarithm of both sides (with any base, but let's use base 10 for simplicity). + # log( 0.99 ^ n ) < log( 0.1 / x ) + # Using the properties of logarithms, we can bring down the exponent: + # n * log( 0.99 ) < log( -0.1 / x ) + # And finally, we can solve for n: + # n > log( -0.1 / x ) / log( 0.99 ) + # We can plug in x = -100: + # n > log( -0.1 / -100 ) / log( 0.99 ) + # n > log( 0.001 ) / log( 0.99 ) + # n > -3 / log( 0.99 ) + # n > 458.22 + minimum-spam-penalty-decay-factor: 0.99 + # The maximum rate at which the spam penalty value of a peer decays. Decay speeds increase + # during sustained malicious activity, leading to a slower recovery of the app-specific score for the penalized node. Conversely, + # decay speeds decrease, allowing faster recoveries, when nodes exhibit fleeting misbehavior. + maximum-spam-penalty-decay-factor: 0.8 + # The threshold for which when the negative penalty is above this value, the decay function will not be called. + # instead, the penalty will be set to 0. This is to prevent the penalty from keeping a small negative value for a long time. + skip-decay-threshold: -0.1 + misbehaviour-penalties: + # The penalty applied to the application specific penalty when a peer conducts a graft misbehaviour. + graft: -10 + # The penalty applied to the application specific penalty when a peer conducts a prune misbehaviour. + prune: -10 + # The penalty applied to the application specific penalty when a peer conducts a iHave misbehaviour. + ihave: -10 + # The penalty applied to the application specific penalty when a peer conducts a iWant misbehaviour. + iwant: -10 + # The penalty applied to the application specific penalty when a peer conducts a rpc publish message misbehaviour. + publish: -10 + # The factor used to reduce the penalty for control message misbehaviours on cluster prefixed topics. This allows a more lenient punishment for nodes + # that fall behind and may need to request old data. + cluster-prefixed-reduction-factor: 0.2 subscription-provider: # The interval for updating the list of subscribed peers to all topics in gossipsub. This is used to keep track of subscriptions # violations and penalize peers accordingly. Recommended value is in the order of a few minutes to avoid contentions; as the operation diff --git a/consensus/hotstuff/committees/static.go b/consensus/hotstuff/committees/static.go index b95c6448dff..3234233cc9e 100644 --- a/consensus/hotstuff/committees/static.go +++ b/consensus/hotstuff/committees/static.go @@ -7,7 +7,6 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/state/protocol" ) @@ -22,7 +21,7 @@ func NewStaticCommittee(participants flow.IdentityList, myID flow.Identifier, dk // NewStaticCommitteeWithDKG returns a new committee with a static participant set. func NewStaticCommitteeWithDKG(participants flow.IdentityList, myID flow.Identifier, dkg protocol.DKG) (*Static, error) { - valid := order.IdentityListCanonical(participants) + valid := flow.IsIdentityListCanonical(participants) if !valid { return nil, fmt.Errorf("participants %v is not in Canonical order", participants) } diff --git a/consensus/hotstuff/signature/block_signer_decoder_test.go b/consensus/hotstuff/signature/block_signer_decoder_test.go index c065e315add..76d13461ccc 100644 --- a/consensus/hotstuff/signature/block_signer_decoder_test.go +++ b/consensus/hotstuff/signature/block_signer_decoder_test.go @@ -11,7 +11,6 @@ import ( hotstuff "github.com/onflow/flow-go/consensus/hotstuff/mocks" "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/state" "github.com/onflow/flow-go/utils/unittest" @@ -32,7 +31,7 @@ type blockSignerDecoderSuite struct { func (s *blockSignerDecoderSuite) SetupTest() { // the default header fixture creates signerIDs for a committee of 10 nodes, so we prepare a committee same as that - s.allConsensus = unittest.IdentityListFixture(40, unittest.WithRole(flow.RoleConsensus)).Sort(order.Canonical) + s.allConsensus = unittest.IdentityListFixture(40, unittest.WithRole(flow.RoleConsensus)).Sort(flow.Canonical) // mock consensus committee s.committee = hotstuff.NewDynamicCommittee(s.T()) diff --git a/consensus/hotstuff/timeoutcollector/timeout_processor.go b/consensus/hotstuff/timeoutcollector/timeout_processor.go index 60f0e785359..5c959ea8b8f 100644 --- a/consensus/hotstuff/timeoutcollector/timeout_processor.go +++ b/consensus/hotstuff/timeoutcollector/timeout_processor.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/consensus/hotstuff/tracker" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/signature" ) @@ -266,8 +265,8 @@ func (p *TimeoutProcessor) buildTC() (*flow.TimeoutCertificate, error) { // we need to canonically order the respective `newestQCView`, so we can properly map signer to `newestQCView` after decoding. // sort data in canonical order - slices.SortFunc(signersData, func(lhs, rhs hotstuff.TimeoutSignerInfo) bool { - return order.IdentifierCanonical(lhs.Signer, rhs.Signer) + slices.SortFunc(signersData, func(lhs, rhs hotstuff.TimeoutSignerInfo) int { + return flow.IdentifierCanonical(lhs.Signer, rhs.Signer) }) // extract signers and data separately diff --git a/consensus/hotstuff/timeoutcollector/timeout_processor_test.go b/consensus/hotstuff/timeoutcollector/timeout_processor_test.go index b37188c5857..1042b1b15ef 100644 --- a/consensus/hotstuff/timeoutcollector/timeout_processor_test.go +++ b/consensus/hotstuff/timeoutcollector/timeout_processor_test.go @@ -22,7 +22,6 @@ import ( "github.com/onflow/flow-go/consensus/hotstuff/votecollector" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/local" msig "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/utils/unittest" @@ -55,7 +54,7 @@ func (s *TimeoutProcessorTestSuite) SetupTest() { s.validator = mocks.NewValidator(s.T()) s.sigAggregator = mocks.NewTimeoutSignatureAggregator(s.T()) s.notifier = mocks.NewTimeoutCollectorConsumer(s.T()) - s.participants = unittest.IdentityListFixture(11, unittest.WithWeight(s.sigWeight)).Sort(order.Canonical) + s.participants = unittest.IdentityListFixture(11, unittest.WithWeight(s.sigWeight)).Sort(flow.Canonical) s.signer = s.participants[0] s.view = (uint64)(rand.Uint32() + 100) s.totalWeight = *atomic.NewUint64(0) @@ -472,7 +471,7 @@ func TestTimeoutProcessor_BuildVerifyTC(t *testing.T) { signers[identity.NodeID] = verification.NewStakingSigner(me) }) // identities must be in canonical order - stakingSigners = stakingSigners.Sort(order.Canonical) + stakingSigners = stakingSigners.Sort(flow.Canonical) // utility function which generates a valid timeout for every signer createTimeouts := func(participants flow.IdentityList, view uint64, newestQC *flow.QuorumCertificate, lastViewTC *flow.TimeoutCertificate) []*model.TimeoutObject { diff --git a/consensus/integration/epoch_test.go b/consensus/integration/epoch_test.go index aa41de368fe..2a218fbd2b8 100644 --- a/consensus/integration/epoch_test.go +++ b/consensus/integration/epoch_test.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/flow/mapfunc" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/state/protocol/inmem" "github.com/onflow/flow-go/utils/unittest" ) @@ -218,7 +217,7 @@ func withNextEpoch( // convert to encodable representation for simple modification encodableSnapshot := snapshot.Encodable() - nextEpochIdentities = nextEpochIdentities.Sort(order.Canonical) + nextEpochIdentities = nextEpochIdentities.Sort(flow.Canonical) currEpoch := &encodableSnapshot.Epochs.Current // take pointer so assignments apply currEpoch.FinalView = currEpoch.FirstView + curEpochViews - 1 // first epoch lasts curEpochViews @@ -255,7 +254,7 @@ func withNextEpoch( nextEpochIdentities. Filter(filter.Not(filter.In(encodableSnapshot.Identities))). Map(mapfunc.WithWeight(0))..., - ).Sort(order.Canonical) + ).Sort(flow.Canonical) return inmem.SnapshotFromEncodable(encodableSnapshot) } diff --git a/consensus/integration/nodes_test.go b/consensus/integration/nodes_test.go index 8b8963cdc32..68e89fc6d4b 100644 --- a/consensus/integration/nodes_test.go +++ b/consensus/integration/nodes_test.go @@ -35,7 +35,6 @@ import ( "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/buffer" builder "github.com/onflow/flow-go/module/builder/consensus" @@ -257,7 +256,7 @@ func createRootBlockData(participantData *run.ParticipantData) (*flow.Block, *fl // add other roles to create a complete identity list participants := unittest.CompleteIdentitySet(consensusParticipants...) - participants.Sort(order.Canonical) + participants.Sort(flow.Canonical) dkgParticipantsKeys := make([]crypto.PublicKey, 0, len(consensusParticipants)) for _, participant := range participants.Filter(filter.HasRole(flow.RoleConsensus)) { @@ -289,7 +288,7 @@ func createRootBlockData(participantData *run.ParticipantData) (*flow.Block, *fl } func createPrivateNodeIdentities(n int) []bootstrap.NodeInfo { - consensus := unittest.IdentityListFixture(n, unittest.WithRole(flow.RoleConsensus)).Sort(order.Canonical) + consensus := unittest.IdentityListFixture(n, unittest.WithRole(flow.RoleConsensus)).Sort(flow.Canonical) infos := make([]bootstrap.NodeInfo, 0, n) for _, node := range consensus { networkPrivKey := unittest.NetworkingPrivKeyFixture() diff --git a/crypto/hash/empty.go b/crypto/hash/empty.go new file mode 100644 index 00000000000..7adc22fef06 --- /dev/null +++ b/crypto/hash/empty.go @@ -0,0 +1 @@ +package hash diff --git a/crypto/random/empty.go b/crypto/random/empty.go new file mode 100644 index 00000000000..eeab28a038e --- /dev/null +++ b/crypto/random/empty.go @@ -0,0 +1 @@ +package random diff --git a/engine/access/access_test.go b/engine/access/access_test.go index 39dbc155634..edbe7cfb038 100644 --- a/engine/access/access_test.go +++ b/engine/access/access_test.go @@ -984,7 +984,7 @@ func (suite *Suite) TestExecuteScript() { Log: suite.log, SnapshotHistoryLimit: backend.DefaultSnapshotHistoryLimit, Communicator: backend.NewNodeCommunicator(false), - ScriptExecutionMode: backend.ScriptExecutionModeExecutionNodesOnly, + ScriptExecutionMode: backend.IndexQueryModeExecutionNodesOnly, TxErrorMessagesCacheSize: 1000, }) require.NoError(suite.T(), err) diff --git a/engine/access/ingestion/engine.go b/engine/access/ingestion/engine.go index 54369ccebdb..d2349487eec 100644 --- a/engine/access/ingestion/engine.go +++ b/engine/access/ingestion/engine.go @@ -26,31 +26,39 @@ import ( "github.com/onflow/flow-go/utils/logging" ) -// time to wait for the all the missing collections to be received at node startup -const collectionCatchupTimeout = 30 * time.Second +const ( + // time to wait for the all the missing collections to be received at node startup + collectionCatchupTimeout = 30 * time.Second -// time to poll the storage to check if missing collections have been received -const collectionCatchupDBPollInterval = 10 * time.Millisecond + // time to poll the storage to check if missing collections have been received + collectionCatchupDBPollInterval = 10 * time.Millisecond -// time to update the FullBlockHeight index -const fullBlockUpdateInterval = 1 * time.Minute + // time to update the FullBlockHeight index + fullBlockRefreshInterval = 1 * time.Second -// a threshold of number of blocks with missing collections beyond which collections should be re-requested -// this is to prevent spamming the collection nodes with request -const missingCollsForBlkThreshold = 100 + // time to request missing collections from the network + missingCollsRequestInterval = 1 * time.Minute -// a threshold of block height beyond which collections should be re-requested (regardless of the number of blocks for which collection are missing) -// this is to ensure that if a collection is missing for a long time (in terms of block height) it is eventually re-requested -const missingCollsForAgeThreshold = 100 + // a threshold of number of blocks with missing collections beyond which collections should be re-requested + // this is to prevent spamming the collection nodes with request + missingCollsForBlkThreshold = 100 -// default queue capacity -const defaultQueueCapacity = 10_000 + // a threshold of block height beyond which collections should be re-requested (regardless of the number of blocks for which collection are missing) + // this is to ensure that if a collection is missing for a long time (in terms of block height) it is eventually re-requested + missingCollsForAgeThreshold = 100 -var defaultCollectionCatchupTimeout = collectionCatchupTimeout -var defaultCollectionCatchupDBPollInterval = collectionCatchupDBPollInterval -var defaultFullBlockUpdateInterval = fullBlockUpdateInterval -var defaultMissingCollsForBlkThreshold = missingCollsForBlkThreshold -var defaultMissingCollsForAgeThreshold = missingCollsForAgeThreshold + // default queue capacity + defaultQueueCapacity = 10_000 +) + +var ( + defaultCollectionCatchupTimeout = collectionCatchupTimeout + defaultCollectionCatchupDBPollInterval = collectionCatchupDBPollInterval + defaultFullBlockRefreshInterval = fullBlockRefreshInterval + defaultMissingCollsRequestInterval = missingCollsRequestInterval + defaultMissingCollsForBlkThreshold = missingCollsForBlkThreshold + defaultMissingCollsForAgeThreshold uint64 = missingCollsForAgeThreshold +) // Engine represents the ingestion engine, used to funnel data from other nodes // to a centralized location that can be queried by a user @@ -200,6 +208,8 @@ func (e *Engine) initLastFullBlockHeightIndex() error { if err != nil { return fmt.Errorf("failed to get root block: %w", err) } + + // insert is a noop if the index has already been initialized and no error is returned err = e.blocks.InsertLastFullBlockHeightIfNotExists(rootBlock.Height) if err != nil { return fmt.Errorf("failed to update last full block height during ingestion engine startup: %w", err) @@ -227,13 +237,30 @@ func (e *Engine) processBackground(ctx irrecoverable.SignalerContext, ready comp } ready() - ticker := time.NewTicker(defaultFullBlockUpdateInterval) + updateTicker := time.NewTicker(defaultFullBlockRefreshInterval) + defer updateTicker.Stop() + + requestTicker := time.NewTicker(defaultMissingCollsRequestInterval) + defer requestTicker.Stop() + for { select { case <-ctx.Done(): return - case <-ticker.C: - e.updateLastFullBlockReceivedIndex() + + // refresh the LastFullBlockReceived index + case <-updateTicker.C: + err := e.updateLastFullBlockReceivedIndex() + if err != nil { + ctx.Throw(err) + } + + // request missing collections from the network + case <-requestTicker.C: + err := e.checkMissingCollections() + if err != nil { + ctx.Throw(err) + } } } } @@ -312,8 +339,6 @@ func (e *Engine) processAvailableFinalizedBlocks(ctx context.Context) error { e.log.Error().Err(err).Hex("block_id", blockID[:]).Msg("failed to process block") continue } - - e.trackFinalizedMetricForBlock(hb) } } @@ -352,7 +377,7 @@ func (e *Engine) SubmitLocal(event interface{}) { // Submit submits the given event from the node with the given origin ID // for processing in a non-blocking manner. It returns instantly and logs // a potential processing error internally when done. -func (e *Engine) Submit(channel channels.Channel, originID flow.Identifier, event interface{}) { +func (e *Engine) Submit(_ channels.Channel, originID flow.Identifier, event interface{}) { err := e.process(originID, event) if err != nil { engine.LogError(e.log, err) @@ -366,7 +391,7 @@ func (e *Engine) ProcessLocal(event interface{}) error { // Process processes the given event from the node with the given origin ID in // a blocking manner. It returns the potential processing error when done. -func (e *Engine) Process(channel channels.Channel, originID flow.Identifier, event interface{}) error { +func (e *Engine) Process(_ channels.Channel, originID flow.Identifier, event interface{}) error { return e.process(originID, event) } @@ -419,20 +444,15 @@ func (e *Engine) processFinalizedBlock(blockID flow.Identifier) error { // queue requesting each of the collections from the collection node e.requestCollectionsInFinalizedBlock(block.Payload.Guarantees) + e.trackFinalizedMetricForBlock(block) + return nil } -func (e *Engine) trackFinalizedMetricForBlock(hb *model.Block) { - // TODO: consider using storage.Index.ByBlockID, the index contains collection id and seals ID - // retrieve the block - block, err := e.blocks.ByID(hb.BlockID) - if err != nil { - e.log.Warn().Err(err).Msg("could not track tx finalized metric: finalized block not found locally") - return - } - - // TODO lookup actual finalization time by looking at the block finalizing `b` +func (e *Engine) trackFinalizedMetricForBlock(block *flow.Block) { + // TODO: lookup actual finalization time by looking at the block finalizing `b` now := time.Now().UTC() + blockID := block.ID() // mark all transactions as finalized // TODO: sample to reduce performance overhead @@ -452,14 +472,14 @@ func (e *Engine) trackFinalizedMetricForBlock(hb *model.Block) { } } - if ti, found := e.blocksToMarkExecuted.ByID(hb.BlockID); found { + if ti, found := e.blocksToMarkExecuted.ByID(blockID); found { e.trackExecutedMetricForBlock(block, ti) e.metrics.UpdateExecutionReceiptMaxHeight(block.Header.Height) - e.blocksToMarkExecuted.Remove(hb.BlockID) + e.blocksToMarkExecuted.Remove(blockID) } } -func (e *Engine) handleExecutionReceipt(originID flow.Identifier, r *flow.ExecutionReceipt) error { +func (e *Engine) handleExecutionReceipt(_ flow.Identifier, r *flow.ExecutionReceipt) error { // persist the execution receipt locally, storing will also index the receipt err := e.executionReceipts.Store(r) if err != nil { @@ -513,17 +533,7 @@ func (e *Engine) trackExecutedMetricForBlock(block *flow.Block, ti time.Time) { } } -// handleCollection handles the response of the a collection request made earlier when a block was received -func (e *Engine) handleCollection(originID flow.Identifier, entity flow.Entity) error { - - // convert the entity to a strictly typed collection - collection, ok := entity.(*flow.Collection) - if !ok { - return fmt.Errorf("invalid entity type (%T)", entity) - } - - light := collection.Light() - +func (e *Engine) trackExecutedMetricForCollection(light *flow.LightCollection) { if ti, found := e.collectionsToMarkFinalized.ByID(light.ID()); found { for _, t := range light.Transactions { e.metrics.TransactionFinalized(t, ti) @@ -537,6 +547,20 @@ func (e *Engine) handleCollection(originID flow.Identifier, entity flow.Entity) } e.collectionsToMarkExecuted.Remove(light.ID()) } +} + +// handleCollection handles the response of the a collection request made earlier when a block was received +func (e *Engine) handleCollection(_ flow.Identifier, entity flow.Entity) error { + + // convert the entity to a strictly typed collection + collection, ok := entity.(*flow.Collection) + if !ok { + return fmt.Errorf("invalid entity type (%T)", entity) + } + + light := collection.Light() + + e.trackExecutedMetricForCollection(&light) // FIX: we can't index guarantees here, as we might have more than one block // with the same collection as long as it is not finalized @@ -613,7 +637,7 @@ func (e *Engine) requestMissingCollections(ctx context.Context) error { missingColls, err := e.missingCollectionsAtHeight(i) if err != nil { - return fmt.Errorf("failed to retreive missing collections by height %d during collection catchup: %w", i, err) + return fmt.Errorf("failed to retrieve missing collections by height %d during collection catchup: %w", i, err) } // request the missing collections @@ -654,14 +678,14 @@ func (e *Engine) requestMissingCollections(ctx context.Context) error { var foundColls []flow.Identifier // query db to find if collections are still missing - for collId := range missingCollMap { - found, err := e.lookupCollection(collId) + for collID := range missingCollMap { + found, err := e.haveCollection(collID) if err != nil { return err } // if collection found in local db, remove it from missingColls later if found { - foundColls = append(foundColls, collId) + foundColls = append(foundColls, collID) } } @@ -676,113 +700,128 @@ func (e *Engine) requestMissingCollections(ctx context.Context) error { return nil } -// updateLastFullBlockReceivedIndex keeps the FullBlockHeight index up to date and requests missing collections if -// the number of blocks missing collection have reached the defaultMissingCollsForBlkThreshold value. -// (The FullBlockHeight index indicates that block for which all collections have been received) -func (e *Engine) updateLastFullBlockReceivedIndex() { +// updateLastFullBlockReceivedIndex finds the next highest height where all previous collections +// have been indexed, and updates the LastFullBlockReceived index to that height +func (e *Engine) updateLastFullBlockReceivedIndex() error { + lastFullHeight, err := e.blocks.GetLastFullBlockHeight() + if err != nil { + return fmt.Errorf("failed to get last full block height: %w", err) + } - logError := func(err error) { - e.log.Error().Err(err).Msg("failed to update the last full block height") + finalBlk, err := e.state.Final().Head() + if err != nil { + return fmt.Errorf("failed to get finalized block: %w", err) } + finalizedHeight := finalBlk.Height - lastFullHeight, err := e.blocks.GetLastFullBlockHeight() + // track the latest contiguous full height + newLastFullHeight, err := e.lowestHeightWithMissingCollection(lastFullHeight, finalizedHeight) if err != nil { - if !errors.Is(err, storage.ErrNotFound) { - logError(err) - return + return fmt.Errorf("failed to find last full block received height: %w", err) + } + + // if more contiguous blocks are now complete, update db + if newLastFullHeight > lastFullHeight { + err = e.blocks.UpdateLastFullBlockHeight(newLastFullHeight) + if err != nil { + return fmt.Errorf("failed to update last full block height") } - // use the root height as the last full height - header, err := e.state.Params().FinalizedRoot() + + e.metrics.UpdateLastFullBlockHeight(newLastFullHeight) + + e.log.Debug(). + Uint64("last_full_block_height", newLastFullHeight). + Msg("updated LastFullBlockReceived index") + } + + return nil +} + +// lowestHeightWithMissingCollection returns the lowest height that is missing collections +func (e *Engine) lowestHeightWithMissingCollection(lastFullHeight, finalizedHeight uint64) (uint64, error) { + newLastFullHeight := lastFullHeight + + for i := lastFullHeight + 1; i <= finalizedHeight; i++ { + missingColls, err := e.missingCollectionsAtHeight(i) if err != nil { - logError(err) - return + return 0, err + } + + // return when we find the first block with missing collections + if len(missingColls) > 0 { + return newLastFullHeight, nil } - lastFullHeight = header.Height + + newLastFullHeight = i } - e.log.Debug().Uint64("last_full_block_height", lastFullHeight).Msg("updating LastFullBlockReceived index...") + return newLastFullHeight, nil +} + +// checkMissingCollections requests missing collections if the number of blocks missing collections +// have reached the defaultMissingCollsForBlkThreshold value. +func (e *Engine) checkMissingCollections() error { + lastFullHeight, err := e.blocks.GetLastFullBlockHeight() + if err != nil { + return err + } finalBlk, err := e.state.Final().Head() if err != nil { - logError(err) - return + return fmt.Errorf("failed to get finalized block: %w", err) } finalizedHeight := finalBlk.Height - // track number of incomplete blocks + // number of blocks with missing collections incompleteBlksCnt := 0 - // track the latest contiguous full height - latestFullHeight := lastFullHeight - // collect all missing collections var allMissingColls []*flow.CollectionGuarantee // start from the next block till we either hit the finalized block or cross the max collection missing threshold for i := lastFullHeight + 1; i <= finalizedHeight && incompleteBlksCnt < defaultMissingCollsForBlkThreshold; i++ { - - // find missing collections for block at height i missingColls, err := e.missingCollectionsAtHeight(i) if err != nil { - logError(err) - return + return fmt.Errorf("failed to find missing collections at height %d: %w", i, err) } - // if there are missing collections - if len(missingColls) > 0 { - - // increment number of incomplete blocks - incompleteBlksCnt++ - - // collect the missing collections for requesting later - allMissingColls = append(allMissingColls, missingColls...) - + if len(missingColls) == 0 { continue } - // if there are no missing collections so far, advance the latestFullHeight pointer - if incompleteBlksCnt == 0 { - latestFullHeight = i - } - } - - // if more contiguous blocks are now complete, update db - if latestFullHeight > lastFullHeight { - err = e.blocks.UpdateLastFullBlockHeight(latestFullHeight) - if err != nil { - logError(err) - return - } + incompleteBlksCnt++ - e.metrics.UpdateLastFullBlockHeight(lastFullHeight) + allMissingColls = append(allMissingColls, missingColls...) } - // additionally, if more than threshold blocks have missing collection OR collections are missing since defaultMissingCollsForAgeThreshold, re-request those collections - if incompleteBlksCnt >= defaultMissingCollsForBlkThreshold || (finalizedHeight-lastFullHeight) > uint64(defaultMissingCollsForAgeThreshold) { + // additionally, if more than threshold blocks have missing collections OR collections are + // missing since defaultMissingCollsForAgeThreshold, re-request those collections + if incompleteBlksCnt >= defaultMissingCollsForBlkThreshold || + (finalizedHeight-lastFullHeight) > defaultMissingCollsForAgeThreshold { // warn log since this should generally not happen e.log.Warn(). + Uint64("finalized_height", finalizedHeight). + Uint64("last_full_blk_height", lastFullHeight). Int("missing_collection_blk_count", incompleteBlksCnt). - Int("threshold", defaultMissingCollsForBlkThreshold). - Uint64("last_full_blk_height", latestFullHeight). + Int("missing_collection_count", len(allMissingColls)). Msg("re-requesting missing collections") e.requestCollectionsInFinalizedBlock(allMissingColls) } - e.log.Debug().Uint64("last_full_blk_height", latestFullHeight).Msg("updated LastFullBlockReceived index") + return nil } // missingCollectionsAtHeight returns all missing collection guarantees at a given height func (e *Engine) missingCollectionsAtHeight(h uint64) ([]*flow.CollectionGuarantee, error) { - blk, err := e.blocks.ByHeight(h) + block, err := e.blocks.ByHeight(h) if err != nil { - return nil, fmt.Errorf("failed to retreive block by height %d: %w", h, err) + return nil, fmt.Errorf("failed to retrieve block by height %d: %w", h, err) } var missingColls []*flow.CollectionGuarantee - for _, guarantee := range blk.Payload.Guarantees { - + for _, guarantee := range block.Payload.Guarantees { collID := guarantee.CollectionID - found, err := e.lookupCollection(collID) + found, err := e.haveCollection(collID) if err != nil { return nil, err } @@ -793,22 +832,21 @@ func (e *Engine) missingCollectionsAtHeight(h uint64) ([]*flow.CollectionGuarant return missingColls, nil } -// lookupCollection looks up the collection from the collection db with collID -func (e *Engine) lookupCollection(collId flow.Identifier) (bool, error) { - _, err := e.collections.LightByID(collId) +// haveCollection looks up the collection from the collection db with collID +func (e *Engine) haveCollection(collID flow.Identifier) (bool, error) { + _, err := e.collections.LightByID(collID) if err == nil { return true, nil } if errors.Is(err, storage.ErrNotFound) { return false, nil } - return false, fmt.Errorf("failed to retreive collection %s: %w", collId.String(), err) + return false, fmt.Errorf("failed to retrieve collection %s: %w", collID.String(), err) } // requestCollectionsInFinalizedBlock registers collection requests with the requester engine func (e *Engine) requestCollectionsInFinalizedBlock(missingColls []*flow.CollectionGuarantee) { for _, cg := range missingColls { - // TODO: move this query out of for loop? guarantors, err := protocol.FindGuarantors(e.state, cg) if err != nil { // failed to find guarantors for guarantees contained in a finalized block is fatal error diff --git a/engine/access/ingestion/engine_test.go b/engine/access/ingestion/engine_test.go index c4d0fb72141..b6753ecb144 100644 --- a/engine/access/ingestion/engine_test.go +++ b/engine/access/ingestion/engine_test.go @@ -64,73 +64,73 @@ func TestIngestEngine(t *testing.T) { suite.Run(t, new(Suite)) } -func (suite *Suite) TearDownTest() { - suite.cancel() +func (s *Suite) TearDownTest() { + s.cancel() } -func (suite *Suite) SetupTest() { +func (s *Suite) SetupTest() { log := zerolog.New(os.Stderr) + ctx, cancel := context.WithCancel(context.Background()) + s.cancel = cancel obsIdentity := unittest.IdentityFixture(unittest.WithRole(flow.RoleAccess)) // mock out protocol state - suite.proto.state = new(protocol.FollowerState) - suite.proto.snapshot = new(protocol.Snapshot) - suite.proto.params = new(protocol.Params) - suite.finalizedBlock = unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) - suite.proto.state.On("Identity").Return(obsIdentity, nil) - suite.proto.state.On("Final").Return(suite.proto.snapshot, nil) - suite.proto.state.On("Params").Return(suite.proto.params) - suite.proto.snapshot.On("Head").Return( + s.proto.state = new(protocol.FollowerState) + s.proto.snapshot = new(protocol.Snapshot) + s.proto.params = new(protocol.Params) + s.finalizedBlock = unittest.BlockHeaderFixture(unittest.WithHeaderHeight(0)) + s.proto.state.On("Identity").Return(obsIdentity, nil) + s.proto.state.On("Final").Return(s.proto.snapshot, nil) + s.proto.state.On("Params").Return(s.proto.params) + s.proto.snapshot.On("Head").Return( func() *flow.Header { - return suite.finalizedBlock + return s.finalizedBlock }, nil, ).Maybe() - suite.me = new(module.Local) - suite.me.On("NodeID").Return(obsIdentity.NodeID) + s.me = new(module.Local) + s.me.On("NodeID").Return(obsIdentity.NodeID) net := new(mocknetwork.Network) conduit := new(mocknetwork.Conduit) net.On("Register", channels.ReceiveReceipts, mock.Anything). Return(conduit, nil). Once() - suite.request = new(module.Requester) - - suite.provider = new(mocknetwork.Engine) - suite.blocks = new(storage.Blocks) - suite.headers = new(storage.Headers) - suite.collections = new(storage.Collections) - suite.transactions = new(storage.Transactions) - suite.receipts = new(storage.ExecutionReceipts) - suite.results = new(storage.ExecutionResults) + s.request = new(module.Requester) + + s.provider = new(mocknetwork.Engine) + s.blocks = new(storage.Blocks) + s.headers = new(storage.Headers) + s.collections = new(storage.Collections) + s.transactions = new(storage.Transactions) + s.receipts = new(storage.ExecutionReceipts) + s.results = new(storage.ExecutionResults) collectionsToMarkFinalized, err := stdmap.NewTimes(100) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) collectionsToMarkExecuted, err := stdmap.NewTimes(100) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) blocksToMarkExecuted, err := stdmap.NewTimes(100) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) - eng, err := New(log, net, suite.proto.state, suite.me, suite.request, suite.blocks, suite.headers, suite.collections, - suite.transactions, suite.results, suite.receipts, metrics.NewNoopCollector(), collectionsToMarkFinalized, collectionsToMarkExecuted, + eng, err := New(log, net, s.proto.state, s.me, s.request, s.blocks, s.headers, s.collections, + s.transactions, s.results, s.receipts, metrics.NewNoopCollector(), collectionsToMarkFinalized, collectionsToMarkExecuted, blocksToMarkExecuted) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) - suite.blocks.On("GetLastFullBlockHeight").Once().Return(uint64(0), errors.New("do nothing")) + s.blocks.On("GetLastFullBlockHeight").Once().Return(uint64(0), errors.New("do nothing")) - ctx, cancel := context.WithCancel(context.Background()) irrecoverableCtx, _ := irrecoverable.WithSignaler(ctx) eng.ComponentManager.Start(irrecoverableCtx) <-eng.Ready() - suite.eng = eng - suite.cancel = cancel + s.eng = eng } // TestOnFinalizedBlock checks that when a block is received, a request for each individual collection is made -func (suite *Suite) TestOnFinalizedBlock() { - suite.blocks.On("GetLastFullBlockHeight").Return(uint64(0), nil).Once() +func (s *Suite) TestOnFinalizedBlock() { + s.blocks.On("GetLastFullBlockHeight").Return(uint64(0), nil).Once() block := unittest.BlockFixture() block.SetPayload(unittest.PayloadFixture( @@ -146,7 +146,7 @@ func (suite *Suite) TestOnFinalizedBlock() { // guarantee signers must be cluster committee members, so that access will fetch collection from // the signers that are specified by guarantee.SignerIndices indices, err := signature.EncodeSignersToIndices(clusterCommittee.NodeIDs(), clusterCommittee.NodeIDs()) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) guarantee.SignerIndices = indices } @@ -155,15 +155,15 @@ func (suite *Suite) TestOnFinalizedBlock() { } // we should query the block once and index the guarantee payload once - suite.blocks.On("ByID", block.ID()).Return(&block, nil).Twice() + s.blocks.On("ByID", block.ID()).Return(&block, nil).Twice() for _, g := range block.Payload.Guarantees { collection := unittest.CollectionFixture(1) light := collection.Light() - suite.collections.On("LightByID", g.CollectionID).Return(&light, nil).Twice() + s.collections.On("LightByID", g.CollectionID).Return(&light, nil).Twice() } // expect that the block storage is indexed with each of the collection guarantee - suite.blocks.On("IndexBlockForCollections", block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() + s.blocks.On("IndexBlockForCollections", block.ID(), []flow.Identifier(flow.GetIDs(block.Payload.Guarantees))).Return(nil).Once() cluster := new(protocol.Cluster) cluster.On("Members").Return(clusterCommittee, nil) @@ -173,8 +173,8 @@ func (suite *Suite) TestOnFinalizedBlock() { epochs.On("Current").Return(epoch) snap := new(protocol.Snapshot) snap.On("Epochs").Return(epochs) - suite.proto.state.On("AtBlockID", refBlockID).Return(snap) - suite.results.On("Index", mock.Anything, mock.Anything).Return(nil) + s.proto.state.On("AtBlockID", refBlockID).Return(snap) + s.results.On("Index", mock.Anything, mock.Anything).Return(nil) // for each of the guarantees, we should request the corresponding collection once needed := make(map[flow.Identifier]struct{}) @@ -185,140 +185,140 @@ func (suite *Suite) TestOnFinalizedBlock() { wg := sync.WaitGroup{} wg.Add(4) - suite.request.On("EntityByID", mock.Anything, mock.Anything).Run( + s.request.On("EntityByID", mock.Anything, mock.Anything).Run( func(args mock.Arguments) { collID := args.Get(0).(flow.Identifier) _, pending := needed[collID] - suite.Assert().True(pending, "collection should be pending (%x)", collID) + s.Assert().True(pending, "collection should be pending (%x)", collID) delete(needed, collID) wg.Done() }, ) // process the block through the finalized callback - suite.eng.OnFinalizedBlock(&hotstuffBlock) - suite.Assertions.Eventually(func() bool { + s.eng.OnFinalizedBlock(&hotstuffBlock) + s.Assertions.Eventually(func() bool { wg.Wait() return true }, time.Millisecond*20, time.Millisecond) // assert that the block was retrieved and all collections were requested - suite.headers.AssertExpectations(suite.T()) - suite.request.AssertNumberOfCalls(suite.T(), "EntityByID", len(block.Payload.Guarantees)) - suite.request.AssertNumberOfCalls(suite.T(), "Index", len(block.Payload.Seals)) + s.headers.AssertExpectations(s.T()) + s.request.AssertNumberOfCalls(s.T(), "EntityByID", len(block.Payload.Guarantees)) + s.request.AssertNumberOfCalls(s.T(), "Index", len(block.Payload.Seals)) } // TestOnCollection checks that when a Collection is received, it is persisted -func (suite *Suite) TestOnCollection() { +func (s *Suite) TestOnCollection() { originID := unittest.IdentifierFixture() collection := unittest.CollectionFixture(5) light := collection.Light() // we should store the light collection and index its transactions - suite.collections.On("StoreLightAndIndexByTransaction", &light).Return(nil).Once() + s.collections.On("StoreLightAndIndexByTransaction", &light).Return(nil).Once() // for each transaction in the collection, we should store it needed := make(map[flow.Identifier]struct{}) for _, txID := range light.Transactions { needed[txID] = struct{}{} } - suite.transactions.On("Store", mock.Anything).Return(nil).Run( + s.transactions.On("Store", mock.Anything).Return(nil).Run( func(args mock.Arguments) { tx := args.Get(0).(*flow.TransactionBody) _, pending := needed[tx.ID()] - suite.Assert().True(pending, "tx not pending (%x)", tx.ID()) + s.Assert().True(pending, "tx not pending (%x)", tx.ID()) }, ) // process the block through the collection callback - suite.eng.OnCollection(originID, &collection) + s.eng.OnCollection(originID, &collection) // check that the collection was stored and indexed, and we stored all transactions - suite.collections.AssertExpectations(suite.T()) - suite.transactions.AssertNumberOfCalls(suite.T(), "Store", len(collection.Transactions)) + s.collections.AssertExpectations(s.T()) + s.transactions.AssertNumberOfCalls(s.T(), "Store", len(collection.Transactions)) } // TestExecutionReceiptsAreIndexed checks that execution receipts are properly indexed -func (suite *Suite) TestExecutionReceiptsAreIndexed() { +func (s *Suite) TestExecutionReceiptsAreIndexed() { originID := unittest.IdentifierFixture() collection := unittest.CollectionFixture(5) light := collection.Light() // we should store the light collection and index its transactions - suite.collections.On("StoreLightAndIndexByTransaction", &light).Return(nil).Once() + s.collections.On("StoreLightAndIndexByTransaction", &light).Return(nil).Once() block := &flow.Block{ Header: &flow.Header{Height: 0}, Payload: &flow.Payload{Guarantees: []*flow.CollectionGuarantee{}}, } - suite.blocks.On("ByID", mock.Anything).Return(block, nil) + s.blocks.On("ByID", mock.Anything).Return(block, nil) // for each transaction in the collection, we should store it needed := make(map[flow.Identifier]struct{}) for _, txID := range light.Transactions { needed[txID] = struct{}{} } - suite.transactions.On("Store", mock.Anything).Return(nil).Run( + s.transactions.On("Store", mock.Anything).Return(nil).Run( func(args mock.Arguments) { tx := args.Get(0).(*flow.TransactionBody) _, pending := needed[tx.ID()] - suite.Assert().True(pending, "tx not pending (%x)", tx.ID()) + s.Assert().True(pending, "tx not pending (%x)", tx.ID()) }, ) er1 := unittest.ExecutionReceiptFixture() er2 := unittest.ExecutionReceiptFixture() - suite.receipts.On("Store", mock.Anything).Return(nil) - suite.blocks.On("ByID", er1.ExecutionResult.BlockID).Return(nil, storerr.ErrNotFound) + s.receipts.On("Store", mock.Anything).Return(nil) + s.blocks.On("ByID", er1.ExecutionResult.BlockID).Return(nil, storerr.ErrNotFound) - suite.receipts.On("Store", mock.Anything).Return(nil) - suite.blocks.On("ByID", er2.ExecutionResult.BlockID).Return(nil, storerr.ErrNotFound) + s.receipts.On("Store", mock.Anything).Return(nil) + s.blocks.On("ByID", er2.ExecutionResult.BlockID).Return(nil, storerr.ErrNotFound) - err := suite.eng.handleExecutionReceipt(originID, er1) - require.NoError(suite.T(), err) + err := s.eng.handleExecutionReceipt(originID, er1) + require.NoError(s.T(), err) - err = suite.eng.handleExecutionReceipt(originID, er2) - require.NoError(suite.T(), err) + err = s.eng.handleExecutionReceipt(originID, er2) + require.NoError(s.T(), err) - suite.receipts.AssertExpectations(suite.T()) - suite.results.AssertExpectations(suite.T()) - suite.receipts.AssertExpectations(suite.T()) + s.receipts.AssertExpectations(s.T()) + s.results.AssertExpectations(s.T()) + s.receipts.AssertExpectations(s.T()) } // TestOnCollection checks that when a duplicate collection is received, the node doesn't // crash but just ignores its transactions. -func (suite *Suite) TestOnCollectionDuplicate() { +func (s *Suite) TestOnCollectionDuplicate() { originID := unittest.IdentifierFixture() collection := unittest.CollectionFixture(5) light := collection.Light() // we should store the light collection and index its transactions - suite.collections.On("StoreLightAndIndexByTransaction", &light).Return(storerr.ErrAlreadyExists).Once() + s.collections.On("StoreLightAndIndexByTransaction", &light).Return(storerr.ErrAlreadyExists).Once() // for each transaction in the collection, we should store it needed := make(map[flow.Identifier]struct{}) for _, txID := range light.Transactions { needed[txID] = struct{}{} } - suite.transactions.On("Store", mock.Anything).Return(nil).Run( + s.transactions.On("Store", mock.Anything).Return(nil).Run( func(args mock.Arguments) { tx := args.Get(0).(*flow.TransactionBody) _, pending := needed[tx.ID()] - suite.Assert().True(pending, "tx not pending (%x)", tx.ID()) + s.Assert().True(pending, "tx not pending (%x)", tx.ID()) }, ) // process the block through the collection callback - suite.eng.OnCollection(originID, &collection) + s.eng.OnCollection(originID, &collection) // check that the collection was stored and indexed, and we stored all transactions - suite.collections.AssertExpectations(suite.T()) - suite.transactions.AssertNotCalled(suite.T(), "Store", "should not store any transactions") + s.collections.AssertExpectations(s.T()) + s.transactions.AssertNotCalled(s.T(), "Store", "should not store any transactions") } // TestRequestMissingCollections tests that the all missing collections are requested on the call to requestMissingCollections -func (suite *Suite) TestRequestMissingCollections() { +func (s *Suite) TestRequestMissingCollections() { blkCnt := 3 startHeight := uint64(1000) @@ -349,14 +349,14 @@ func (suite *Suite) TestRequestMissingCollections() { // guarantee signers must be cluster committee members, so that access will fetch collection from // the signers that are specified by guarantee.SignerIndices indices, err := signature.EncodeSignersToIndices(clusterCommittee.NodeIDs(), clusterCommittee.NodeIDs()) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) c.SignerIndices = indices } } // setup the block storage mock // each block should be queried by height - suite.blocks.On("ByHeight", mock.IsType(uint64(0))).Return( + s.blocks.On("ByHeight", mock.IsType(uint64(0))).Return( func(h uint64) *flow.Block { // simulate a db lookup return heightMap[h] @@ -368,9 +368,9 @@ func (suite *Suite) TestRequestMissingCollections() { return storerr.ErrNotFound }) // consider collections are missing for all blocks - suite.blocks.On("GetLastFullBlockHeight").Return(startHeight-1, nil) + s.blocks.On("GetLastFullBlockHeight").Return(startHeight-1, nil) // consider the last test block as the head - suite.finalizedBlock = blocks[blkCnt-1].Header + s.finalizedBlock = blocks[blkCnt-1].Header // p is the probability of not receiving the collection before the next poll and it // helps simulate the slow trickle of the requested collections being received @@ -381,7 +381,7 @@ func (suite *Suite) TestRequestMissingCollections() { // for the first lookup call for each collection, it will be reported as missing from db // for the subsequent calls, it will be reported as present with the probability p - suite.collections.On("LightByID", mock.Anything).Return( + s.collections.On("LightByID", mock.Anything).Return( func(cID flow.Identifier) *flow.LightCollection { return nil // the actual collection object return is never really read }, @@ -400,10 +400,10 @@ func (suite *Suite) TestRequestMissingCollections() { // setup the requester engine mock // entityByID should be called once per collection for _, c := range collIDs { - suite.request.On("EntityByID", c, mock.Anything).Return() + s.request.On("EntityByID", c, mock.Anything).Return() } // force should be called once - suite.request.On("Force").Return() + s.request.On("Force").Return() cluster := new(protocol.Cluster) cluster.On("Members").Return(clusterCommittee, nil) @@ -413,17 +413,17 @@ func (suite *Suite) TestRequestMissingCollections() { epochs.On("Current").Return(epoch) snap := new(protocol.Snapshot) snap.On("Epochs").Return(epochs) - suite.proto.state.On("AtBlockID", refBlockID).Return(snap) + s.proto.state.On("AtBlockID", refBlockID).Return(snap) assertExpectations := func() { - suite.request.AssertExpectations(suite.T()) - suite.collections.AssertExpectations(suite.T()) - suite.proto.snapshot.AssertExpectations(suite.T()) - suite.blocks.AssertExpectations(suite.T()) + s.request.AssertExpectations(s.T()) + s.collections.AssertExpectations(s.T()) + s.proto.snapshot.AssertExpectations(s.T()) + s.blocks.AssertExpectations(s.T()) } // test 1 - collections are not received before timeout - suite.Run("timeout before all missing collections are received", func() { + s.Run("timeout before all missing collections are received", func() { // simulate that collection are never received p = 1 @@ -432,15 +432,15 @@ func (suite *Suite) TestRequestMissingCollections() { ctx, cancel := context.WithTimeout(context.Background(), 100*defaultCollectionCatchupDBPollInterval) defer cancel() - err := suite.eng.requestMissingCollections(ctx) + err := s.eng.requestMissingCollections(ctx) - require.Error(suite.T(), err) - require.Contains(suite.T(), err.Error(), "context deadline exceeded") + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), "context deadline exceeded") assertExpectations() }) // test 2 - all collections are eventually received before the deadline - suite.Run("all missing collections are received", func() { + s.Run("all missing collections are received", func() { // 90% of the time, collections are reported as not received when the collection storage is queried p = 0.9 @@ -448,18 +448,19 @@ func (suite *Suite) TestRequestMissingCollections() { ctx, cancel := context.WithTimeout(context.Background(), defaultCollectionCatchupTimeout) defer cancel() - err := suite.eng.requestMissingCollections(ctx) + err := s.eng.requestMissingCollections(ctx) - require.NoError(suite.T(), err) - require.Len(suite.T(), rcvdColl, len(collIDs)) + require.NoError(s.T(), err) + require.Len(s.T(), rcvdColl, len(collIDs)) assertExpectations() }) } -// TestUpdateLastFullBlockReceivedIndex tests that UpdateLastFullBlockReceivedIndex function keeps the FullBlockIndex -// upto date and request collections if blocks with missing collections exceed the threshold. -func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { +// TestProcessBackgroundCalls tests that updateLastFullBlockReceivedIndex and checkMissingCollections +// function calls keep the FullBlockIndex up-to-date and request collections if blocks with missing +// collections exceed the threshold. +func (s *Suite) TestProcessBackgroundCalls() { blkCnt := 3 collPerBlk := 10 startHeight := uint64(1000) @@ -485,7 +486,7 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { // guarantee signers must be cluster committee members, so that access will fetch collection from // the signers that are specified by guarantee.SignerIndices indices, err := signature.EncodeSignersToIndices(clusterCommittee.NodeIDs(), clusterCommittee.NodeIDs()) - require.NoError(suite.T(), err) + require.NoError(s.T(), err) cg.SignerIndices = indices guarantees[j] = cg } @@ -505,7 +506,7 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { // setup the block storage mock // each block should be queried by height - suite.blocks.On("ByHeight", mock.IsType(uint64(0))).Return( + s.blocks.On("ByHeight", mock.IsType(uint64(0))).Return( func(h uint64) *flow.Block { // simulate a db lookup return heightMap[h] @@ -525,7 +526,7 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { epochs.On("Current").Return(epoch) snap := new(protocol.Snapshot) snap.On("Epochs").Return(epochs) - suite.proto.state.On("AtBlockID", refBlockID).Return(snap) + s.proto.state.On("AtBlockID", refBlockID).Return(snap) // blkMissingColl controls which collections are reported as missing by the collections storage mock blkMissingColl := make([]bool, blkCnt) @@ -533,7 +534,7 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { blkMissingColl[i] = false for _, cg := range blocks[i].Payload.Guarantees { j := i - suite.collections.On("LightByID", cg.CollectionID).Return( + s.collections.On("LightByID", cg.CollectionID).Return( func(cID flow.Identifier) *flow.LightCollection { return collMap[cID] }, @@ -546,54 +547,38 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { } } - var lastFullBlockHeight uint64 - var rtnErr error - suite.blocks.On("GetLastFullBlockHeight").Return( - func() uint64 { - return lastFullBlockHeight - }, - func() error { - return rtnErr - }) - // consider the last test block as the head - suite.finalizedBlock = finalizedBlk.Header + s.finalizedBlock = finalizedBlk.Header - suite.Run("full block height index is created and advanced if not present", func() { - // simulate the absence of the full block height index - lastFullBlockHeight = 0 - rtnErr = storerr.ErrNotFound - suite.proto.params.On("FinalizedRoot").Return(rootBlk.Header, nil) - suite.blocks.On("UpdateLastFullBlockHeight", finalizedHeight).Return(nil).Once() - - suite.eng.updateLastFullBlockReceivedIndex() - - suite.blocks.AssertExpectations(suite.T()) - }) - - suite.Run("full block height index is advanced if newer full blocks are discovered", func() { - rtnErr = nil + s.Run("full block height index is advanced if newer full blocks are discovered", func() { block := blocks[1] - lastFullBlockHeight = block.Header.Height - suite.blocks.On("UpdateLastFullBlockHeight", finalizedHeight).Return(nil).Once() + s.blocks.On("UpdateLastFullBlockHeight", finalizedHeight).Return(nil).Once() + s.blocks.On("GetLastFullBlockHeight").Return(func() (uint64, error) { + return block.Header.Height, nil + }).Once() - suite.eng.updateLastFullBlockReceivedIndex() + err := s.eng.updateLastFullBlockReceivedIndex() + s.Require().NoError(err) - suite.blocks.AssertExpectations(suite.T()) + s.blocks.AssertExpectations(s.T()) }) - suite.Run("full block height index is not advanced beyond finalized blocks", func() { - rtnErr = nil - lastFullBlockHeight = finalizedHeight + s.Run("full block height index is not advanced beyond finalized blocks", func() { + s.blocks.On("GetLastFullBlockHeight").Return(func() (uint64, error) { + return finalizedHeight, nil + }).Once() + + err := s.eng.updateLastFullBlockReceivedIndex() + s.Require().NoError(err) - suite.eng.updateLastFullBlockReceivedIndex() - suite.blocks.AssertExpectations(suite.T()) // not new call to UpdateLastFullBlockHeight should be made + s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made }) - suite.Run("missing collections are requested when count exceeds defaultMissingCollsForBlkThreshold", func() { + s.Run("missing collections are requested when count exceeds defaultMissingCollsForBlkThreshold", func() { // root block is the last complete block - rtnErr = nil - lastFullBlockHeight = rootBlkHeight + s.blocks.On("GetLastFullBlockHeight").Return(func() (uint64, error) { + return rootBlkHeight, nil + }).Once() // lower the block threshold to request missing collections defaultMissingCollsForBlkThreshold = 2 @@ -603,23 +588,25 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { blkMissingColl[i] = true // setup receive engine expectations for _, cg := range blocks[i].Payload.Guarantees { - suite.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() + s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() } } - suite.eng.updateLastFullBlockReceivedIndex() + err := s.eng.checkMissingCollections() + s.Require().NoError(err) // assert that missing collections are requested - suite.request.AssertExpectations(suite.T()) + s.request.AssertExpectations(s.T()) // last full blk index is not advanced - suite.blocks.AssertExpectations(suite.T()) // no new call to UpdateLastFullBlockHeight should be made + s.blocks.AssertExpectations(s.T()) // no new call to UpdateLastFullBlockHeight should be made }) - suite.Run("missing collections are requested when count exceeds defaultMissingCollsForAgeThreshold", func() { + s.Run("missing collections are requested when count exceeds defaultMissingCollsForAgeThreshold", func() { // root block is the last complete block - rtnErr = nil - lastFullBlockHeight = rootBlkHeight + s.blocks.On("GetLastFullBlockHeight").Return(func() (uint64, error) { + return rootBlkHeight, nil + }).Once() // lower the height threshold to request missing collections defaultMissingCollsForAgeThreshold = 1 @@ -632,23 +619,25 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { blkMissingColl[i] = true // setup receive engine expectations for _, cg := range blocks[i].Payload.Guarantees { - suite.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() + s.request.On("EntityByID", cg.CollectionID, mock.Anything).Return().Once() } } - suite.eng.updateLastFullBlockReceivedIndex() + err := s.eng.checkMissingCollections() + s.Require().NoError(err) // assert that missing collections are requested - suite.request.AssertExpectations(suite.T()) + s.request.AssertExpectations(s.T()) // last full blk index is not advanced - suite.blocks.AssertExpectations(suite.T()) // not new call to UpdateLastFullBlockHeight should be made + s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made }) - suite.Run("missing collections are not requested if defaultMissingCollsForBlkThreshold not reached", func() { + s.Run("missing collections are not requested if defaultMissingCollsForBlkThreshold not reached", func() { // root block is the last complete block - rtnErr = nil - lastFullBlockHeight = rootBlkHeight + s.blocks.On("GetLastFullBlockHeight").Return(func() (uint64, error) { + return rootBlkHeight, nil + }).Once() // raise the thresholds to avoid requesting missing collections defaultMissingCollsForAgeThreshold = 3 @@ -659,22 +648,23 @@ func (suite *Suite) TestUpdateLastFullBlockReceivedIndex() { blkMissingColl[i] = true } - suite.eng.updateLastFullBlockReceivedIndex() + err := s.eng.checkMissingCollections() + s.Require().NoError(err) // assert that missing collections are not requested even though there are collections missing - suite.request.AssertExpectations(suite.T()) + s.request.AssertExpectations(s.T()) // last full blk index is not advanced - suite.blocks.AssertExpectations(suite.T()) // not new call to UpdateLastFullBlockHeight should be made + s.blocks.AssertExpectations(s.T()) // not new call to UpdateLastFullBlockHeight should be made }) } -func (suite *Suite) TestComponentShutdown() { +func (s *Suite) TestComponentShutdown() { // start then shut down the engine - unittest.AssertClosesBefore(suite.T(), suite.eng.Ready(), 10*time.Millisecond) - suite.cancel() - unittest.AssertClosesBefore(suite.T(), suite.eng.Done(), 10*time.Millisecond) + unittest.AssertClosesBefore(s.T(), s.eng.Ready(), 10*time.Millisecond) + s.cancel() + unittest.AssertClosesBefore(s.T(), s.eng.Done(), 10*time.Millisecond) - err := suite.eng.ProcessLocal(&flow.ExecutionReceipt{}) - suite.Assert().ErrorIs(err, component.ErrComponentShutdown) + err := s.eng.ProcessLocal(&flow.ExecutionReceipt{}) + s.Assert().ErrorIs(err, component.ErrComponentShutdown) } diff --git a/engine/access/integration_unsecure_grpc_server_test.go b/engine/access/integration_unsecure_grpc_server_test.go index 810868e36fe..674a9ad4d1e 100644 --- a/engine/access/integration_unsecure_grpc_server_test.go +++ b/engine/access/integration_unsecure_grpc_server_test.go @@ -63,6 +63,7 @@ type SameGRPCPortTestSuite struct { // storage blocks *storagemock.Blocks headers *storagemock.Headers + events *storagemock.Events collections *storagemock.Collections transactions *storagemock.Transactions receipts *storagemock.ExecutionReceipts @@ -101,6 +102,7 @@ func (suite *SameGRPCPortTestSuite) SetupTest() { suite.snapshot.On("Epochs").Return(suite.epochQuery).Maybe() suite.blocks = new(storagemock.Blocks) suite.headers = new(storagemock.Headers) + suite.events = new(storagemock.Events) suite.transactions = new(storagemock.Transactions) suite.collections = new(storagemock.Collections) suite.receipts = new(storagemock.ExecutionReceipts) @@ -241,6 +243,7 @@ func (suite *SameGRPCPortTestSuite) SetupTest() { conf, suite.state, suite.headers, + suite.events, suite.seals, suite.results, nil, @@ -249,6 +252,7 @@ func (suite *SameGRPCPortTestSuite) SetupTest() { rootBlock.Header.Height, rootBlock.Header.Height, suite.registers, + false, ) assert.NoError(suite.T(), err) diff --git a/engine/access/rest/routes/subscribe_events_test.go b/engine/access/rest/routes/subscribe_events_test.go index 0b5626c64b2..6d5b731d6b7 100644 --- a/engine/access/rest/routes/subscribe_events_test.go +++ b/engine/access/rest/routes/subscribe_events_test.go @@ -43,10 +43,11 @@ type testType struct { headers http.Header } +var chainID = flow.Testnet var testEventTypes = []flow.EventType{ - "A.0123456789abcdef.flow.event", - "B.0123456789abcdef.flow.event", - "C.0123456789abcdef.flow.event", + unittest.EventTypeFixture(chainID), + unittest.EventTypeFixture(chainID), + unittest.EventTypeFixture(chainID), } type SubscribeEventsSuite struct { @@ -83,6 +84,8 @@ func (s *SubscribeEventsSuite) SetupTest() { // update payloads with valid CCF encoded data for i := range blockEvents.Events { blockEvents.Events[i].Payload = eventsGenerator.New().Payload + + s.T().Logf("block events %d %v => %v", block.Header.Height, block.ID(), blockEvents.Events[i].Type) } s.blocks = append(s.blocks, block) @@ -143,7 +146,6 @@ func (s *SubscribeEventsSuite) TestSubscribeEvents() { }, }, } - chain := flow.MonotonicEmulator.Chain() // create variations for each of the base test tests := make([]testType, 0, len(testVectors)*2) @@ -159,7 +161,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEvents() { t3 := test t3.name = fmt.Sprintf("%s - non existing events", test.name) - t3.eventTypes = []string{"A.0123456789abcdff.flow.event"} + t3.eventTypes = []string{fmt.Sprintf("%s_new", testEventTypes[0])} tests = append(tests, t3) } @@ -170,7 +172,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEvents() { filter, err := state_stream.NewEventFilter( state_stream.DefaultEventFilterConfig, - chain, + chainID.Chain(), test.eventTypes, test.addresses, test.contracts) @@ -245,9 +247,9 @@ func (s *SubscribeEventsSuite) TestSubscribeEvents() { // closing the connection after 1 second go func() { time.Sleep(1 * time.Second) - close(respRecorder.closed) + respRecorder.Close() }() - executeWsRequest(req, stateStreamBackend, respRecorder) + executeWsRequest(req, stateStreamBackend, respRecorder, chainID.Chain()) requireResponse(s.T(), respRecorder, expectedEventsResponses) }) } @@ -259,7 +261,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEventsHandlesErrors() { req, err := getSubscribeEventsRequest(s.T(), s.blocks[0].ID(), s.blocks[0].Header.Height, nil, nil, nil, 1, nil) require.NoError(s.T(), err) respRecorder := newTestHijackResponseRecorder() - executeWsRequest(req, stateStreamBackend, respRecorder) + executeWsRequest(req, stateStreamBackend, respRecorder, chainID.Chain()) requireError(s.T(), respRecorder, "can only provide either block ID or start height") }) @@ -284,7 +286,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEventsHandlesErrors() { req, err := getSubscribeEventsRequest(s.T(), invalidBlock.ID(), request.EmptyHeight, nil, nil, nil, 1, nil) require.NoError(s.T(), err) respRecorder := newTestHijackResponseRecorder() - executeWsRequest(req, stateStreamBackend, respRecorder) + executeWsRequest(req, stateStreamBackend, respRecorder, chainID.Chain()) requireError(s.T(), respRecorder, "stream encountered an error: subscription error") }) @@ -293,7 +295,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEventsHandlesErrors() { req, err := getSubscribeEventsRequest(s.T(), s.blocks[0].ID(), request.EmptyHeight, []string{"foo"}, nil, nil, 1, nil) require.NoError(s.T(), err) respRecorder := newTestHijackResponseRecorder() - executeWsRequest(req, stateStreamBackend, respRecorder) + executeWsRequest(req, stateStreamBackend, respRecorder, chainID.Chain()) requireError(s.T(), respRecorder, "invalid event type format") }) @@ -318,7 +320,7 @@ func (s *SubscribeEventsSuite) TestSubscribeEventsHandlesErrors() { req, err := getSubscribeEventsRequest(s.T(), s.blocks[0].ID(), request.EmptyHeight, nil, nil, nil, 1, nil) require.NoError(s.T(), err) respRecorder := newTestHijackResponseRecorder() - executeWsRequest(req, stateStreamBackend, respRecorder) + executeWsRequest(req, stateStreamBackend, respRecorder, chainID.Chain()) requireError(s.T(), respRecorder, "subscription channel closed") }) } diff --git a/engine/access/rest/routes/test_helpers.go b/engine/access/rest/routes/test_helpers.go index 1bedb07f383..ebe40fa48df 100644 --- a/engine/access/rest/routes/test_helpers.go +++ b/engine/access/rest/routes/test_helpers.go @@ -55,6 +55,7 @@ func (c fakeNetConn) Close() error { } return nil } + func (c fakeNetConn) LocalAddr() net.Addr { return localAddr } func (c fakeNetConn) RemoteAddr() net.Addr { return remoteAddr } func (c fakeNetConn) SetDeadline(t time.Time) error { return nil } @@ -101,6 +102,15 @@ func (w *testHijackResponseRecorder) Hijack() (net.Conn, *bufio.ReadWriter, erro return fakeNetConn{w.responseBuff, w.closed}, bufio.NewReadWriter(br, bw), nil } +func (w *testHijackResponseRecorder) Close() error { + select { + case <-w.closed: + default: + close(w.closed) + } + return nil +} + // newTestHijackResponseRecorder creates a new instance of testHijackResponseRecorder. func newTestHijackResponseRecorder() *testHijackResponseRecorder { return &testHijackResponseRecorder{ @@ -122,7 +132,7 @@ func executeRequest(req *http.Request, backend access.API) *httptest.ResponseRec return rr } -func executeWsRequest(req *http.Request, stateStreamApi state_stream.API, responseRecorder *testHijackResponseRecorder) { +func executeWsRequest(req *http.Request, stateStreamApi state_stream.API, responseRecorder *testHijackResponseRecorder, chain flow.Chain) { restCollector := metrics.NewNoopCollector() config := backend.Config{ @@ -133,7 +143,7 @@ func executeWsRequest(req *http.Request, stateStreamApi state_stream.API, respon router := NewRouterBuilder(unittest.Logger(), restCollector).AddWsRoutes( stateStreamApi, - flow.Testnet.Chain(), config).Build() + chain, config).Build() router.ServeHTTP(responseRecorder, req) } diff --git a/engine/access/rpc/backend/backend.go b/engine/access/rpc/backend/backend.go index 4e5838d234d..d082277cd8d 100644 --- a/engine/access/rpc/backend/backend.go +++ b/engine/access/rpc/backend/backend.go @@ -83,6 +83,7 @@ type Params struct { HistoricalAccessNodes []accessproto.AccessAPIClient Blocks storage.Blocks Headers storage.Headers + Events storage.Events Collections storage.Collections Transactions storage.Transactions ExecutionReceipts storage.ExecutionReceipts @@ -101,7 +102,8 @@ type Params struct { TxResultCacheSize uint TxErrorMessagesCacheSize uint ScriptExecutor execution.ScriptExecutor - ScriptExecutionMode ScriptExecutionMode + ScriptExecutionMode IndexQueryMode + EventQueryMode IndexQueryMode } // New creates backend instance @@ -146,11 +148,11 @@ func New(params Params) (*Backend, error) { state: params.State, // create the sub-backends backendScripts: backendScripts{ + log: params.Log, headers: params.Headers, executionReceipts: params.ExecutionReceipts, connFactory: params.ConnFactory, state: params.State, - log: params.Log, metrics: params.AccessMetrics, loggedScripts: loggedScripts, nodeCommunicator: params.Communicator, @@ -158,6 +160,7 @@ func New(params Params) (*Backend, error) { scriptExecMode: params.ScriptExecutionMode, }, backendTransactions: backendTransactions{ + log: params.Log, staticCollectionRPC: params.CollectionRPC, state: params.State, chainID: params.ChainID, @@ -171,19 +174,21 @@ func New(params Params) (*Backend, error) { retry: retry, connFactory: params.ConnFactory, previousAccessNodes: params.HistoricalAccessNodes, - log: params.Log, nodeCommunicator: params.Communicator, txResultCache: txResCache, txErrorMessagesCache: txErrorMessagesCache, }, backendEvents: backendEvents{ + log: params.Log, + chain: params.ChainID.Chain(), state: params.State, headers: params.Headers, + events: params.Events, executionReceipts: params.ExecutionReceipts, connFactory: params.ConnFactory, - log: params.Log, maxHeightRange: params.MaxHeightRange, nodeCommunicator: params.Communicator, + queryMode: params.EventQueryMode, }, backendBlockHeaders: backendBlockHeaders{ headers: params.Headers, @@ -194,11 +199,11 @@ func New(params Params) (*Backend, error) { state: params.State, }, backendAccounts: backendAccounts{ + log: params.Log, state: params.State, headers: params.Headers, executionReceipts: params.ExecutionReceipts, connFactory: params.ConnFactory, - log: params.Log, nodeCommunicator: params.Communicator, scriptExecutor: params.ScriptExecutor, scriptExecMode: params.ScriptExecutionMode, diff --git a/engine/access/rpc/backend/backend_accounts.go b/engine/access/rpc/backend/backend_accounts.go index dbb2c03ab64..2e58b904a48 100644 --- a/engine/access/rpc/backend/backend_accounts.go +++ b/engine/access/rpc/backend/backend_accounts.go @@ -31,7 +31,7 @@ type backendAccounts struct { connFactory connection.ConnectionFactory nodeCommunicator Communicator scriptExecutor execution.ScriptExecutor - scriptExecMode ScriptExecutionMode + scriptExecMode IndexQueryMode } // GetAccount returns the account details at the latest sealed block. @@ -66,14 +66,12 @@ func (b *backendAccounts) GetAccountAtBlockHeight( address flow.Address, height uint64, ) (*flow.Account, error) { - header, err := b.headers.ByHeight(height) + blockID, err := b.headers.BlockIDByHeight(height) if err != nil { return nil, rpc.ConvertStorageError(err) } - blockID := header.ID() - - account, err := b.getAccountAtBlock(ctx, address, blockID, header.Height) + account, err := b.getAccountAtBlock(ctx, address, blockID, height) if err != nil { b.log.Debug().Err(err).Msgf("failed to get account at height: %d", height) return nil, err @@ -93,13 +91,13 @@ func (b *backendAccounts) getAccountAtBlock( height uint64, ) (*flow.Account, error) { switch b.scriptExecMode { - case ScriptExecutionModeExecutionNodesOnly: + case IndexQueryModeExecutionNodesOnly: return b.getAccountFromAnyExeNode(ctx, address, blockID) - case ScriptExecutionModeLocalOnly: + case IndexQueryModeLocalOnly: return b.getAccountFromLocalStorage(ctx, address, height) - case ScriptExecutionModeFailover: + case IndexQueryModeFailover: localResult, localErr := b.getAccountFromLocalStorage(ctx, address, height) if localErr == nil { return localResult, nil @@ -110,7 +108,7 @@ func (b *backendAccounts) getAccountAtBlock( return execResult, execErr - case ScriptExecutionModeCompare: + case IndexQueryModeCompare: execResult, execErr := b.getAccountFromAnyExeNode(ctx, address, blockID) // Only compare actual get account errors from the EN, not system errors if execErr != nil && !isInvalidArgumentError(execErr) { diff --git a/engine/access/rpc/backend/backend_accounts_test.go b/engine/access/rpc/backend/backend_accounts_test.go index 614f91bfb49..d8428022228 100644 --- a/engine/access/rpc/backend/backend_accounts_test.go +++ b/engine/access/rpc/backend/backend_accounts_test.go @@ -139,7 +139,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromExecutionNode_HappyPath() { s.setupENSuccessResponse(s.block.ID()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeExecutionNodesOnly + backend.scriptExecMode = IndexQueryModeExecutionNodesOnly s.Run("GetAccount - happy path", func() { s.testGetAccount(ctx, backend, codes.OK) @@ -166,7 +166,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromExecutionNode_Fails() { s.setupENFailingResponse(s.block.ID(), errToReturn) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeExecutionNodesOnly + backend.scriptExecMode = IndexQueryModeExecutionNodesOnly s.Run("GetAccount - fails with backend err", func() { s.testGetAccount(ctx, backend, statusCode) @@ -190,7 +190,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromStorage_HappyPath() { Return(s.account, nil) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor s.Run("GetAccount - happy path", func() { @@ -214,7 +214,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromStorage_Fails() { scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor testCases := []struct { @@ -264,7 +264,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromFailover_HappyPath() { scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeFailover + backend.scriptExecMode = IndexQueryModeFailover backend.scriptExecutor = scriptExecutor for _, errToReturn := range []error{execution.ErrDataNotAvailable, storage.ErrNotFound} { @@ -302,7 +302,7 @@ func (s *BackendAccountsSuite) TestGetAccountFromFailover_ReturnsENErrors() { Return(nil, execution.ErrDataNotAvailable) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeFailover + backend.scriptExecMode = IndexQueryModeFailover backend.scriptExecutor = scriptExecutor s.Run("GetAccount - fails with backend err", func() { @@ -324,7 +324,7 @@ func (s *BackendAccountsSuite) TestGetAccountAtLatestBlockFromStorage_Inconsiste scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor s.Run(fmt.Sprintf("GetAccountAtLatestBlock - fails with %v", "inconsistent node's state"), func() { @@ -376,7 +376,7 @@ func (s *BackendAccountsSuite) testGetAccountAtLatestBlock(ctx context.Context, func (s *BackendAccountsSuite) testGetAccountAtBlockHeight(ctx context.Context, backend *backendAccounts, statusCode codes.Code) { height := s.block.Header.Height - s.headers.On("ByHeight", height).Return(s.block.Header, nil).Once() + s.headers.On("BlockIDByHeight", height).Return(s.block.Header.ID(), nil).Once() if statusCode == codes.OK { actual, err := backend.GetAccountAtBlockHeight(ctx, s.account.Address, height) diff --git a/engine/access/rpc/backend/backend_events.go b/engine/access/rpc/backend/backend_events.go index a5bebada2e7..e35442ca966 100644 --- a/engine/access/rpc/backend/backend_events.go +++ b/engine/access/rpc/backend/backend_events.go @@ -5,6 +5,7 @@ import ( "encoding/hex" "errors" "fmt" + "sort" "time" "github.com/onflow/flow/protobuf/go/flow/entities" @@ -17,6 +18,7 @@ import ( "github.com/onflow/flow-go/engine/access/rpc/connection" "github.com/onflow/flow-go/engine/common/rpc" "github.com/onflow/flow-go/engine/common/rpc/convert" + "github.com/onflow/flow-go/model/events" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/state/protocol" @@ -25,12 +27,23 @@ import ( type backendEvents struct { headers storage.Headers + events storage.Events executionReceipts storage.ExecutionReceipts state protocol.State + chain flow.Chain connFactory connection.ConnectionFactory log zerolog.Logger maxHeightRange uint nodeCommunicator Communicator + queryMode IndexQueryMode +} + +// blockMetadata is used to capture information about requested blocks to avoid repeated blockID +// calculations and passing around full block headers. +type blockMetadata struct { + ID flow.Identifier + Height uint64 + Timestamp time.Time } // GetEventsForHeightRange retrieves events for all sealed blocks between the start block height and @@ -43,47 +56,67 @@ func (b *backendEvents) GetEventsForHeightRange( ) ([]flow.BlockEvents, error) { if endHeight < startHeight { - return nil, status.Error(codes.InvalidArgument, "invalid start or end height") + return nil, status.Error(codes.InvalidArgument, "start height must not be larger than end height") } rangeSize := endHeight - startHeight + 1 // range is inclusive on both ends if rangeSize > uint64(b.maxHeightRange) { - return nil, status.Errorf(codes.InvalidArgument, "requested block range (%d) exceeded maximum (%d)", rangeSize, b.maxHeightRange) + return nil, status.Errorf(codes.InvalidArgument, + "requested block range (%d) exceeded maximum (%d)", rangeSize, b.maxHeightRange) } // get the latest sealed block header - head, err := b.state.Sealed().Head() + sealed, err := b.state.Sealed().Head() if err != nil { - // sealed block must be in the store, so return an Internal code even if we got NotFound + // sealed block must be in the store, so throw an exception for any error err := irrecoverable.NewExceptionf("failed to lookup sealed header: %w", err) irrecoverable.Throw(ctx, err) return nil, err } // start height should not be beyond the last sealed height - if head.Height < startHeight { + if startHeight > sealed.Height { return nil, status.Errorf(codes.OutOfRange, - "start height %d is greater than the last sealed block height %d", startHeight, head.Height) + "start height %d is greater than the last sealed block height %d", startHeight, sealed.Height) } // limit max height to last sealed block in the chain - if head.Height < endHeight { - endHeight = head.Height + // + // Note: this causes unintuitive behavior for clients making requests through a proxy that + // fronts multiple nodes. With that setup, clients may receive responses for a smaller range + // than requested because the node serving the request has a slightly delayed view of the chain. + // + // An alternative option is to return an error here, but that's likely to cause more pain for + // these clients since the requests would intermittently fail. it's recommended instead to + // check the block height of the last message in the response. this will be the last block + // height searched, and can be used to determine the start height for the next range. + if endHeight > sealed.Height { + endHeight = sealed.Height } // find the block headers for all the blocks between min and max height (inclusive) - blockHeaders := make([]*flow.Header, 0) + blockHeaders := make([]blockMetadata, 0, endHeight-startHeight+1) for i := startHeight; i <= endHeight; i++ { - header, err := b.headers.ByHeight(i) + // this looks inefficient, but is actually what's done under the covers by `headers.ByHeight` + // and avoids calculating header.ID() for each block. + blockID, err := b.headers.BlockIDByHeight(i) if err != nil { - return nil, rpc.ConvertStorageError(fmt.Errorf("failed to get events: %w", err)) + return nil, rpc.ConvertStorageError(fmt.Errorf("failed to get blockID for %d: %w", i, err)) + } + header, err := b.headers.ByBlockID(blockID) + if err != nil { + return nil, rpc.ConvertStorageError(fmt.Errorf("failed to get block header for %d: %w", i, err)) } - blockHeaders = append(blockHeaders, header) + blockHeaders = append(blockHeaders, blockMetadata{ + ID: blockID, + Height: header.Height, + Timestamp: header.Timestamp, + }) } - return b.getBlockEventsFromExecutionNode(ctx, blockHeaders, eventType, requiredEventEncodingVersion) + return b.getBlockEvents(ctx, blockHeaders, eventType, requiredEventEncodingVersion) } // GetEventsForBlockIDs retrieves events for all the specified block IDs that have the given type @@ -99,31 +132,159 @@ func (b *backendEvents) GetEventsForBlockIDs( } // find the block headers for all the block IDs - blockHeaders := make([]*flow.Header, 0) + blockHeaders := make([]blockMetadata, 0, len(blockIDs)) for _, blockID := range blockIDs { header, err := b.headers.ByBlockID(blockID) if err != nil { - return nil, rpc.ConvertStorageError(fmt.Errorf("failed to get events: %w", err)) + return nil, rpc.ConvertStorageError(fmt.Errorf("failed to get block header for %s: %w", blockID, err)) + } + + blockHeaders = append(blockHeaders, blockMetadata{ + ID: blockID, + Height: header.Height, + Timestamp: header.Timestamp, + }) + } + + return b.getBlockEvents(ctx, blockHeaders, eventType, requiredEventEncodingVersion) +} + +// getBlockEvents retrieves events for all the specified blocks that have the given type +// It gets all events available in storage, and requests the rest from an execution node. +func (b *backendEvents) getBlockEvents( + ctx context.Context, + blockInfos []blockMetadata, + eventType string, + requiredEventEncodingVersion entities.EventEncodingVersion, +) ([]flow.BlockEvents, error) { + target := flow.EventType(eventType) + + if _, err := events.ValidateEvent(target, b.chain); err != nil { + return nil, status.Errorf(codes.InvalidArgument, "invalid event type: %v", err) + } + + switch b.queryMode { + case IndexQueryModeExecutionNodesOnly: + return b.getBlockEventsFromExecutionNode(ctx, blockInfos, eventType, requiredEventEncodingVersion) + + case IndexQueryModeLocalOnly: + localResponse, missingBlocks, err := b.getBlockEventsFromStorage(ctx, blockInfos, target, requiredEventEncodingVersion) + if err != nil { + return nil, err + } + // all blocks should be available. + if len(missingBlocks) > 0 { + return nil, status.Errorf(codes.NotFound, "events not found in local storage for %d blocks", len(missingBlocks)) + } + return localResponse, nil + + case IndexQueryModeFailover: + localResponse, missingBlocks, err := b.getBlockEventsFromStorage(ctx, blockInfos, target, requiredEventEncodingVersion) + if err != nil { + // if there was an error, request all blocks from execution nodes + missingBlocks = blockInfos + b.log.Debug().Err(err).Msg("failed to get events from local storage") + } + + if len(missingBlocks) == 0 { + return localResponse, nil + } + + b.log.Debug(). + Int("missing_blocks", len(missingBlocks)). + Msg("querying execution nodes for events from missing blocks") + + enResponse, err := b.getBlockEventsFromExecutionNode(ctx, missingBlocks, eventType, requiredEventEncodingVersion) + if err != nil { + return nil, err } - blockHeaders = append(blockHeaders, header) + // sort ascending by block height + // this is needed because some blocks may be retrieved from storage and others from execution nodes. + // most likely, the earlier blocks will all be found in local storage, but that's not guaranteed, + // especially for nodes started after a spork, or once pruning is enabled. + // Note: this may not match the order of the original request for clients using GetEventsForBlockIDs + // that provide out of order block IDs + response := append(localResponse, enResponse...) + sort.Slice(response, func(i, j int) bool { + return response[i].BlockHeight < response[j].BlockHeight + }) + return response, nil + + default: + return nil, status.Errorf(codes.Internal, "unknown event query mode: %v", b.queryMode) } +} - // forward the request to the execution node - return b.getBlockEventsFromExecutionNode(ctx, blockHeaders, eventType, requiredEventEncodingVersion) +// getBlockEventsFromStorage retrieves events for all the specified blocks that have the given type +// from the local storage +func (b *backendEvents) getBlockEventsFromStorage( + ctx context.Context, + blockInfos []blockMetadata, + eventType flow.EventType, + requiredEventEncodingVersion entities.EventEncodingVersion, +) ([]flow.BlockEvents, []blockMetadata, error) { + missing := make([]blockMetadata, 0) + resp := make([]flow.BlockEvents, 0) + for _, blockInfo := range blockInfos { + if ctx.Err() != nil { + return nil, nil, rpc.ConvertError(ctx.Err(), "failed to get events from storage", codes.Canceled) + } + + events, err := b.events.ByBlockID(blockInfo.ID) + if err != nil { + // Note: if there are no events for a block, an empty slice is returned + if errors.Is(err, storage.ErrNotFound) { + missing = append(missing, blockInfo) + continue + } + err = fmt.Errorf("failed to get events for block %s: %w", blockInfo.ID, err) + return nil, nil, rpc.ConvertError(err, "failed to get events from storage", codes.Internal) + } + + filteredEvents := make([]flow.Event, 0) + for _, e := range events { + if e.Type != eventType { + continue + } + + // events are encoded in CCF format in storage. convert to JSON-CDC if requested + if requiredEventEncodingVersion == entities.EventEncodingVersion_JSON_CDC_V0 { + payload, err := convert.CcfPayloadToJsonPayload(e.Payload) + if err != nil { + err = fmt.Errorf("failed to convert event payload for block %s: %w", blockInfo.ID, err) + return nil, nil, rpc.ConvertError(err, "failed to convert event payload", codes.Internal) + } + e.Payload = payload + } + + filteredEvents = append(filteredEvents, e) + } + + resp = append(resp, flow.BlockEvents{ + BlockID: blockInfo.ID, + BlockHeight: blockInfo.Height, + BlockTimestamp: blockInfo.Timestamp, + Events: filteredEvents, + }) + } + + return resp, missing, nil } +// getBlockEventsFromExecutionNode retrieves events for all the specified blocks that have the given type +// from an execution node func (b *backendEvents) getBlockEventsFromExecutionNode( ctx context.Context, - blockHeaders []*flow.Header, + blockInfos []blockMetadata, eventType string, requiredEventEncodingVersion entities.EventEncodingVersion, ) ([]flow.BlockEvents, error) { // create an execution API request for events at block ID - blockIDs := make([]flow.Identifier, len(blockHeaders)) - for i := range blockIDs { - blockIDs[i] = blockHeaders[i].ID() + blockIDs := make([]flow.Identifier, len(blockInfos)) + for i := range blockInfos { + blockIDs[i] = blockInfos[i].ID } if len(blockIDs) == 0 { @@ -140,7 +301,6 @@ func (b *backendEvents) getBlockEventsFromExecutionNode( execNodes, err := executionNodesForBlockID(ctx, lastBlockID, b.executionReceipts, b.state, b.log) if err != nil { - b.log.Error().Err(err).Msg("failed to retrieve events from execution node") return nil, rpc.ConvertError(err, "failed to retrieve events from execution node", codes.Internal) } @@ -148,7 +308,6 @@ func (b *backendEvents) getBlockEventsFromExecutionNode( var successfulNode *flow.Identity resp, successfulNode, err = b.getEventsFromAnyExeNode(ctx, execNodes, req) if err != nil { - b.log.Error().Err(err).Msg("failed to retrieve events from execution nodes") return nil, rpc.ConvertError(err, "failed to retrieve events from execution nodes", codes.Internal) } b.log.Trace(). @@ -159,7 +318,7 @@ func (b *backendEvents) getBlockEventsFromExecutionNode( // convert execution node api result to access node api result results, err := verifyAndConvertToAccessEvents( resp.GetResults(), - blockHeaders, + blockInfos, resp.GetEventEncodingVersion(), requiredEventEncodingVersion, ) @@ -170,31 +329,31 @@ func (b *backendEvents) getBlockEventsFromExecutionNode( return results, nil } -// verifyAndConvertToAccessEvents converts execution node api result to access node api result, and verifies that the results contains -// results from each block that was requested +// verifyAndConvertToAccessEvents converts execution node api result to access node api result, +// and verifies that the results contains results from each block that was requested func verifyAndConvertToAccessEvents( execEvents []*execproto.GetEventsForBlockIDsResponse_Result, - requestedBlockHeaders []*flow.Header, + requestedBlockInfos []blockMetadata, from entities.EventEncodingVersion, to entities.EventEncodingVersion, ) ([]flow.BlockEvents, error) { - if len(execEvents) != len(requestedBlockHeaders) { + if len(execEvents) != len(requestedBlockInfos) { return nil, errors.New("number of results does not match number of blocks requested") } - requestedBlockHeaderSet := map[string]*flow.Header{} - for _, header := range requestedBlockHeaders { - requestedBlockHeaderSet[header.ID().String()] = header + requestedBlockInfoSet := map[string]blockMetadata{} + for _, header := range requestedBlockInfos { + requestedBlockInfoSet[header.ID.String()] = header } results := make([]flow.BlockEvents, len(execEvents)) for i, result := range execEvents { - header, expected := requestedBlockHeaderSet[hex.EncodeToString(result.GetBlockId())] + blockInfo, expected := requestedBlockInfoSet[hex.EncodeToString(result.GetBlockId())] if !expected { return nil, fmt.Errorf("unexpected blockID from exe node %x", result.GetBlockId()) } - if result.GetBlockHeight() != header.Height { + if result.GetBlockHeight() != blockInfo.Height { return nil, fmt.Errorf("unexpected block height %d for block %x from exe node", result.GetBlockHeight(), result.GetBlockId()) @@ -207,9 +366,9 @@ func verifyAndConvertToAccessEvents( } results[i] = flow.BlockEvents{ - BlockID: header.ID(), - BlockHeight: header.Height, - BlockTimestamp: header.Timestamp, + BlockID: blockInfo.ID, + BlockHeight: blockInfo.Height, + BlockTimestamp: blockInfo.Timestamp, Events: events, } } @@ -223,7 +382,8 @@ func verifyAndConvertToAccessEvents( // error aggregating all failures is returned. func (b *backendEvents) getEventsFromAnyExeNode(ctx context.Context, execNodes flow.IdentityList, - req *execproto.GetEventsForBlockIDsRequest) (*execproto.GetEventsForBlockIDsResponse, *flow.Identity, error) { + req *execproto.GetEventsForBlockIDsRequest, +) (*execproto.GetEventsForBlockIDsResponse, *flow.Identity, error) { var resp *execproto.GetEventsForBlockIDsResponse var execNode *flow.Identity errToReturn := b.nodeCommunicator.CallAvailableNode( @@ -259,16 +419,13 @@ func (b *backendEvents) getEventsFromAnyExeNode(ctx context.Context, func (b *backendEvents) tryGetEvents(ctx context.Context, execNode *flow.Identity, - req *execproto.GetEventsForBlockIDsRequest) (*execproto.GetEventsForBlockIDsResponse, error) { + req *execproto.GetEventsForBlockIDsRequest, +) (*execproto.GetEventsForBlockIDsResponse, error) { execRPCClient, closer, err := b.connFactory.GetExecutionAPIClient(execNode.Address) if err != nil { return nil, err } defer closer.Close() - resp, err := execRPCClient.GetEventsForBlockIDs(ctx, req) - if err != nil { - return nil, err - } - return resp, nil + return execRPCClient.GetEventsForBlockIDs(ctx, req) } diff --git a/engine/access/rpc/backend/backend_events_test.go b/engine/access/rpc/backend/backend_events_test.go new file mode 100644 index 00000000000..10306a304aa --- /dev/null +++ b/engine/access/rpc/backend/backend_events_test.go @@ -0,0 +1,465 @@ +package backend + +import ( + "context" + "fmt" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/onflow/cadence/encoding/ccf" + jsoncdc "github.com/onflow/cadence/encoding/json" + "github.com/onflow/flow/protobuf/go/flow/entities" + execproto "github.com/onflow/flow/protobuf/go/flow/execution" + + access "github.com/onflow/flow-go/engine/access/mock" + connectionmock "github.com/onflow/flow-go/engine/access/rpc/connection/mock" + "github.com/onflow/flow-go/engine/common/rpc/convert" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" + protocol "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/storage" + storagemock "github.com/onflow/flow-go/storage/mock" + "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/generator" +) + +var targetEvent string + +type testCase struct { + encoding entities.EventEncodingVersion + queryMode IndexQueryMode +} + +type BackendEventsSuite struct { + suite.Suite + + log zerolog.Logger + state *protocol.State + snapshot *protocol.Snapshot + params *protocol.Params + rootHeader *flow.Header + + events *storagemock.Events + headers *storagemock.Headers + receipts *storagemock.ExecutionReceipts + connectionFactory *connectionmock.ConnectionFactory + chainID flow.ChainID + + executionNodes flow.IdentityList + execClient *access.ExecutionAPIClient + + sealedHead *flow.Header + blocks []*flow.Block + blockIDs []flow.Identifier + blockEvents []flow.Event + + testCases []testCase +} + +func TestBackendEventsSuite(t *testing.T) { + suite.Run(t, new(BackendEventsSuite)) +} + +func (s *BackendEventsSuite) SetupTest() { + s.log = unittest.Logger() + s.state = protocol.NewState(s.T()) + s.snapshot = protocol.NewSnapshot(s.T()) + s.rootHeader = unittest.BlockHeaderFixture() + s.params = protocol.NewParams(s.T()) + s.events = storagemock.NewEvents(s.T()) + s.headers = storagemock.NewHeaders(s.T()) + s.receipts = storagemock.NewExecutionReceipts(s.T()) + s.connectionFactory = connectionmock.NewConnectionFactory(s.T()) + s.chainID = flow.Testnet + + s.execClient = access.NewExecutionAPIClient(s.T()) + s.executionNodes = unittest.IdentityListFixture(2, unittest.WithRole(flow.RoleExecution)) + + blockCount := 5 + s.blocks = make([]*flow.Block, blockCount) + s.blockIDs = make([]flow.Identifier, blockCount) + + for i := 0; i < blockCount; i++ { + var header *flow.Header + if i == 0 { + header = unittest.BlockHeaderFixture() + } else { + header = unittest.BlockHeaderWithParentFixture(s.blocks[i-1].Header) + } + + payload := unittest.PayloadFixture() + header.PayloadHash = payload.Hash() + block := &flow.Block{ + Header: header, + Payload: &payload, + } + // the last block is sealed + if i == blockCount-1 { + s.sealedHead = header + } + + s.blocks[i] = block + s.blockIDs[i] = block.ID() + + s.T().Logf("block %d: %s", header.Height, block.ID()) + } + + s.blockEvents = generator.GetEventsWithEncoding(10, entities.EventEncodingVersion_CCF_V0) + targetEvent = string(s.blockEvents[0].Type) + + s.events.On("ByBlockID", mock.Anything).Return(func(blockID flow.Identifier) ([]flow.Event, error) { + for _, headerID := range s.blockIDs { + if blockID == headerID { + return s.blockEvents, nil + } + } + return nil, storage.ErrNotFound + }).Maybe() + + s.headers.On("BlockIDByHeight", mock.Anything).Return(func(height uint64) (flow.Identifier, error) { + for _, block := range s.blocks { + if height == block.Header.Height { + return block.ID(), nil + } + } + return flow.ZeroID, storage.ErrNotFound + }).Maybe() + + s.headers.On("ByBlockID", mock.Anything).Return(func(blockID flow.Identifier) (*flow.Header, error) { + for _, block := range s.blocks { + if blockID == block.ID() { + return block.Header, nil + } + } + return nil, storage.ErrNotFound + }).Maybe() + + s.testCases = make([]testCase, 0) + + for _, encoding := range []entities.EventEncodingVersion{ + entities.EventEncodingVersion_CCF_V0, + entities.EventEncodingVersion_JSON_CDC_V0, + } { + for _, queryMode := range []IndexQueryMode{ + IndexQueryModeExecutionNodesOnly, + IndexQueryModeLocalOnly, + IndexQueryModeFailover, + } { + s.testCases = append(s.testCases, testCase{ + encoding: encoding, + queryMode: queryMode, + }) + } + } +} + +func (s *BackendEventsSuite) defaultBackend() *backendEvents { + return &backendEvents{ + log: s.log, + chain: s.chainID.Chain(), + state: s.state, + events: s.events, + headers: s.headers, + executionReceipts: s.receipts, + connFactory: s.connectionFactory, + nodeCommunicator: NewNodeCommunicator(false), + maxHeightRange: DefaultMaxHeightRange, + queryMode: IndexQueryModeExecutionNodesOnly, + } +} + +// setupExecutionNodes sets up the mocks required to test against an EN backend +func (s *BackendEventsSuite) setupExecutionNodes(block *flow.Block) { + s.params.On("FinalizedRoot").Return(s.rootHeader, nil) + s.state.On("Params").Return(s.params) + s.state.On("Final").Return(s.snapshot) + s.snapshot.On("Identities", mock.Anything).Return(s.executionNodes, nil) + + // this line causes a S1021 lint error because receipts is explicitly declared. this is required + // to ensure the mock library handles the response type correctly + var receipts flow.ExecutionReceiptList //nolint:gosimple + receipts = unittest.ReceiptsForBlockFixture(block, s.executionNodes.NodeIDs()) + s.receipts.On("ByBlockID", block.ID()).Return(receipts, nil) + + s.connectionFactory.On("GetExecutionAPIClient", mock.Anything). + Return(s.execClient, &mockCloser{}, nil) +} + +// setupENSuccessResponse configures the execution node client to return a successful response +func (s *BackendEventsSuite) setupENSuccessResponse(eventType string, blocks []*flow.Block) { + s.setupExecutionNodes(blocks[len(blocks)-1]) + + ids := make([][]byte, len(blocks)) + results := make([]*execproto.GetEventsForBlockIDsResponse_Result, len(blocks)) + + events := make([]*entities.Event, 0) + for _, event := range s.blockEvents { + if string(event.Type) == eventType { + events = append(events, convert.EventToMessage(event)) + } + } + + for i, block := range blocks { + id := block.ID() + ids[i] = id[:] + results[i] = &execproto.GetEventsForBlockIDsResponse_Result{ + BlockId: id[:], + BlockHeight: block.Header.Height, + Events: events, + } + } + expectedExecRequest := &execproto.GetEventsForBlockIDsRequest{ + Type: eventType, + BlockIds: ids, + } + expectedResponse := &execproto.GetEventsForBlockIDsResponse{ + Results: results, + EventEncodingVersion: entities.EventEncodingVersion_CCF_V0, + } + + s.execClient.On("GetEventsForBlockIDs", mock.Anything, expectedExecRequest). + Return(expectedResponse, nil) +} + +// setupENFailingResponse configures the execution node client to return an error +func (s *BackendEventsSuite) setupENFailingResponse(eventType string, headers []*flow.Header, err error) { + ids := make([][]byte, len(headers)) + for i, header := range headers { + id := header.ID() + ids[i] = id[:] + } + failingRequest := &execproto.GetEventsForBlockIDsRequest{ + Type: eventType, + BlockIds: ids, + } + + s.execClient.On("GetEventsForBlockIDs", mock.Anything, failingRequest). + Return(nil, err) +} + +// TestGetEvents_HappyPaths tests the happy paths for GetEventsForBlockIDs and GetEventsForHeightRange +// across all queryModes and encodings +func (s *BackendEventsSuite) TestGetEvents_HappyPaths() { + ctx := context.Background() + + startHeight := s.blocks[0].Header.Height + endHeight := s.sealedHead.Height + + s.state.On("Sealed").Return(s.snapshot) + s.snapshot.On("Head").Return(s.sealedHead, nil) + + s.Run("GetEventsForHeightRange - end height updated", func() { + backend := s.defaultBackend() + backend.queryMode = IndexQueryModeFailover + endHeight := startHeight + 20 // should still return 5 responses + encoding := entities.EventEncodingVersion_CCF_V0 + + response, err := backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, encoding) + s.Require().NoError(err) + + s.assertResponse(response, encoding) + }) + + for _, tt := range s.testCases { + s.Run(fmt.Sprintf("all from storage - %s - %s", tt.encoding.String(), tt.queryMode), func() { + switch tt.queryMode { + case IndexQueryModeExecutionNodesOnly: + // not applicable + return + case IndexQueryModeLocalOnly, IndexQueryModeFailover: + // only calls to local storage + } + + backend := s.defaultBackend() + backend.queryMode = tt.queryMode + + response, err := backend.GetEventsForBlockIDs(ctx, targetEvent, s.blockIDs, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + + response, err = backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + }) + + s.Run(fmt.Sprintf("all from en - %s - %s", tt.encoding.String(), tt.queryMode), func() { + events := storagemock.NewEvents(s.T()) + + switch tt.queryMode { + case IndexQueryModeLocalOnly: + // not applicable + return + case IndexQueryModeExecutionNodesOnly: + // only calls to EN, no calls to storage + case IndexQueryModeFailover: + // all calls to storage fail + events.On("ByBlockID", mock.Anything).Return(nil, storage.ErrNotFound) + } + + backend := s.defaultBackend() + backend.queryMode = tt.queryMode + backend.events = events + + s.setupENSuccessResponse(targetEvent, s.blocks) + + response, err := backend.GetEventsForBlockIDs(ctx, targetEvent, s.blockIDs, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + + response, err = backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + }) + + s.Run(fmt.Sprintf("mixed storage & en - %s - %s", tt.encoding.String(), tt.queryMode), func() { + events := storagemock.NewEvents(s.T()) + + switch tt.queryMode { + case IndexQueryModeLocalOnly, IndexQueryModeExecutionNodesOnly: + // not applicable + return + case IndexQueryModeFailover: + // only failing blocks queried from EN + s.setupENSuccessResponse(targetEvent, s.blocks[0:2]) + } + + // the first 2 blocks are not available from storage, and should be fetched from the EN + events.On("ByBlockID", s.blockIDs[0]).Return(nil, storage.ErrNotFound) + events.On("ByBlockID", s.blockIDs[1]).Return(nil, storage.ErrNotFound) + events.On("ByBlockID", s.blockIDs[2]).Return(s.blockEvents, nil) + events.On("ByBlockID", s.blockIDs[3]).Return(s.blockEvents, nil) + events.On("ByBlockID", s.blockIDs[4]).Return(s.blockEvents, nil) + + backend := s.defaultBackend() + backend.queryMode = tt.queryMode + backend.events = events + + response, err := backend.GetEventsForBlockIDs(ctx, targetEvent, s.blockIDs, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + + response, err = backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, tt.encoding) + s.Require().NoError(err) + s.assertResponse(response, tt.encoding) + }) + } +} + +func (s *BackendEventsSuite) TestGetEventsForHeightRange_HandlesErrors() { + ctx := context.Background() + + startHeight := s.blocks[0].Header.Height + endHeight := s.sealedHead.Height + encoding := entities.EventEncodingVersion_CCF_V0 + + s.Run("returns error for endHeight < startHeight", func() { + backend := s.defaultBackend() + endHeight := startHeight - 1 + + response, err := backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, encoding) + s.Assert().Equal(codes.InvalidArgument, status.Code(err)) + s.Assert().Nil(response) + }) + + s.Run("returns error for range larger than max", func() { + backend := s.defaultBackend() + endHeight := startHeight + DefaultMaxHeightRange + + response, err := backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, encoding) + s.Assert().Equal(codes.InvalidArgument, status.Code(err)) + s.Assert().Nil(response) + }) + + s.Run("throws irrecoverable if sealed header not available", func() { + s.state.On("Sealed").Return(s.snapshot) + s.snapshot.On("Head").Return(nil, storage.ErrNotFound).Once() + + signCtxErr := irrecoverable.NewExceptionf("failed to lookup sealed header: %w", storage.ErrNotFound) + signalerCtx := irrecoverable.WithSignalerContext(context.Background(), + irrecoverable.NewMockSignalerContextExpectError(s.T(), ctx, signCtxErr)) + + backend := s.defaultBackend() + + response, err := backend.GetEventsForHeightRange(signalerCtx, targetEvent, startHeight, endHeight, encoding) + // these will never be returned in production + s.Assert().Equal(codes.Unknown, status.Code(err)) + s.Assert().Nil(response) + }) + + s.state.On("Sealed").Return(s.snapshot) + s.snapshot.On("Head").Return(s.sealedHead, nil) + + s.Run("returns error for startHeight > sealed height", func() { + backend := s.defaultBackend() + startHeight := s.sealedHead.Height + 1 + endHeight := startHeight + 1 + + response, err := backend.GetEventsForHeightRange(ctx, targetEvent, startHeight, endHeight, encoding) + s.Assert().Equal(codes.OutOfRange, status.Code(err)) + s.Assert().Nil(response) + }) +} + +func (s *BackendEventsSuite) TestGetEventsForBlockIDs_HandlesErrors() { + ctx := context.Background() + + encoding := entities.EventEncodingVersion_CCF_V0 + + s.Run("returns error when too many blockIDs requested", func() { + backend := s.defaultBackend() + backend.maxHeightRange = 3 + + response, err := backend.GetEventsForBlockIDs(ctx, targetEvent, s.blockIDs, encoding) + s.Assert().Equal(codes.InvalidArgument, status.Code(err)) + s.Assert().Nil(response) + }) + + s.Run("returns error for missing header", func() { + headers := storagemock.NewHeaders(s.T()) + backend := s.defaultBackend() + backend.headers = headers + + for i, blockID := range s.blockIDs { + // return error on the last header + if i == len(s.blocks)-1 { + headers.On("ByBlockID", blockID).Return(nil, storage.ErrNotFound) + continue + } + + headers.On("ByBlockID", blockID).Return(s.blocks[i].Header, nil) + } + + response, err := backend.GetEventsForBlockIDs(ctx, targetEvent, s.blockIDs, encoding) + s.Assert().Equal(codes.NotFound, status.Code(err)) + s.Assert().Nil(response) + }) +} + +func (s *BackendEventsSuite) assertResponse(response []flow.BlockEvents, encoding entities.EventEncodingVersion) { + s.Assert().Len(response, len(s.blocks)) + for i, block := range s.blocks { + s.Assert().Equal(block.Header.Height, response[i].BlockHeight) + s.Assert().Equal(block.Header.ID(), response[i].BlockID) + s.Assert().Len(response[i].Events, 1) + + s.assertEncoding(&response[i].Events[0], encoding) + } +} + +func (s *BackendEventsSuite) assertEncoding(event *flow.Event, encoding entities.EventEncodingVersion) { + var err error + switch encoding { + case entities.EventEncodingVersion_CCF_V0: + _, err = ccf.Decode(nil, event.Payload) + case entities.EventEncodingVersion_JSON_CDC_V0: + _, err = jsoncdc.Decode(nil, event.Payload) + default: + s.T().Errorf("unknown encoding: %s", encoding.String()) + } + s.Require().NoError(err) +} diff --git a/engine/access/rpc/backend/backend_scripts.go b/engine/access/rpc/backend/backend_scripts.go index 54e97817d89..3158575e098 100644 --- a/engine/access/rpc/backend/backend_scripts.go +++ b/engine/access/rpc/backend/backend_scripts.go @@ -37,7 +37,7 @@ type backendScripts struct { loggedScripts *lru.Cache[[md5.Size]byte, time.Time] nodeCommunicator Communicator scriptExecutor execution.ScriptExecutor - scriptExecMode ScriptExecutionMode + scriptExecMode IndexQueryMode } // scriptExecutionRequest encapsulates the data needed to execute a script to make it easier @@ -118,15 +118,15 @@ func (b *backendScripts) executeScript( scriptRequest *scriptExecutionRequest, ) ([]byte, error) { switch b.scriptExecMode { - case ScriptExecutionModeExecutionNodesOnly: + case IndexQueryModeExecutionNodesOnly: result, _, err := b.executeScriptOnAvailableExecutionNodes(ctx, scriptRequest) return result, err - case ScriptExecutionModeLocalOnly: + case IndexQueryModeLocalOnly: result, _, err := b.executeScriptLocally(ctx, scriptRequest) return result, err - case ScriptExecutionModeFailover: + case IndexQueryModeFailover: localResult, localDuration, localErr := b.executeScriptLocally(ctx, scriptRequest) if localErr == nil || isInvalidArgumentError(localErr) || status.Code(localErr) == codes.Canceled { return localResult, localErr @@ -143,7 +143,7 @@ func (b *backendScripts) executeScript( return execResult, execErr - case ScriptExecutionModeCompare: + case IndexQueryModeCompare: execResult, execDuration, execErr := b.executeScriptOnAvailableExecutionNodes(ctx, scriptRequest) // we can only compare the results if there were either no errors or a cadence error // since we cannot distinguish the EN error as caused by the block being pruned or some other reason, diff --git a/engine/access/rpc/backend/backend_scripts_test.go b/engine/access/rpc/backend/backend_scripts_test.go index 0563b760cbf..61df658359f 100644 --- a/engine/access/rpc/backend/backend_scripts_test.go +++ b/engine/access/rpc/backend/backend_scripts_test.go @@ -158,7 +158,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptOnExecutionNode_HappyPath() { s.setupENSuccessResponse(s.block.ID()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeExecutionNodesOnly + backend.scriptExecMode = IndexQueryModeExecutionNodesOnly s.Run("GetAccount", func() { s.testExecuteScriptAtLatestBlock(ctx, backend, codes.OK) @@ -186,7 +186,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptOnExecutionNode_Fails() { s.setupENFailingResponse(s.block.ID(), errToReturn) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeExecutionNodesOnly + backend.scriptExecMode = IndexQueryModeExecutionNodesOnly s.Run("GetAccount", func() { s.testExecuteScriptAtLatestBlock(ctx, backend, statusCode) @@ -211,7 +211,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptFromStorage_HappyPath() { Return(expectedResponse, nil) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor s.Run("GetAccount - happy path", func() { @@ -235,7 +235,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptFromStorage_Fails() { scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor testCases := []struct { @@ -300,7 +300,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptWithFailover_HappyPath() { scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeFailover + backend.scriptExecMode = IndexQueryModeFailover backend.scriptExecutor = scriptExecutor for _, errToReturn := range errors { @@ -331,7 +331,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptWithFailover_SkippedForCorrectCod scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeFailover + backend.scriptExecMode = IndexQueryModeFailover backend.scriptExecutor = scriptExecutor testCases := []struct { @@ -386,7 +386,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptWithFailover_ReturnsENErrors() { Return(nil, execution.ErrDataNotAvailable) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeFailover + backend.scriptExecMode = IndexQueryModeFailover backend.scriptExecutor = scriptExecutor s.Run("ExecuteScriptAtLatestBlock", func() { @@ -408,7 +408,7 @@ func (s *BackendScriptsSuite) TestExecuteScriptAtLatestBlockFromStorage_Inconsis scriptExecutor := execmock.NewScriptExecutor(s.T()) backend := s.defaultBackend() - backend.scriptExecMode = ScriptExecutionModeLocalOnly + backend.scriptExecMode = IndexQueryModeLocalOnly backend.scriptExecutor = scriptExecutor s.Run(fmt.Sprintf("ExecuteScriptAtLatestBlock - fails with %v", "inconsistent node's state"), func() { diff --git a/engine/access/rpc/backend/backend_test.go b/engine/access/rpc/backend/backend_test.go index fb434fb62b2..0647e3e9119 100644 --- a/engine/access/rpc/backend/backend_test.go +++ b/engine/access/rpc/backend/backend_test.go @@ -8,6 +8,7 @@ import ( "github.com/dgraph-io/badger/v2" accessproto "github.com/onflow/flow/protobuf/go/flow/access" + "github.com/onflow/flow/protobuf/go/flow/entities" entitiesproto "github.com/onflow/flow/protobuf/go/flow/entities" execproto "github.com/onflow/flow/protobuf/go/flow/execution" "github.com/rs/zerolog" @@ -65,7 +66,6 @@ type Suite struct { colClient *access.AccessAPIClient execClient *access.ExecutionAPIClient historicalAccessClient *access.AccessAPIClient - archiveClient *access.AccessAPIClient connectionFactory *connectionmock.ConnectionFactory communicator *backendmock.Communicator @@ -97,7 +97,6 @@ func (suite *Suite) SetupTest() { suite.receipts = new(storagemock.ExecutionReceipts) suite.results = new(storagemock.ExecutionResults) suite.colClient = new(access.AccessAPIClient) - suite.archiveClient = new(access.AccessAPIClient) suite.execClient = new(access.ExecutionAPIClient) suite.transactionResults = storagemock.NewLightTransactionResults(suite.T()) suite.chainID = flow.Testnet @@ -1473,159 +1472,6 @@ type mockCloser struct{} func (mc *mockCloser) Close() error { return nil } -func (suite *Suite) TestGetEventsForBlockIDs() { - suite.state.On("Sealed").Return(suite.snapshot, nil).Maybe() - suite.state.On("Final").Return(suite.snapshot, nil).Maybe() - - exeNodeEventEncodingVersion := entitiesproto.EventEncodingVersion_CCF_V0 - events := generator.GetEventsWithEncoding(10, exeNodeEventEncodingVersion) - validExecutorIdentities := flow.IdentityList{} - - setupStorage := func(n int) []*flow.Header { - headers := make([]*flow.Header, n) - ids := unittest.IdentityListFixture(2, unittest.WithRole(flow.RoleExecution)) - - for i := 0; i < n; i++ { - b := unittest.BlockFixture() - suite.headers. - On("ByBlockID", b.ID()). - Return(b.Header, nil).Once() - - headers[i] = b.Header - - receipt1 := unittest.ReceiptForBlockFixture(&b) - receipt1.ExecutorID = ids[0].NodeID - receipt2 := unittest.ReceiptForBlockFixture(&b) - receipt2.ExecutorID = ids[1].NodeID - receipt1.ExecutionResult = receipt2.ExecutionResult - suite.receipts. - On("ByBlockID", b.ID()). - Return(flow.ExecutionReceiptList{receipt1, receipt2}, nil).Once() - validExecutorIdentities = append(validExecutorIdentities, ids...) - } - - return headers - } - blockHeaders := setupStorage(5) - - suite.snapshot.On("Identities", mock.Anything).Return(validExecutorIdentities, nil) - validENIDs := flow.IdentifierList(validExecutorIdentities.NodeIDs()) - - // create a mock connection factory - connFactory := connectionmock.NewConnectionFactory(suite.T()) - connFactory.On("GetExecutionAPIClient", mock.Anything).Return(suite.execClient, &mockCloser{}, nil) - - // create the expected results from execution node and access node - exeResults := make([]*execproto.GetEventsForBlockIDsResponse_Result, len(blockHeaders)) - - for i := 0; i < len(blockHeaders); i++ { - exeResults[i] = &execproto.GetEventsForBlockIDsResponse_Result{ - BlockId: convert.IdentifierToMessage(blockHeaders[i].ID()), - BlockHeight: blockHeaders[i].Height, - Events: convert.EventsToMessages(events), - } - } - - expected := make([]flow.BlockEvents, len(blockHeaders)) - expectedEvents := generator.GetEventsWithEncoding(10, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - for i := 0; i < len(blockHeaders); i++ { - expected[i] = flow.BlockEvents{ - BlockID: blockHeaders[i].ID(), - BlockHeight: blockHeaders[i].Height, - BlockTimestamp: blockHeaders[i].Timestamp, - Events: expectedEvents, - } - } - - // create the execution node response - exeResp := &execproto.GetEventsForBlockIDsResponse{ - Results: exeResults, - EventEncodingVersion: exeNodeEventEncodingVersion, - } - - ctx := context.Background() - - blockIDs := make([]flow.Identifier, len(blockHeaders)) - for i, header := range blockHeaders { - blockIDs[i] = header.ID() - } - exeReq := &execproto.GetEventsForBlockIDsRequest{ - BlockIds: convert.IdentifiersToMessages(blockIDs), - Type: string(flow.EventAccountCreated), - } - - // create receipt mocks that always returns empty - receipts := new(storagemock.ExecutionReceipts) - receipts. - On("ByBlockID", mock.Anything). - Return(flow.ExecutionReceiptList{}, nil) - - // expect two calls to the executor api client (one for each of the following 2 test cases) - suite.execClient. - On("GetEventsForBlockIDs", ctx, exeReq). - Return(exeResp, nil). - Once() - - suite.Run("with an execution node chosen using block ID form the list of Fixed ENs", func() { - - params := suite.defaultBackendParams() - params.ConnFactory = connFactory - // set the fixed EN Identifiers to the generated execution IDs - params.FixedExecutionNodeIDs = validENIDs.Strings() - - // create the handler - backend, err := New(params) - suite.Require().NoError(err) - - // execute request - actual, err := backend.GetEventsForBlockIDs(ctx, string(flow.EventAccountCreated), blockIDs, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.checkResponse(actual, err) - - suite.Require().Equal(expected, actual) - }) - - suite.Run("with an empty block ID list", func() { - - params := suite.defaultBackendParams() - params.ExecutionReceipts = receipts - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = validENIDs.Strings() - - // create the handler - backend, err := New(params) - suite.Require().NoError(err) - - // execute request with an empty block id list and expect an empty list of events and no error - resp, err := backend.GetEventsForBlockIDs(ctx, string(flow.EventAccountCreated), []flow.Identifier{}, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - require.NoError(suite.T(), err) - require.Empty(suite.T(), resp) - }) - - for _, version := range eventEncodingVersions { - suite.Run(fmt.Sprintf("test %s event encoding version for GetEventsForBlockIDs", version.String()), func() { - params := suite.defaultBackendParams() - params.ExecutionReceipts = receipts - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = validENIDs.Strings() - - // create the handler - backend, err := New(params) - suite.Require().NoError(err) - - // execute request with an empty block id list and expect an empty list of events and no error - result, err := backend.GetEventsForBlockIDs(ctx, string(flow.EventAccountCreated), []flow.Identifier{}, version) - expectedResult := generator.GetEventsWithEncoding(1, version) - suite.checkResponse(result, err) - - for _, blockEvent := range result { - suite.Assert().Equal(blockEvent.Events, expectedResult) - } - }) - } - - suite.assertAllExpectations() -} - func (suite *Suite) TestGetExecutionResultByID() { suite.state.On("Sealed").Return(suite.snapshot, nil).Maybe() @@ -1748,300 +1594,6 @@ func (suite *Suite) TestGetExecutionResultByBlockID() { suite.assertAllExpectations() } -func (suite *Suite) TestGetEventsForHeightRange() { - ctx := context.Background() - const minHeight uint64 = 5 - const maxHeight uint64 = 10 - var headHeight uint64 - var blockHeaders []*flow.Header - var nodeIdentities flow.IdentityList - - headersDB := make(map[uint64]*flow.Header) // backend for storage.Headers - var head *flow.Header // backend for Snapshot.Head - - state := new(protocol.State) - snapshot := new(protocol.Snapshot) - state.On("Final").Return(snapshot, nil) - state.On("Sealed").Return(snapshot, nil) - - rootHeader := unittest.BlockHeaderFixture() - stateParams := new(protocol.Params) - stateParams.On("FinalizedRoot").Return(rootHeader, nil) - state.On("Params").Return(stateParams).Maybe() - - snapshot.On("Identities", mock.Anything).Return( - func(_ flow.IdentityFilter) flow.IdentityList { - return nodeIdentities - }, - func(flow.IdentityFilter) error { return nil }, - ) - - // mock Headers to pull from Headers backend - suite.headers.On("ByHeight", mock.Anything).Return( - func(height uint64) *flow.Header { - return headersDB[height] - }, - func(height uint64) error { - _, ok := headersDB[height] - if !ok { - return storage.ErrNotFound - } - return nil - }).Maybe() - - setupHeadHeight := func(height uint64) { - header := unittest.BlockHeaderFixture() // create a mock header - header.Height = height // set the header height - head = header - } - - setupStorage := func(min uint64, max uint64) ([]*flow.Header, []*flow.ExecutionReceipt, flow.IdentityList) { - headersDB = make(map[uint64]*flow.Header) // reset backend - - var headers []*flow.Header - var ers []*flow.ExecutionReceipt - var enIDs flow.IdentityList - for i := min; i <= max; i++ { - block := unittest.BlockFixture() - header := block.Header - headersDB[i] = header - headers = append(headers, header) - newErs, ids := suite.setupReceipts(&block) - ers = append(ers, newErs...) - enIDs = append(enIDs, ids...) - } - return headers, ers, enIDs - } - - setupExecClient := func() []flow.BlockEvents { - blockIDs := make([]flow.Identifier, len(blockHeaders)) - for i, header := range blockHeaders { - blockIDs[i] = header.ID() - } - execReq := &execproto.GetEventsForBlockIDsRequest{ - BlockIds: convert.IdentifiersToMessages(blockIDs), - Type: string(flow.EventAccountCreated), - } - - results := make([]flow.BlockEvents, len(blockHeaders)) - exeResults := make([]*execproto.GetEventsForBlockIDsResponse_Result, len(blockHeaders)) - - exeNodeEventEncodingVersion := entitiesproto.EventEncodingVersion_CCF_V0 - - for i, header := range blockHeaders { - events := generator.GetEventsWithEncoding(1, exeNodeEventEncodingVersion) - height := header.Height - - results[i] = flow.BlockEvents{ - BlockID: header.ID(), - BlockHeight: height, - BlockTimestamp: header.Timestamp, - Events: generator.GetEventsWithEncoding(1, entitiesproto.EventEncodingVersion_JSON_CDC_V0), - } - - exeResults[i] = &execproto.GetEventsForBlockIDsResponse_Result{ - BlockId: convert.IdentifierToMessage(header.ID()), - BlockHeight: header.Height, - Events: convert.EventsToMessages(events), - } - } - - exeResp := &execproto.GetEventsForBlockIDsResponse{ - Results: exeResults, - EventEncodingVersion: exeNodeEventEncodingVersion, - } - - suite.execClient. - On("GetEventsForBlockIDs", ctx, execReq). - Return(exeResp, nil). - Once() - - return results - } - - // tests that signaler context received error when node state is inconsistent - suite.Run("inconsistent node's state", func() { - headHeight = maxHeight - 1 - setupHeadHeight(headHeight) - - // setup mocks - stateParams.On("SporkID").Return(unittest.IdentifierFixture(), nil) - stateParams.On("ProtocolVersion").Return(uint(unittest.Uint64InRange(10, 30)), nil) - stateParams.On("SporkRootBlockHeight").Return(headHeight, nil) - stateParams.On("SealedRoot").Return(head, nil) - - params := suite.defaultBackendParams() - params.State = state - - backend, err := New(params) - suite.Require().NoError(err) - - err = fmt.Errorf("inconsistent node's state") - snapshot.On("Head").Return(nil, err).Once() - - signCtxErr := irrecoverable.NewExceptionf("failed to lookup sealed header: %w", err) - signalerCtx := irrecoverable.WithSignalerContext(context.Background(), - irrecoverable.NewMockSignalerContextExpectError(suite.T(), context.Background(), signCtxErr)) - - actual, err := backend.GetEventsForHeightRange(signalerCtx, string(flow.EventAccountCreated), minHeight, maxHeight, - entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.Require().Error(err) - suite.Require().Nil(actual) - }) - - connFactory := suite.setupConnectionFactory() - // mock snapshot to return head backend - snapshot.On("Head").Return( - func() *flow.Header { return head }, - func() error { return nil }, - ) - - //suite.state = state - suite.Run("invalid request max height < min height", func() { - params := suite.defaultBackendParams() - params.ConnFactory = connFactory - - backend, err := New(params) - suite.Require().NoError(err) - - _, err = backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), maxHeight, minHeight, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.Require().Error(err) - - suite.assertAllExpectations() // assert that request was not sent to execution node - }) - - suite.Run("valid request with min_height < max_height < last_sealed_block_height", func() { - - headHeight = maxHeight + 1 - - // setup mocks - setupHeadHeight(headHeight) - blockHeaders, _, nodeIdentities = setupStorage(minHeight, maxHeight) - expectedResp := setupExecClient() - fixedENIdentifiersStr := flow.IdentifierList(nodeIdentities.NodeIDs()).Strings() - - stateParams.On("SporkID").Return(unittest.IdentifierFixture(), nil) - stateParams.On("ProtocolVersion").Return(uint(unittest.Uint64InRange(10, 30)), nil) - stateParams.On("SporkRootBlockHeight").Return(headHeight, nil) - stateParams.On("SealedRoot").Return(head, nil) - - params := suite.defaultBackendParams() - params.State = state - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIdentifiersStr - - backend, err := New(params) - suite.Require().NoError(err) - - // execute request - actualResp, err := backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), minHeight, maxHeight, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - - // check response - suite.checkResponse(actualResp, err) - suite.assertAllExpectations() - suite.Require().Equal(expectedResp, actualResp) - }) - - suite.Run("valid request with max_height > last_sealed_block_height", func() { - headHeight = maxHeight - 1 - setupHeadHeight(headHeight) - blockHeaders, _, nodeIdentities = setupStorage(minHeight, headHeight) - expectedResp := setupExecClient() - fixedENIdentifiersStr := flow.IdentifierList(nodeIdentities.NodeIDs()).Strings() - - stateParams.On("SporkID").Return(unittest.IdentifierFixture(), nil) - stateParams.On("ProtocolVersion").Return(uint(unittest.Uint64InRange(10, 30)), nil) - stateParams.On("SporkRootBlockHeight").Return(headHeight, nil) - stateParams.On("SealedRoot").Return(head, nil) - - params := suite.defaultBackendParams() - params.State = state - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIdentifiersStr - - backend, err := New(params) - suite.Require().NoError(err) - - actualResp, err := backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), minHeight, maxHeight, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.checkResponse(actualResp, err) - - suite.assertAllExpectations() - suite.Require().Equal(expectedResp, actualResp) - }) - - // set max height range to 1 and request range of 2 - suite.Run("invalid request exceeding max height range", func() { - headHeight = maxHeight - 1 - setupHeadHeight(headHeight) - blockHeaders, _, nodeIdentities = setupStorage(minHeight, headHeight) - fixedENIdentifiersStr := flow.IdentifierList(nodeIdentities.NodeIDs()).Strings() - - params := suite.defaultBackendParams() - params.ConnFactory = connFactory - params.MaxHeightRange = 1 - params.FixedExecutionNodeIDs = fixedENIdentifiersStr - - backend, err := New(params) - suite.Require().NoError(err) - - _, err = backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), minHeight, minHeight+1, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.Require().Error(err) - }) - - suite.Run("invalid request last_sealed_block_height < min height", func() { - - // set sealed height to one less than the request start height - headHeight = minHeight - 1 - - // setup mocks - setupHeadHeight(headHeight) - blockHeaders, _, nodeIdentities = setupStorage(minHeight, maxHeight) - fixedENIdentifiersStr := flow.IdentifierList(nodeIdentities.NodeIDs()).Strings() - - params := suite.defaultBackendParams() - params.State = state - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIdentifiersStr - - backend, err := New(params) - suite.Require().NoError(err) - - _, err = backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), minHeight, maxHeight, entitiesproto.EventEncodingVersion_JSON_CDC_V0) - suite.Require().Error(err) - }) - - for _, version := range eventEncodingVersions { - suite.Run(fmt.Sprintf("test %s event encoding version for GetEventsForHeightRange", version.String()), func() { - headHeight = maxHeight - 1 - setupHeadHeight(headHeight) - blockHeaders, _, nodeIdentities = setupStorage(minHeight, headHeight) - _ = setupExecClient() - fixedENIdentifiersStr := nodeIdentities.NodeIDs().Strings() - - stateParams.On("SporkID").Return(unittest.IdentifierFixture(), nil) - stateParams.On("ProtocolVersion").Return(uint(unittest.Uint64InRange(10, 30)), nil) - stateParams.On("SporkRootBlockHeight").Return(headHeight, nil) - stateParams.On("SealedRoot").Return(head, nil) - - params := suite.defaultBackendParams() - params.State = state - params.ConnFactory = connFactory - params.FixedExecutionNodeIDs = fixedENIdentifiersStr - - backend, err := New(params) - suite.Require().NoError(err) - - result, err := backend.GetEventsForHeightRange(ctx, string(flow.EventAccountCreated), minHeight, maxHeight, version) - expectedResult := generator.GetEventsWithEncoding(1, version) - suite.checkResponse(result, err) - - for _, blockEvent := range result { - suite.Assert().Equal(blockEvent.Events, expectedResult) - } - }) - } -} - func (suite *Suite) TestGetNodeVersionInfo() { sporkRootBlock := unittest.BlockHeaderFixture() nodeRootBlock := unittest.BlockHeaderFixture(unittest.WithHeaderHeight(sporkRootBlock.Height + 100)) @@ -2382,15 +1934,14 @@ func (suite *Suite) TestGetTransactionResultEventEncodingVersion() { backend, err := New(params) suite.Require().NoError(err) - exeNodeEventEncodingVersion := entitiesproto.EventEncodingVersion_CCF_V0 - events := generator.GetEventsWithEncoding(1, exeNodeEventEncodingVersion) - eventMessages := convert.EventsToMessages(events) + ccfEvents, jsoncdcEvents := generateEncodedEvents(suite.T(), 1) + eventMessages := convert.EventsToMessages(ccfEvents) for _, version := range eventEncodingVersions { suite.Run(fmt.Sprintf("test %s event encoding version for GetTransactionResult", version.String()), func() { exeEventResp := &execproto.GetTransactionResultResponse{ Events: eventMessages, - EventEncodingVersion: exeNodeEventEncodingVersion, + EventEncodingVersion: entitiesproto.EventEncodingVersion_CCF_V0, } suite.execClient. @@ -2402,9 +1953,16 @@ func (suite *Suite) TestGetTransactionResultEventEncodingVersion() { Once() result, err := backend.GetTransactionResult(ctx, txId, blockId, flow.ZeroID, version) - expectedResult := generator.GetEventsWithEncoding(1, version) suite.checkResponse(result, err) + var expectedResult []flow.Event + switch version { + case entitiesproto.EventEncodingVersion_CCF_V0: + expectedResult = append(expectedResult, ccfEvents...) + case entitiesproto.EventEncodingVersion_JSON_CDC_V0: + expectedResult = append(expectedResult, jsoncdcEvents...) + } + suite.Assert().Equal(result.Events, expectedResult) }) } @@ -2443,8 +2001,8 @@ func (suite *Suite) TestGetTransactionResultByIndexAndBlockIdEventEncodingVersio suite.Require().NoError(err) exeNodeEventEncodingVersion := entitiesproto.EventEncodingVersion_CCF_V0 - events := generator.GetEventsWithEncoding(1, exeNodeEventEncodingVersion) - eventMessages := convert.EventsToMessages(events) + ccfEvents, jsoncdcEvents := generateEncodedEvents(suite.T(), 1) + eventMessages := convert.EventsToMessages(ccfEvents) for _, version := range eventEncodingVersions { suite.Run(fmt.Sprintf("test %s event encoding version for GetTransactionResultByIndex", version.String()), func() { @@ -2464,8 +2022,15 @@ func (suite *Suite) TestGetTransactionResultByIndexAndBlockIdEventEncodingVersio result, err := backend.GetTransactionResultByIndex(ctx, blockId, index, version) suite.checkResponse(result, err) - expectedResult := generator.GetEventsWithEncoding(1, version) - suite.Assert().Equal(result.Events, expectedResult) + var expectedResult []flow.Event + switch version { + case entitiesproto.EventEncodingVersion_CCF_V0: + expectedResult = append(expectedResult, ccfEvents...) + case entitiesproto.EventEncodingVersion_JSON_CDC_V0: + expectedResult = append(expectedResult, jsoncdcEvents...) + } + + suite.Assert().Equal(expectedResult, result.Events) }) suite.Run(fmt.Sprintf("test %s event encoding version for GetTransactionResultsByBlockID", version.String()), func() { @@ -2488,7 +2053,14 @@ func (suite *Suite) TestGetTransactionResultByIndexAndBlockIdEventEncodingVersio results, err := backend.GetTransactionResultsByBlockID(ctx, blockId, version) suite.checkResponse(results, err) - expectedResult := generator.GetEventsWithEncoding(1, version) + var expectedResult []flow.Event + switch version { + case entitiesproto.EventEncodingVersion_CCF_V0: + expectedResult = append(expectedResult, ccfEvents...) + case entitiesproto.EventEncodingVersion_JSON_CDC_V0: + expectedResult = append(expectedResult, jsoncdcEvents...) + } + for _, result := range results { suite.Assert().Equal(result.Events, expectedResult) } @@ -2592,6 +2164,17 @@ func getEvents(n int) []flow.Event { return events } +func generateEncodedEvents(t *testing.T, n int) ([]flow.Event, []flow.Event) { + ccfEvents := generator.GetEventsWithEncoding(n, entities.EventEncodingVersion_CCF_V0) + jsonEvents := make([]flow.Event, n) + for i, e := range ccfEvents { + jsonEvent, err := convert.CcfEventToJsonEvent(e) + require.NoError(t, err) + jsonEvents[i] = *jsonEvent + } + return ccfEvents, jsonEvents +} + func (suite *Suite) defaultBackendParams() Params { return Params{ State: suite.state, diff --git a/engine/access/rpc/backend/backend_transactions.go b/engine/access/rpc/backend/backend_transactions.go index bd20d419df9..6d0303d5ed8 100644 --- a/engine/access/rpc/backend/backend_transactions.go +++ b/engine/access/rpc/backend/backend_transactions.go @@ -91,7 +91,7 @@ func (b *backendTransactions) trySendTransaction(ctx context.Context, tx *flow.T } // otherwise choose all collection nodes to try - collNodes, err := b.chooseCollectionNodes(tx) + collNodes, err := b.chooseCollectionNodes(tx.ID()) if err != nil { return fmt.Errorf("failed to determine collection node for tx %x: %w", tx, err) } @@ -122,7 +122,7 @@ func (b *backendTransactions) trySendTransaction(ctx context.Context, tx *flow.T // chooseCollectionNodes finds a random subset of size sampleSize of collection node addresses from the // collection node cluster responsible for the given tx -func (b *backendTransactions) chooseCollectionNodes(tx *flow.TransactionBody) (flow.IdentityList, error) { +func (b *backendTransactions) chooseCollectionNodes(txID flow.Identifier) (flow.IdentityList, error) { // retrieve the set of collector clusters clusters, err := b.state.Final().Epochs().Current().Clustering() @@ -131,18 +131,20 @@ func (b *backendTransactions) chooseCollectionNodes(tx *flow.TransactionBody) (f } // get the cluster responsible for the transaction - targetNodes, ok := clusters.ByTxID(tx.ID()) + targetNodes, ok := clusters.ByTxID(txID) if !ok { - return nil, fmt.Errorf("could not get local cluster by txID: %x", tx.ID()) + return nil, fmt.Errorf("could not get local cluster by txID: %x", txID) } return targetNodes, nil } // sendTransactionToCollection sends the transaction to the given collection node via grpc -func (b *backendTransactions) sendTransactionToCollector(ctx context.Context, +func (b *backendTransactions) sendTransactionToCollector( + ctx context.Context, tx *flow.TransactionBody, - collectionNodeAddr string) error { + collectionNodeAddr string, +) error { collectionRPC, closer, err := b.connFactory.GetAccessAPIClient(collectionNodeAddr, nil) if err != nil { diff --git a/engine/access/rpc/backend/config.go b/engine/access/rpc/backend/config.go index fd4d54bcab9..2cdbe3bb8cc 100644 --- a/engine/access/rpc/backend/config.go +++ b/engine/access/rpc/backend/config.go @@ -17,52 +17,53 @@ type Config struct { FixedExecutionNodeIDs []string // fixed list of execution node IDs to choose from if no node ID can be chosen from the PreferredExecutionNodeIDs CircuitBreakerConfig connection.CircuitBreakerConfig // the configuration for circuit breaker ScriptExecutionMode string // the mode in which scripts are executed + EventQueryMode string // the mode in which events are queried } -type ScriptExecutionMode int +type IndexQueryMode int const ( - // ScriptExecutionModeLocalOnly executes scripts and gets accounts using only local storage - ScriptExecutionModeLocalOnly ScriptExecutionMode = iota + 1 + // IndexQueryModeLocalOnly executes scripts and gets accounts using only local storage + IndexQueryModeLocalOnly IndexQueryMode = iota + 1 - // ScriptExecutionModeExecutionNodesOnly executes scripts and gets accounts using only + // IndexQueryModeExecutionNodesOnly executes scripts and gets accounts using only // execution nodes - ScriptExecutionModeExecutionNodesOnly + IndexQueryModeExecutionNodesOnly - // ScriptExecutionModeFailover executes scripts and gets accounts using local storage first, + // IndexQueryModeFailover executes scripts and gets accounts using local storage first, // then falls back to execution nodes if data is not available for the height or if request // failed due to a non-user error. - ScriptExecutionModeFailover + IndexQueryModeFailover - // ScriptExecutionModeCompare executes scripts and gets accounts using both local storage and + // IndexQueryModeCompare executes scripts and gets accounts using both local storage and // execution nodes and compares the results. The execution node result is always returned. - ScriptExecutionModeCompare + IndexQueryModeCompare ) -func ParseScriptExecutionMode(s string) (ScriptExecutionMode, error) { +func ParseIndexQueryMode(s string) (IndexQueryMode, error) { switch s { - case ScriptExecutionModeLocalOnly.String(): - return ScriptExecutionModeLocalOnly, nil - case ScriptExecutionModeExecutionNodesOnly.String(): - return ScriptExecutionModeExecutionNodesOnly, nil - case ScriptExecutionModeFailover.String(): - return ScriptExecutionModeFailover, nil - case ScriptExecutionModeCompare.String(): - return ScriptExecutionModeCompare, nil + case IndexQueryModeLocalOnly.String(): + return IndexQueryModeLocalOnly, nil + case IndexQueryModeExecutionNodesOnly.String(): + return IndexQueryModeExecutionNodesOnly, nil + case IndexQueryModeFailover.String(): + return IndexQueryModeFailover, nil + case IndexQueryModeCompare.String(): + return IndexQueryModeCompare, nil default: return 0, errors.New("invalid script execution mode") } } -func (m ScriptExecutionMode) String() string { +func (m IndexQueryMode) String() string { switch m { - case ScriptExecutionModeLocalOnly: + case IndexQueryModeLocalOnly: return "local-only" - case ScriptExecutionModeExecutionNodesOnly: + case IndexQueryModeExecutionNodesOnly: return "execution-nodes-only" - case ScriptExecutionModeFailover: + case IndexQueryModeFailover: return "failover" - case ScriptExecutionModeCompare: + case IndexQueryModeCompare: return "compare" default: return "" diff --git a/engine/access/rpc/backend/script_comparer.go b/engine/access/rpc/backend/script_comparer.go index 4bd219c4c96..e7ec9f3c489 100644 --- a/engine/access/rpc/backend/script_comparer.go +++ b/engine/access/rpc/backend/script_comparer.go @@ -53,7 +53,7 @@ func (c *scriptResultComparison) compare(execResult, localResult *scriptResult) if isOutOfRangeError(localResult.err) { c.metrics.ScriptExecutionNotIndexed() c.logComparison(execResult, localResult, - "script execution results do not match EN because data is not indexed yet") + "script execution results do not match EN because data is not indexed yet", false) return false } @@ -66,7 +66,7 @@ func (c *scriptResultComparison) compare(execResult, localResult *scriptResult) c.metrics.ScriptExecutionErrorMismatch() c.logComparison(execResult, localResult, - "cadence errors from local execution do not match and EN") + "cadence errors from local execution do not match EN", true) return false } @@ -77,12 +77,12 @@ func (c *scriptResultComparison) compare(execResult, localResult *scriptResult) c.metrics.ScriptExecutionResultMismatch() c.logComparison(execResult, localResult, - "script execution results from local execution do not match EN") + "script execution results from local execution do not match EN", true) return false } // logScriptExecutionComparison logs the script execution comparison between local execution and execution node -func (c *scriptResultComparison) logComparison(execResult, localResult *scriptResult, msg string) { +func (c *scriptResultComparison) logComparison(execResult, localResult *scriptResult, msg string, useError bool) { args := make([]string, len(c.request.arguments)) for i, arg := range c.request.arguments { args[i] = string(arg) @@ -109,7 +109,11 @@ func (c *scriptResultComparison) logComparison(execResult, localResult *scriptRe lgCtx = lgCtx.Dur("local_duration_ms", localResult.duration) lg := lgCtx.Logger() - lg.Debug().Msg(msg) + if useError { + lg.Error().Msg(msg) + } else { + lg.Debug().Msg(msg) + } } func isOutOfRangeError(err error) bool { diff --git a/engine/access/state_stream/backend/backend.go b/engine/access/state_stream/backend/backend.go index 1afc020a5fd..f2a35ebe97d 100644 --- a/engine/access/state_stream/backend/backend.go +++ b/engine/access/state_stream/backend/backend.go @@ -90,6 +90,7 @@ func New( config Config, state protocol.State, headers storage.Headers, + events storage.Events, seals storage.Seals, results storage.ExecutionResults, execDataStore execution_data.ExecutionDataStore, @@ -98,6 +99,7 @@ func New( rootHeight uint64, highestAvailableHeight uint64, registers *execution.RegistersAsyncStore, + useEventsIndex bool, ) (*StateStreamBackend, error) { logger := log.With().Str("module", "state_stream_api").Logger() @@ -136,12 +138,15 @@ func New( b.EventsBackend = EventsBackend{ log: logger, + events: events, + headers: headers, broadcaster: broadcaster, sendTimeout: config.ClientSendTimeout, responseLimit: config.ResponseLimit, sendBufferSize: int(config.ClientSendBufferSize), getExecutionData: b.getExecutionData, getStartHeight: b.getStartHeight, + useIndex: useEventsIndex, } return b, nil diff --git a/engine/access/state_stream/backend/backend_events.go b/engine/access/state_stream/backend/backend_events.go index ac1297f0e0f..303f8e09e32 100644 --- a/engine/access/state_stream/backend/backend_events.go +++ b/engine/access/state_stream/backend/backend_events.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/engine/access/state_stream" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" ) @@ -21,6 +22,8 @@ type EventsResponse struct { type EventsBackend struct { log zerolog.Logger + events storage.Events + headers storage.Headers broadcaster *engine.Broadcaster sendTimeout time.Duration responseLimit float64 @@ -28,6 +31,8 @@ type EventsBackend struct { getExecutionData GetExecutionDataFunc getStartHeight GetStartHeightFunc + + useIndex bool } func (b EventsBackend) SubscribeEvents(ctx context.Context, startBlockID flow.Identifier, startHeight uint64, filter state_stream.EventFilter) state_stream.Subscription { @@ -43,27 +48,61 @@ func (b EventsBackend) SubscribeEvents(ctx context.Context, startBlockID flow.Id return sub } +// getResponseFactory returns a function function that returns the event response for a given height. func (b EventsBackend) getResponseFactory(filter state_stream.EventFilter) GetDataByHeightFunc { - return func(ctx context.Context, height uint64) (interface{}, error) { - executionData, err := b.getExecutionData(ctx, height) - if err != nil { - return nil, fmt.Errorf("could not get execution data for block %d: %w", height, err) + return func(ctx context.Context, height uint64) (response interface{}, err error) { + if b.useIndex { + response, err = b.getEventsFromStorage(height, filter) + } else { + response, err = b.getEventsFromExecutionData(ctx, height, filter) } - events := []flow.Event{} - for _, chunkExecutionData := range executionData.ChunkExecutionDatas { - events = append(events, filter.Filter(chunkExecutionData.Events)...) + if err == nil && b.log.GetLevel() == zerolog.TraceLevel { + eventsResponse := response.(*EventsResponse) + b.log.Trace(). + Hex("block_id", logging.ID(eventsResponse.BlockID)). + Uint64("height", height). + Int("events", len(eventsResponse.Events)). + Msg("sending events") } + return + } +} + +// getEventsFromExecutionData returns the events for a given height extractd from the execution data. +func (b EventsBackend) getEventsFromExecutionData(ctx context.Context, height uint64, filter state_stream.EventFilter) (*EventsResponse, error) { + executionData, err := b.getExecutionData(ctx, height) + if err != nil { + return nil, fmt.Errorf("could not get execution data for block %d: %w", height, err) + } + + var events flow.EventsList + for _, chunkExecutionData := range executionData.ChunkExecutionDatas { + events = append(events, filter.Filter(chunkExecutionData.Events)...) + } - b.log.Trace(). - Hex("block_id", logging.ID(executionData.BlockID)). - Uint64("height", height). - Msgf("sending %d events", len(events)) + return &EventsResponse{ + BlockID: executionData.BlockID, + Height: height, + Events: events, + }, nil +} - return &EventsResponse{ - BlockID: executionData.BlockID, - Height: height, - Events: events, - }, nil +// getEventsFromStorage returns the events for a given height from the index storage. +func (b EventsBackend) getEventsFromStorage(height uint64, filter state_stream.EventFilter) (*EventsResponse, error) { + blockID, err := b.headers.BlockIDByHeight(height) + if err != nil { + return nil, fmt.Errorf("could not get header for height %d: %w", height, err) } + + events, err := b.events.ByBlockID(blockID) + if err != nil { + return nil, fmt.Errorf("could not get events for block %d: %w", height, err) + } + + return &EventsResponse{ + BlockID: blockID, + Height: height, + Events: filter.Filter(events), + }, nil } diff --git a/engine/access/state_stream/backend/backend_events_test.go b/engine/access/state_stream/backend/backend_events_test.go index c9a093ebe82..d01fbbad781 100644 --- a/engine/access/state_stream/backend/backend_events_test.go +++ b/engine/access/state_stream/backend/backend_events_test.go @@ -7,6 +7,7 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "google.golang.org/grpc/codes" @@ -15,6 +16,7 @@ import ( "github.com/onflow/flow-go/engine/access/state_stream" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/mocks" ) type BackendEventsSuite struct { @@ -29,8 +31,24 @@ func (s *BackendEventsSuite) SetupTest() { s.BackendExecutionDataSuite.SetupTest() } -// TestSubscribeEvents tests the SubscribeEvents method happy path -func (s *BackendEventsSuite) TestSubscribeEvents() { +// TestSubscribeEventsFromExecutionData tests the SubscribeEvents method happy path for events +// extracted from ExecutionData +func (s *BackendEventsSuite) TestSubscribeEventsFromExecutionData() { + s.runTestSubscribeEvents() +} + +// TestSubscribeEventsFromLocalStorage tests the SubscribeEvents method happy path for events +// extracted from local storage +func (s *BackendEventsSuite) TestSubscribeEventsFromLocalStorage() { + s.backend.useIndex = true + s.events.On("ByBlockID", mock.AnythingOfType("flow.Identifier")).Return( + mocks.StorageMapGetter(s.blockEvents), + ) + + s.runTestSubscribeEvents() +} + +func (s *BackendEventsSuite) runTestSubscribeEvents() { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -77,9 +95,6 @@ func (s *BackendEventsSuite) TestSubscribeEvents() { }, } - // supports simple address comparisions for testing - chain := flow.MonotonicEmulator.Chain() - // create variations for each of the base test tests := make([]testType, 0, len(baseTests)*3) for _, test := range baseTests { @@ -90,13 +105,13 @@ func (s *BackendEventsSuite) TestSubscribeEvents() { t2 := test t2.name = fmt.Sprintf("%s - some events", test.name) - t2.filters, err = state_stream.NewEventFilter(state_stream.DefaultEventFilterConfig, chain, []string{string(testEventTypes[0])}, nil, nil) + t2.filters, err = state_stream.NewEventFilter(state_stream.DefaultEventFilterConfig, chainID.Chain(), []string{string(testEventTypes[0])}, nil, nil) require.NoError(s.T(), err) tests = append(tests, t2) t3 := test t3.name = fmt.Sprintf("%s - no events", test.name) - t3.filters, err = state_stream.NewEventFilter(state_stream.DefaultEventFilterConfig, chain, []string{"A.0x1.NonExistent.Event"}, nil, nil) + t3.filters, err = state_stream.NewEventFilter(state_stream.DefaultEventFilterConfig, chainID.Chain(), []string{"A.0x1.NonExistent.Event"}, nil, nil) require.NoError(s.T(), err) tests = append(tests, t3) } @@ -126,7 +141,7 @@ func (s *BackendEventsSuite) TestSubscribeEvents() { s.broadcaster.Publish() } - expectedEvents := flow.EventsList{} + var expectedEvents flow.EventsList for _, event := range s.blockEvents[b.ID()] { if test.filters.Match(event) { expectedEvents = append(expectedEvents, event) diff --git a/engine/access/state_stream/backend/backend_executiondata_test.go b/engine/access/state_stream/backend/backend_executiondata_test.go index 4a4d72a20c3..5b3ba2d5a4b 100644 --- a/engine/access/state_stream/backend/backend_executiondata_test.go +++ b/engine/access/state_stream/backend/backend_executiondata_test.go @@ -28,12 +28,14 @@ import ( "github.com/onflow/flow-go/storage" storagemock "github.com/onflow/flow-go/storage/mock" "github.com/onflow/flow-go/utils/unittest" + "github.com/onflow/flow-go/utils/unittest/mocks" ) +var chainID = flow.MonotonicEmulator var testEventTypes = []flow.EventType{ - "A.0x1.Foo.Bar", - "A.0x2.Zoo.Moo", - "A.0x3.Goo.Hoo", + unittest.EventTypeFixture(chainID), + unittest.EventTypeFixture(chainID), + unittest.EventTypeFixture(chainID), } type BackendExecutionDataSuite struct { @@ -43,6 +45,7 @@ type BackendExecutionDataSuite struct { params *protocolmock.Params snapshot *protocolmock.Snapshot headers *storagemock.Headers + events *storagemock.Events seals *storagemock.Seals results *storagemock.ExecutionResults registers *storagemock.RegisterIndex @@ -56,7 +59,7 @@ type BackendExecutionDataSuite struct { backend *StateStreamBackend blocks []*flow.Block - blockEvents map[flow.Identifier]flow.EventsList + blockEvents map[flow.Identifier][]flow.Event execDataMap map[flow.Identifier]*execution_data.BlockExecutionDataEntity blockMap map[uint64]*flow.Block sealMap map[flow.Identifier]*flow.Seal @@ -75,6 +78,7 @@ func (s *BackendExecutionDataSuite) SetupTest() { s.snapshot = protocolmock.NewSnapshot(s.T()) s.params = protocolmock.NewParams(s.T()) s.headers = storagemock.NewHeaders(s.T()) + s.events = storagemock.NewEvents(s.T()) s.seals = storagemock.NewSeals(s.T()) s.results = storagemock.NewExecutionResults(s.T()) @@ -96,7 +100,7 @@ func (s *BackendExecutionDataSuite) SetupTest() { blockCount := 5 s.execDataMap = make(map[flow.Identifier]*execution_data.BlockExecutionDataEntity, blockCount) - s.blockEvents = make(map[flow.Identifier]flow.EventsList, blockCount) + s.blockEvents = make(map[flow.Identifier][]flow.Event, blockCount) s.blockMap = make(map[uint64]*flow.Block, blockCount) s.sealMap = make(map[flow.Identifier]*flow.Seal, blockCount) s.resultMap = make(map[flow.Identifier]*flow.ExecutionResult, blockCount) @@ -170,82 +174,36 @@ func (s *BackendExecutionDataSuite) SetupTest() { s.snapshot.On("Head").Return(s.blocks[0].Header, nil).Maybe() s.seals.On("FinalizedSealForBlock", mock.AnythingOfType("flow.Identifier")).Return( - func(blockID flow.Identifier) *flow.Seal { - if seal, ok := s.sealMap[blockID]; ok { - return seal - } - return nil - }, - func(blockID flow.Identifier) error { - if _, ok := s.sealMap[blockID]; ok { - return nil - } - return storage.ErrNotFound - }, + mocks.StorageMapGetter(s.sealMap), ).Maybe() s.results.On("ByID", mock.AnythingOfType("flow.Identifier")).Return( - func(resultID flow.Identifier) *flow.ExecutionResult { - if result, ok := s.resultMap[resultID]; ok { - return result - } - return nil - }, - func(resultID flow.Identifier) error { - if _, ok := s.resultMap[resultID]; ok { - return nil - } - return storage.ErrNotFound - }, + mocks.StorageMapGetter(s.resultMap), ).Maybe() s.headers.On("ByBlockID", mock.AnythingOfType("flow.Identifier")).Return( - func(blockID flow.Identifier) *flow.Header { - for _, block := range s.blockMap { - if block.ID() == blockID { - return block.Header - } - } - return nil - }, - func(blockID flow.Identifier) error { + func(blockID flow.Identifier) (*flow.Header, error) { for _, block := range s.blockMap { if block.ID() == blockID { - return nil + return block.Header, nil } } - return storage.ErrNotFound + return nil, storage.ErrNotFound }, ).Maybe() s.headers.On("ByHeight", mock.AnythingOfType("uint64")).Return( - func(height uint64) *flow.Header { - if block, ok := s.blockMap[height]; ok { - return block.Header - } - return nil - }, - func(height uint64) error { - if _, ok := s.blockMap[height]; ok { - return nil - } - return storage.ErrNotFound - }, + mocks.ConvertStorageOutput( + mocks.StorageMapGetter(s.blockMap), + func(block *flow.Block) *flow.Header { return block.Header }, + ), ).Maybe() s.headers.On("BlockIDByHeight", mock.AnythingOfType("uint64")).Return( - func(height uint64) flow.Identifier { - if block, ok := s.blockMap[height]; ok { - return block.Header.ID() - } - return flow.ZeroID - }, - func(height uint64) error { - if _, ok := s.blockMap[height]; ok { - return nil - } - return storage.ErrNotFound - }, + mocks.ConvertStorageOutput( + mocks.StorageMapGetter(s.blockMap), + func(block *flow.Block) flow.Identifier { return block.ID() }, + ), ).Maybe() s.backend, err = New( @@ -253,6 +211,7 @@ func (s *BackendExecutionDataSuite) SetupTest() { conf, s.state, s.headers, + s.events, s.seals, s.results, s.eds, @@ -261,6 +220,7 @@ func (s *BackendExecutionDataSuite) SetupTest() { rootBlock.Header.Height, rootBlock.Header.Height, // initialize with no downloaded data s.registersAsync, + false, ) require.NoError(s.T(), err) } diff --git a/engine/access/state_stream/backend/handler_test.go b/engine/access/state_stream/backend/handler_test.go index 0d645c5e580..3cf9d656f8a 100644 --- a/engine/access/state_stream/backend/handler_test.go +++ b/engine/access/state_stream/backend/handler_test.go @@ -198,8 +198,7 @@ func TestGetExecutionDataByBlockID(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ccfEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_CCF_V0) - jsonEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_JSON_CDC_V0) + ccfEvents, jsonEvents := generateEvents(t, 3) tests := []struct { eventVersion entities.EventEncodingVersion @@ -345,8 +344,7 @@ func TestExecutionDataStream(t *testing.T) { } } - ccfEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_CCF_V0) - jsonEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_JSON_CDC_V0) + ccfEvents, jsonEvents := generateEvents(t, 3) tests := []struct { eventVersion entities.EventEncodingVersion @@ -472,9 +470,7 @@ func TestEventStream(t *testing.T) { } // generate events with a payload to include - // generators will produce identical event payloads (before encoding) - ccfEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_CCF_V0) - jsonEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_JSON_CDC_V0) + ccfEvents, jsonEvents := generateEvents(t, 3) tests := []struct { eventVersion entities.EventEncodingVersion @@ -605,6 +601,17 @@ func TestGetRegisterValues(t *testing.T) { }) } +func generateEvents(t *testing.T, n int) ([]flow.Event, []flow.Event) { + ccfEvents := generator.GetEventsWithEncoding(n, entities.EventEncodingVersion_CCF_V0) + jsonEvents := make([]flow.Event, len(ccfEvents)) + for i, e := range ccfEvents { + jsonEvent, err := convert.CcfEventToJsonEvent(e) + require.NoError(t, err) + jsonEvents[i] = *jsonEvent + } + return ccfEvents, jsonEvents +} + func makeConfig(maxGlobalStreams uint32) Config { return Config{ EventFilterConfig: state_stream.DefaultEventFilterConfig, diff --git a/engine/access/state_stream/event_test.go b/engine/access/state_stream/event_test.go deleted file mode 100644 index 3dbccd34406..00000000000 --- a/engine/access/state_stream/event_test.go +++ /dev/null @@ -1,79 +0,0 @@ -package state_stream_test - -import ( - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/engine/access/state_stream" - "github.com/onflow/flow-go/model/flow" -) - -func TestParseEvent(t *testing.T) { - t.Parallel() - - tests := []struct { - name string - eventType flow.EventType - expected state_stream.ParsedEvent - }{ - { - name: "flow event", - eventType: "flow.AccountCreated", - expected: state_stream.ParsedEvent{ - Type: state_stream.ProtocolEventType, - EventType: "flow.AccountCreated", - Contract: "flow", - ContractName: "flow", - Name: "AccountCreated", - }, - }, - { - name: "account event", - eventType: "A.0000000000000001.Contract1.EventA", - expected: state_stream.ParsedEvent{ - Type: state_stream.AccountEventType, - EventType: "A.0000000000000001.Contract1.EventA", - Address: "0000000000000001", - Contract: "A.0000000000000001.Contract1", - ContractName: "Contract1", - Name: "EventA", - }, - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - event, err := state_stream.ParseEvent(test.eventType) - require.NoError(t, err) - - assert.Equal(t, test.expected.Type, event.Type) - assert.Equal(t, test.expected.EventType, event.EventType) - assert.Equal(t, test.expected.Address, event.Address) - assert.Equal(t, test.expected.Contract, event.Contract) - assert.Equal(t, test.expected.Name, event.Name) - }) - } -} - -func TestParseEvent_Invalid(t *testing.T) { - t.Parallel() - - eventTypes := []flow.EventType{ - "", // not enough parts - "invalid", // not enough parts - "invalid.event", // invalid first part - "B.0000000000000001.invalid.event", // invalid first part - "flow", // incorrect number of parts for protocol event - "flow.invalid.event", // incorrect number of parts for protocol event - "A.0000000000000001.invalid", // incorrect number of parts for account event - "A.0000000000000001.invalid.a.b", // incorrect number of parts for account event - - } - - for _, eventType := range eventTypes { - _, err := state_stream.ParseEvent(eventType) - assert.Error(t, err, "expected error for event type: %s", eventType) - } -} diff --git a/engine/access/state_stream/filter.go b/engine/access/state_stream/filter.go index ab90b98240c..8936ba49e0a 100644 --- a/engine/access/state_stream/filter.go +++ b/engine/access/state_stream/filter.go @@ -4,6 +4,7 @@ import ( "fmt" "strings" + "github.com/onflow/flow-go/model/events" "github.com/onflow/flow-go/model/flow" ) @@ -71,7 +72,7 @@ func NewEventFilter( // with criteria that will never match. for _, event := range eventTypes { eventType := flow.EventType(event) - if err := validateEventType(eventType); err != nil { + if err := validateEventType(eventType, chain); err != nil { return EventFilter{}, err } f.EventTypes[eventType] = struct{}{} @@ -120,7 +121,7 @@ func (f *EventFilter) Match(event flow.Event) bool { return true } - parsed, err := ParseEvent(event.Type) + parsed, err := events.ParseEvent(event.Type) if err != nil { // TODO: log this error return false @@ -130,7 +131,7 @@ func (f *EventFilter) Match(event flow.Event) bool { return true } - if parsed.Type == AccountEventType { + if parsed.Type == events.AccountEventType { _, ok := f.Addresses[parsed.Address] return ok } @@ -139,8 +140,8 @@ func (f *EventFilter) Match(event flow.Event) bool { } // validateEventType ensures that the event type matches the expected format -func validateEventType(eventType flow.EventType) error { - _, err := ParseEvent(flow.EventType(eventType)) +func validateEventType(eventType flow.EventType, chain flow.Chain) error { + _, err := events.ValidateEvent(flow.EventType(eventType), chain) if err != nil { return fmt.Errorf("invalid event type %s: %w", eventType, err) } diff --git a/engine/common/follower/cache/cache_test.go b/engine/common/follower/cache/cache_test.go index d5c42f5ea80..b94a7cc31ce 100644 --- a/engine/common/follower/cache/cache_test.go +++ b/engine/common/follower/cache/cache_test.go @@ -239,8 +239,8 @@ func (s *CacheSuite) TestConcurrentAdd() { unittest.RequireReturnsBefore(s.T(), wg.Wait, time.Millisecond*500, "should submit blocks before timeout") require.Len(s.T(), allCertifiedBlocks, len(blocks)-1) - slices.SortFunc(allCertifiedBlocks, func(lhs *flow.Block, rhs *flow.Block) bool { - return lhs.Header.Height < rhs.Header.Height + slices.SortFunc(allCertifiedBlocks, func(lhs *flow.Block, rhs *flow.Block) int { + return int(lhs.Header.Height) - int(rhs.Header.Height) }) require.Equal(s.T(), blocks[:len(blocks)-1], allCertifiedBlocks) } diff --git a/engine/common/follower/pending_tree/pending_tree_test.go b/engine/common/follower/pending_tree/pending_tree_test.go index ac482871aa4..66b53058b23 100644 --- a/engine/common/follower/pending_tree/pending_tree_test.go +++ b/engine/common/follower/pending_tree/pending_tree_test.go @@ -154,8 +154,8 @@ func (s *PendingTreeSuite) TestBatchWithSkipsAndInRandomOrder() { require.NoError(s.T(), err) // restore view based order since that's what we will get from PendingTree - slices.SortFunc(blocks, func(lhs flow.CertifiedBlock, rhs flow.CertifiedBlock) bool { - return lhs.View() < rhs.View() + slices.SortFunc(blocks, func(lhs flow.CertifiedBlock, rhs flow.CertifiedBlock) int { + return int(lhs.View()) - int(rhs.View()) }) assert.Equal(s.T(), blocks, connectedBlocks) diff --git a/engine/common/rpc/convert/execution_data_test.go b/engine/common/rpc/convert/execution_data_test.go index 198d45481c8..d32ae94d008 100644 --- a/engine/common/rpc/convert/execution_data_test.go +++ b/engine/common/rpc/convert/execution_data_test.go @@ -21,7 +21,12 @@ import ( func TestConvertBlockExecutionDataEventPayloads(t *testing.T) { // generators will produce identical event payloads (before encoding) ccfEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_CCF_V0) - jsonEvents := generator.GetEventsWithEncoding(3, entities.EventEncodingVersion_JSON_CDC_V0) + jsonEvents := make([]flow.Event, len(ccfEvents)) + for i, e := range ccfEvents { + jsonEvent, err := convert.CcfEventToJsonEvent(e) + require.NoError(t, err) + jsonEvents[i] = *jsonEvent + } // generate BlockExecutionData with CCF encoded events executionData := unittest.BlockExecutionDataFixture( diff --git a/engine/consensus/approvals/assignment_collector_tree.go b/engine/consensus/approvals/assignment_collector_tree.go index e161a75faa4..c31282b3e8a 100644 --- a/engine/consensus/approvals/assignment_collector_tree.go +++ b/engine/consensus/approvals/assignment_collector_tree.go @@ -160,11 +160,10 @@ func (t *AssignmentCollectorTree) selectCollectorsForFinalizedFork(startHeight, var fork []*assignmentCollectorVertex for height := startHeight; height <= finalizedHeight; height++ { iter := t.forest.GetVerticesAtLevel(height) - finalizedBlock, err := t.headers.ByHeight(height) + finalizedBlockID, err := t.headers.BlockIDByHeight(height) if err != nil { return nil, fmt.Errorf("could not retrieve finalized block at height %d: %w", height, err) } - finalizedBlockID := finalizedBlock.ID() for iter.HasNext() { vertex := iter.NextVertex().(*assignmentCollectorVertex) if finalizedBlockID == vertex.collector.BlockID() { diff --git a/engine/consensus/approvals/testutil.go b/engine/consensus/approvals/testutil.go index df5e98fa36b..610ffceeb43 100644 --- a/engine/consensus/approvals/testutil.go +++ b/engine/consensus/approvals/testutil.go @@ -134,20 +134,22 @@ func (s *BaseAssignmentCollectorTestSuite) SetupTest() { return realstorage.ErrNotFound } }) - s.Headers.On("ByHeight", mock.Anything).Return( - func(height uint64) *flow.Header { + s.Headers.On("BlockIDByHeight", mock.Anything).Return( + func(height uint64) (flow.Identifier, error) { if block, found := s.FinalizedAtHeight[height]; found { - return block + return block.ID(), nil } else { - return nil + return flow.ZeroID, realstorage.ErrNotFound } }, - func(height uint64) error { - _, found := s.FinalizedAtHeight[height] - if !found { - return realstorage.ErrNotFound + ) + s.Headers.On("ByHeight", mock.Anything).Return( + func(height uint64) (*flow.Header, error) { + if block, found := s.FinalizedAtHeight[height]; found { + return block, nil + } else { + return nil, realstorage.ErrNotFound } - return nil }, ) diff --git a/engine/consensus/sealing/core.go b/engine/consensus/sealing/core.go index 1bf9350e09f..32e912a3344 100644 --- a/engine/consensus/sealing/core.go +++ b/engine/consensus/sealing/core.go @@ -257,11 +257,11 @@ func (c *Core) processIncorporatedResult(incRes *flow.IncorporatedResult) error // For incorporating blocks at heights that are already finalized, we check that the incorporating block // is on the finalized fork. Otherwise, the incorporating block is orphaned, and we can drop the result. if incorporatedAtHeight <= c.counterLastFinalizedHeight.Value() { - finalized, err := c.headers.ByHeight(incorporatedAtHeight) + finalizedID, err := c.headers.BlockIDByHeight(incorporatedAtHeight) if err != nil { return fmt.Errorf("could not retrieve finalized block at height %d: %w", incorporatedAtHeight, err) } - if finalized.ID() != incRes.IncorporatedBlockID { + if finalizedID != incRes.IncorporatedBlockID { // it means that we got incorporated incRes for a block which doesn't extend our chain // and should be discarded from future processing return engine.NewOutdatedInputErrorf("won't process incorporated incRes from orphan block %s", incRes.IncorporatedBlockID) diff --git a/engine/execution/checker/core.go b/engine/execution/checker/core.go new file mode 100644 index 00000000000..78ca7475dd9 --- /dev/null +++ b/engine/execution/checker/core.go @@ -0,0 +1,150 @@ +package checker + +import ( + "context" + "errors" + "fmt" + + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/engine/execution/state" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/state/protocol" + "github.com/onflow/flow-go/storage" +) + +// Core is the core logic of the checker engine that checks if the execution result matches the sealed result. +type Core struct { + log zerolog.Logger + state protocol.State + execState state.ExecutionState +} + +func NewCore( + logger zerolog.Logger, + state protocol.State, + execState state.ExecutionState, +) *Core { + e := &Core{ + log: logger.With().Str("engine", "checker").Logger(), + state: state, + execState: execState, + } + + return e +} + +// checkMyCommitWithSealedCommit is the main check of the checker engine +func checkMyCommitWithSealedCommit( + executedBlock *flow.Header, + myCommit flow.StateCommitment, + sealedCommit flow.StateCommitment, +) error { + if myCommit != sealedCommit { + // mismatch + return fmt.Errorf("execution result is different from the sealed result, height: %v, block_id: %v, sealed_commit: %v, my_commit: %v", + executedBlock.Height, + executedBlock.ID(), + sealedCommit, + myCommit, + ) + } + + // match + return nil +} + +// RunCheck skips when the last sealed has not been executed, and last executed has not been finalized. +func (c *Core) RunCheck() error { + // find last sealed block + lastSealedBlock, lastFinal, seal, err := c.findLastSealedBlock() + if err != nil { + return err + } + + mycommitAtLastSealed, err := c.execState.StateCommitmentByBlockID(lastSealedBlock.ID()) + if err == nil { + // if last sealed block has been executed, then check if they match + return checkMyCommitWithSealedCommit(lastSealedBlock, mycommitAtLastSealed, seal.FinalState) + } + + // if last sealed block has not been executed, then check if recent executed block has + // been sealed already, if yes, check if they match. + lastExecutedHeight, err := c.findLastExecutedBlockHeight() + if err != nil { + return err + } + + if lastExecutedHeight > lastFinal.Height { + // last executed block has not been finalized yet, + // can't check since unfinalized block is also unsealed, skip + return nil + } + + // TODO: better to query seals from protocol state, + // switch to state.Final().LastSealed() when available + sealedExecuted, seal, err := c.findLatestSealedAtHeight(lastExecutedHeight) + if err != nil { + return fmt.Errorf("could not get the last sealed block at height: %v, err: %w", lastExecutedHeight, err) + } + + sealedCommit := seal.FinalState + + mycommit, err := c.execState.StateCommitmentByBlockID(seal.BlockID) + if errors.Is(err, storage.ErrNotFound) { + // have not executed the sealed block yet + // in other words, this can't detect execution fork, if the execution is behind + // the sealing + return nil + } + + if err != nil { + return fmt.Errorf("could not get my state commitment OnFinalizedBlock, blockID: %v", seal.BlockID) + } + + return checkMyCommitWithSealedCommit(sealedExecuted, mycommit, sealedCommit) +} + +// findLastSealedBlock finds the last sealed block +func (c *Core) findLastSealedBlock() (*flow.Header, *flow.Header, *flow.Seal, error) { + finalized := c.state.Final() + lastFinal, err := finalized.Head() + if err != nil { + return nil, nil, nil, err + } + + _, lastSeal, err := finalized.SealedResult() + if err != nil { + return nil, nil, nil, fmt.Errorf("could not get the last sealed for the finalized block: %w", err) + } + + lastSealed, err := c.state.AtBlockID(lastSeal.BlockID).Head() + if err != nil { + return nil, nil, nil, fmt.Errorf("could not get the last sealed block: %w", err) + } + + return lastSealed, lastFinal, lastSeal, nil +} + +// findLastExecutedBlockHeight finds the last executed block height +func (c *Core) findLastExecutedBlockHeight() (uint64, error) { + height, _, err := c.execState.GetHighestExecutedBlockID(context.Background()) + if err != nil { + return 0, fmt.Errorf("could not get the last executed block: %w", err) + } + return height, nil +} + +// findLatestSealedAtHeight finds the latest sealed block at the given height +func (c *Core) findLatestSealedAtHeight(finalizedHeight uint64) (*flow.Header, *flow.Seal, error) { + _, seal, err := c.state.AtHeight(finalizedHeight).SealedResult() + if err != nil { + return nil, nil, fmt.Errorf("could not get the last sealed for the finalized block: %w", err) + } + + sealed, err := c.state.AtBlockID(seal.BlockID).Head() + if err != nil { + return nil, nil, fmt.Errorf("could not get the last sealed block: %w", err) + } + return sealed, seal, nil +} diff --git a/engine/execution/checker/core_test.go b/engine/execution/checker/core_test.go new file mode 100644 index 00000000000..cd27c5cabdc --- /dev/null +++ b/engine/execution/checker/core_test.go @@ -0,0 +1,156 @@ +package checker_test + +import ( + "testing" + + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/engine/execution/checker" + stateMock "github.com/onflow/flow-go/engine/execution/state/mock" + "github.com/onflow/flow-go/model/flow" + protocol "github.com/onflow/flow-go/state/protocol/mock" + "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/utils/unittest" +) + +func makeCore(t *testing.T) (*checker.Core, *protocol.State, *stateMock.ExecutionState) { + logger := unittest.Logger() + state := protocol.NewState(t) + execState := stateMock.NewExecutionState(t) + core := checker.NewCore(logger, state, execState) + return core, state, execState +} + +func mockFinalizedBlock(t *testing.T, state *protocol.State, finalized *flow.Header) *protocol.Snapshot { + finalizedSnapshot := protocol.NewSnapshot(t) + finalizedSnapshot.On("Head").Return(finalized, nil) + state.On("Final").Return(finalizedSnapshot) + return finalizedSnapshot +} + +func mockAtBlockID(t *testing.T, state *protocol.State, header *flow.Header) *protocol.Snapshot { + snapshot := protocol.NewSnapshot(t) + snapshot.On("Head").Return(header, nil) + state.On("AtBlockID", header.ID()).Return(snapshot) + return snapshot +} + +func mockSealedBlock(t *testing.T, state *protocol.State, finalized *protocol.Snapshot, sealed *flow.Header) (*flow.ExecutionResult, *flow.Seal) { + lastSealResult := unittest.ExecutionResultFixture(func(r *flow.ExecutionResult) { + r.BlockID = sealed.ID() + }) + lastSeal := unittest.Seal.Fixture(unittest.Seal.WithResult(lastSealResult)) + finalized.On("SealedResult").Return(lastSealResult, lastSeal, nil) + return lastSealResult, lastSeal +} + +func mockFinalizedSealedBlock(t *testing.T, state *protocol.State, finalized *flow.Header, sealed *flow.Header) (*flow.ExecutionResult, *flow.Seal) { + finalizedSnapshot := mockFinalizedBlock(t, state, finalized) + return mockSealedBlock(t, state, finalizedSnapshot, sealed) +} + +func mockSealedBlockAtHeight(t *testing.T, state *protocol.State, height uint64, lastSealed *flow.Header) (*flow.ExecutionResult, *flow.Seal) { + snapshotAtHeight := protocol.NewSnapshot(t) + lastSealedResultAtHeight := unittest.ExecutionResultFixture(func(r *flow.ExecutionResult) { + r.BlockID = lastSealed.ID() + }) + lastSealAtHeight := unittest.Seal.Fixture(unittest.Seal.WithResult(lastSealedResultAtHeight)) + snapshotAtHeight.On("SealedResult").Return(lastSealedResultAtHeight, lastSealAtHeight, nil) + state.On("AtHeight", height).Return(snapshotAtHeight, nil) + return lastSealedResultAtHeight, lastSealAtHeight +} + +func mockExecutedBlock(t *testing.T, es *stateMock.ExecutionState, executed *flow.Header, result *flow.ExecutionResult) { + commit, err := result.FinalStateCommitment() + require.NoError(t, err) + es.On("StateCommitmentByBlockID", executed.ID()).Return(commit, nil) +} + +func mockUnexecutedBlock(t *testing.T, es *stateMock.ExecutionState, unexecuted *flow.Header) { + es.On("StateCommitmentByBlockID", unexecuted.ID()).Return(nil, storage.ErrNotFound) +} + +func TestCheckPassIfLastSealedIsExecutedAndMatch(t *testing.T) { + // ..<- LastSealed(executed) <- .. <- LastFinalized <- .. <- LastExecuted <- ... + chain, _, _ := unittest.ChainFixture(10) + lastFinal := chain[7].Header + lastSealed := chain[5].Header + + core, state, es := makeCore(t) + lastSealedResult, _ := mockFinalizedSealedBlock(t, state, lastFinal, lastSealed) + mockAtBlockID(t, state, lastSealed) + mockExecutedBlock(t, es, lastSealed, lastSealedResult) + + require.NoError(t, core.RunCheck()) +} + +func TestCheckFailIfLastSealedIsExecutedButMismatch(t *testing.T) { + // ..<- LastSealed(executed) <- .. <- LastFinalized <- .. <- LastExecuted <- ... + chain, _, _ := unittest.ChainFixture(10) + lastFinal := chain[7].Header + lastSealed := chain[5].Header + + core, state, es := makeCore(t) + _, _ = mockFinalizedSealedBlock(t, state, lastFinal, lastSealed) + mockAtBlockID(t, state, lastSealed) + + mismatchingResult := unittest.ExecutionResultFixture() + + mockExecutedBlock(t, es, lastSealed, mismatchingResult) + + require.Error(t, core.RunCheck()) + require.Contains(t, core.RunCheck().Error(), "execution result is different from the sealed result") +} + +func TestCheckPassIfLastSealedIsNotExecutedAndLastExecutedMatch(t *testing.T) { + // LastSealedExecuted (sealed) <..<- LastExecuted(finalized) <..<- LastSealed(not executed) <..<- LastFinalized + chain, _, _ := unittest.ChainFixture(10) + lastFinal := chain[7].Header + lastSealed := chain[5].Header + lastExecuted := chain[3].Header + lastSealedExecuted := chain[1].Header + + core, state, es := makeCore(t) + // mock that last sealed is not executed + mockFinalizedSealedBlock(t, state, lastFinal, lastSealed) + mockAtBlockID(t, state, lastSealed) + mockUnexecutedBlock(t, es, lastSealed) + + // mock the last sealed and is also executed + es.On("GetHighestExecutedBlockID", mock.Anything).Return(lastExecuted.Height, lastExecuted.ID(), nil) + lastSealedResultAtExecutedHeight, _ := mockSealedBlockAtHeight(t, state, lastExecuted.Height, lastSealedExecuted) + mockAtBlockID(t, state, lastSealedExecuted) + + // mock with matching result + mockExecutedBlock(t, es, lastSealedExecuted, lastSealedResultAtExecutedHeight) + + require.NoError(t, core.RunCheck()) +} + +func TestCheckFailIfLastSealedIsNotExecutedAndLastExecutedMismatch(t *testing.T) { + // LastSealedExecuted (sealed) <..<- LastExecuted(finalized) <..<- LastSealed(not executed) <..<- LastFinalized + chain, _, _ := unittest.ChainFixture(10) + lastFinal := chain[7].Header + lastSealed := chain[5].Header + lastExecuted := chain[3].Header + lastSealedExecuted := chain[1].Header + + core, state, es := makeCore(t) + // mock that last sealed is not executed + mockFinalizedSealedBlock(t, state, lastFinal, lastSealed) + mockAtBlockID(t, state, lastSealed) + mockUnexecutedBlock(t, es, lastSealed) + + // mock the last sealed and is also executed + es.On("GetHighestExecutedBlockID", mock.Anything).Return(lastExecuted.Height, lastExecuted.ID(), nil) + mockSealedBlockAtHeight(t, state, lastExecuted.Height, lastSealedExecuted) + mockAtBlockID(t, state, lastSealedExecuted) + + // mock with mismatching result + mismatchingResult := unittest.ExecutionResultFixture() + mockExecutedBlock(t, es, lastSealedExecuted, mismatchingResult) + + require.Error(t, core.RunCheck()) + require.Contains(t, core.RunCheck().Error(), "execution result is different from the sealed result") +} diff --git a/engine/execution/checker/engine.go b/engine/execution/checker/engine.go index a1a96184105..5c9a5bc1404 100644 --- a/engine/execution/checker/engine.go +++ b/engine/execution/checker/engine.go @@ -1,108 +1,57 @@ package checker import ( - "errors" - "fmt" + "context" + "time" - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/consensus/hotstuff/model" - "github.com/onflow/flow-go/consensus/hotstuff/notifications" - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/execution/state" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/state/protocol" - "github.com/onflow/flow-go/storage" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" ) type Engine struct { - notifications.NoopConsumer // satisfy the FinalizationConsumer interface - - unit *engine.Unit - log zerolog.Logger - state protocol.State - execState state.ExecutionState - sealsDB storage.Seals -} - -func New( - logger zerolog.Logger, - state protocol.State, - execState state.ExecutionState, - sealsDB storage.Seals, -) *Engine { - return &Engine{ - unit: engine.NewUnit(), - log: logger.With().Str("engine", "checker").Logger(), - state: state, - execState: execState, - sealsDB: sealsDB, - } + *component.ComponentManager + core *Core } -func (e *Engine) Ready() <-chan struct{} { - // make sure we will run into a crashloop if result gets inconsistent - // with sealed result. +// DefaultTimeInterval triggers the check once every minute, +const DefaultTimeInterval = time.Minute * 1 - finalized, err := e.state.Final().Head() - - if err != nil { - e.log.Fatal().Err(err).Msg("could not get finalized block on startup") +func NewEngine(core *Core) *Engine { + e := &Engine{ + core: core, } - err = e.checkLastSealed(finalized.ID()) - if err != nil { - e.log.Fatal().Err(err).Msg("execution consistency check failed on startup") - } - return e.unit.Ready() -} - -func (e *Engine) Done() <-chan struct{} { - return e.unit.Done() -} + e.ComponentManager = component.NewComponentManagerBuilder(). + AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + err := e.runLoop(ctx, DefaultTimeInterval) + if err != nil { + ctx.Throw(err) + } + }). + Build() -// when a block is finalized check if the last sealed has been executed, -// if it has been executed, check whether if the sealed result is consistent -// with the executed result -func (e *Engine) OnFinalizedBlock(block *model.Block) { - err := e.checkLastSealed(block.BlockID) - if err != nil { - e.log.Fatal().Err(err).Msg("execution consistency check failed") - } + return e } -func (e *Engine) checkLastSealed(finalizedID flow.Identifier) error { - // TODO: better to query seals from protocol state, - // switch to state.Final().LastSealed() when available - seal, err := e.sealsDB.HighestInFork(finalizedID) - if err != nil { - return fmt.Errorf("could not get the last sealed for the finalized block: %w", err) - } - - blockID := seal.BlockID - sealedCommit := seal.FinalState - - mycommit, err := e.execState.StateCommitmentByBlockID(blockID) - if errors.Is(err, storage.ErrNotFound) { - // have not executed the sealed block yet - // in other words, this can't detect execution fork, if the execution is behind - // the sealing - return nil - } - - if err != nil { - return fmt.Errorf("could not get my state commitment OnFinalizedBlock, blockID: %v", blockID) - } - - if mycommit != sealedCommit { - sealed, err := e.state.AtBlockID(blockID).Head() - if err != nil { - return fmt.Errorf("could not get sealed block when checkLastSealed: %v, err: %w", blockID, err) +// runLoop runs the check every minute. +// Why using a timer instead of listening to finalized and executed events? +// because it's simpler as it doesn't need to subscribe to those events. +// It also runs less checks, note: the checker doesn't need to find the +// first mismatched block, as long as it can find a mismatch, it's good enough. +// A timer could reduce the number of checks, as it only checks once every minute. +func (e *Engine) runLoop(ctx context.Context, tickInterval time.Duration) error { + ticker := time.NewTicker(tickInterval) + defer ticker.Stop() // critical for ticker to be garbage collected + for { + select { + case <-ticker.C: + err := e.core.RunCheck() + if err != nil { + return err + } + case <-ctx.Done(): + return nil } - - return fmt.Errorf("execution result is different from the sealed result, height: %v, block_id: %v, sealed_commit: %x, my_commit: %x", - sealed.Height, blockID, sealedCommit, mycommit) } - - return nil } diff --git a/engine/execution/computation/computer/computer_test.go b/engine/execution/computation/computer/computer_test.go index 260ee5bec2d..30cae148f81 100644 --- a/engine/execution/computation/computer/computer_test.go +++ b/engine/execution/computation/computer/computer_test.go @@ -601,7 +601,8 @@ func TestBlockExecutor_ExecuteBlock(t *testing.T) { serviceEvents := systemcontracts.ServiceEventsForChain(execCtx.Chain.ChainID()) - payload, err := ccf.Decode(nil, unittest.EpochSetupFixtureCCF) + randomSource := unittest.EpochSetupRandomSourceFixture() + payload, err := ccf.Decode(nil, unittest.EpochSetupFixtureCCF(randomSource)) require.NoError(t, err) serviceEventA, ok := payload.(cadence.Event) @@ -1247,7 +1248,7 @@ func Test_ExecutingSystemCollection(t *testing.T) { noopCollector := metrics.NewNoopCollector() expectedNumberOfEvents := 3 - expectedEventSize := 1484 + expectedEventSize := 1497 // bootstrapping does not cache programs expectedCachedPrograms := 0 diff --git a/engine/execution/computation/manager.go b/engine/execution/computation/manager.go index 36fca70d833..1a0754ee2f9 100644 --- a/engine/execution/computation/manager.go +++ b/engine/execution/computation/manager.go @@ -239,6 +239,7 @@ func DefaultFVMOptions(chainID flow.ChainID, cadenceTracing bool, extensiveTraci AttachmentsEnabled: chainID != flow.Mainnet, }, )), + fvm.WithEVMEnabled(true), } if extensiveTracing { diff --git a/engine/execution/execution_test.go b/engine/execution/execution_test.go index 064948b7711..12a79e90706 100644 --- a/engine/execution/execution_test.go +++ b/engine/execution/execution_test.go @@ -17,7 +17,6 @@ import ( "github.com/onflow/flow-go/engine/testutil" testmock "github.com/onflow/flow-go/engine/testutil/mock" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/network/channels" @@ -60,7 +59,7 @@ func TestExecutionFlow(t *testing.T) { unittest.WithKeys, ) - identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID).Sort(order.Canonical) + identities := unittest.CompleteIdentitySet(colID, conID, exeID, verID).Sort(flow.Canonical) // create execution node exeNode := testutil.ExecutionNode(t, hub, exeID, identities, 21, chainID) diff --git a/engine/execution/ingestion/engine.go b/engine/execution/ingestion/engine.go index 25100a43dd1..72188166d51 100644 --- a/engine/execution/ingestion/engine.go +++ b/engine/execution/ingestion/engine.go @@ -25,7 +25,6 @@ import ( "github.com/onflow/flow-go/module/mempool/stdmap" "github.com/onflow/flow-go/module/trace" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/channels" psEvents "github.com/onflow/flow-go/state/protocol/events" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/utils/logging" @@ -37,7 +36,6 @@ type Engine struct { unit *engine.Unit log zerolog.Logger - me module.Local collectionFetcher CollectionFetcher headers storage.Headers // see comments on getHeaderByHeight for why we need it blocks storage.Blocks @@ -60,7 +58,6 @@ func New( unit *engine.Unit, logger zerolog.Logger, net network.EngineRegistry, - me module.Local, collectionFetcher CollectionFetcher, headers storage.Headers, blocks storage.Blocks, @@ -83,7 +80,6 @@ func New( eng := Engine{ unit: unit, log: log, - me: me, collectionFetcher: collectionFetcher, headers: headers, blocks: blocks, @@ -130,51 +126,11 @@ func (e *Engine) Done() <-chan struct{} { return e.unit.Done() } -// SubmitLocal submits an event originating on the local node. -func (e *Engine) SubmitLocal(event interface{}) { - e.unit.Launch(func() { - err := e.process(e.me.NodeID(), event) - if err != nil { - engine.LogError(e.log, err) - } - }) -} - -// Submit submits the given event from the node with the given origin ID -// for processing in a non-blocking manner. It returns instantly and logs -// a potential processing error internally when done. -func (e *Engine) Submit( - channel channels.Channel, - originID flow.Identifier, - event interface{}, -) { - e.unit.Launch(func() { - err := e.process(originID, event) - if err != nil { - engine.LogError(e.log, err) - } - }) -} - // ProcessLocal processes an event originating on the local node. func (e *Engine) ProcessLocal(event interface{}) error { return fmt.Errorf("ingestion error does not process local events") } -func (e *Engine) Process( - channel channels.Channel, - originID flow.Identifier, - event interface{}, -) error { - return e.unit.Do(func() error { - return e.process(originID, event) - }) -} - -func (e *Engine) process(originID flow.Identifier, event interface{}) error { - return nil -} - // on nodes startup, we need to load all the unexecuted blocks to the execution queues. // blocks have to be loaded in the way that the parent has been loaded before loading its children func (e *Engine) reloadUnexecutedBlocks() error { diff --git a/engine/execution/ingestion/engine_test.go b/engine/execution/ingestion/engine_test.go index 64d66aeec1f..8ad8551ab8f 100644 --- a/engine/execution/ingestion/engine_test.go +++ b/engine/execution/ingestion/engine_test.go @@ -25,7 +25,6 @@ import ( uploadermock "github.com/onflow/flow-go/engine/execution/ingestion/uploader/mock" provider "github.com/onflow/flow-go/engine/execution/provider/mock" stateMock "github.com/onflow/flow-go/engine/execution/state/mock" - "github.com/onflow/flow-go/engine/testutil/mocklocal" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/metrics" @@ -69,8 +68,6 @@ func runWithEngine(t *testing.T, f func(testingContext)) { myIdentity.Role = flow.RoleExecution myIdentity.StakingPubKey = sk.PublicKey() - me := mocklocal.NewMockLocal(sk, myIdentity.ID(), t) - headers := storage.NewHeaders(t) blocks := storage.NewBlocks(t) collections := mocks.NewMockCollectionStore() @@ -119,7 +116,6 @@ func runWithEngine(t *testing.T, f func(testingContext)) { unit, log, net, - me, fetcher, headers, blocks, diff --git a/engine/execution/ingestion/loader/unexecuted_loader.go b/engine/execution/ingestion/loader/unexecuted_loader.go index a9eba76115f..7d32ba11ec4 100644 --- a/engine/execution/ingestion/loader/unexecuted_loader.go +++ b/engine/execution/ingestion/loader/unexecuted_loader.go @@ -17,7 +17,7 @@ import ( type UnexecutedLoader struct { log zerolog.Logger state protocol.State - headers storage.Headers // see comments on getHeaderByHeight for why we need it + headers storage.Headers execState state.ExecutionState } @@ -157,12 +157,12 @@ func (e *UnexecutedLoader) finalizedUnexecutedBlocks(ctx context.Context, finali } for ; lastExecuted > rootBlock.Height; lastExecuted-- { - header, err := e.getHeaderByHeight(lastExecuted) + finalizedID, err := e.headers.BlockIDByHeight(lastExecuted) if err != nil { return nil, fmt.Errorf("could not get header at height: %v, %w", lastExecuted, err) } - executed, err := e.execState.IsBlockExecuted(header.Height, header.ID()) + executed, err := e.execState.IsBlockExecuted(lastExecuted, finalizedID) if err != nil { return nil, fmt.Errorf("could not check whether block is executed: %w", err) } @@ -179,12 +179,12 @@ func (e *UnexecutedLoader) finalizedUnexecutedBlocks(ctx context.Context, finali // starting from the first unexecuted block, go through each unexecuted and finalized block // reload its block to execution queues for height := firstUnexecuted; height <= final.Height; height++ { - header, err := e.getHeaderByHeight(height) + finalizedID, err := e.headers.BlockIDByHeight(height) if err != nil { return nil, fmt.Errorf("could not get header at height: %v, %w", height, err) } - unexecuted = append(unexecuted, header.ID()) + unexecuted = append(unexecuted, finalizedID) } e.log.Info(). @@ -227,12 +227,3 @@ func (e *UnexecutedLoader) pendingUnexecutedBlocks(ctx context.Context, finalize return unexecuted, nil } - -// if the EN is dynamically bootstrapped, the finalized blocks at height range: -// [ sealedRoot.Height, finalizedRoot.Height - 1] can not be retrieved from -// protocol state, but only from headers -func (e *UnexecutedLoader) getHeaderByHeight(height uint64) (*flow.Header, error) { - // we don't use protocol state because for dynamic boostrapped execution node - // the last executed and sealed block is below the finalized root block - return e.headers.ByHeight(height) -} diff --git a/engine/execution/ingestion/loader/unexecuted_loader_test.go b/engine/execution/ingestion/loader/unexecuted_loader_test.go index 23779394c5b..df5ef452606 100644 --- a/engine/execution/ingestion/loader/unexecuted_loader_test.go +++ b/engine/execution/ingestion/loader/unexecuted_loader_test.go @@ -200,7 +200,7 @@ func TestLoadingUnexecutedBlocks(t *testing.T) { loader := loader.NewUnexecutedLoader(log, ps, headers, es) // block C is the only finalized block, index its header by its height - headers.EXPECT().ByHeight(blockC.Header.Height).Return(blockC.Header, nil) + headers.EXPECT().BlockIDByHeight(blockC.Header.Height).Return(blockC.Header.ID(), nil) es.ExecuteBlock(t, blockA) es.ExecuteBlock(t, blockB) @@ -238,7 +238,7 @@ func TestLoadingUnexecutedBlocks(t *testing.T) { loader := loader.NewUnexecutedLoader(log, ps, headers, es) // block C is finalized, index its header by its height - headers.EXPECT().ByHeight(blockC.Header.Height).Return(blockC.Header, nil) + headers.EXPECT().BlockIDByHeight(blockC.Header.Height).Return(blockC.Header.ID(), nil) es.ExecuteBlock(t, blockA) es.ExecuteBlock(t, blockB) @@ -278,7 +278,7 @@ func TestLoadingUnexecutedBlocks(t *testing.T) { loader := loader.NewUnexecutedLoader(log, ps, headers, es) // block A is finalized, index its header by its height - headers.EXPECT().ByHeight(blockA.Header.Height).Return(blockA.Header, nil) + headers.EXPECT().BlockIDByHeight(blockA.Header.Height).Return(blockA.Header.ID(), nil) es.ExecuteBlock(t, blockA) es.ExecuteBlock(t, blockB) @@ -346,7 +346,7 @@ func TestLoadingUnexecutedBlocks(t *testing.T) { loader := loader.NewUnexecutedLoader(log, ps, headers, es) // block C is finalized, index its header by its height - headers.EXPECT().ByHeight(blockC.Header.Height).Return(blockC.Header, nil) + headers.EXPECT().BlockIDByHeight(blockC.Header.Height).Return(blockC.Header.ID(), nil) es.ExecuteBlock(t, blockA) es.ExecuteBlock(t, blockB) diff --git a/engine/execution/ingestion/loader/unfinalized_loader.go b/engine/execution/ingestion/loader/unfinalized_loader.go index ed90efec40d..d71c737cbc3 100644 --- a/engine/execution/ingestion/loader/unfinalized_loader.go +++ b/engine/execution/ingestion/loader/unfinalized_loader.go @@ -15,7 +15,7 @@ import ( type UnfinalizedLoader struct { log zerolog.Logger state protocol.State - headers storage.Headers // see comments on getHeaderByHeight for why we need it + headers storage.Headers execState state.FinalizedExecutionState } @@ -53,19 +53,21 @@ func (e *UnfinalizedLoader) LoadUnexecuted(ctx context.Context) ([]flow.Identifi lg.Info().Msgf("start loading unfinalized blocks") - // TODO: dynamically bootstrapped execution node will reload blocks from + // dynamically bootstrapped execution node will have highest finalized executed as sealed root, + // which is lower than finalized root. so we will reload blocks from + // [sealedRoot.Height + 1, finalizedRoot.Height] and execute them on startup. unexecutedFinalized := make([]flow.Identifier, 0) // starting from the first unexecuted block, go through each unexecuted and finalized block // reload its block to execution queues // loading finalized blocks for height := lastExecuted + 1; height <= final.Height; height++ { - header, err := e.getHeaderByHeight(height) + finalizedID, err := e.headers.BlockIDByHeight(height) if err != nil { return nil, fmt.Errorf("could not get header at height: %v, %w", height, err) } - unexecutedFinalized = append(unexecutedFinalized, header.ID()) + unexecutedFinalized = append(unexecutedFinalized, finalizedID) } // loaded all pending blocks @@ -85,12 +87,3 @@ func (e *UnfinalizedLoader) LoadUnexecuted(ctx context.Context) ([]flow.Identifi return unexecuted, nil } - -// if the EN is dynamically bootstrapped, the finalized blocks at height range: -// [ sealedRoot.Height, finalizedRoot.Height - 1] can not be retrieved from -// protocol state, but only from headers -func (e *UnfinalizedLoader) getHeaderByHeight(height uint64) (*flow.Header, error) { - // we don't use protocol state because for dynamic boostrapped execution node - // the last executed and sealed block is below the finalized root block - return e.headers.ByHeight(height) -} diff --git a/engine/execution/ingestion/loader/unfinalized_loader_test.go b/engine/execution/ingestion/loader/unfinalized_loader_test.go index 3c8b84aed40..c5ded941006 100644 --- a/engine/execution/ingestion/loader/unfinalized_loader_test.go +++ b/engine/execution/ingestion/loader/unfinalized_loader_test.go @@ -37,9 +37,9 @@ func TestLoadingUnfinalizedBlocks(t *testing.T) { es := new(stateMock.FinalizedExecutionState) es.On("GetHighestFinalizedExecuted").Return(genesis.Header.Height) headers := new(storage.Headers) - headers.On("ByHeight", blockA.Header.Height).Return(blockA.Header, nil) - headers.On("ByHeight", blockB.Header.Height).Return(blockB.Header, nil) - headers.On("ByHeight", blockC.Header.Height).Return(blockC.Header, nil) + headers.On("BlockIDByHeight", blockA.Header.Height).Return(blockA.Header.ID(), nil) + headers.On("BlockIDByHeight", blockB.Header.Height).Return(blockB.Header.ID(), nil) + headers.On("BlockIDByHeight", blockC.Header.Height).Return(blockC.Header.ID(), nil) loader := loader.NewUnfinalizedLoader(unittest.Logger(), ps, headers, es) diff --git a/engine/execution/ingestion/stop/stop_control.go b/engine/execution/ingestion/stop/stop_control.go index bb14e8905d5..b91738e5d31 100644 --- a/engine/execution/ingestion/stop/stop_control.go +++ b/engine/execution/ingestion/stop/stop_control.go @@ -148,7 +148,7 @@ func (s stopBoundary) String() string { // StopControlHeaders is an interface for fetching headers // Its jut a small subset of storage.Headers for comments see storage.Headers type StopControlHeaders interface { - ByHeight(height uint64) (*flow.Header, error) + BlockIDByHeight(height uint64) (flow.Identifier, error) } // NewStopControl creates new StopControl. @@ -476,12 +476,12 @@ func (s *StopControl) blockFinalized( // Let's find the ID of the block that should be executed last // which is the parent of the block at the stopHeight - header, err := s.headers.ByHeight(s.stopBoundary.StopBeforeHeight - 1) + finalizedID, err := s.headers.BlockIDByHeight(s.stopBoundary.StopBeforeHeight - 1) if err != nil { handleErr(fmt.Errorf("failed to get header by height: %w", err)) return } - parentID = header.ID() + parentID = finalizedID } s.stopBoundary.stopAfterExecuting = parentID diff --git a/engine/execution/ingestion/stop/stop_control_test.go b/engine/execution/ingestion/stop/stop_control_test.go index 6698c3cc7b8..12900d56dad 100644 --- a/engine/execution/ingestion/stop/stop_control_test.go +++ b/engine/execution/ingestion/stop/stop_control_test.go @@ -162,12 +162,12 @@ type stopControlMockHeaders struct { headers map[uint64]*flow.Header } -func (m *stopControlMockHeaders) ByHeight(height uint64) (*flow.Header, error) { +func (m *stopControlMockHeaders) BlockIDByHeight(height uint64) (flow.Identifier, error) { h, ok := m.headers[height] if !ok { - return nil, fmt.Errorf("header not found") + return flow.ZeroID, fmt.Errorf("header not found") } - return h, nil + return h.ID(), nil } func TestAddStopForPastBlocks(t *testing.T) { @@ -865,4 +865,9 @@ func Test_StopControlWorkers(t *testing.T) { func TestPatchedVersion(t *testing.T) { require.True(t, semver.New("0.31.20").LessThan(*semver.New("0.31.21"))) require.True(t, semver.New("0.31.20-patch.1").LessThan(*semver.New("0.31.20"))) // be careful with this one + require.True(t, semver.New("0.31.20-without-netgo").LessThan(*semver.New("0.31.20"))) + + // a special build created with "+" would not change the version priority for standard and pre-release versions + require.True(t, semver.New("0.31.20+without-netgo").Equal(*semver.New("0.31.20"))) + require.True(t, semver.New("0.31.20-patch.1+without-netgo").Equal(*semver.New("0.31.20-patch.1"))) } diff --git a/engine/execution/mock/register_store_notifier.go b/engine/execution/mock/register_store_notifier.go new file mode 100644 index 00000000000..7959e9e766b --- /dev/null +++ b/engine/execution/mock/register_store_notifier.go @@ -0,0 +1,30 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mock + +import mock "github.com/stretchr/testify/mock" + +// RegisterStoreNotifier is an autogenerated mock type for the RegisterStoreNotifier type +type RegisterStoreNotifier struct { + mock.Mock +} + +// OnFinalizedAndExecutedHeightUpdated provides a mock function with given fields: height +func (_m *RegisterStoreNotifier) OnFinalizedAndExecutedHeightUpdated(height uint64) { + _m.Called(height) +} + +type mockConstructorTestingTNewRegisterStoreNotifier interface { + mock.TestingT + Cleanup(func()) +} + +// NewRegisterStoreNotifier creates a new instance of RegisterStoreNotifier. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewRegisterStoreNotifier(t mockConstructorTestingTNewRegisterStoreNotifier) *RegisterStoreNotifier { + mock := &RegisterStoreNotifier{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/engine/execution/state/bootstrap/bootstrap.go b/engine/execution/state/bootstrap/bootstrap.go index 7bc96dddb6c..97656092d09 100644 --- a/engine/execution/state/bootstrap/bootstrap.go +++ b/engine/execution/state/bootstrap/bootstrap.go @@ -144,10 +144,10 @@ func (b *Bootstrapper) BootstrapExecutionDatabase( return nil } -func ImportRegistersFromCheckpoint(logger zerolog.Logger, checkpointFile string, checkpointHeight uint64, pdb *pebble.DB, workerCount int) error { - logger.Info().Msgf("importing registers from checkpoint file %s at height %d", checkpointFile, checkpointHeight) +func ImportRegistersFromCheckpoint(logger zerolog.Logger, checkpointFile string, checkpointHeight uint64, checkpointRootHash ledger.RootHash, pdb *pebble.DB, workerCount int) error { + logger.Info().Msgf("importing registers from checkpoint file %s at height %d with root hash: %v", checkpointFile, checkpointHeight, checkpointRootHash) - bootstrap, err := pStorage.NewRegisterBootstrap(pdb, checkpointFile, checkpointHeight, logger) + bootstrap, err := pStorage.NewRegisterBootstrap(pdb, checkpointFile, checkpointHeight, checkpointRootHash, logger) if err != nil { return fmt.Errorf("could not create registers bootstrapper: %w", err) } diff --git a/engine/execution/state/bootstrap/bootstrap_test.go b/engine/execution/state/bootstrap/bootstrap_test.go index 0318e0bf195..584ee2215e0 100644 --- a/engine/execution/state/bootstrap/bootstrap_test.go +++ b/engine/execution/state/bootstrap/bootstrap_test.go @@ -53,7 +53,7 @@ func TestBootstrapLedger(t *testing.T) { } func TestBootstrapLedger_ZeroTokenSupply(t *testing.T) { - expectedStateCommitmentBytes, _ := hex.DecodeString("e1f393d24398b4e4de149073f126a93f309f34a50e80d7421e27afd9a387972f") + expectedStateCommitmentBytes, _ := hex.DecodeString("63e2f2cfd7d22aeebafe460e4b08aa6debcae8add3b285a4d4f06955554fc8c2") expectedStateCommitment, err := flow.ToStateCommitment(expectedStateCommitmentBytes) require.NoError(t, err) diff --git a/engine/execution/state/state.go b/engine/execution/state/state.go index 47152585256..9f2de2807ef 100644 --- a/engine/execution/state/state.go +++ b/engine/execution/state/state.go @@ -487,11 +487,11 @@ func (s *state) GetHighestExecutedBlockID(ctx context.Context) (uint64, flow.Ide // when storehouse is enabled, the highest executed block is consisted as // the highest finalized and executed block height := s.GetHighestFinalizedExecuted() - header, err := s.headers.ByHeight(height) + finalizedID, err := s.headers.BlockIDByHeight(height) if err != nil { return 0, flow.ZeroID, fmt.Errorf("could not get header by height %v: %w", height, err) } - return height, header.ID(), nil + return height, finalizedID, nil } var blockID flow.Identifier diff --git a/engine/execution/state/state_storehouse_test.go b/engine/execution/state/state_storehouse_test.go index 60cd4851f5c..cbbf1fe671b 100644 --- a/engine/execution/state/state_storehouse_test.go +++ b/engine/execution/state/state_storehouse_test.go @@ -114,7 +114,7 @@ func withRegisterStore(t *testing.T, fn func( log := unittest.Logger() var wal execution.ExecutedFinalizedWAL finalized, headerByHeight, highest := testutil.NewMockFinalizedReader(10, 100) - rs, err := storehouse.NewRegisterStore(diskStore, wal, finalized, log) + rs, err := storehouse.NewRegisterStore(diskStore, wal, finalized, log, storehouse.NewNoopNotifier()) require.NoError(t, err) fn(t, rs, diskStore, finalized, 10, highest, headerByHeight) }) diff --git a/engine/execution/storehouse.go b/engine/execution/storehouse.go index d6682ea5e43..47864217c36 100644 --- a/engine/execution/storehouse.go +++ b/engine/execution/storehouse.go @@ -57,6 +57,11 @@ type RegisterStore interface { IsBlockExecuted(height uint64, blockID flow.Identifier) (bool, error) } +// RegisterStoreNotifier is the interface for register store to notify when a block is finalized and executed +type RegisterStoreNotifier interface { + OnFinalizedAndExecutedHeightUpdated(height uint64) +} + type FinalizedReader interface { // FinalizedBlockIDAtHeight returns the block ID of the finalized block at the given height. // It return storage.NotFound if the given height has not been finalized yet diff --git a/engine/execution/storehouse/register_store.go b/engine/execution/storehouse/register_store.go index fafe0903b74..e52e4d1fb5f 100644 --- a/engine/execution/storehouse/register_store.go +++ b/engine/execution/storehouse/register_store.go @@ -20,16 +20,30 @@ type RegisterStore struct { finalized execution.FinalizedReader log zerolog.Logger finalizing *atomic.Bool // making sure only one goroutine is finalizing at a time + notifier execution.RegisterStoreNotifier } var _ execution.RegisterStore = (*RegisterStore)(nil) +type NoopNotifier struct{} + +func NewNoopNotifier() *NoopNotifier { return &NoopNotifier{} } + +func (n *NoopNotifier) OnFinalizedAndExecutedHeightUpdated(height uint64) {} + +var _ execution.RegisterStoreNotifier = (*NoopNotifier)(nil) + func NewRegisterStore( diskStore execution.OnDiskRegisterStore, wal execution.ExecutedFinalizedWAL, finalized execution.FinalizedReader, log zerolog.Logger, + notifier execution.RegisterStoreNotifier, ) (*RegisterStore, error) { + if notifier == nil { + return nil, fmt.Errorf("notifier is empty, use NoopNotifier if you don't need it") + } + // replay the executed and finalized blocks from the write ahead logs // to the OnDiskRegisterStore height, err := syncDiskStore(wal, diskStore, log) @@ -55,6 +69,7 @@ func NewRegisterStore( finalized: finalized, finalizing: atomic.NewBool(false), log: log.With().Str("module", "register-store").Logger(), + notifier: notifier, }, nil } @@ -195,6 +210,8 @@ func (r *RegisterStore) onBlockFinalized() error { return fmt.Errorf("cannot save %v registers to disk store for height %v: %w", len(regs), next, err) } + r.notifier.OnFinalizedAndExecutedHeightUpdated(next) + err = r.memStore.Prune(next, blockID) if err != nil { return fmt.Errorf("cannot prune memStore for height %v: %w", next, err) diff --git a/engine/execution/storehouse/register_store_metrics.go b/engine/execution/storehouse/register_store_metrics.go new file mode 100644 index 00000000000..1542b29d384 --- /dev/null +++ b/engine/execution/storehouse/register_store_metrics.go @@ -0,0 +1,22 @@ +package storehouse + +import ( + "github.com/onflow/flow-go/engine/execution" + "github.com/onflow/flow-go/module" +) + +type RegisterStoreMetrics struct { + collector module.ExecutionMetrics +} + +var _ execution.RegisterStoreNotifier = (*RegisterStoreMetrics)(nil) + +func NewRegisterStoreMetrics(collector module.ExecutionMetrics) *RegisterStoreMetrics { + return &RegisterStoreMetrics{ + collector: collector, + } +} + +func (m *RegisterStoreMetrics) OnFinalizedAndExecutedHeightUpdated(height uint64) { + m.collector.ExecutionLastFinalizedExecutedBlockHeight(height) +} diff --git a/engine/execution/storehouse/register_store_test.go b/engine/execution/storehouse/register_store_test.go index 5b2f9594c08..7435842311e 100644 --- a/engine/execution/storehouse/register_store_test.go +++ b/engine/execution/storehouse/register_store_test.go @@ -15,6 +15,14 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +type notifier struct { + height uint64 +} + +func (n *notifier) OnFinalizedAndExecutedHeightUpdated(height uint64) { + n.height = height +} + func withRegisterStore(t *testing.T, fn func( t *testing.T, rs *storehouse.RegisterStore, @@ -23,14 +31,16 @@ func withRegisterStore(t *testing.T, fn func( rootHeight uint64, endHeight uint64, headers map[uint64]*flow.Header, + n *notifier, )) { pebble.RunWithRegistersStorageAtInitialHeights(t, 10, 10, func(diskStore *pebble.Registers) { log := unittest.Logger() var wal execution.ExecutedFinalizedWAL finalized, headerByHeight, highest := testutil.NewMockFinalizedReader(10, 100) - rs, err := storehouse.NewRegisterStore(diskStore, wal, finalized, log) + n := ¬ifier{height: 10} + rs, err := storehouse.NewRegisterStore(diskStore, wal, finalized, log, n) require.NoError(t, err) - fn(t, rs, diskStore, finalized, 10, highest, headerByHeight) + fn(t, rs, diskStore, finalized, 10, highest, headerByHeight, n) }) } @@ -49,6 +59,7 @@ func TestRegisterStoreGetRegisterFail(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // unknown block _, err := rs.GetRegister(rootHeight+1, unknownBlock, unknownReg.Key) @@ -88,6 +99,7 @@ func TestRegisterStoreSaveRegistersShouldFail(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { wrongParent := unittest.BlockHeaderFixture(unittest.WithHeaderHeight(rootHeight + 1)) err := rs.SaveRegisters(wrongParent, flow.RegisterEntries{}) @@ -117,6 +129,7 @@ func TestRegisterStoreSaveRegistersShouldOK(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // not executed executed, err := rs.IsBlockExecuted(rootHeight+1, headerByHeight[rootHeight+1].ID()) @@ -169,6 +182,7 @@ func TestRegisterStoreIsBlockExecuted(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // save block 11 reg := makeReg("X", "1") @@ -214,6 +228,7 @@ func TestRegisterStoreReadingFromDisk(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // R <- 11 (X: 1, Y: 2) <- 12 (Y: 3) <- 13 (X: 4) @@ -229,9 +244,13 @@ func TestRegisterStoreReadingFromDisk(t *testing.T) { err = rs.SaveRegisters(headerByHeight[rootHeight+3], flow.RegisterEntries{makeReg("X", "4")}) require.NoError(t, err) + require.Equal(t, rootHeight, n.height) + require.NoError(t, finalized.MockFinal(rootHeight+2)) require.NoError(t, rs.OnBlockFinalized()) // notify 12 is finalized + require.Equal(t, rootHeight+2, n.height) + val, err := rs.GetRegister(rootHeight+1, headerByHeight[rootHeight+1].ID(), makeReg("Y", "2").Key) require.NoError(t, err) // value at block 11 is now stored in OnDiskRegisterStore, which is 2 @@ -262,6 +281,7 @@ func TestRegisterStoreReadingFromInMemStore(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // R <- 11 (X: 1, Y: 2) <- 12 (Y: 3) @@ -321,6 +341,7 @@ func TestRegisterStoreReadRegisterAtPrunedHeight(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // R <- 11 (X: 1) @@ -371,6 +392,7 @@ func TestRegisterStoreExecuteFinalizedBlockOrFinalizeExecutedBlockShouldNotCallF rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { require.Equal(t, 1, finalized.FinalizedCalled()) // called by NewRegisterStore @@ -428,6 +450,7 @@ func TestRegisterStoreExecuteFirstFinalizeLater(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { // save block 11 err := rs.SaveRegisters(headerByHeight[rootHeight+1], flow.RegisterEntries{makeReg("X", "1")}) @@ -444,17 +467,22 @@ func TestRegisterStoreExecuteFirstFinalizeLater(t *testing.T) { require.NoError(t, err) require.Equal(t, rootHeight, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight, n.height) + require.NoError(t, finalized.MockFinal(rootHeight+1)) require.NoError(t, rs.OnBlockFinalized()) // notify 11 is finalized require.Equal(t, rootHeight+1, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+1, n.height) require.NoError(t, finalized.MockFinal(rootHeight+2)) require.NoError(t, rs.OnBlockFinalized()) // notify 12 is finalized require.Equal(t, rootHeight+2, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+2, n.height) require.NoError(t, finalized.MockFinal(rootHeight+3)) require.NoError(t, rs.OnBlockFinalized()) // notify 13 is finalized require.Equal(t, rootHeight+3, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+3, n.height) }) } @@ -474,6 +502,7 @@ func TestRegisterStoreFinalizeFirstExecuteLater(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { require.NoError(t, finalized.MockFinal(rootHeight+1)) require.NoError(t, rs.OnBlockFinalized()) // notify 11 is finalized @@ -487,20 +516,25 @@ func TestRegisterStoreFinalizeFirstExecuteLater(t *testing.T) { require.NoError(t, rs.OnBlockFinalized()) // notify 13 is finalized require.Equal(t, rootHeight, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight, n.height) + // save block 11 err := rs.SaveRegisters(headerByHeight[rootHeight+1], flow.RegisterEntries{makeReg("X", "1")}) require.NoError(t, err) require.Equal(t, rootHeight+1, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+1, n.height) // save block 12 err = rs.SaveRegisters(headerByHeight[rootHeight+2], flow.RegisterEntries{makeReg("X", "2")}) require.NoError(t, err) require.Equal(t, rootHeight+2, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+2, n.height) // save block 13 err = rs.SaveRegisters(headerByHeight[rootHeight+3], flow.RegisterEntries{makeReg("X", "3")}) require.NoError(t, err) require.Equal(t, rootHeight+3, rs.LastFinalizedAndExecutedHeight()) + require.Equal(t, rootHeight+3, n.height) }) } @@ -517,6 +551,7 @@ func TestRegisterStoreConcurrentFinalizeAndExecute(t *testing.T) { rootHeight uint64, endHeight uint64, headerByHeight map[uint64]*flow.Header, + n *notifier, ) { var wg sync.WaitGroup diff --git a/engine/testutil/nodes.go b/engine/testutil/nodes.go index 8b3d98bbfa0..ff3c331ff66 100644 --- a/engine/testutil/nodes.go +++ b/engine/testutil/nodes.go @@ -643,7 +643,7 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit require.NoError(t, err) checkpointHeight := uint64(0) - require.NoError(t, esbootstrap.ImportRegistersFromCheckpoint(node.Log, checkpointFile, checkpointHeight, pebbledb, 2)) + require.NoError(t, esbootstrap.ImportRegistersFromCheckpoint(node.Log, checkpointFile, checkpointHeight, matchTrie.RootHash(), pebbledb, 2)) diskStore, err := storagepebble.NewRegisters(pebbledb) require.NoError(t, err) @@ -653,7 +653,9 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit diskStore, nil, // TOOD(leo): replace with real WAL reader, - node.Log) + node.Log, + storehouse.NewNoopNotifier(), + ) require.NoError(t, err) storehouseEnabled := true @@ -761,7 +763,6 @@ func ExecutionNode(t *testing.T, hub *stub.Hub, identity *flow.Identity, identit unit, node.Log, node.Net, - node.Me, fetcher, node.Headers, node.Blocks, diff --git a/fvm/bootstrap.go b/fvm/bootstrap.go index 4b6fad3fc2e..b47100b0ac5 100644 --- a/fvm/bootstrap.go +++ b/fvm/bootstrap.go @@ -76,7 +76,13 @@ type BootstrapParams struct { minimumStorageReservation cadence.UFix64 storagePerFlow cadence.UFix64 restrictedAccountCreationEnabled cadence.Bool - setupEVMEnabled cadence.Bool + + // `setupEVMEnabled` == true && `evmAbiOnly` == true will enable the ABI-only EVM + // `setupEVMEnabled` == true && `evmAbiOnly` == false will enable the full EVM functionality + // `setupEVMEnabled` == false will disable EVM + // This will allow to quickly disable the ABI-only EVM, in case there's a bug or something. + setupEVMEnabled cadence.Bool + evmAbiOnly cadence.Bool // versionFreezePeriod is the number of blocks in the future where the version // changes are frozen. The Node version beacon manages the freeze period, @@ -219,6 +225,13 @@ func WithSetupEVMEnabled(enabled cadence.Bool) BootstrapProcedureOption { } } +func WithEVMABIOnly(evmAbiOnly cadence.Bool) BootstrapProcedureOption { + return func(bp *BootstrapProcedure) *BootstrapProcedure { + bp.evmAbiOnly = evmAbiOnly + return bp + } +} + func WithRestrictedContractDeployment(restricted *bool) BootstrapProcedureOption { return func(bp *BootstrapProcedure) *BootstrapProcedure { bp.restrictedContractDeployment = restricted @@ -829,7 +842,7 @@ func (b *bootstrapExecutor) setupEVM(serviceAddress, fungibleTokenAddress, flowT evmAcc := b.createAccount(nil) // account for storage tx := blueprints.DeployContractTransaction( serviceAddress, - stdlib.ContractCode(flowTokenAddress), + stdlib.ContractCode(flowTokenAddress, bool(b.evmAbiOnly)), stdlib.ContractName, ) // WithEVMEnabled should only be used after we create an account for storage diff --git a/fvm/errors/errors.go b/fvm/errors/errors.go index 0ff5ee1b37a..30c6464b2d4 100644 --- a/fvm/errors/errors.go +++ b/fvm/errors/errors.go @@ -6,6 +6,7 @@ import ( "github.com/hashicorp/go-multierror" "github.com/onflow/cadence/runtime" + "github.com/onflow/cadence/runtime/errors" ) type Unwrappable interface { @@ -213,3 +214,29 @@ func NewEventEncodingError(err error) CodedError { ErrCodeEventEncodingError, "error while encoding emitted event: %w ", err) } + +// EVMError needs to satisfy the user error interface +// in order for Cadence to correctly handle the error +var _ errors.UserError = &(EVMError{}) + +type EVMError struct { + CodedError +} + +func (e EVMError) IsUserError() {} + +// NewEVMError constructs a new CodedError which captures a +// collection of errors provided by (non-fatal) evm runtime. +func NewEVMError(err error) EVMError { + return EVMError{ + WrapCodedError( + ErrEVMExecutionError, + err, + "evm runtime error"), + } +} + +// IsEVMError returns true if error is an EVM error +func IsEVMError(err error) bool { + return HasErrorCode(err, ErrEVMExecutionError) +} diff --git a/fvm/errors/execution.go b/fvm/errors/execution.go index e741345c67c..d70e47e6b7c 100644 --- a/fvm/errors/execution.go +++ b/fvm/errors/execution.go @@ -289,17 +289,3 @@ func NewInvalidInternalStateAccessError( opType, id) } - -// NewEVMError constructs a new CodedError which captures a -// collection of errors provided by (non-fatal) evm runtime. -func NewEVMError(err error) CodedError { - return WrapCodedError( - ErrEVMExecutionError, - err, - "evm runtime error") -} - -// IsEVMError returns true if error is an EVM error -func IsEVMError(err error) bool { - return HasErrorCode(err, ErrEVMExecutionError) -} diff --git a/fvm/evm/emulator/config.go b/fvm/evm/emulator/config.go index 74f012839a9..b15ede521e9 100644 --- a/fvm/evm/emulator/config.go +++ b/fvm/evm/emulator/config.go @@ -69,7 +69,10 @@ func defaultConfig() *Config { EVMConfig: vm.Config{ NoBaseFee: true, }, - TxContext: &vm.TxContext{}, + TxContext: &vm.TxContext{ + GasPrice: new(big.Int), + BlobFeeCap: new(big.Int), + }, BlockContext: &vm.BlockContext{ CanTransfer: core.CanTransfer, Transfer: core.Transfer, diff --git a/fvm/evm/emulator/database/database.go b/fvm/evm/emulator/database/database.go deleted file mode 100644 index 8b18e56e7bc..00000000000 --- a/fvm/evm/emulator/database/database.go +++ /dev/null @@ -1,424 +0,0 @@ -package database - -import ( - stdErrors "errors" - "runtime" - "sync" - - gethCommon "github.com/ethereum/go-ethereum/common" - gethTypes "github.com/ethereum/go-ethereum/core/types" - gethDB "github.com/ethereum/go-ethereum/ethdb" - "github.com/onflow/atree" - - "github.com/onflow/flow-go/fvm/errors" - "github.com/onflow/flow-go/fvm/evm/types" - "github.com/onflow/flow-go/model/flow" -) - -const ( - FlowEVMRootSlabKey = "RootSlabKey" - FlowEVMRootHashKey = "RootHash" - StorageIDSize = 16 -) - -// Database is where EVM data is stored. -// -// under the hood, databases uses an Atree map -// stored under account `flowEVMRootAddress` -// each key value pairs inserted into this map is -// of type of ByteStringValue; we use this type instead -// of atree array, given the EVM environment is not smart enough -// to interact with a portion of the value and would load everything under a key -// before opearting on it. This means it could lead to having large slabs for a single value. -type Database struct { - flowEVMRootAddress flow.Address - led atree.Ledger - storage *atree.PersistentSlabStorage - baseStorage *atree.LedgerBaseStorage - atreemap *atree.OrderedMap - rootIDBytesToBeStored []byte // if is empty means we don't need to store anything - // Ramtin: other database implementations for EVM uses a lock - // to protect the storage against concurrent operations - // though one might do more research to see if we need - // these type of locking if the underlying structure (atree) - // has such protections or if EVM really needs it - lock sync.RWMutex -} - -var _ types.Database = &Database{} - -// NewDatabase returns a wrapped map that implements all the required database interface methods. -func NewDatabase(led atree.Ledger, flowEVMRootAddress flow.Address) (*Database, error) { - baseStorage := atree.NewLedgerBaseStorage(led) - - storage, err := NewPersistentSlabStorage(baseStorage) - if err != nil { - return nil, handleError(err) - } - - db := &Database{ - led: led, - baseStorage: baseStorage, - flowEVMRootAddress: flowEVMRootAddress, - storage: storage, - } - - err = db.retrieveOrCreateMapRoot() - if err != nil { - return nil, handleError(err) - } - return db, nil -} - -func (db *Database) retrieveOrCreateMapRoot() error { - rootIDBytes, err := db.led.GetValue(db.flowEVMRootAddress.Bytes(), []byte(FlowEVMRootSlabKey)) - if err != nil { - return err - } - - var m *atree.OrderedMap - if len(rootIDBytes) == 0 { - m, err = atree.NewMap(db.storage, atree.Address(db.flowEVMRootAddress), atree.NewDefaultDigesterBuilder(), emptyTypeInfo{}) - if err != nil { - return err - } - rootIDBytes := make([]byte, StorageIDSize) - _, err := m.StorageID().ToRawBytes(rootIDBytes) - if err != nil { - return err - } - db.rootIDBytesToBeStored = rootIDBytes - } else { - storageID, err := atree.NewStorageIDFromRawBytes(rootIDBytes) - if err != nil { - return err - } - m, err = atree.NewMapWithRootID(db.storage, storageID, atree.NewDefaultDigesterBuilder()) - if err != nil { - return err - } - } - db.atreemap = m - return nil -} - -// Get retrieves the given key if it's present in the key-value store. -func (db *Database) Get(key []byte) ([]byte, error) { - db.lock.RLock() - defer db.lock.RUnlock() - - value, err := db.get(key) - return value, handleError(err) -} - -func (db *Database) get(key []byte) ([]byte, error) { - data, err := db.atreemap.Get(compare, hashInputProvider, NewByteStringValue(key)) - if err != nil { - return nil, err - } - - v, err := data.StoredValue(db.atreemap.Storage) - if err != nil { - return nil, err - } - - return v.(ByteStringValue).Bytes(), nil -} - -// Put inserts the given value into the key-value store. -func (db *Database) Put(key []byte, value []byte) error { - db.lock.Lock() - defer db.lock.Unlock() - - err := db.put(key, value) - return handleError(err) -} - -func (db *Database) put(key []byte, value []byte) error { - existingValueStorable, err := db.atreemap.Set(compare, hashInputProvider, NewByteStringValue(key), NewByteStringValue(value)) - if err != nil { - return err - } - - if id, ok := existingValueStorable.(atree.StorageIDStorable); ok { - // NOTE: deep remove isn't necessary because value is ByteStringValue (not container) - err := db.storage.Remove(atree.StorageID(id)) - if err != nil { - return err - } - } - - return nil -} - -// Has checks if a key is present in the key-value store. -func (db *Database) Has(key []byte) (bool, error) { - db.lock.RLock() - defer db.lock.RUnlock() - has, err := db.has(key) - return has, handleError(err) -} - -func (db *Database) has(key []byte) (bool, error) { - has, err := db.atreemap.Has(compare, hashInputProvider, NewByteStringValue(key)) - if err != nil { - return false, err - } - return has, nil -} - -// Delete removes the key from the key-value store. -func (db *Database) Delete(key []byte) error { - db.lock.Lock() - defer db.lock.Unlock() - err := db.delete(key) - return handleError(err) -} - -func (db *Database) delete(key []byte) error { - removedMapKeyStorable, removedMapValueStorable, err := db.atreemap.Remove(compare, hashInputProvider, NewByteStringValue(key)) - if err != nil { - return err - } - - if id, ok := removedMapKeyStorable.(atree.StorageIDStorable); ok { - // NOTE: deep remove isn't necessary because key is ByteStringValue (not container) - err := db.storage.Remove(atree.StorageID(id)) - if err != nil { - return err - } - } - - if id, ok := removedMapValueStorable.(atree.StorageIDStorable); ok { - // NOTE: deep remove isn't necessary because value is ByteStringValue (not container) - err := db.storage.Remove(atree.StorageID(id)) - if err != nil { - return err - } - } - return nil -} - -// ApplyBatch applys changes from a batch into the database -func (db *Database) ApplyBatch(b *batch) error { - db.lock.Lock() - defer db.lock.Unlock() - err := db.applyBatch(b) - return err -} - -func (db *Database) applyBatch(b *batch) error { - var err error - for _, keyvalue := range b.writes { - if err != nil { - return err - } - if keyvalue.delete { - err = db.delete(keyvalue.key) - continue - } - err = db.put(keyvalue.key, keyvalue.value) - } - return err -} - -// GetRootHash returns the active root hash -func (db *Database) GetRootHash() (gethCommon.Hash, error) { - db.lock.Lock() - defer db.lock.Unlock() - - hash, err := db.getRootHash() - return hash, handleError(err) -} - -func (db *Database) getRootHash() (gethCommon.Hash, error) { - data, err := db.led.GetValue(db.flowEVMRootAddress[:], []byte(FlowEVMRootHashKey)) - if err != nil { - return gethCommon.Hash{}, handleError(err) - } - if len(data) == 0 { - return gethTypes.EmptyRootHash, nil - } - return gethCommon.BytesToHash(data), nil -} - -// Commits the changes from atree into the underlying storage -// -// this method can be merged as part of batcher -func (db *Database) Commit(root gethCommon.Hash) error { - db.lock.Lock() - defer db.lock.Unlock() - - err := db.commit(root) - return handleError(err) -} - -func (db *Database) commit(root gethCommon.Hash) error { - err := db.storage.FastCommit(runtime.NumCPU()) - if err != nil { - return err - } - - // check if we have to store the rootID - if len(db.rootIDBytesToBeStored) > 0 { - err = db.led.SetValue(db.flowEVMRootAddress.Bytes(), []byte(FlowEVMRootSlabKey), db.rootIDBytesToBeStored[:]) - if err != nil { - return err - } - } - - err = db.led.SetValue(db.flowEVMRootAddress[:], []byte(FlowEVMRootHashKey), root[:]) - if err != nil { - return err - } - return nil -} - -// Close is a no-op -func (db *Database) Close() error { - return nil -} - -// NewBatch creates a write-only key-value store that buffers changes to its host -// database until a final write is called. -func (db *Database) NewBatch() gethDB.Batch { - return &batch{ - db: db, - } -} - -// NewBatchWithSize creates a write-only database batch with pre-allocated buffer. -func (db *Database) NewBatchWithSize(size int) gethDB.Batch { - return &batch{ - db: db, - writes: make([]keyvalue, 0, size), - } -} - -// NewIterator is not supported in this database -// if needed in the future we could implement it using atree iterators -func (db *Database) NewIterator(prefix []byte, start []byte) gethDB.Iterator { - panic(types.ErrNotImplemented) -} - -// NewSnapshot is not supported -func (db *Database) NewSnapshot() (gethDB.Snapshot, error) { - return nil, types.ErrNotImplemented -} - -// Stat method is not supported -func (db *Database) Stat(property string) (string, error) { - return "", types.ErrNotImplemented -} - -func (db *Database) BytesRetrieved() int { - return db.baseStorage.BytesRetrieved() -} - -func (db *Database) BytesStored() int { - return db.baseStorage.BytesStored() -} -func (db *Database) ResetReporter() { - db.baseStorage.ResetReporter() -} - -// Compact is not supported on a memory database, but there's no need either as -// a memory database doesn't waste space anyway. -// Compact is a no op -func (db *Database) Compact(start []byte, limit []byte) error { - return nil -} - -// Len returns the number of entries currently present in the memory database. -// -// Note, this method is only used for testing (i.e. not public in general) and -// does not have explicit checks for closed-ness to allow simpler testing code. -func (db *Database) Len() int { - db.lock.RLock() - defer db.lock.RUnlock() - - return int(db.atreemap.Count()) -} - -// keyvalue is a key-value tuple tagged with a deletion field to allow creating -// memory-database write batches. -type keyvalue struct { - key []byte - value []byte - delete bool -} - -// batch is a write-only memory batch that commits changes to its host -// database when Write is called. A batch cannot be used concurrently. -type batch struct { - db *Database - writes []keyvalue - size int -} - -// Put inserts the given value into the batch for later committing. -func (b *batch) Put(key, value []byte) error { - return b.set(key, value, false) -} - -// Delete inserts the a key removal into the batch for later committing. -func (b *batch) Delete(key []byte) error { - return b.set(key, nil, true) -} - -func (b *batch) set(key []byte, value []byte, delete bool) error { - b.writes = append(b.writes, keyvalue{gethCommon.CopyBytes(key), gethCommon.CopyBytes(value), delete}) - b.size += len(key) + len(value) - return nil -} - -// DropCache drops the database read cache -func (db *Database) DropCache() { - db.storage.DropCache() -} - -// ValueSize retrieves the amount of data queued up for writing. -func (b *batch) ValueSize() int { - return b.size -} - -// Write flushes any accumulated data to the memory database. -func (b *batch) Write() error { - return b.db.ApplyBatch(b) -} - -// Reset resets the batch for reuse. -func (b *batch) Reset() { - // TODO: reset writes elements to release memory if value is large. - b.writes = b.writes[:0] - b.size = 0 -} - -// Replay replays the batch contents. -func (b *batch) Replay(w gethDB.KeyValueWriter) error { - for _, keyvalue := range b.writes { - if keyvalue.delete { - if err := w.Delete(keyvalue.key); err != nil { - return err - } - continue - } - if err := w.Put(keyvalue.key, keyvalue.value); err != nil { - return err - } - } - return nil -} - -func handleError(err error) error { - if err == nil { - return nil - } - var atreeFatalError *atree.FatalError - // if is a atree fatal error or fvm fatal error (the second one captures external errors) - if stdErrors.As(err, &atreeFatalError) || errors.IsFailure(err) { - return types.NewFatalError(err) - } - // wrap the non-fatal error with DB error - return types.NewDatabaseError(err) -} diff --git a/fvm/evm/emulator/database/database_test.go b/fvm/evm/emulator/database/database_test.go deleted file mode 100644 index a23e38d4295..00000000000 --- a/fvm/evm/emulator/database/database_test.go +++ /dev/null @@ -1,177 +0,0 @@ -package database_test - -import ( - "fmt" - "testing" - - gethCommon "github.com/ethereum/go-ethereum/common" - gethTypes "github.com/ethereum/go-ethereum/core/types" - "github.com/onflow/atree" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/fvm/errors" - "github.com/onflow/flow-go/fvm/evm/emulator/database" - "github.com/onflow/flow-go/fvm/evm/testutils" - "github.com/onflow/flow-go/fvm/evm/types" - "github.com/onflow/flow-go/model/flow" -) - -func TestDatabase(t *testing.T) { - - key1 := []byte("ABC") - key2 := []byte("DEF") - value1 := []byte{1, 2, 3, 4, 5, 6, 7, 8} - value2 := []byte{9, 10, 11} - - t.Run("test basic database functionality", func(t *testing.T) { - testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { - testutils.RunWithTestFlowEVMRootAddress(t, backend, func(flowEVMRoot flow.Address) { - db, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - - err = db.Put(key1, value1) - require.NoError(t, err) - - err = db.Commit(gethTypes.EmptyRootHash) - require.NoError(t, err) - - newdb, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - - has, err := newdb.Has(key1) - require.NoError(t, err) - require.True(t, has) - - retValue, err := newdb.Get(key1) - require.NoError(t, err) - - require.Equal(t, value1, retValue) - - err = newdb.Delete(key1) - require.NoError(t, err) - - has, err = newdb.Has(key1) - require.NoError(t, err) - require.False(t, has) - - h, err := newdb.GetRootHash() - require.NoError(t, err) - require.Equal(t, gethTypes.EmptyRootHash, h) - - newRoot := gethCommon.Hash{1, 2, 3} - err = newdb.Commit(newRoot) - require.NoError(t, err) - - h, err = newdb.GetRootHash() - require.NoError(t, err) - require.Equal(t, newRoot, h) - }) - }) - }) - - t.Run("test batch functionality", func(t *testing.T) { - testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { - testutils.RunWithTestFlowEVMRootAddress(t, backend, func(flowEVMRoot flow.Address) { - db, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - - err = db.Put(key1, value1) - require.NoError(t, err) - - has, err := db.Has(key1) - require.NoError(t, err) - require.True(t, has) - - batch := db.NewBatch() - err = batch.Delete(key1) - require.NoError(t, err) - - err = batch.Put(key2, value2) - require.NoError(t, err) - - has, err = db.Has(key2) - require.NoError(t, err) - require.False(t, has) - - err = batch.Write() - require.NoError(t, err) - - retVal, err := db.Get(key2) - require.NoError(t, err) - require.Equal(t, value2, retVal) - - has, err = db.Has(key1) - require.NoError(t, err) - require.False(t, has) - }) - }) - }) - - t.Run("test non fatal error", func(t *testing.T) { - ledger := &testutils.TestValueStore{ - GetValueFunc: func(_, _ []byte) ([]byte, error) { - return nil, errors.NewLedgerInteractionLimitExceededError(0, 0) - }, - SetValueFunc: func(owner, key, value []byte) error { - return nil - }, - } - testutils.RunWithTestFlowEVMRootAddress(t, ledger, func(flowEVMRoot flow.Address) { - _, err := database.NewDatabase(ledger, flowEVMRoot) - require.Error(t, err) - require.True(t, types.IsADatabaseError(err)) - }) - }) - - t.Run("test fatal error (get value)", func(t *testing.T) { - ledger := &testutils.TestValueStore{ - GetValueFunc: func(_, _ []byte) ([]byte, error) { - return nil, fmt.Errorf("a fatal error") - }, - SetValueFunc: func(owner, key, value []byte) error { - return nil - }, - } - testutils.RunWithTestFlowEVMRootAddress(t, ledger, func(flowEVMRoot flow.Address) { - _, err := database.NewDatabase(ledger, flowEVMRoot) - require.Error(t, err) - require.True(t, types.IsAFatalError(err)) - }) - }) - - t.Run("test fatal error (storage id allocation)", func(t *testing.T) { - ledger := &testutils.TestValueStore{ - AllocateStorageIndexFunc: func(_ []byte) (atree.StorageIndex, error) { - return atree.StorageIndex{}, fmt.Errorf("a fatal error") - }, - GetValueFunc: func(_, _ []byte) ([]byte, error) { - return nil, nil - }, - SetValueFunc: func(owner, key, value []byte) error { - return nil - }, - } - testutils.RunWithTestFlowEVMRootAddress(t, ledger, func(flowEVMRoot flow.Address) { - _, err := database.NewDatabase(ledger, flowEVMRoot) - require.Error(t, err) - require.True(t, types.IsAFatalError(err)) - }) - }) - - t.Run("test fatal error (not implemented methods)", func(t *testing.T) { - testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { - testutils.RunWithTestFlowEVMRootAddress(t, backend, func(flowEVMRoot flow.Address) { - db, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - - _, err = db.Stat("") - require.Error(t, err) - require.True(t, types.IsAFatalError(err)) - - _, err = db.NewSnapshot() - require.Error(t, err) - require.True(t, types.IsAFatalError(err)) - }) - }) - }) -} diff --git a/fvm/evm/emulator/database/metered_database.go b/fvm/evm/emulator/database/metered_database.go deleted file mode 100644 index ab8853c5ae7..00000000000 --- a/fvm/evm/emulator/database/metered_database.go +++ /dev/null @@ -1,42 +0,0 @@ -package database - -import ( - "github.com/onflow/atree" - - "github.com/onflow/flow-go/model/flow" -) - -// MeteredDatabase wrapper around the database purposely built for testing and benchmarking. -type MeteredDatabase struct { - *Database -} - -// NewMeteredDatabase create a database wrapper purposely built for testing and benchmarking. -func NewMeteredDatabase(led atree.Ledger, flowEVMRootAddress flow.Address) (*MeteredDatabase, error) { - database, err := NewDatabase(led, flowEVMRootAddress) - if err != nil { - return nil, err - } - - return &MeteredDatabase{ - Database: database, - }, nil -} - -func (m *MeteredDatabase) DropCache() { - m.storage.DropCache() -} - -func (m *MeteredDatabase) BytesRead() int { - return m.baseStorage.BytesRetrieved() -} - -func (m *MeteredDatabase) BytesWritten() int { - return m.baseStorage.BytesStored() -} - -func (m *MeteredDatabase) ResetReporter() { - m.baseStorage.ResetReporter() - m.baseStorage.Size() - m.storage.Count() -} diff --git a/fvm/evm/emulator/database/storable.go b/fvm/evm/emulator/database/storable.go deleted file mode 100644 index 520c47573f1..00000000000 --- a/fvm/evm/emulator/database/storable.go +++ /dev/null @@ -1,242 +0,0 @@ -package database - -import ( - "bytes" - "encoding/binary" - "fmt" - "math" - - "github.com/fxamacker/cbor/v2" - - "github.com/onflow/atree" -) - -type ByteStringValue struct { - data []byte - size uint32 -} - -var _ atree.Value = &ByteStringValue{} -var _ atree.Storable = &ByteStringValue{} - -func NewByteStringValue(data []byte) ByteStringValue { - size := atree.GetUintCBORSize(uint64(len(data))) + uint32(len(data)) - return ByteStringValue{data: data, size: size} -} - -func (v ByteStringValue) ChildStorables() []atree.Storable { - return nil -} - -func (v ByteStringValue) StoredValue(_ atree.SlabStorage) (atree.Value, error) { - return v, nil -} - -func (v ByteStringValue) Storable(storage atree.SlabStorage, address atree.Address, maxInlineSize uint64) (atree.Storable, error) { - if uint64(v.ByteSize()) > maxInlineSize { - - // Create StorableSlab - id, err := storage.GenerateStorageID(address) - if err != nil { - return nil, err - } - - slab := &atree.StorableSlab{ - StorageID: id, - Storable: v, - } - - // Store StorableSlab in storage - err = storage.Store(id, slab) - if err != nil { - return nil, err - } - - // Return storage id as storable - return atree.StorageIDStorable(id), nil - } - - return v, nil -} - -func (v ByteStringValue) Encode(enc *atree.Encoder) error { - return enc.CBOR.EncodeBytes(v.data) -} - -func (v ByteStringValue) getHashInput(scratch []byte) ([]byte, error) { - - const cborTypeByteString = 0x40 - - buf := scratch - if uint32(len(buf)) < v.size { - buf = make([]byte, v.size) - } else { - buf = buf[:v.size] - } - - slen := len(v.data) - - if slen <= 23 { - buf[0] = cborTypeByteString | byte(slen) - copy(buf[1:], v.data) - return buf, nil - } - - if slen <= math.MaxUint8 { - buf[0] = cborTypeByteString | byte(24) - buf[1] = byte(slen) - copy(buf[2:], v.data) - return buf, nil - } - - if slen <= math.MaxUint16 { - buf[0] = cborTypeByteString | byte(25) - binary.BigEndian.PutUint16(buf[1:], uint16(slen)) - copy(buf[3:], v.data) - return buf, nil - } - - if slen <= math.MaxUint32 { - buf[0] = cborTypeByteString | byte(26) - binary.BigEndian.PutUint32(buf[1:], uint32(slen)) - copy(buf[5:], v.data) - return buf, nil - } - - buf[0] = cborTypeByteString | byte(27) - binary.BigEndian.PutUint64(buf[1:], uint64(slen)) - copy(buf[9:], v.data) - return buf, nil -} - -func (v ByteStringValue) ByteSize() uint32 { - return v.size -} - -func (v ByteStringValue) String() string { - return string(v.data) -} - -func (v ByteStringValue) Bytes() []byte { - return v.data -} - -func decodeStorable(dec *cbor.StreamDecoder, _ atree.StorageID) (atree.Storable, error) { - t, err := dec.NextType() - if err != nil { - return nil, err - } - - switch t { - case cbor.ByteStringType: - s, err := dec.DecodeBytes() - if err != nil { - return nil, err - } - return NewByteStringValue(s), nil - - case cbor.TagType: - tagNumber, err := dec.DecodeTagNumber() - if err != nil { - return nil, err - } - - switch tagNumber { - - case atree.CBORTagStorageID: - return atree.DecodeStorageIDStorable(dec) - - default: - return nil, fmt.Errorf("invalid tag number %d", tagNumber) - } - - default: - return nil, fmt.Errorf("invalid cbor type %s for storable", t) - } -} - -func compare(storage atree.SlabStorage, value atree.Value, storable atree.Storable) (bool, error) { - switch v := value.(type) { - - case ByteStringValue: - other, ok := storable.(ByteStringValue) - if ok { - return bytes.Equal(other.data, v.data), nil - } - - // Retrieve value from storage - otherValue, err := storable.StoredValue(storage) - if err != nil { - return false, err - } - other, ok = otherValue.(ByteStringValue) - if ok { - return bytes.Equal(other.data, v.data), nil - } - - return false, nil - } - - return false, fmt.Errorf("value %T not supported for comparison", value) -} - -func hashInputProvider(value atree.Value, buffer []byte) ([]byte, error) { - switch v := value.(type) { - case ByteStringValue: - return v.getHashInput(buffer) - } - - return nil, fmt.Errorf("value %T not supported for hash input", value) -} - -func NewPersistentSlabStorage(baseStorage atree.BaseStorage) (*atree.PersistentSlabStorage, error) { - - encMode, err := cbor.EncOptions{}.EncMode() - if err != nil { - return nil, err - } - - decMode, err := cbor.DecOptions{}.DecMode() - if err != nil { - return nil, err - } - - return atree.NewPersistentSlabStorage( - baseStorage, - encMode, - decMode, - decodeStorable, - decodeTypeInfo, - ), nil - -} - -type emptyTypeInfo struct{} - -var _ atree.TypeInfo = emptyTypeInfo{} - -func (emptyTypeInfo) Encode(e *cbor.StreamEncoder) error { - return e.EncodeNil() -} - -func (i emptyTypeInfo) Equal(other atree.TypeInfo) bool { - _, ok := other.(emptyTypeInfo) - return ok -} - -func decodeTypeInfo(dec *cbor.StreamDecoder) (atree.TypeInfo, error) { - ty, err := dec.NextType() - if err != nil { - return nil, err - } - switch ty { - case cbor.NilType: - err := dec.DecodeNil() - if err != nil { - return nil, err - } - return emptyTypeInfo{}, nil - } - - return nil, fmt.Errorf("not supported type info") -} diff --git a/fvm/evm/emulator/emulator.go b/fvm/evm/emulator/emulator.go index eef720912f7..69ad180d043 100644 --- a/fvm/evm/emulator/emulator.go +++ b/fvm/evm/emulator/emulator.go @@ -5,28 +5,32 @@ import ( gethCommon "github.com/ethereum/go-ethereum/common" gethCore "github.com/ethereum/go-ethereum/core" - gethRawDB "github.com/ethereum/go-ethereum/core/rawdb" - gethState "github.com/ethereum/go-ethereum/core/state" gethTypes "github.com/ethereum/go-ethereum/core/types" gethVM "github.com/ethereum/go-ethereum/core/vm" gethCrypto "github.com/ethereum/go-ethereum/crypto" + "github.com/onflow/atree" + "github.com/onflow/flow-go/fvm/evm/emulator/state" "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" ) // Emulator handles operations against evm runtime type Emulator struct { - Database types.Database + rootAddr flow.Address + ledger atree.Ledger } var _ types.Emulator = &Emulator{} // NewEmulator constructs a new EVM Emulator func NewEmulator( - db types.Database, + ledger atree.Ledger, + rootAddr flow.Address, ) *Emulator { return &Emulator{ - Database: db, + rootAddr: rootAddr, + ledger: ledger, } } @@ -40,7 +44,7 @@ func newConfig(ctx types.BlockContext) *Config { // NewReadOnlyBlockView constructs a new readonly block view func (em *Emulator) NewReadOnlyBlockView(ctx types.BlockContext) (types.ReadOnlyBlockView, error) { - execState, err := newState(em.Database) + execState, err := state.NewStateDB(em.ledger, em.rootAddr) return &ReadOnlyBlockView{ state: execState, }, err @@ -51,14 +55,15 @@ func (em *Emulator) NewBlockView(ctx types.BlockContext) (types.BlockView, error cfg := newConfig(ctx) return &BlockView{ config: cfg, - database: em.Database, + rootAddr: em.rootAddr, + ledger: em.ledger, }, nil } // ReadOnlyBlockView provides a read only view of a block // could be used multiple times for queries type ReadOnlyBlockView struct { - state *gethState.StateDB + state types.StateDB } // BalanceOf returns the balance of the given address @@ -82,7 +87,8 @@ func (bv *ReadOnlyBlockView) NonceOf(address types.Address) (uint64, error) { // TODO: add block level commit (separation of trie commit to storage) type BlockView struct { config *Config - database types.Database + rootAddr flow.Address + ledger atree.Ledger } // DirectCall executes a direct call @@ -100,10 +106,7 @@ func (bl *BlockView) DirectCall(call *types.DirectCall) (*types.Result, error) { default: res, err = proc.run(call.Message(), types.DirectCallTxType) } - if err != nil { - return res, err - } - return res, bl.commit(res.StateRootHash) + return res, err } // RunTransaction runs an evm transaction @@ -126,15 +129,11 @@ func (bl *BlockView) RunTransaction( // update tx context origin proc.evm.TxContext.Origin = msg.From res, err := proc.run(msg, tx.Type()) - if err != nil { - return res, err - } - - return res, bl.commit(res.StateRootHash) + return res, err } func (bl *BlockView) newProcedure() (*procedure, error) { - execState, err := newState(bl.database) + execState, err := state.NewStateDB(bl.ledger, bl.rootAddr) if err != nil { return nil, err } @@ -148,59 +147,27 @@ func (bl *BlockView) newProcedure() (*procedure, error) { cfg.ChainConfig, cfg.EVMConfig, ), - state: execState, - database: bl.database, + state: execState, }, nil } -func (bl *BlockView) commit(rootHash gethCommon.Hash) error { - // commit atree changes back to the backend - err := bl.database.Commit(rootHash) - return handleCommitError(err) -} - type procedure struct { - config *Config - evm *gethVM.EVM - state *gethState.StateDB - database types.Database + config *Config + evm *gethVM.EVM + state types.StateDB } // commit commits the changes to the state. -func (proc *procedure) commit() (gethCommon.Hash, error) { - // commits the changes from the journal into the in memory trie. - // in the future if we want to move this to the block level we could use finalize - // to get the root hash - newRoot, err := proc.state.Commit(true) - if err != nil { - return gethTypes.EmptyRootHash, handleCommitError(err) - } - - // flush the trie to the lower level db - // the reason we have to do this, is the original database - // is designed to keep changes in memory until the state.Commit - // is called then the changes moves into the trie, but the trie - // would stay in memory for faster transaction execution. you - // have to explicitly ask the trie to commit to the underlying storage - err = proc.state.Database().TrieDB().Commit(newRoot, false) - if err != nil { - return gethTypes.EmptyRootHash, handleCommitError(err) - } - - // // remove the read registers (no history tracking) - // err = proc.database.DeleteAndCleanReadKey() - // if err != nil { - // return gethTypes.EmptyRootHash, types.NewFatalError(err) - // } - return newRoot, nil +func (proc *procedure) commit() error { + return handleCommitError(proc.state.Commit()) } func handleCommitError(err error) error { if err == nil { return nil } - // if known types (database errors) don't do anything and return - if types.IsAFatalError(err) || types.IsADatabaseError(err) { + // if known types (state errors) don't do anything and return + if types.IsAFatalError(err) || types.IsAStateError(err) { return err } @@ -209,7 +176,6 @@ func handleCommitError(err error) error { } func (proc *procedure) mintTo(address types.Address, amount *big.Int) (*types.Result, error) { - var err error addr := address.ToCommon() res := &types.Result{ GasConsumed: proc.config.DirectCallBaseGasUsage, @@ -225,13 +191,10 @@ func (proc *procedure) mintTo(address types.Address, amount *big.Int) (*types.Re proc.state.AddBalance(addr, amount) // we don't need to increment any nonce, given the origin doesn't exist - res.StateRootHash, err = proc.commit() - - return res, err + return res, proc.commit() } func (proc *procedure) withdrawFrom(address types.Address, amount *big.Int) (*types.Result, error) { - var err error addr := address.ToCommon() res := &types.Result{ @@ -264,8 +227,7 @@ func (proc *procedure) withdrawFrom(address types.Address, amount *big.Int) (*ty nonce := proc.state.GetNonce(addr) proc.state.SetNonce(addr, nonce+1) - res.StateRootHash, err = proc.commit() - return res, err + return res, proc.commit() } func (proc *procedure) run(msg *gethCore.Message, txType uint8) (*types.Result, error) { @@ -281,8 +243,8 @@ func (proc *procedure) run(msg *gethCore.Message, txType uint8) (*types.Result, ).TransitionDb() if err != nil { res.Failed = true - // if the error is a fatal error or a non-fatal database error return it - if types.IsAFatalError(err) || types.IsADatabaseError(err) { + // if the error is a fatal error or a non-fatal state error return it + if types.IsAFatalError(err) || types.IsAStateError(err) { return &res, err } // otherwise is a validation error (pre-check failure) @@ -299,31 +261,21 @@ func (proc *procedure) run(msg *gethCore.Message, txType uint8) (*types.Result, if msg.To == nil { res.DeployedContractAddress = types.NewAddress(gethCrypto.CreateAddress(msg.From, msg.Nonce)) } - res.Logs = proc.state.Logs() + res.Logs = proc.state.Logs( + // TODO pass proper hash values + gethCommon.Hash{}, + proc.config.BlockContext.BlockNumber.Uint64(), + gethCommon.Hash{}, + 0, + ) } else { res.Failed = true err = types.NewEVMExecutionError(execResult.Err) } } - var commitErr error - res.StateRootHash, commitErr = proc.commit() + commitErr := proc.commit() if commitErr != nil { return &res, commitErr } return &res, err } - -// Ramtin: this is the part of the code that we have to update if we hit performance problems -// the NewDatabase from the RawDB might have to change. -func newState(database types.Database) (*gethState.StateDB, error) { - root, err := database.GetRootHash() - if err != nil { - return nil, err - } - - return gethState.New(root, - gethState.NewDatabase( - gethRawDB.NewDatabase(database), - ), - nil) -} diff --git a/fvm/evm/emulator/emulator_test.go b/fvm/evm/emulator/emulator_test.go index caaf5853cac..3cca27b0906 100644 --- a/fvm/evm/emulator/emulator_test.go +++ b/fvm/evm/emulator/emulator_test.go @@ -1,7 +1,6 @@ package emulator_test import ( - "fmt" "math" "math/big" "testing" @@ -12,7 +11,6 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/fvm/evm/emulator" - "github.com/onflow/flow-go/fvm/evm/emulator/database" "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" @@ -21,18 +19,8 @@ import ( var blockNumber = big.NewInt(10) var defaultCtx = types.NewDefaultBlockContext(blockNumber.Uint64()) -func RunWithTestDB(t testing.TB, f func(types.Database)) { - testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { - testutils.RunWithTestFlowEVMRootAddress(t, backend, func(flowEVMRoot flow.Address) { - db, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - f(db) - }) - }) -} - -func RunWithNewEmulator(t testing.TB, db types.Database, f func(*emulator.Emulator)) { - env := emulator.NewEmulator(db) +func RunWithNewEmulator(t testing.TB, backend *testutils.TestBackend, rootAddr flow.Address, f func(*emulator.Emulator)) { + env := emulator.NewEmulator(backend, rootAddr) f(env) } @@ -49,339 +37,306 @@ func RunWithNewReadOnlyBlockView(t testing.TB, em *emulator.Emulator, f func(blk } func TestNativeTokenBridging(t *testing.T) { - RunWithTestDB(t, func(db types.Database) { - originalBalance := big.NewInt(10000) - testAccount := types.NewAddressFromString("test") + testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { + testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { + originalBalance := big.NewInt(10000) + testAccount := types.NewAddressFromString("test") - t.Run("mint tokens to the first account", func(t *testing.T) { - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall(types.NewDepositCall(testAccount, originalBalance)) - require.NoError(t, err) - require.Equal(t, defaultCtx.DirectCallBaseGasUsage, res.GasConsumed) + t.Run("mint tokens to the first account", func(t *testing.T) { + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall(types.NewDepositCall(testAccount, originalBalance)) + require.NoError(t, err) + require.Equal(t, defaultCtx.DirectCallBaseGasUsage, res.GasConsumed) + }) }) }) - }) - t.Run("tokens withdraw", func(t *testing.T) { - amount := big.NewInt(1000) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { - retBalance, err := blk.BalanceOf(testAccount) - require.NoError(t, err) - require.Equal(t, originalBalance, retBalance) + t.Run("tokens withdraw", func(t *testing.T) { + amount := big.NewInt(1000) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { + retBalance, err := blk.BalanceOf(testAccount) + require.NoError(t, err) + require.Equal(t, originalBalance, retBalance) + }) }) - }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall(types.NewWithdrawCall(testAccount, amount)) - require.NoError(t, err) - require.Equal(t, defaultCtx.DirectCallBaseGasUsage, res.GasConsumed) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall(types.NewWithdrawCall(testAccount, amount)) + require.NoError(t, err) + require.Equal(t, defaultCtx.DirectCallBaseGasUsage, res.GasConsumed) + }) }) - }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { - retBalance, err := blk.BalanceOf(testAccount) - require.NoError(t, err) - require.Equal(t, amount.Sub(originalBalance, amount), retBalance) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { + retBalance, err := blk.BalanceOf(testAccount) + require.NoError(t, err) + require.Equal(t, amount.Sub(originalBalance, amount), retBalance) + }) }) }) }) }) - } func TestContractInteraction(t *testing.T) { - RunWithTestDB(t, func(db types.Database) { - - testContract := testutils.GetStorageTestContract(t) + testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { + testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - testAccount := types.NewAddressFromString("test") - amount := big.NewInt(0).Mul(big.NewInt(1337), big.NewInt(gethParams.Ether)) - amountToBeTransfered := big.NewInt(0).Mul(big.NewInt(100), big.NewInt(gethParams.Ether)) + testContract := testutils.GetStorageTestContract(t) - // fund test account - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(testAccount, amount)) - require.NoError(t, err) - }) - }) - - var contractAddr types.Address + testAccount := types.NewAddressFromString("test") + amount := big.NewInt(0).Mul(big.NewInt(1337), big.NewInt(gethParams.Ether)) + amountToBeTransfered := big.NewInt(0).Mul(big.NewInt(100), big.NewInt(gethParams.Ether)) - t.Run("deploy contract", func(t *testing.T) { - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { + // fund test account + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall( - types.NewDeployCall( - testAccount, - testContract.ByteCode, - math.MaxUint64, - amountToBeTransfered), - ) + _, err := blk.DirectCall(types.NewDepositCall(testAccount, amount)) require.NoError(t, err) - contractAddr = res.DeployedContractAddress - }) - RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { - require.NotNil(t, contractAddr) - retCode, err := blk.CodeOf(contractAddr) - require.NoError(t, err) - require.NotEmpty(t, retCode) - - retBalance, err := blk.BalanceOf(contractAddr) - require.NoError(t, err) - require.Equal(t, amountToBeTransfered, retBalance) - - retBalance, err = blk.BalanceOf(testAccount) - require.NoError(t, err) - require.Equal(t, amount.Sub(amount, amountToBeTransfered), retBalance) }) }) - }) - - t.Run("call contract", func(t *testing.T) { - num := big.NewInt(10) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall( - types.NewContractCall( - testAccount, - contractAddr, - testContract.MakeCallData(t, "store", num), - 1_000_000, - big.NewInt(0), // this should be zero because the contract doesn't have receiver - ), - ) - require.NoError(t, err) - require.GreaterOrEqual(t, res.GasConsumed, uint64(40_000)) + var contractAddr types.Address + + t.Run("deploy contract", func(t *testing.T) { + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall( + types.NewDeployCall( + testAccount, + testContract.ByteCode, + math.MaxUint64, + amountToBeTransfered), + ) + require.NoError(t, err) + contractAddr = res.DeployedContractAddress + }) + RunWithNewReadOnlyBlockView(t, env, func(blk types.ReadOnlyBlockView) { + require.NotNil(t, contractAddr) + retCode, err := blk.CodeOf(contractAddr) + require.NoError(t, err) + require.NotEmpty(t, retCode) + + retBalance, err := blk.BalanceOf(contractAddr) + require.NoError(t, err) + require.Equal(t, amountToBeTransfered, retBalance) + + retBalance, err = blk.BalanceOf(testAccount) + require.NoError(t, err) + require.Equal(t, amount.Sub(amount, amountToBeTransfered), retBalance) + }) }) }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall( - types.NewContractCall( - testAccount, - contractAddr, - testContract.MakeCallData(t, "retrieve"), - 1_000_000, - big.NewInt(0), // this should be zero because the contract doesn't have receiver - ), - ) - require.NoError(t, err) - - ret := new(big.Int).SetBytes(res.ReturnedValue) - require.Equal(t, num, ret) - require.GreaterOrEqual(t, res.GasConsumed, uint64(23_000)) + t.Run("call contract", func(t *testing.T) { + num := big.NewInt(10) + + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall( + types.NewContractCall( + testAccount, + contractAddr, + testContract.MakeCallData(t, "store", num), + 1_000_000, + big.NewInt(0), // this should be zero because the contract doesn't have receiver + ), + ) + require.NoError(t, err) + require.GreaterOrEqual(t, res.GasConsumed, uint64(40_000)) + }) }) - }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - res, err := blk.DirectCall( - types.NewContractCall( - testAccount, - contractAddr, - testContract.MakeCallData(t, "blockNumber"), - 1_000_000, - big.NewInt(0), // this should be zero because the contract doesn't have receiver - ), - ) - require.NoError(t, err) - - ret := new(big.Int).SetBytes(res.ReturnedValue) - require.Equal(t, blockNumber, ret) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall( + types.NewContractCall( + testAccount, + contractAddr, + testContract.MakeCallData(t, "retrieve"), + 1_000_000, + big.NewInt(0), // this should be zero because the contract doesn't have receiver + ), + ) + require.NoError(t, err) + + ret := new(big.Int).SetBytes(res.ReturnedValue) + require.Equal(t, num, ret) + require.GreaterOrEqual(t, res.GasConsumed, uint64(23_000)) + }) }) - }) - - }) - t.Run("test sending transactions (happy case)", func(t *testing.T) { - account := testutils.GetTestEOAAccount(t, testutils.EOATestAccount1KeyHex) - fAddr := account.Address() - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(fAddr, amount)) - require.NoError(t, err) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + res, err := blk.DirectCall( + types.NewContractCall( + testAccount, + contractAddr, + testContract.MakeCallData(t, "blockNumber"), + 1_000_000, + big.NewInt(0), // this should be zero because the contract doesn't have receiver + ), + ) + require.NoError(t, err) + + ret := new(big.Int).SetBytes(res.ReturnedValue) + require.Equal(t, blockNumber, ret) + }) }) + }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) - ctx.GasFeeCollector = types.NewAddressFromString("coinbase") - coinbaseOrgBalance := gethCommon.Big1 - // small amount of money to create account - RunWithNewBlockView(t, env, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(ctx.GasFeeCollector, coinbaseOrgBalance)) - require.NoError(t, err) + t.Run("test sending transactions (happy case)", func(t *testing.T) { + account := testutils.GetTestEOAAccount(t, testutils.EOATestAccount1KeyHex) + fAddr := account.Address() + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + _, err := blk.DirectCall(types.NewDepositCall(fAddr, amount)) + require.NoError(t, err) + }) }) - blk, err := env.NewBlockView(ctx) - require.NoError(t, err) - tx := account.PrepareAndSignTx( - t, - testAccount.ToCommon(), // to - nil, // data - big.NewInt(1000), // amount - gethParams.TxGas, // gas limit - gethCommon.Big1, // gas fee - - ) - _, err = blk.RunTransaction(tx) - require.NoError(t, err) - - // check the balance of coinbase - RunWithNewReadOnlyBlockView(t, env, func(blk2 types.ReadOnlyBlockView) { - bal, err := blk2.BalanceOf(ctx.GasFeeCollector) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) + ctx.GasFeeCollector = types.NewAddressFromString("coinbase") + coinbaseOrgBalance := gethCommon.Big1 + // small amount of money to create account + RunWithNewBlockView(t, env, func(blk types.BlockView) { + _, err := blk.DirectCall(types.NewDepositCall(ctx.GasFeeCollector, coinbaseOrgBalance)) + require.NoError(t, err) + }) + + blk, err := env.NewBlockView(ctx) require.NoError(t, err) - expected := gethParams.TxGas*gethCommon.Big1.Uint64() + gethCommon.Big1.Uint64() - require.Equal(t, expected, bal.Uint64()) + tx := account.PrepareAndSignTx( + t, + testAccount.ToCommon(), // to + nil, // data + big.NewInt(1000), // amount + gethParams.TxGas, // gas limit + gethCommon.Big1, // gas fee - nonce, err := blk2.NonceOf(fAddr) + ) + _, err = blk.RunTransaction(tx) require.NoError(t, err) - require.Equal(t, 1, int(nonce)) + + // check the balance of coinbase + RunWithNewReadOnlyBlockView(t, env, func(blk2 types.ReadOnlyBlockView) { + bal, err := blk2.BalanceOf(ctx.GasFeeCollector) + require.NoError(t, err) + expected := gethParams.TxGas*gethCommon.Big1.Uint64() + gethCommon.Big1.Uint64() + require.Equal(t, expected, bal.Uint64()) + + nonce, err := blk2.NonceOf(fAddr) + require.NoError(t, err) + require.Equal(t, 1, int(nonce)) + }) }) }) - }) - t.Run("test sending transactions (invalid nonce)", func(t *testing.T) { - account := testutils.GetTestEOAAccount(t, testutils.EOATestAccount1KeyHex) - fAddr := account.Address() - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - RunWithNewBlockView(t, env, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(fAddr, amount)) - require.NoError(t, err) + t.Run("test sending transactions (invalid nonce)", func(t *testing.T) { + account := testutils.GetTestEOAAccount(t, testutils.EOATestAccount1KeyHex) + fAddr := account.Address() + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + RunWithNewBlockView(t, env, func(blk types.BlockView) { + _, err := blk.DirectCall(types.NewDepositCall(fAddr, amount)) + require.NoError(t, err) + }) }) - }) - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) - blk, err := env.NewBlockView(ctx) - require.NoError(t, err) - tx := account.SignTx(t, - gethTypes.NewTransaction( - 100, // nonce - testAccount.ToCommon(), // to - big.NewInt(1000), // amount - gethParams.TxGas, // gas limit - gethCommon.Big1, // gas fee - nil, // data - ), - ) - _, err = blk.RunTransaction(tx) - require.Error(t, err) - require.True(t, types.IsEVMValidationError(err)) + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) + blk, err := env.NewBlockView(ctx) + require.NoError(t, err) + tx := account.SignTx(t, + gethTypes.NewTransaction( + 100, // nonce + testAccount.ToCommon(), // to + big.NewInt(1000), // amount + gethParams.TxGas, // gas limit + gethCommon.Big1, // gas fee + nil, // data + ), + ) + _, err = blk.RunTransaction(tx) + require.Error(t, err) + require.True(t, types.IsEVMValidationError(err)) + }) }) - }) - t.Run("test sending transactions (bad signature)", func(t *testing.T) { - RunWithNewEmulator(t, db, func(env *emulator.Emulator) { - ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) - blk, err := env.NewBlockView(ctx) - require.NoError(t, err) - tx := gethTypes.NewTx(&gethTypes.LegacyTx{ - Nonce: 0, - GasPrice: gethCommon.Big1, - Gas: gethParams.TxGas, // gas limit - To: nil, // to - Value: big.NewInt(1000), // amount - Data: nil, // data - V: big.NewInt(1), - R: big.NewInt(2), - S: big.NewInt(3), + t.Run("test sending transactions (bad signature)", func(t *testing.T) { + RunWithNewEmulator(t, backend, rootAddr, func(env *emulator.Emulator) { + ctx := types.NewDefaultBlockContext(blockNumber.Uint64()) + blk, err := env.NewBlockView(ctx) + require.NoError(t, err) + tx := gethTypes.NewTx(&gethTypes.LegacyTx{ + Nonce: 0, + GasPrice: gethCommon.Big1, + Gas: gethParams.TxGas, // gas limit + To: nil, // to + Value: big.NewInt(1000), // amount + Data: nil, // data + V: big.NewInt(1), + R: big.NewInt(2), + S: big.NewInt(3), + }) + _, err = blk.RunTransaction(tx) + require.Error(t, err) + require.True(t, types.IsEVMValidationError(err)) }) - _, err = blk.RunTransaction(tx) - require.Error(t, err) - require.True(t, types.IsEVMValidationError(err)) }) - }) + }) }) } func TestTransfers(t *testing.T) { - RunWithTestDB(t, func(db types.Database) { - testAccount1 := types.NewAddressFromString("test1") - testAccount2 := types.NewAddressFromString("test2") - - amount := big.NewInt(0).Mul(big.NewInt(1337), big.NewInt(gethParams.Ether)) - amountToBeTransfered := big.NewInt(0).Mul(big.NewInt(100), big.NewInt(gethParams.Ether)) - - RunWithNewEmulator(t, db, func(em *emulator.Emulator) { - RunWithNewBlockView(t, em, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(testAccount1, amount)) - require.NoError(t, err) - }) - }) + testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { + testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - RunWithNewEmulator(t, db, func(em *emulator.Emulator) { - RunWithNewBlockView(t, em, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewTransferCall(testAccount1, testAccount2, amountToBeTransfered)) - require.NoError(t, err) - }) - }) + testAccount1 := types.NewAddressFromString("test1") + testAccount2 := types.NewAddressFromString("test2") - RunWithNewEmulator(t, db, func(em *emulator.Emulator) { - RunWithNewReadOnlyBlockView(t, em, func(blk types.ReadOnlyBlockView) { - bal, err := blk.BalanceOf(testAccount2) - require.NoError(t, err) - require.Equal(t, amountToBeTransfered.Uint64(), bal.Uint64()) + amount := big.NewInt(0).Mul(big.NewInt(1337), big.NewInt(gethParams.Ether)) + amountToBeTransfered := big.NewInt(0).Mul(big.NewInt(100), big.NewInt(gethParams.Ether)) - bal, err = blk.BalanceOf(testAccount1) - require.NoError(t, err) - require.Equal(t, new(big.Int).Sub(amount, amountToBeTransfered).Uint64(), bal.Uint64()) + RunWithNewEmulator(t, backend, rootAddr, func(em *emulator.Emulator) { + RunWithNewBlockView(t, em, func(blk types.BlockView) { + _, err := blk.DirectCall(types.NewDepositCall(testAccount1, amount)) + require.NoError(t, err) + }) }) - }) - }) -} - -func TestDatabaseErrorHandling(t *testing.T) { - t.Run("test non-fatal db error handling", func(t *testing.T) { - db := &testutils.TestDatabase{ - GetRootHashFunc: func() (gethCommon.Hash, error) { - return gethTypes.EmptyRootHash, types.NewDatabaseError(fmt.Errorf("some non-fatal error")) - }, - } - - RunWithNewEmulator(t, db, func(em *emulator.Emulator) { - RunWithNewBlockView(t, em, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(types.EmptyAddress, big.NewInt(1))) - require.Error(t, err) - require.True(t, types.IsADatabaseError(err)) + RunWithNewEmulator(t, backend, rootAddr, func(em *emulator.Emulator) { + RunWithNewBlockView(t, em, func(blk types.BlockView) { + _, err := blk.DirectCall(types.NewTransferCall(testAccount1, testAccount2, amountToBeTransfered)) + require.NoError(t, err) + }) }) - }) - }) - t.Run("test fatal db error handling", func(t *testing.T) { - db := &testutils.TestDatabase{ - GetRootHashFunc: func() (gethCommon.Hash, error) { - return gethTypes.EmptyRootHash, types.NewFatalError(fmt.Errorf("some non-fatal error")) - }, - } + RunWithNewEmulator(t, backend, rootAddr, func(em *emulator.Emulator) { + RunWithNewReadOnlyBlockView(t, em, func(blk types.ReadOnlyBlockView) { + bal, err := blk.BalanceOf(testAccount2) + require.NoError(t, err) + require.Equal(t, amountToBeTransfered.Uint64(), bal.Uint64()) - RunWithNewEmulator(t, db, func(em *emulator.Emulator) { - RunWithNewBlockView(t, em, func(blk types.BlockView) { - _, err := blk.DirectCall(types.NewDepositCall(types.EmptyAddress, big.NewInt(1))) - require.Error(t, err) - require.True(t, types.IsAFatalError(err)) + bal, err = blk.BalanceOf(testAccount1) + require.NoError(t, err) + require.Equal(t, new(big.Int).Sub(amount, amountToBeTransfered).Uint64(), bal.Uint64()) + }) }) }) }) } func TestStorageNoSideEffect(t *testing.T) { - t.Skip("we need to fix this issue ") - testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { testutils.RunWithTestFlowEVMRootAddress(t, backend, func(flowEVMRoot flow.Address) { - db, err := database.NewDatabase(backend, flowEVMRoot) - require.NoError(t, err) - - em := emulator.NewEmulator(db) + var err error + em := emulator.NewEmulator(backend, flowEVMRoot) testAccount := types.NewAddressFromString("test") - amount := big.NewInt(100) + amount := big.NewInt(10) RunWithNewBlockView(t, em, func(blk types.BlockView) { _, err = blk.DirectCall(types.NewDepositCall(testAccount, amount)) require.NoError(t, err) diff --git a/fvm/evm/emulator/state/account.go b/fvm/evm/emulator/state/account.go new file mode 100644 index 00000000000..489d944b798 --- /dev/null +++ b/fvm/evm/emulator/state/account.go @@ -0,0 +1,62 @@ +package state + +import ( + "math/big" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rlp" +) + +// Account holds the metadata of an address and provides (de)serialization functionality +// +// Note that code and storage slots of an address is not part of this data structure +type Account struct { + // address + Address gethCommon.Address + // balance of the address + Balance *big.Int + // nonce of the address + Nonce uint64 + // hash of the code + // if no code the gethTypes.EmptyCodeHash is stored + CodeHash gethCommon.Hash + // the id of the collection holds storage slots for this account + // this value is nil for EOA accounts + CollectionID []byte +} + +// NewAccount constructs a new account +func NewAccount( + address gethCommon.Address, + balance *big.Int, + nonce uint64, + codeHash gethCommon.Hash, + collectionID []byte, +) *Account { + return &Account{ + Address: address, + Balance: balance, + Nonce: nonce, + CodeHash: codeHash, + CollectionID: collectionID, + } +} + +func (a *Account) HasCode() bool { + return a.CodeHash != gethTypes.EmptyCodeHash +} + +// Encode encodes the account +func (a *Account) Encode() ([]byte, error) { + return rlp.EncodeToBytes(a) +} + +// DecodeAccount constructs a new account from the encoded data +func DecodeAccount(inp []byte) (*Account, error) { + if len(inp) == 0 { + return nil, nil + } + a := &Account{} + return a, rlp.DecodeBytes(inp, a) +} diff --git a/fvm/evm/emulator/state/account_test.go b/fvm/evm/emulator/state/account_test.go new file mode 100644 index 00000000000..c3e2fca047e --- /dev/null +++ b/fvm/evm/emulator/state/account_test.go @@ -0,0 +1,28 @@ +package state_test + +import ( + "testing" + + "github.com/ethereum/go-ethereum/common" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" +) + +func TestAccountEncoding(t *testing.T) { + acc := state.NewAccount( + testutils.RandomCommonAddress(t), + testutils.RandomBigInt(1000), + uint64(2), + common.BytesToHash([]byte{1}), + []byte{2}, + ) + + encoded, err := acc.Encode() + require.NoError(t, err) + + ret, err := state.DecodeAccount(encoded) + require.NoError(t, err) + require.Equal(t, acc, ret) +} diff --git a/fvm/evm/emulator/state/base.go b/fvm/evm/emulator/state/base.go new file mode 100644 index 00000000000..173aa6d40f0 --- /dev/null +++ b/fvm/evm/emulator/state/base.go @@ -0,0 +1,501 @@ +package state + +import ( + "fmt" + "math/big" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + "github.com/onflow/atree" + + "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" +) + +const ( + // AccountsStorageIDKey is the path where we store the collection ID for accounts + AccountsStorageIDKey = "AccountsStorageIDKey" + // CodesStorageIDKey is the path where we store the collection ID for codes + CodesStorageIDKey = "CodesStorageIDKey" +) + +// BaseView implements a types.BaseView +// it acts as the base layer of state queries for the stateDB +// it stores accounts, codes and storage slots. +// +// under the hood it uses a set of collections, +// one for account's meta data, one for codes +// and one for each of account storage space. +type BaseView struct { + rootAddress flow.Address + ledger atree.Ledger + collectionProvider *CollectionProvider + + // collections + accounts *Collection + codes *Collection + slots map[gethCommon.Address]*Collection + + // cached values + cachedAccounts map[gethCommon.Address]*Account + cachedCodes map[gethCommon.Address][]byte + cachedSlots map[types.SlotAddress]gethCommon.Hash + + // flags + accountSetupOnCommit bool + codeSetupOnCommit bool +} + +var _ types.BaseView = &BaseView{} + +// NewBaseView constructs a new base view +func NewBaseView(ledger atree.Ledger, rootAddress flow.Address) (*BaseView, error) { + cp, err := NewCollectionProvider(atree.Address(rootAddress), ledger) + if err != nil { + return nil, err + } + + view := &BaseView{ + ledger: ledger, + rootAddress: rootAddress, + collectionProvider: cp, + + slots: make(map[gethCommon.Address]*Collection), + + cachedAccounts: make(map[gethCommon.Address]*Account), + cachedCodes: make(map[gethCommon.Address][]byte), + cachedSlots: make(map[types.SlotAddress]gethCommon.Hash), + } + + // fetch the account collection, if not exist, create one + view.accounts, view.accountSetupOnCommit, err = view.fetchOrCreateCollection(AccountsStorageIDKey) + if err != nil { + return nil, err + } + + // fetch the code collection, if not exist, create one + view.codes, view.codeSetupOnCommit, err = view.fetchOrCreateCollection(CodesStorageIDKey) + if err != nil { + return nil, err + } + + return view, nil +} + +// Exist returns true if the address exist in the state +func (v *BaseView) Exist(addr gethCommon.Address) (bool, error) { + acc, err := v.getAccount(addr) + return acc != nil, err +} + +// IsCreated returns true if the address has been created in the context of this transaction +func (v *BaseView) IsCreated(gethCommon.Address) bool { + return false +} + +// HasSelfDestructed returns true if an address is flagged for destruction at the end of transaction +func (v *BaseView) HasSelfDestructed(gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) +} + +// GetBalance returns the balance of an address +// +// for non-existent accounts it returns a balance of zero +func (v *BaseView) GetBalance(addr gethCommon.Address) (*big.Int, error) { + acc, err := v.getAccount(addr) + bal := big.NewInt(0) + if acc != nil { + bal = acc.Balance + } + return bal, err +} + +// GetNonce returns the nonce of an address +// +// for non-existent accounts it returns zero +func (v *BaseView) GetNonce(addr gethCommon.Address) (uint64, error) { + acc, err := v.getAccount(addr) + nonce := uint64(0) + if acc != nil { + nonce = acc.Nonce + } + return nonce, err +} + +// GetCode returns the code of an address +// +// for non-existent accounts or accounts without a code (e.g. EOAs) it returns nil +func (v *BaseView) GetCode(addr gethCommon.Address) ([]byte, error) { + return v.getCode(addr) +} + +// GetCodeHash returns the code hash of an address +// +// for non-existent accounts or accounts without a code (e.g. EOAs) it returns default empty +// hash value (gethTypes.EmptyCodeHash) +func (v *BaseView) GetCodeHash(addr gethCommon.Address) (gethCommon.Hash, error) { + acc, err := v.getAccount(addr) + codeHash := gethTypes.EmptyCodeHash + if acc != nil { + codeHash = acc.CodeHash + } + return codeHash, err +} + +// GetCodeSize returns the code size of an address +// +// for non-existent accounts or accounts without a code (e.g. EOAs) it returns zero +func (v *BaseView) GetCodeSize(addr gethCommon.Address) (int, error) { + code, err := v.GetCode(addr) + return len(code), err +} + +// GetState returns values for a slot in the main storage +// +// for non-existent slots it returns the default empty hash value (gethTypes.EmptyCodeHash) +func (v *BaseView) GetState(sk types.SlotAddress) (gethCommon.Hash, error) { + return v.getSlot(sk) +} + +// UpdateSlot updates the value for a slot +func (v *BaseView) UpdateSlot(sk types.SlotAddress, value gethCommon.Hash) error { + return v.storeSlot(sk, value) +} + +// GetRefund returns the total amount of (gas) refund +// +// this method returns the value of zero +func (v *BaseView) GetRefund() uint64 { + return 0 +} + +// GetTransientState returns values for an slot transient storage +// +// transient storage is not a functionality for the base view so it always +// returns the default value for non-existent slots +func (v *BaseView) GetTransientState(types.SlotAddress) gethCommon.Hash { + return gethCommon.Hash{} +} + +// AddressInAccessList checks if an address is in the access list +// +// access list control is not a functionality of the base view +// it always returns false +func (v *BaseView) AddressInAccessList(gethCommon.Address) bool { + return false +} + +// SlotInAccessList checks if a slot is in the access list +// +// access list control is not a functionality of the base view +// it always returns false +func (v *BaseView) SlotInAccessList(types.SlotAddress) (addressOk bool, slotOk bool) { + return false, false +} + +// CreateAccount creates a new account +func (v *BaseView) CreateAccount( + addr gethCommon.Address, + balance *big.Int, + nonce uint64, + code []byte, + codeHash gethCommon.Hash, +) error { + var colID []byte + // if is an smart contract account + if len(code) > 0 { + err := v.storeCode(addr, code) + if err != nil { + return err + } + } + + // create a new account and store it + acc := NewAccount(addr, balance, nonce, codeHash, colID) + + // no need to update the cache , storeAccount would update the cache + return v.storeAccount(acc) +} + +// UpdateAccount updates an account's meta data +func (v *BaseView) UpdateAccount( + addr gethCommon.Address, + balance *big.Int, + nonce uint64, + code []byte, + codeHash gethCommon.Hash, +) error { + acc, err := v.getAccount(addr) + if err != nil { + return err + } + // if update is called on a non existing account + // we gracefully call the create account + // TODO: but we might need to revisit this action in the future + if acc == nil { + return v.CreateAccount(addr, balance, nonce, code, codeHash) + } + // if it has a code change + if codeHash != acc.CodeHash { + err := v.storeCode(addr, code) + if err != nil { + return err + } + // TODO: maybe purge the state in the future as well + // currently the behaviour of stateDB doesn't purge the data + // We don't need to check if the code is empty and we purge the state + // this is not possible right now. + } + newAcc := NewAccount(addr, balance, nonce, codeHash, acc.CollectionID) + // no need to update the cache , storeAccount would update the cache + return v.storeAccount(newAcc) +} + +// DeleteAccount deletes an account's meta data, code, and +// storage slots associated with that address +func (v *BaseView) DeleteAccount(addr gethCommon.Address) error { + // 1. check account exists + acc, err := v.getAccount(addr) + if err != nil { + return err + } + if acc == nil { + return fmt.Errorf("account doesn't exist to be deleted") + } + + // 2. update the cache + delete(v.cachedAccounts, addr) + + // 3. collections + err = v.accounts.Remove(addr.Bytes()) + if err != nil { + return err + } + + // 4. remove the code + if acc.HasCode() { + err = v.storeCode(addr, nil) + if err != nil { + return err + } + } + + // 5. remove storage slots + if len(acc.CollectionID) > 0 { + col, found := v.slots[addr] + if !found { + col, err = v.collectionProvider.CollectionByID(acc.CollectionID) + if err != nil { + return err + } + } + // delete all slots related to this account (eip-6780) + keys, err := col.Destroy() + if err != nil { + return err + } + + delete(v.slots, addr) + + for _, key := range keys { + delete(v.cachedSlots, types.SlotAddress{ + Address: addr, + Key: gethCommon.BytesToHash(key), + }) + } + } + return nil +} + +// Commit commits the changes to the underlying storage layers +func (v *BaseView) Commit() error { + // commit collection changes + err := v.collectionProvider.Commit() + if err != nil { + return err + } + + // if this is the first time we are setting up an + // account collection, store its collection id. + if v.accountSetupOnCommit { + err = v.ledger.SetValue(v.rootAddress[:], []byte(AccountsStorageIDKey), v.accounts.CollectionID()) + if err != nil { + return err + } + v.accountSetupOnCommit = false + + } + + // if this is the first time we are setting up an + // code collection, store its collection id. + if v.codeSetupOnCommit { + err = v.ledger.SetValue(v.rootAddress[:], []byte(CodesStorageIDKey), v.codes.CollectionID()) + if err != nil { + return err + } + v.codeSetupOnCommit = false + } + return nil +} + +func (v *BaseView) fetchOrCreateCollection(path string) (collection *Collection, created bool, error error) { + collectionID, err := v.ledger.GetValue(v.rootAddress[:], []byte(path)) + if err != nil { + return nil, false, err + } + if len(collectionID) == 0 { + collection, err = v.collectionProvider.NewCollection() + return collection, true, err + } + collection, err = v.collectionProvider.CollectionByID(collectionID) + return collection, false, err +} + +func (v *BaseView) getAccount(addr gethCommon.Address) (*Account, error) { + // check cached accounts first + acc, found := v.cachedAccounts[addr] + if found { + return acc, nil + } + + // then collect it from the account collection + data, err := v.accounts.Get(addr.Bytes()) + if err != nil { + return nil, err + } + // decode it + acc, err = DecodeAccount(data) + if err != nil { + return nil, err + } + // cache it + if acc != nil { + v.cachedAccounts[addr] = acc + } + return acc, nil +} + +func (v *BaseView) storeAccount(acc *Account) error { + data, err := acc.Encode() + if err != nil { + return err + } + // update the cache + v.cachedAccounts[acc.Address] = acc + return v.accounts.Set(acc.Address.Bytes(), data) +} + +func (v *BaseView) getCode(addr gethCommon.Address) ([]byte, error) { + // check the cache first + code, found := v.cachedCodes[addr] + if found { + return code, nil + } + // check if account exist in cache and has codeHash + acc, found := v.cachedAccounts[addr] + if found && !acc.HasCode() { + return nil, nil + } + // then collect it from the code collection + code, err := v.codes.Get(addr.Bytes()) + if err != nil { + return nil, err + } + if code != nil { + v.cachedCodes[addr] = code + } + return code, nil +} + +func (v *BaseView) storeCode(addr gethCommon.Address, code []byte) error { + if len(code) == 0 { + delete(v.cachedCodes, addr) + return v.codes.Remove(addr.Bytes()) + } + v.cachedCodes[addr] = code + return v.codes.Set(addr.Bytes(), code) +} + +func (v *BaseView) getSlot(sk types.SlotAddress) (gethCommon.Hash, error) { + value, found := v.cachedSlots[sk] + if found { + return value, nil + } + + acc, err := v.getAccount(sk.Address) + if err != nil { + return gethCommon.Hash{}, err + } + if acc == nil || len(acc.CollectionID) == 0 { + return gethCommon.Hash{}, nil + } + + col, err := v.getSlotCollection(acc) + if err != nil { + return gethCommon.Hash{}, err + } + + val, err := col.Get(sk.Key.Bytes()) + if err != nil { + return gethCommon.Hash{}, err + } + value = gethCommon.BytesToHash(val) + v.cachedSlots[sk] = value + return value, nil +} + +func (v *BaseView) storeSlot(sk types.SlotAddress, data gethCommon.Hash) error { + acc, err := v.getAccount(sk.Address) + if err != nil { + return err + } + if acc == nil { + return fmt.Errorf("slot belongs to a non-existing account") + } + if !acc.HasCode() { + return fmt.Errorf("slot belongs to a non-smart contract account") + } + col, err := v.getSlotCollection(acc) + if err != nil { + return err + } + + emptyValue := gethCommon.Hash{} + if data == emptyValue { + delete(v.cachedSlots, sk) + return col.Remove(sk.Key.Bytes()) + } + v.cachedSlots[sk] = data + return col.Set(sk.Key.Bytes(), data.Bytes()) +} + +func (v *BaseView) getSlotCollection(acc *Account) (*Collection, error) { + var err error + + if len(acc.CollectionID) == 0 { + // create a new collection for slots + col, err := v.collectionProvider.NewCollection() + if err != nil { + return nil, err + } + // cache collection + v.slots[acc.Address] = col + // update account's collection ID + acc.CollectionID = col.CollectionID() + err = v.storeAccount(acc) + if err != nil { + return nil, err + } + return col, nil + } + + col, found := v.slots[acc.Address] + if !found { + col, err = v.collectionProvider.CollectionByID(acc.CollectionID) + if err != nil { + return nil, err + } + v.slots[acc.Address] = col + } + return col, nil +} diff --git a/fvm/evm/emulator/state/base_test.go b/fvm/evm/emulator/state/base_test.go new file mode 100644 index 00000000000..c216d365b55 --- /dev/null +++ b/fvm/evm/emulator/state/base_test.go @@ -0,0 +1,247 @@ +package state_test + +import ( + "math/big" + "testing" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" +) + +func TestBaseView(t *testing.T) { + t.Parallel() + + t.Run("test account functionalities", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + rootAddr := flow.Address{1, 2, 3, 4, 5, 6, 7, 8} + view, err := state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + addr1 := testutils.RandomCommonAddress(t) + + // data calls for a non-existent account + checkAccount(t, + view, + addr1, + false, + big.NewInt(0), + uint64(0), + nil, + gethTypes.EmptyCodeHash, + ) + + // create an account with code + newBal := big.NewInt(10) + newNonce := uint64(5) + newCode := []byte("some code") + newCodeHash := gethCommon.Hash{1, 2} + + err = view.CreateAccount(addr1, newBal, newNonce, newCode, newCodeHash) + require.NoError(t, err) + + // check data from cache + checkAccount(t, + view, + addr1, + true, + newBal, + newNonce, + newCode, + newCodeHash, + ) + + // commit the changes and create a new baseview + err = view.Commit() + require.NoError(t, err) + + view, err = state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + checkAccount(t, + view, + addr1, + true, + newBal, + newNonce, + newCode, + newCodeHash, + ) + + // test update account + + newBal = big.NewInt(12) + newNonce = uint64(6) + newCode = []byte("some new code") + newCodeHash = gethCommon.Hash{2, 3} + err = view.UpdateAccount(addr1, newBal, newNonce, newCode, newCodeHash) + require.NoError(t, err) + + // check data from cache + checkAccount(t, + view, + addr1, + true, + newBal, + newNonce, + newCode, + newCodeHash, + ) + + // commit the changes and create a new baseview + err = view.Commit() + require.NoError(t, err) + + view, err = state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + checkAccount(t, + view, + addr1, + true, + newBal, + newNonce, + newCode, + newCodeHash, + ) + + // test delete account + + err = view.DeleteAccount(addr1) + require.NoError(t, err) + + // check from cache + checkAccount(t, + view, + addr1, + false, + big.NewInt(0), + uint64(0), + nil, + gethTypes.EmptyCodeHash, + ) + + // commit the changes and create a new baseview + err = view.Commit() + require.NoError(t, err) + + view, err = state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + checkAccount(t, + view, + addr1, + false, + big.NewInt(0), + uint64(0), + nil, + gethTypes.EmptyCodeHash, + ) + }) + + t.Run("test slot storage", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + rootAddr := flow.Address{1, 2, 3, 4, 5, 6, 7, 8} + view, err := state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + addr1 := testutils.RandomCommonAddress(t) + key1 := testutils.RandomCommonHash(t) + slot1 := types.SlotAddress{ + Address: addr1, + Key: key1, + } + + // non-existent account + value, err := view.GetState(slot1) + require.NoError(t, err) + require.Equal(t, value, gethCommon.Hash{}) + + // store a new value + newValue := testutils.RandomCommonHash(t) + + // updating slot for non-existent account should fail + err = view.UpdateSlot(slot1, newValue) + require.Error(t, err) + + // account should have code to have slots + err = view.CreateAccount(addr1, big.NewInt(10), 0, []byte("ABC"), gethCommon.Hash{1, 2, 3}) + require.NoError(t, err) + + err = view.UpdateSlot(slot1, newValue) + require.NoError(t, err) + + // return result from the cache + value, err = view.GetState(slot1) + require.NoError(t, err) + require.Equal(t, newValue, value) + + // commit changes + err = view.Commit() + require.NoError(t, err) + + view2, err := state.NewBaseView(ledger, rootAddr) + require.NoError(t, err) + + // return state from ledger + value, err = view2.GetState(slot1) + require.NoError(t, err) + require.Equal(t, newValue, value) + }) + + t.Run("default values method calls", func(t *testing.T) { + // calls to these method that has always same value + view, err := state.NewBaseView(testutils.GetSimpleValueStore(), flow.Address{1, 2, 3, 4}) + require.NoError(t, err) + + dest, bal := view.HasSelfDestructed(gethCommon.Address{}) + require.Equal(t, false, dest) + require.Equal(t, new(big.Int), bal) + require.Equal(t, false, view.IsCreated(gethCommon.Address{})) + require.Equal(t, uint64(0), view.GetRefund()) + require.Equal(t, gethCommon.Hash{}, view.GetTransientState(types.SlotAddress{})) + require.Equal(t, false, view.AddressInAccessList(gethCommon.Address{})) + addrFound, slotFound := view.SlotInAccessList(types.SlotAddress{}) + require.Equal(t, false, addrFound) + require.Equal(t, false, slotFound) + }) +} + +func checkAccount(t *testing.T, + view *state.BaseView, + addr gethCommon.Address, + exists bool, + balance *big.Int, + nonce uint64, + code []byte, + codeHash gethCommon.Hash, +) { + ex, err := view.Exist(addr) + require.NoError(t, err) + require.Equal(t, exists, ex) + + bal, err := view.GetBalance(addr) + require.NoError(t, err) + require.Equal(t, balance, bal) + + no, err := view.GetNonce(addr) + require.NoError(t, err) + require.Equal(t, nonce, no) + + cd, err := view.GetCode(addr) + require.NoError(t, err) + require.Equal(t, code, cd) + + cs, err := view.GetCodeSize(addr) + require.NoError(t, err) + require.Equal(t, len(code), cs) + + ch, err := view.GetCodeHash(addr) + require.NoError(t, err) + require.Equal(t, codeHash, ch) +} diff --git a/fvm/evm/emulator/state/collection.go b/fvm/evm/emulator/state/collection.go new file mode 100644 index 00000000000..b2bb605c903 --- /dev/null +++ b/fvm/evm/emulator/state/collection.go @@ -0,0 +1,426 @@ +package state + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "math" + "runtime" + + "github.com/fxamacker/cbor/v2" + "github.com/onflow/atree" +) + +const ( + storageIDSize = 16 +) + +// CollectionProvider provides access to collections +type CollectionProvider struct { + rootAddr atree.Address + storage *atree.PersistentSlabStorage +} + +// NewCollectionProvider constructs a new CollectionProvider +func NewCollectionProvider( + rootAddr atree.Address, + ledger atree.Ledger, +) (*CollectionProvider, error) { + // empty address is not allowed (causes issues with atree) + if rootAddr == atree.AddressUndefined { + return nil, fmt.Errorf("empty address as root is not allowed") + } + baseStorage := atree.NewLedgerBaseStorage(ledger) + storage, err := NewPersistentSlabStorage(baseStorage) + return &CollectionProvider{ + rootAddr: rootAddr, + storage: storage, + }, err +} + +// CollectionByID returns the collection by collection ID +// +// if no collection is found with that collection id, it return error +// Warning: this method should only used only once for each collection and +// the returned pointer should be kept for the future. +// calling twice for the same collection might result in odd-behaviours +// currently collection provider doesn't do any internal caching to protect aginast these cases +func (cp *CollectionProvider) CollectionByID(collectionID []byte) (*Collection, error) { + storageID, err := atree.NewStorageIDFromRawBytes(collectionID) + if err != nil { + return nil, err + } + // sanity check the storage ID address + if storageID.Address != cp.rootAddr { + return nil, fmt.Errorf("root address mismatch %x != %x", storageID.Address, cp.rootAddr) + } + + omap, err := atree.NewMapWithRootID(cp.storage, storageID, atree.NewDefaultDigesterBuilder()) + if err != nil { + return nil, err + } + return &Collection{ + omap: omap, + storage: cp.storage, + collectionID: collectionID, + }, nil +} + +// NewCollection constructs a new collection +func (cp *CollectionProvider) NewCollection() (*Collection, error) { + omap, err := atree.NewMap(cp.storage, cp.rootAddr, atree.NewDefaultDigesterBuilder(), emptyTypeInfo{}) + if err != nil { + return nil, err + } + storageIDBytes := make([]byte, storageIDSize) + _, err = omap.StorageID().ToRawBytes(storageIDBytes) + if err != nil { + return nil, err + } + return &Collection{ + storage: cp.storage, + omap: omap, + collectionID: storageIDBytes, // we reuse the storageID bytes as collectionID + }, nil +} + +// Commit commits all changes to the collections with changes +func (cp *CollectionProvider) Commit() error { + return cp.storage.FastCommit(runtime.NumCPU()) +} + +// Collection provides a persistent and compact way of storing key/value pairs +// each collection has a unique collectionID that can be used to fetch the collection +// +// TODO(ramtin): we might not need any extra hashing on the atree side +// and optimize this to just use the key given the keys are hashed ? +type Collection struct { + omap *atree.OrderedMap + storage *atree.PersistentSlabStorage + collectionID []byte +} + +// CollectionID returns the unique id for the collection +func (c *Collection) CollectionID() []byte { + return c.collectionID +} + +// Get gets the value for the given key +// +// if key doesn't exist it returns nil (no error) +func (c *Collection) Get(key []byte) ([]byte, error) { + data, err := c.omap.Get(compare, hashInputProvider, NewByteStringValue(key)) + if err != nil { + var keyNotFoundError *atree.KeyNotFoundError + if errors.As(err, &keyNotFoundError) { + return nil, nil + } + return nil, err + } + + value, err := data.StoredValue(c.omap.Storage) + if err != nil { + return nil, err + } + + return value.(ByteStringValue).Bytes(), nil +} + +// Set sets the value for the given key +// +// if a value already stored at the given key it replaces the value +func (c *Collection) Set(key, value []byte) error { + existingValueStorable, err := c.omap.Set(compare, hashInputProvider, NewByteStringValue(key), NewByteStringValue(value)) + if err != nil { + return err + } + + if id, ok := existingValueStorable.(atree.StorageIDStorable); ok { + // NOTE: deep remove isn't necessary because value is ByteStringValue (not container) + err := c.storage.Remove(atree.StorageID(id)) + if err != nil { + return err + } + } + return nil +} + +// Remove removes a key from the collection +// +// if the key doesn't exist it return no error +func (c *Collection) Remove(key []byte) error { + _, existingValueStorable, err := c.omap.Remove(compare, hashInputProvider, NewByteStringValue(key)) + if err != nil { + var keyNotFoundError *atree.KeyNotFoundError + if errors.As(err, &keyNotFoundError) { + return nil + } + return err + } + + if id, ok := existingValueStorable.(atree.StorageIDStorable); ok { + // NOTE: deep remove isn't necessary because value is ByteStringValue (not container) + err := c.storage.Remove(atree.StorageID(id)) + if err != nil { + return err + } + } + return nil +} + +// Destroy destroys the whole collection +func (c *Collection) Destroy() ([][]byte, error) { + var cachedErr error + keys := make([][]byte, c.omap.Count()) + i := 0 + err := c.omap.PopIterate(func(keyStorable atree.Storable, valueStorable atree.Storable) { + if id, ok := valueStorable.(atree.StorageIDStorable); ok { + err := c.storage.Remove(atree.StorageID(id)) + if err != nil && cachedErr == nil { + cachedErr = err + } + } + key, err := keyStorable.StoredValue(c.omap.Storage) + if err != nil && cachedErr == nil { + cachedErr = err + } + keys[i] = key.(ByteStringValue).Bytes() + i++ + }) + if cachedErr != nil { + return keys, cachedErr + } + if err != nil { + return keys, err + } + return keys, c.storage.Remove(c.omap.StorageID()) +} + +type ByteStringValue struct { + data []byte + size uint32 +} + +var _ atree.Value = &ByteStringValue{} +var _ atree.Storable = &ByteStringValue{} + +func NewByteStringValue(data []byte) ByteStringValue { + size := atree.GetUintCBORSize(uint64(len(data))) + uint32(len(data)) + return ByteStringValue{data: data, size: size} +} + +func (v ByteStringValue) ChildStorables() []atree.Storable { + return nil +} + +func (v ByteStringValue) StoredValue(_ atree.SlabStorage) (atree.Value, error) { + return v, nil +} + +func (v ByteStringValue) Storable(storage atree.SlabStorage, address atree.Address, maxInlineSize uint64) (atree.Storable, error) { + if uint64(v.ByteSize()) <= maxInlineSize { + return v, nil + } + + // Create StorableSlab + id, err := storage.GenerateStorageID(address) + if err != nil { + return nil, err + } + + slab := &atree.StorableSlab{ + StorageID: id, + Storable: v, + } + + // Store StorableSlab in storage + err = storage.Store(id, slab) + if err != nil { + return nil, err + } + + // Return storage id as storable + return atree.StorageIDStorable(id), nil +} + +func (v ByteStringValue) Encode(enc *atree.Encoder) error { + return enc.CBOR.EncodeBytes(v.data) +} + +func (v ByteStringValue) getHashInput(scratch []byte) ([]byte, error) { + + const cborTypeByteString = 0x40 + + buf := scratch + if uint32(len(buf)) < v.size { + buf = make([]byte, v.size) + } else { + buf = buf[:v.size] + } + + slen := len(v.data) + + if slen <= 23 { + buf[0] = cborTypeByteString | byte(slen) + copy(buf[1:], v.data) + return buf, nil + } + + if slen <= math.MaxUint8 { + buf[0] = cborTypeByteString | byte(24) + buf[1] = byte(slen) + copy(buf[2:], v.data) + return buf, nil + } + + if slen <= math.MaxUint16 { + buf[0] = cborTypeByteString | byte(25) + binary.BigEndian.PutUint16(buf[1:], uint16(slen)) + copy(buf[3:], v.data) + return buf, nil + } + + if slen <= math.MaxUint32 { + buf[0] = cborTypeByteString | byte(26) + binary.BigEndian.PutUint32(buf[1:], uint32(slen)) + copy(buf[5:], v.data) + return buf, nil + } + + buf[0] = cborTypeByteString | byte(27) + binary.BigEndian.PutUint64(buf[1:], uint64(slen)) + copy(buf[9:], v.data) + return buf, nil +} + +func (v ByteStringValue) ByteSize() uint32 { + return v.size +} + +func (v ByteStringValue) String() string { + return string(v.data) +} + +func (v ByteStringValue) Bytes() []byte { + return v.data +} + +func decodeStorable(dec *cbor.StreamDecoder, _ atree.StorageID) (atree.Storable, error) { + t, err := dec.NextType() + if err != nil { + return nil, err + } + + switch t { + case cbor.ByteStringType: + s, err := dec.DecodeBytes() + if err != nil { + return nil, err + } + return NewByteStringValue(s), nil + + case cbor.TagType: + tagNumber, err := dec.DecodeTagNumber() + if err != nil { + return nil, err + } + + switch tagNumber { + + case atree.CBORTagStorageID: + return atree.DecodeStorageIDStorable(dec) + + default: + return nil, fmt.Errorf("invalid tag number %d", tagNumber) + } + + default: + return nil, fmt.Errorf("invalid cbor type %s for storable", t) + } +} + +func compare(storage atree.SlabStorage, value atree.Value, storable atree.Storable) (bool, error) { + switch v := value.(type) { + + case ByteStringValue: + other, ok := storable.(ByteStringValue) + if ok { + return bytes.Equal(other.data, v.data), nil + } + + // Retrieve value from storage + otherValue, err := storable.StoredValue(storage) + if err != nil { + return false, err + } + other, ok = otherValue.(ByteStringValue) + if ok { + return bytes.Equal(other.data, v.data), nil + } + + return false, nil + } + + return false, fmt.Errorf("value %T not supported for comparison", value) +} + +func hashInputProvider(value atree.Value, buffer []byte) ([]byte, error) { + switch v := value.(type) { + case ByteStringValue: + return v.getHashInput(buffer) + } + + return nil, fmt.Errorf("value %T not supported for hash input", value) +} + +func NewPersistentSlabStorage(baseStorage atree.BaseStorage) (*atree.PersistentSlabStorage, error) { + encMode, err := cbor.EncOptions{}.EncMode() + if err != nil { + return nil, err + } + + decMode, err := cbor.DecOptions{}.DecMode() + if err != nil { + return nil, err + } + + return atree.NewPersistentSlabStorage( + baseStorage, + encMode, + decMode, + decodeStorable, + decodeTypeInfo, + ), nil +} + +type emptyTypeInfo struct{} + +var _ atree.TypeInfo = emptyTypeInfo{} + +func (emptyTypeInfo) Encode(e *cbor.StreamEncoder) error { + return e.EncodeNil() +} + +func (i emptyTypeInfo) Equal(other atree.TypeInfo) bool { + _, ok := other.(emptyTypeInfo) + return ok +} + +func decodeTypeInfo(dec *cbor.StreamDecoder) (atree.TypeInfo, error) { + ty, err := dec.NextType() + if err != nil { + return nil, err + } + switch ty { + case cbor.NilType: + err := dec.DecodeNil() + if err != nil { + return nil, err + } + return emptyTypeInfo{}, nil + default: + } + + return nil, fmt.Errorf("not supported type info") +} diff --git a/fvm/evm/emulator/state/collection_test.go b/fvm/evm/emulator/state/collection_test.go new file mode 100644 index 00000000000..526d9e94a3e --- /dev/null +++ b/fvm/evm/emulator/state/collection_test.go @@ -0,0 +1,69 @@ +package state_test + +import ( + "testing" + + "github.com/onflow/atree" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" +) + +func TestCollection(t *testing.T) { + + cp := setupTestCollection(t) + c1, err := cp.NewCollection() + require.NoError(t, err) + + key1 := []byte("A") + key2 := []byte("B") + value1 := []byte{1} + value2 := []byte{2} + + // get value for A + ret, err := c1.Get(key1) + require.NoError(t, err) + require.Empty(t, ret) + + // set value1 for A + err = c1.Set(key1, value1) + require.NoError(t, err) + + ret, err = c1.Get(key1) + require.NoError(t, err) + require.Equal(t, ret, value1) + + err = c1.Remove(key1) + require.NoError(t, err) + + ret, err = c1.Get(key1) + require.NoError(t, err) + require.Empty(t, ret) + + err = c1.Set(key2, value2) + require.NoError(t, err) + + c2, err := cp.CollectionByID(c1.CollectionID()) + require.NoError(t, err) + + ret, err = c2.Get(key2) + require.NoError(t, err) + require.Equal(t, value2, ret) + + // destroy + keys, err := c1.Destroy() + require.NoError(t, err) + require.Len(t, keys, 1) + require.Equal(t, key2, keys[0]) + + _, err = cp.CollectionByID(c1.CollectionID()) + require.Error(t, err) +} + +func setupTestCollection(t *testing.T) *state.CollectionProvider { + ledger := testutils.GetSimpleValueStore() + cp, err := state.NewCollectionProvider(atree.Address{1, 2, 3, 4, 5, 6, 7, 8}, ledger) + require.NoError(t, err) + return cp +} diff --git a/fvm/evm/emulator/state/delta.go b/fvm/evm/emulator/state/delta.go new file mode 100644 index 00000000000..50575789d50 --- /dev/null +++ b/fvm/evm/emulator/state/delta.go @@ -0,0 +1,502 @@ +package state + +import ( + "fmt" + "math/big" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + gethCrypto "github.com/ethereum/go-ethereum/crypto" + + "github.com/onflow/flow-go/fvm/evm/types" +) + +// DeltaView captures the changes to the state during the execution +// +// for most of the read calls it checks its change logs and if no record is +// found it would redirect the call to the parent view. +type DeltaView struct { + parent types.ReadOnlyView + + // dirtyAddresses keeps a set of addresses with changes + dirtyAddresses map[gethCommon.Address]struct{} + // created keeps a set of recently created addresses + created map[gethCommon.Address]struct{} + // toBeDestructed keeps a set of addresses flagged to be destructed at the + // end of transaction, it also keeps the balance of the addresses before destruction + toBeDestructed map[gethCommon.Address]*big.Int + // is a flag used to track accounts that has been flagged for + // destruction but recreated later + recreated map[gethCommon.Address]struct{} + // balances keeps the changes to the account balances + balances map[gethCommon.Address]*big.Int + // nonces keeps the changes to the account nonces + nonces map[gethCommon.Address]uint64 + // codes keeps the changes to the account codes + codes map[gethCommon.Address][]byte + // codeHashes keeps the changes to account code hashes + codeHashes map[gethCommon.Address]gethCommon.Hash + + // slots keeps a set of slots that has been changed in this view + slots map[types.SlotAddress]gethCommon.Hash + + // transient storage + transient map[types.SlotAddress]gethCommon.Hash + + // access lists + accessListAddresses map[gethCommon.Address]struct{} + accessListSlots map[types.SlotAddress]struct{} + + // logs + logs []*gethTypes.Log + + // preimages + preimages map[gethCommon.Hash][]byte + + // refund + refund uint64 +} + +var _ types.HotView = &DeltaView{} + +// NewDeltaView constructs a new delta view +func NewDeltaView(parent types.ReadOnlyView) *DeltaView { + return &DeltaView{ + parent: parent, + + dirtyAddresses: make(map[gethCommon.Address]struct{}), + created: make(map[gethCommon.Address]struct{}), + toBeDestructed: make(map[gethCommon.Address]*big.Int), + recreated: make(map[gethCommon.Address]struct{}), + balances: make(map[gethCommon.Address]*big.Int), + nonces: make(map[gethCommon.Address]uint64), + codes: make(map[gethCommon.Address][]byte), + codeHashes: make(map[gethCommon.Address]gethCommon.Hash), + + slots: make(map[types.SlotAddress]gethCommon.Hash), + + // for refund we just copy the data + refund: parent.GetRefund(), + } +} + +// NewChildView constructs a new delta view having the current view as parent +func (d *DeltaView) NewChildView() *DeltaView { + return NewDeltaView(d) +} + +// Exist returns true if address exists +// +// it also returns true for both newly created accounts or accounts that has been flagged for deletion +func (d *DeltaView) Exist(addr gethCommon.Address) (bool, error) { + _, found := d.created[addr] + if found { + return true, nil + } + _, found = d.toBeDestructed[addr] + if found { + return true, nil + } + return d.parent.Exist(addr) +} + +// CreateAccount creates a new account for the given address +// +// if address already extists (even if destructed), carry over the balance +// and reset the data from the orginal account. +func (d *DeltaView) CreateAccount(addr gethCommon.Address) error { + // if is already created return + if d.IsCreated(addr) { + return nil + } + exist, err := d.Exist(addr) + if err != nil { + return err + } + if exist { + // check if already destructed + destructed, balance := d.HasSelfDestructed(addr) + if !destructed { + balance, err = d.GetBalance(addr) + if err != nil { + return err + } + err = d.SelfDestruct(addr) + if err != nil { + return err + } + } + + d.nonces[addr] = 0 + d.codes[addr] = nil + d.codeHashes[addr] = gethTypes.EmptyCodeHash + // carrying over the balance. (legacy behaviour of the Geth stateDB) + d.balances[addr] = balance + + // flag addr as recreated, this flag helps with postponing deletion of slabs + // otherwise we have to iterate over all slabs of this account and set the to nil + d.recreated[addr] = struct{}{} + + // remove slabs from cache related to this account + for k := range d.slots { + if k.Address == addr { + delete(d.slots, k) + } + } + } + d.dirtyAddresses[addr] = struct{}{} + d.created[addr] = struct{}{} + return nil +} + +// IsCreated returns true if address has been created in this tx +func (d *DeltaView) IsCreated(addr gethCommon.Address) bool { + _, found := d.created[addr] + if found { + return true + } + return d.parent.IsCreated(addr) +} + +// HasSelfDestructed returns true if address has been flagged for destruction +// it also returns the balance of the address before the destruction call +func (d *DeltaView) HasSelfDestructed(addr gethCommon.Address) (bool, *big.Int) { + bal, found := d.toBeDestructed[addr] + if found { + return true, bal + } + return d.parent.HasSelfDestructed(addr) +} + +// SelfDestruct sets a flag to destruct the account at the end of transaction +// +// if an account has been created in this transaction, it would return an error +func (d *DeltaView) SelfDestruct(addr gethCommon.Address) error { + // if it has been recently created, calling self destruct is not a valid operation + if d.IsCreated(addr) { + return fmt.Errorf("invalid operation, can't selfdestruct an account that is just created") + } + + // if it doesn't exist, return false + exists, err := d.Exist(addr) + if err != nil { + return err + } + if !exists { + return nil + } + + // flag the account for destruction and capture the balance + // before destruction + d.toBeDestructed[addr], err = d.GetBalance(addr) + if err != nil { + return err + } + // flag the address as dirty + d.dirtyAddresses[addr] = struct{}{} + + // set balance to zero + d.balances[addr] = new(big.Int) + return nil +} + +// GetBalance returns the balance of the given address +func (d *DeltaView) GetBalance(addr gethCommon.Address) (*big.Int, error) { + val, found := d.balances[addr] + if found { + return val, nil + } + // if newly created and no balance is set yet + _, newlyCreated := d.created[addr] + if newlyCreated { + return big.NewInt(0), nil + } + return d.parent.GetBalance(addr) +} + +// AddBalance adds the amount to the current balance of the given address +func (d *DeltaView) AddBalance(addr gethCommon.Address, amount *big.Int) error { + // if amount is 0 skip + if amount.Sign() == 0 { + return nil + } + // get the latest balance + orgBalance, err := d.GetBalance(addr) + if err != nil { + return err + } + // update the balance + newBalance := new(big.Int).Add(orgBalance, amount) + d.balances[addr] = newBalance + + // flag the address as dirty + d.dirtyAddresses[addr] = struct{}{} + return nil +} + +// SubBalance subtracts the amount from the current balance of the given address +func (d *DeltaView) SubBalance(addr gethCommon.Address, amount *big.Int) error { + // if amount is 0 skip + if amount.Sign() == 0 { + return nil + } + + // get the latest balance + orgBalance, err := d.GetBalance(addr) + if err != nil { + return err + } + + // update the new balance + newBalance := new(big.Int).Sub(orgBalance, amount) + + // if new balance is negative error + if newBalance.Sign() < 0 { + return fmt.Errorf("account balance is negative %d", newBalance) + } + + // update the balance + d.balances[addr] = newBalance + + // flag the address as dirty + d.dirtyAddresses[addr] = struct{}{} + return nil +} + +// GetNonce returns the nonce of the given address +func (d *DeltaView) GetNonce(addr gethCommon.Address) (uint64, error) { + val, found := d.nonces[addr] + if found { + return val, nil + } + // if newly created + _, newlyCreated := d.created[addr] + if newlyCreated { + return 0, nil + } + return d.parent.GetNonce(addr) +} + +// SetNonce sets the nonce for the given address +func (d *DeltaView) SetNonce(addr gethCommon.Address, nonce uint64) error { + // update the nonce + d.nonces[addr] = nonce + + // flag the address as dirty + d.dirtyAddresses[addr] = struct{}{} + return nil +} + +// GetCode returns the code of the given address +func (d *DeltaView) GetCode(addr gethCommon.Address) ([]byte, error) { + code, found := d.codes[addr] + if found { + return code, nil + } + // if newly created + _, newlyCreated := d.created[addr] + if newlyCreated { + return nil, nil + } + return d.parent.GetCode(addr) +} + +// GetCodeSize returns the code size of the given address +func (d *DeltaView) GetCodeSize(addr gethCommon.Address) (int, error) { + code, err := d.GetCode(addr) + return len(code), err +} + +// GetCodeHash returns the code hash of the given address +func (d *DeltaView) GetCodeHash(addr gethCommon.Address) (gethCommon.Hash, error) { + codeHash, found := d.codeHashes[addr] + if found { + return codeHash, nil + } + // if newly created + _, newlyCreated := d.created[addr] + if newlyCreated { + return gethTypes.EmptyCodeHash, nil + } + return d.parent.GetCodeHash(addr) +} + +// SetCode sets the code for the given address +func (d *DeltaView) SetCode(addr gethCommon.Address, code []byte) error { + // update code + d.codes[addr] = code + + // update code hash + codeHash := gethTypes.EmptyCodeHash + if len(code) > 0 { + codeHash = gethCrypto.Keccak256Hash(code) + } + d.codeHashes[addr] = codeHash + + // flag the address as dirty + d.dirtyAddresses[addr] = struct{}{} + return nil +} + +// GetState returns the value of the slot of the main state +func (d *DeltaView) GetState(sk types.SlotAddress) (gethCommon.Hash, error) { + val, found := d.slots[sk] + if found { + return val, nil + } + // if address is deleted in the scope of this delta view, + // don't go backward. this has been done to skip the step to iterate + // over all the state slabs and delete them. + _, recreated := d.recreated[sk.Address] + if recreated { + return gethCommon.Hash{}, nil + } + return d.parent.GetState(sk) +} + +// SetState adds sets a value for the given slot of the main storage +func (d *DeltaView) SetState(sk types.SlotAddress, value gethCommon.Hash) error { + lastValue, err := d.GetState(sk) + if err != nil { + return err + } + // if the value hasn't changed, skip + if value == lastValue { + return nil + } + d.slots[sk] = value + return nil +} + +// GetTransientState returns the value of the slot of the transient state +func (d *DeltaView) GetTransientState(sk types.SlotAddress) gethCommon.Hash { + if d.transient != nil { + val, found := d.transient[sk] + if found { + return val + } + } + return d.parent.GetTransientState(sk) +} + +// SetTransientState adds sets a value for the given slot of the transient storage +func (d *DeltaView) SetTransientState(sk types.SlotAddress, value gethCommon.Hash) { + if d.transient == nil { + d.transient = make(map[types.SlotAddress]gethCommon.Hash) + } + d.transient[sk] = value +} + +// GetRefund returns the total (gas) refund +func (d *DeltaView) GetRefund() uint64 { + return d.refund +} + +// AddRefund adds the amount to the total (gas) refund +func (d *DeltaView) AddRefund(amount uint64) error { + d.refund += amount + return nil +} + +// SubRefund subtracts the amount from the total (gas) refund +func (d *DeltaView) SubRefund(amount uint64) error { + if amount > d.refund { + return fmt.Errorf("refund counter below zero (gas: %d > refund: %d)", amount, d.refund) + } + d.refund -= amount + return nil +} + +// AddressInAccessList checks if the address is in the access list +func (d *DeltaView) AddressInAccessList(addr gethCommon.Address) bool { + if d.accessListAddresses != nil { + _, addressFound := d.accessListAddresses[addr] + if addressFound { + return true + } + } + return d.parent.AddressInAccessList(addr) +} + +// AddAddressToAccessList adds an address to the access list +func (d *DeltaView) AddAddressToAccessList(addr gethCommon.Address) bool { + if d.accessListAddresses == nil { + d.accessListAddresses = make(map[gethCommon.Address]struct{}) + } + + addrPresent := d.AddressInAccessList(addr) + d.accessListAddresses[addr] = struct{}{} + return !addrPresent +} + +// SlotInAccessList checks if the slot is in the access list +func (d *DeltaView) SlotInAccessList(sk types.SlotAddress) (addressOk bool, slotOk bool) { + addressFound := d.AddressInAccessList(sk.Address) + if d.accessListSlots != nil { + _, slotFound := d.accessListSlots[sk] + if slotFound { + return addressFound, true + } + } + _, slotFound := d.parent.SlotInAccessList(sk) + return addressFound, slotFound +} + +// AddSlotToAccessList adds a slot to the access list +// it also adds the address to the address list +func (d *DeltaView) AddSlotToAccessList(sk types.SlotAddress) (addrAdded bool, slotAdded bool) { + addrPresent, slotPresent := d.SlotInAccessList(sk) + if d.accessListAddresses == nil { + d.accessListAddresses = make(map[gethCommon.Address]struct{}) + } + d.accessListAddresses[sk.Address] = struct{}{} + if d.accessListSlots == nil { + d.accessListSlots = make(map[types.SlotAddress]struct{}) + } + d.accessListSlots[sk] = struct{}{} + return !addrPresent, !slotPresent +} + +// AddLog appends a log to the log collection +func (d *DeltaView) AddLog(log *gethTypes.Log) { + if d.logs == nil { + d.logs = make([]*gethTypes.Log, 0) + } + d.logs = append(d.logs, log) +} + +// Logs returns the logs that has been captured in this view +func (d *DeltaView) Logs() []*gethTypes.Log { + return d.logs +} + +// AddPreimage adds a preimage +func (d *DeltaView) AddPreimage(hash gethCommon.Hash, preimage []byte) { + if d.preimages == nil { + d.preimages = make(map[gethCommon.Hash][]byte) + } + + // make a copy (legacy behaviour) + pi := make([]byte, len(preimage)) + copy(pi, preimage) + d.preimages[hash] = pi +} + +// Preimages returns a map of preimages +func (d *DeltaView) Preimages() map[gethCommon.Hash][]byte { + return d.preimages +} + +// DirtyAddresses returns a set of addresses that has been updated in this view +func (d *DeltaView) DirtyAddresses() map[gethCommon.Address]struct{} { + return d.dirtyAddresses +} + +// DirtySlots returns a set of slots that has been updated in this view +func (d *DeltaView) DirtySlots() map[types.SlotAddress]struct{} { + dirtySlots := make(map[types.SlotAddress]struct{}) + for sk := range d.slots { + dirtySlots[sk] = struct{}{} + } + return dirtySlots +} diff --git a/fvm/evm/emulator/state/delta_test.go b/fvm/evm/emulator/state/delta_test.go new file mode 100644 index 00000000000..820ba2c2ce1 --- /dev/null +++ b/fvm/evm/emulator/state/delta_test.go @@ -0,0 +1,849 @@ +package state_test + +import ( + "fmt" + "math/big" + "testing" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + gethCrypto "github.com/ethereum/go-ethereum/crypto" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/fvm/evm/types" +) + +var emptyRefund = func() uint64 { + return 0 +} + +func TestDeltaView(t *testing.T) { + t.Parallel() + + t.Run("test account exist/creation/self-destruct functionality", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + addr2 := testutils.RandomCommonAddress(t) + addr3 := testutils.RandomCommonAddress(t) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + switch addr { + case addr1: + return true, nil + case addr2: + return false, nil + default: + return false, fmt.Errorf("some error") + } + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + GetBalanceFunc: func(gethCommon.Address) (*big.Int, error) { + return new(big.Int), nil + }, + HasSelfDestructedFunc: func(gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) + }, + }) + + // check existing account on the parent + found, err := view.Exist(addr1) + require.NoError(t, err) + require.True(t, found) + + // account doesn't exist on parent + found, err = view.Exist(addr2) + require.NoError(t, err) + require.False(t, found) + + // handling error on the parent + _, err = view.Exist(addr3) + require.Error(t, err) + + // create a account at address 2 + err = view.CreateAccount(addr2) + require.NoError(t, err) + require.True(t, view.IsCreated(addr2)) + + // now it should be found + found, err = view.Exist(addr2) + require.NoError(t, err) + require.True(t, found) + + // test HasSelfDestructed first + success, _ := view.HasSelfDestructed(addr1) + require.False(t, success) + + // set addr1 for deletion + err = view.SelfDestruct(addr1) + require.NoError(t, err) + + // check HasSelfDestructed now + success, _ = view.HasSelfDestructed(addr1) + require.True(t, success) + + // addr1 should still exist after self destruct call + found, err = view.Exist(addr1) + require.NoError(t, err) + require.True(t, found) + }) + + t.Run("test account balance functionality", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + addr1InitBal := big.NewInt(10) + addr2 := testutils.RandomCommonAddress(t) + addr2InitBal := big.NewInt(5) + addr3 := testutils.RandomCommonAddress(t) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + switch addr { + case addr1, addr2: + return true, nil + default: + return false, nil + } + }, + HasSelfDestructedFunc: func(a gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + GetBalanceFunc: func(addr gethCommon.Address) (*big.Int, error) { + switch addr { + case addr1: + return addr1InitBal, nil + case addr2: + return addr2InitBal, nil + default: + return nil, fmt.Errorf("some error") + } + }, + }) + + // get balance through parent + bal, err := view.GetBalance(addr1) + require.NoError(t, err) + require.Equal(t, addr1InitBal, bal) + + // call self destruct on addr + err = view.SelfDestruct(addr1) + require.NoError(t, err) + + // now it should return balance of zero + bal, err = view.GetBalance(addr1) + require.NoError(t, err) + require.Equal(t, big.NewInt(0), bal) + + // add balance to addr2 + amount := big.NewInt(7) + expected := new(big.Int).Add(addr2InitBal, amount) + err = view.AddBalance(addr2, amount) + require.NoError(t, err) + newBal, err := view.GetBalance(addr2) + require.NoError(t, err) + require.Equal(t, expected, newBal) + + // sub balance from addr2 + amount = big.NewInt(9) + expected = new(big.Int).Sub(newBal, amount) + err = view.SubBalance(addr2, amount) + require.NoError(t, err) + bal, err = view.GetBalance(addr2) + require.NoError(t, err) + require.Equal(t, expected, bal) + + // negative balance error + err = view.SubBalance(addr2, big.NewInt(100)) + require.Error(t, err) + + // handling error on the parent + _, err = view.GetBalance(addr3) + require.Error(t, err) + }) + + t.Run("test nonce functionality", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + addr1InitNonce := uint64(1) + addr2 := testutils.RandomCommonAddress(t) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + switch addr { + case addr1: + return true, nil + default: + return false, nil + } + }, + HasSelfDestructedFunc: func(a gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + GetBalanceFunc: func(a gethCommon.Address) (*big.Int, error) { + return new(big.Int), nil + }, + GetNonceFunc: func(addr gethCommon.Address) (uint64, error) { + switch addr { + case addr1: + return addr1InitNonce, nil + default: + return 0, fmt.Errorf("some error") + } + }, + }) + + // get nonce through parent + nonce, err := view.GetNonce(addr1) + require.NoError(t, err) + require.Equal(t, addr1InitNonce, nonce) + + // set nonce + new := uint64(100) + err = view.SetNonce(addr1, new) + require.NoError(t, err) + nonce, err = view.GetNonce(addr1) + require.NoError(t, err) + require.Equal(t, new, nonce) + + // handling error on the parent + _, err = view.GetNonce(addr2) + require.Error(t, err) + + // create a new account at addr2 + err = view.CreateAccount(addr2) + require.NoError(t, err) + + // now the nonce should return 0 + nonce, err = view.GetNonce(addr2) + require.NoError(t, err) + require.Equal(t, uint64(0), nonce) + }) + + t.Run("test code functionality", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + addr1InitCode := []byte("code1") + addr1IntiCodeHash := gethCommon.BytesToHash([]byte{1, 2}) + addr2 := testutils.RandomCommonAddress(t) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + switch addr { + case addr1: + return true, nil + default: + return false, nil + } + }, + HasSelfDestructedFunc: func(a gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + GetBalanceFunc: func(a gethCommon.Address) (*big.Int, error) { + return new(big.Int), nil + }, + GetCodeFunc: func(addr gethCommon.Address) ([]byte, error) { + switch addr { + case addr1: + return addr1InitCode, nil + default: + return nil, fmt.Errorf("some error") + } + }, + GetCodeSizeFunc: func(addr gethCommon.Address) (int, error) { + switch addr { + case addr1: + return len(addr1InitCode), nil + default: + return 0, fmt.Errorf("some error") + } + }, + GetCodeHashFunc: func(addr gethCommon.Address) (gethCommon.Hash, error) { + switch addr { + case addr1: + return addr1IntiCodeHash, nil + default: + return gethCommon.Hash{}, fmt.Errorf("some error") + } + }, + }) + + // get code through parent + code, err := view.GetCode(addr1) + require.NoError(t, err) + require.Equal(t, addr1InitCode, code) + + // get code size through parent + codeSize, err := view.GetCodeSize(addr1) + require.NoError(t, err) + require.Equal(t, len(addr1InitCode), codeSize) + + // get code hash through parent + codeHash, err := view.GetCodeHash(addr1) + require.NoError(t, err) + require.Equal(t, addr1IntiCodeHash, codeHash) + + // set code for addr1 + newCode := []byte("new code") + err = view.SetCode(addr1, newCode) + require.NoError(t, err) + + code, err = view.GetCode(addr1) + require.NoError(t, err) + require.Equal(t, newCode, code) + + codeSize, err = view.GetCodeSize(addr1) + require.NoError(t, err) + require.Equal(t, len(newCode), codeSize) + + codeHash, err = view.GetCodeHash(addr1) + require.NoError(t, err) + require.Equal(t, gethCrypto.Keccak256Hash(code), codeHash) + + // handling error on the parent + _, err = view.GetCode(addr2) + require.Error(t, err) + + // create a new account at addr2 + err = view.CreateAccount(addr2) + require.NoError(t, err) + + // now the code should return empty code + code, err = view.GetCode(addr2) + require.NoError(t, err) + require.Len(t, code, 0) + + codeHash, err = view.GetCodeHash(addr2) + require.NoError(t, err) + require.Equal(t, gethTypes.EmptyCodeHash, codeHash) + }) + + t.Run("test state access functionality", func(t *testing.T) { + slot1 := types.SlotAddress{ + Address: testutils.RandomCommonAddress(t), + Key: gethCommon.BytesToHash([]byte{1, 2}), + } + + slot1InitValue := gethCommon.BytesToHash([]byte{3, 4}) + + slot2 := types.SlotAddress{ + Address: testutils.RandomCommonAddress(t), + Key: gethCommon.BytesToHash([]byte{5, 6}), + } + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + + GetStateFunc: func(slot types.SlotAddress) (gethCommon.Hash, error) { + switch slot { + case slot1: + return slot1InitValue, nil + default: + return gethCommon.Hash{}, fmt.Errorf("some error") + } + }, + }) + + // get state through parent + value, err := view.GetState(slot1) + require.NoError(t, err) + require.Equal(t, slot1InitValue, value) + + // handle error from parent + _, err = view.GetState(slot2) + require.Error(t, err) + + // check dirty slots + dirtySlots := view.DirtySlots() + require.Empty(t, dirtySlots) + + // set slot1 with some new value + newValue := gethCommon.BytesToHash([]byte{9, 8}) + err = view.SetState(slot1, newValue) + require.NoError(t, err) + + value, err = view.GetState(slot1) + require.NoError(t, err) + require.Equal(t, newValue, value) + + // check dirty slots + dirtySlots = view.DirtySlots() + require.Len(t, dirtySlots, 1) + + _, found := dirtySlots[slot1] + require.True(t, found) + }) + + t.Run("test transient state access functionality", func(t *testing.T) { + slot1 := types.SlotAddress{ + Address: testutils.RandomCommonAddress(t), + Key: gethCommon.BytesToHash([]byte{1, 2}), + } + + slot1InitValue := gethCommon.BytesToHash([]byte{3, 4}) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + GetTransientStateFunc: func(slot types.SlotAddress) gethCommon.Hash { + switch slot { + case slot1: + return slot1InitValue + default: + return gethCommon.Hash{} + } + }, + }) + + // get state through parent + value := view.GetTransientState(slot1) + require.Equal(t, slot1InitValue, value) + + // set slot1 with some new value + newValue := gethCommon.BytesToHash([]byte{9, 8}) + view.SetTransientState(slot1, newValue) + + value = view.GetTransientState(slot1) + require.Equal(t, newValue, value) + }) + + t.Run("test refund functionality", func(t *testing.T) { + initRefund := uint64(10) + view := state.NewDeltaView( + &MockedReadOnlyView{ + GetRefundFunc: func() uint64 { + return initRefund + }, + }) + + // get refund through parent + value := view.GetRefund() + require.Equal(t, initRefund, value) + + // add refund + addition := uint64(7) + err := view.AddRefund(addition) + require.NoError(t, err) + require.Equal(t, initRefund+addition, view.GetRefund()) + + // sub refund + subtract := uint64(2) + err = view.SubRefund(subtract) + require.NoError(t, err) + require.Equal(t, initRefund+addition-subtract, view.GetRefund()) + + // refund goes negative + err = view.SubRefund(1000) + require.Error(t, err) + }) + + t.Run("test access list functionality", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + addr2 := testutils.RandomCommonAddress(t) + slot1 := types.SlotAddress{ + Address: testutils.RandomCommonAddress(t), + Key: gethCommon.BytesToHash([]byte{1, 2}), + } + + slot2 := types.SlotAddress{ + Address: testutils.RandomCommonAddress(t), + Key: gethCommon.BytesToHash([]byte{3, 4}), + } + + view := state.NewDeltaView( + &MockedReadOnlyView{ + GetRefundFunc: emptyRefund, + AddressInAccessListFunc: func(addr gethCommon.Address) bool { + switch addr { + case addr1: + return true + default: + return false + } + }, + SlotInAccessListFunc: func(slot types.SlotAddress) (addressOk bool, slotOk bool) { + switch slot { + case slot1: + return false, true + default: + return false, false + } + }, + }) + + // check address through parent + require.True(t, view.AddressInAccessList(addr1)) + + // add addr 2 to the list + require.False(t, view.AddressInAccessList(addr2)) + added := view.AddAddressToAccessList(addr2) + require.True(t, added) + require.True(t, view.AddressInAccessList(addr2)) + + // adding again + added = view.AddAddressToAccessList(addr2) + require.False(t, added) + + // check slot through parent + addrFound, slotFound := view.SlotInAccessList(slot1) + require.False(t, addrFound) + require.True(t, slotFound) + + // add slot 2 to the list + addrFound, slotFound = view.SlotInAccessList(slot2) + require.False(t, addrFound) + require.False(t, slotFound) + + addressAdded, slotAdded := view.AddSlotToAccessList(slot2) + require.True(t, addressAdded) + require.True(t, slotAdded) + + addrFound, slotFound = view.SlotInAccessList(slot2) + require.True(t, addrFound) + require.True(t, slotFound) + + // adding again + addressAdded, slotAdded = view.AddSlotToAccessList(slot2) + require.False(t, addressAdded) + require.False(t, slotAdded) + }) + + t.Run("test log functionality", func(t *testing.T) { + view := state.NewDeltaView( + &MockedReadOnlyView{ + GetRefundFunc: emptyRefund, + }) + + logs := view.Logs() + require.Empty(t, logs) + + log1 := &gethTypes.Log{ + Address: testutils.RandomCommonAddress(t), + } + view.AddLog(log1) + + log2 := &gethTypes.Log{ + Address: testutils.RandomCommonAddress(t), + } + view.AddLog(log2) + + logs = view.Logs() + require.Equal(t, []*gethTypes.Log{log1, log2}, logs) + }) + + t.Run("test preimage functionality", func(t *testing.T) { + view := state.NewDeltaView( + &MockedReadOnlyView{ + GetRefundFunc: emptyRefund, + }) + + preimages := view.Preimages() + require.Empty(t, preimages) + + preimage1 := []byte{1, 2} + hash1 := gethCommon.BytesToHash([]byte{2, 3}) + view.AddPreimage(hash1, preimage1) + + preimage2 := []byte{4, 5} + hash2 := gethCommon.BytesToHash([]byte{6, 7}) + view.AddPreimage(hash2, preimage2) + + expected := make(map[gethCommon.Hash][]byte) + expected[hash1] = preimage1 + expected[hash2] = preimage2 + + preimages = view.Preimages() + require.Equal(t, expected, preimages) + }) + + t.Run("test dirty addresses functionality", func(t *testing.T) { + addrCount := 6 + addresses := make([]gethCommon.Address, addrCount) + for i := 0; i < addrCount; i++ { + addresses[i] = testutils.RandomCommonAddress(t) + } + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + return true, nil + }, + GetBalanceFunc: func(addr gethCommon.Address) (*big.Int, error) { + return big.NewInt(10), nil + }, + GetNonceFunc: func(addr gethCommon.Address) (uint64, error) { + return 0, nil + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + HasSelfDestructedFunc: func(gethCommon.Address) (bool, *big.Int) { + return false, new(big.Int) + }, + }) + + // check dirty addresses + dirtyAddresses := view.DirtyAddresses() + require.Empty(t, dirtyAddresses) + + // create a account at address 1 + err := view.CreateAccount(addresses[0]) + require.NoError(t, err) + + // self destruct address 2 + err = view.SelfDestruct(addresses[1]) + require.NoError(t, err) + + // add balance for address 3 + err = view.AddBalance(addresses[2], big.NewInt(5)) + require.NoError(t, err) + + // sub balance for address 4 + err = view.AddBalance(addresses[3], big.NewInt(5)) + require.NoError(t, err) + + // set nonce for address 5 + err = view.SetNonce(addresses[4], 5) + require.NoError(t, err) + + // set code for address 6 + err = view.SetCode(addresses[5], []byte{1, 2}) + require.NoError(t, err) + + // now check dirty addresses + dirtyAddresses = view.DirtyAddresses() + require.Len(t, dirtyAddresses, addrCount) + for _, addr := range addresses { + _, found := dirtyAddresses[addr] + require.True(t, found) + } + }) + + t.Run("test account creation after selfdestruct call", func(t *testing.T) { + addr1 := testutils.RandomCommonAddress(t) + + view := state.NewDeltaView( + &MockedReadOnlyView{ + // we need get refund for parent + GetRefundFunc: emptyRefund, + ExistFunc: func(addr gethCommon.Address) (bool, error) { + return true, nil + }, + HasSelfDestructedFunc: func(gethCommon.Address) (bool, *big.Int) { + return true, big.NewInt(2) + }, + IsCreatedFunc: func(a gethCommon.Address) bool { + return false + }, + GetBalanceFunc: func(addr gethCommon.Address) (*big.Int, error) { + return new(big.Int), nil + }, + GetStateFunc: func(sa types.SlotAddress) (gethCommon.Hash, error) { + return gethCommon.Hash{}, nil + }, + }) + + found, err := view.Exist(addr1) + require.NoError(t, err) + require.True(t, found) + + // set balance + initBalance := big.NewInt(10) + err = view.AddBalance(addr1, initBalance) + require.NoError(t, err) + + bal, err := view.GetBalance(addr1) + require.NoError(t, err) + require.Equal(t, initBalance, bal) + + // set code + code := []byte{1, 2, 3} + err = view.SetCode(addr1, code) + require.NoError(t, err) + + ret, err := view.GetCode(addr1) + require.NoError(t, err) + require.Equal(t, code, ret) + + // set key values + key := testutils.RandomCommonHash(t) + value := testutils.RandomCommonHash(t) + sk := types.SlotAddress{Address: addr1, Key: key} + err = view.SetState(sk, value) + require.NoError(t, err) + + vret, err := view.GetState(sk) + require.NoError(t, err) + require.Equal(t, value, vret) + + err = view.SelfDestruct(addr1) + require.NoError(t, err) + + // balance should be returned zero + bal, err = view.GetBalance(addr1) + require.NoError(t, err) + require.Equal(t, new(big.Int), bal) + + // get code should still work + ret, err = view.GetCode(addr1) + require.NoError(t, err) + require.Equal(t, code, ret) + + // get state should also still work + vret, err = view.GetState(sk) + require.NoError(t, err) + require.Equal(t, value, vret) + + // now re-create account + err = view.CreateAccount(addr1) + require.NoError(t, err) + + // it should carry over the balance + bal, err = view.GetBalance(addr1) + require.NoError(t, err) + require.Equal(t, initBalance, bal) + + ret, err = view.GetCode(addr1) + require.NoError(t, err) + require.Len(t, ret, 0) + + vret, err = view.GetState(sk) + require.NoError(t, err) + emptyValue := gethCommon.Hash{} + require.Equal(t, emptyValue, vret) + }) +} + +type MockedReadOnlyView struct { + ExistFunc func(gethCommon.Address) (bool, error) + HasSelfDestructedFunc func(gethCommon.Address) (bool, *big.Int) + IsCreatedFunc func(gethCommon.Address) bool + GetBalanceFunc func(gethCommon.Address) (*big.Int, error) + GetNonceFunc func(gethCommon.Address) (uint64, error) + GetCodeFunc func(gethCommon.Address) ([]byte, error) + GetCodeHashFunc func(gethCommon.Address) (gethCommon.Hash, error) + GetCodeSizeFunc func(gethCommon.Address) (int, error) + GetStateFunc func(types.SlotAddress) (gethCommon.Hash, error) + GetTransientStateFunc func(types.SlotAddress) gethCommon.Hash + GetRefundFunc func() uint64 + AddressInAccessListFunc func(gethCommon.Address) bool + SlotInAccessListFunc func(types.SlotAddress) (addressOk bool, slotOk bool) +} + +var _ types.ReadOnlyView = &MockedReadOnlyView{} + +func (v *MockedReadOnlyView) Exist(addr gethCommon.Address) (bool, error) { + if v.ExistFunc == nil { + panic("Exist is not set in this mocked view") + } + return v.ExistFunc(addr) +} + +func (v *MockedReadOnlyView) IsCreated(addr gethCommon.Address) bool { + if v.IsCreatedFunc == nil { + panic("IsCreated is not set in this mocked view") + } + return v.IsCreatedFunc(addr) +} + +func (v *MockedReadOnlyView) HasSelfDestructed(addr gethCommon.Address) (bool, *big.Int) { + if v.HasSelfDestructedFunc == nil { + panic("HasSelfDestructed is not set in this mocked view") + } + return v.HasSelfDestructedFunc(addr) +} + +func (v *MockedReadOnlyView) GetBalance(addr gethCommon.Address) (*big.Int, error) { + if v.GetBalanceFunc == nil { + panic("GetBalance is not set in this mocked view") + } + return v.GetBalanceFunc(addr) +} + +func (v *MockedReadOnlyView) GetNonce(addr gethCommon.Address) (uint64, error) { + if v.GetNonceFunc == nil { + panic("GetNonce is not set in this mocked view") + } + return v.GetNonceFunc(addr) +} + +func (v *MockedReadOnlyView) GetCode(addr gethCommon.Address) ([]byte, error) { + if v.GetCodeFunc == nil { + panic("GetCode is not set in this mocked view") + } + return v.GetCodeFunc(addr) +} + +func (v *MockedReadOnlyView) GetCodeHash(addr gethCommon.Address) (gethCommon.Hash, error) { + if v.GetCodeHashFunc == nil { + panic("GetCodeHash is not set in this mocked view") + } + return v.GetCodeHashFunc(addr) +} + +func (v *MockedReadOnlyView) GetCodeSize(addr gethCommon.Address) (int, error) { + if v.GetCodeSizeFunc == nil { + panic("GetCodeSize is not set in this mocked view") + } + return v.GetCodeSizeFunc(addr) +} + +func (v *MockedReadOnlyView) GetState(slot types.SlotAddress) (gethCommon.Hash, error) { + if v.GetStateFunc == nil { + panic("GetState is not set in this mocked view") + } + return v.GetStateFunc(slot) +} + +func (v *MockedReadOnlyView) GetTransientState(slot types.SlotAddress) gethCommon.Hash { + if v.GetTransientStateFunc == nil { + panic("GetTransientState is not set in this mocked view") + } + return v.GetTransientStateFunc(slot) +} + +func (v *MockedReadOnlyView) GetRefund() uint64 { + if v.GetRefundFunc == nil { + panic("GetRefund is not set in this mocked view") + } + return v.GetRefundFunc() +} + +func (v *MockedReadOnlyView) AddressInAccessList(addr gethCommon.Address) bool { + if v.AddressInAccessListFunc == nil { + panic("AddressInAccessList is not set in this mocked view") + } + return v.AddressInAccessListFunc(addr) +} + +func (v *MockedReadOnlyView) SlotInAccessList(slot types.SlotAddress) (addressOk bool, slotOk bool) { + if v.SlotInAccessListFunc == nil { + panic("SlotInAccessList is not set in this mocked view") + } + return v.SlotInAccessListFunc(slot) +} diff --git a/fvm/evm/emulator/state/stateDB.go b/fvm/evm/emulator/state/stateDB.go new file mode 100644 index 00000000000..3488f26cda0 --- /dev/null +++ b/fvm/evm/emulator/state/stateDB.go @@ -0,0 +1,469 @@ +package state + +import ( + "bytes" + stdErrors "errors" + "fmt" + "math/big" + "sort" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + gethParams "github.com/ethereum/go-ethereum/params" + "github.com/onflow/atree" + + "github.com/onflow/flow-go/fvm/errors" + "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" +) + +// StateDB implements a types.StateDB interface +// +// stateDB interface defined by the Geth doesn't support returning errors +// when state calls are happening, and requires stateDB to cache the error +// and return it at a later time (when commit is called). Only the first error +// is expected to be returned. +// Warning: current implementation of the StateDB is considered +// to be used for a single EVM transaction execution and is not +// thread safe. yet the current design supports addition of concurrency in the +// future if needed +type StateDB struct { + ledger atree.Ledger + root flow.Address + baseView types.BaseView + views []*DeltaView + cachedError error +} + +var _ types.StateDB = &StateDB{} + +// NewStateDB constructs a new StateDB +func NewStateDB(ledger atree.Ledger, root flow.Address) (*StateDB, error) { + bv, err := NewBaseView(ledger, root) + if err != nil { + return nil, err + } + return &StateDB{ + ledger: ledger, + root: root, + baseView: bv, + views: []*DeltaView{NewDeltaView(bv)}, + cachedError: nil, + }, nil +} + +// Exist returns true if the given address exists in state. +// +// this should also return true for self destructed accounts during the transaction execution. +func (db *StateDB) Exist(addr gethCommon.Address) bool { + exist, err := db.lastestView().Exist(addr) + db.handleError(err) + return exist +} + +// Empty returns whether the given account is empty. +// +// Empty is defined according to EIP161 (balance = nonce = code = 0). +func (db *StateDB) Empty(addr gethCommon.Address) bool { + if !db.Exist(addr) { + return true + } + return db.GetNonce(addr) == 0 && + db.GetBalance(addr).Sign() == 0 && + bytes.Equal(db.GetCodeHash(addr).Bytes(), gethTypes.EmptyCodeHash.Bytes()) +} + +// CreateAccount creates a new account for the given address +// it sets the nonce to zero +func (db *StateDB) CreateAccount(addr gethCommon.Address) { + err := db.lastestView().CreateAccount(addr) + db.handleError(err) +} + +// IsCreated returns true if address is recently created (context of a transaction) +func (db *StateDB) IsCreated(addr gethCommon.Address) bool { + return db.lastestView().IsCreated(addr) +} + +// SelfDestruct flags the address for deletion. +// +// while this address exists for the rest of transaction, +// the balance of this account is return zero after the SelfDestruct call. +func (db *StateDB) SelfDestruct(addr gethCommon.Address) { + err := db.lastestView().SelfDestruct(addr) + db.handleError(err) +} + +// Selfdestruct6780 would only follow the self destruct steps if account is created +func (db *StateDB) Selfdestruct6780(addr gethCommon.Address) { + if db.IsCreated(addr) { + db.SelfDestruct(addr) + } +} + +// HasSelfDestructed returns true if address is flaged with self destruct. +func (db *StateDB) HasSelfDestructed(addr gethCommon.Address) bool { + destructed, _ := db.lastestView().HasSelfDestructed(addr) + return destructed +} + +// SubBalance substitutes the amount from the balance of the given address +func (db *StateDB) SubBalance(addr gethCommon.Address, amount *big.Int) { + err := db.lastestView().SubBalance(addr, amount) + db.handleError(err) +} + +// AddBalance adds the amount from the balance of the given address +func (db *StateDB) AddBalance(addr gethCommon.Address, amount *big.Int) { + err := db.lastestView().AddBalance(addr, amount) + db.handleError(err) +} + +// GetBalance returns the balance of the given address +func (db *StateDB) GetBalance(addr gethCommon.Address) *big.Int { + bal, err := db.lastestView().GetBalance(addr) + db.handleError(err) + return bal +} + +// GetNonce returns the nonce of the given address +func (db *StateDB) GetNonce(addr gethCommon.Address) uint64 { + nonce, err := db.lastestView().GetNonce(addr) + db.handleError(err) + return nonce +} + +// SetNonce sets the nonce value for the given address +func (db *StateDB) SetNonce(addr gethCommon.Address, nonce uint64) { + err := db.lastestView().SetNonce(addr, nonce) + db.handleError(err) +} + +// GetCodeHash returns the code hash of the given address +func (db *StateDB) GetCodeHash(addr gethCommon.Address) gethCommon.Hash { + hash, err := db.lastestView().GetCodeHash(addr) + db.handleError(err) + return hash +} + +// GetCode returns the code for the given address +func (db *StateDB) GetCode(addr gethCommon.Address) []byte { + code, err := db.lastestView().GetCode(addr) + db.handleError(err) + return code +} + +// GetCodeSize returns the size of the code for the given address +func (db *StateDB) GetCodeSize(addr gethCommon.Address) int { + codeSize, err := db.lastestView().GetCodeSize(addr) + db.handleError(err) + return codeSize +} + +// SetCode sets the code for the given address +func (db *StateDB) SetCode(addr gethCommon.Address, code []byte) { + err := db.lastestView().SetCode(addr, code) + db.handleError(err) +} + +// AddRefund adds the amount to the total (gas) refund +func (db *StateDB) AddRefund(amount uint64) { + err := db.lastestView().AddRefund(amount) + db.handleError(err) +} + +// SubRefund subtracts the amount from the total (gas) refund +func (db *StateDB) SubRefund(amount uint64) { + err := db.lastestView().SubRefund(amount) + db.handleError(err) +} + +// GetRefund returns the total (gas) refund +func (db *StateDB) GetRefund() uint64 { + return db.lastestView().GetRefund() +} + +// GetCommittedState returns the value for the given storage slot considering only the commited state and not +// changes in the scope of current transaction. +func (db *StateDB) GetCommittedState(addr gethCommon.Address, key gethCommon.Hash) gethCommon.Hash { + value, err := db.baseView.GetState(types.SlotAddress{Address: addr, Key: key}) + db.handleError(err) + return value +} + +// GetState returns the value for the given storage slot +func (db *StateDB) GetState(addr gethCommon.Address, key gethCommon.Hash) gethCommon.Hash { + state, err := db.lastestView().GetState(types.SlotAddress{Address: addr, Key: key}) + db.handleError(err) + return state +} + +// SetState sets a value for the given storage slot +func (db *StateDB) SetState(addr gethCommon.Address, key gethCommon.Hash, value gethCommon.Hash) { + err := db.lastestView().SetState(types.SlotAddress{Address: addr, Key: key}, value) + db.handleError(err) +} + +// GetTransientState returns the value for the given key of the transient storage +func (db *StateDB) GetTransientState(addr gethCommon.Address, key gethCommon.Hash) gethCommon.Hash { + return db.lastestView().GetTransientState(types.SlotAddress{Address: addr, Key: key}) +} + +// SetTransientState sets a value for the given key of the transient storage +func (db *StateDB) SetTransientState(addr gethCommon.Address, key, value gethCommon.Hash) { + db.lastestView().SetTransientState(types.SlotAddress{Address: addr, Key: key}, value) +} + +// AddressInAccessList checks if an address is in the access list +func (db *StateDB) AddressInAccessList(addr gethCommon.Address) bool { + return db.lastestView().AddressInAccessList(addr) +} + +// SlotInAccessList checks if the given (address,slot) is in the access list +func (db *StateDB) SlotInAccessList(addr gethCommon.Address, key gethCommon.Hash) (addressOk bool, slotOk bool) { + return db.lastestView().SlotInAccessList(types.SlotAddress{Address: addr, Key: key}) +} + +// AddAddressToAccessList adds the given address to the access list. +func (db *StateDB) AddAddressToAccessList(addr gethCommon.Address) { + db.lastestView().AddAddressToAccessList(addr) +} + +// AddSlotToAccessList adds the given (address,slot) to the access list. +func (db *StateDB) AddSlotToAccessList(addr gethCommon.Address, key gethCommon.Hash) { + db.lastestView().AddSlotToAccessList(types.SlotAddress{Address: addr, Key: key}) +} + +// AddLog appends a lot to the collection of logs +func (db *StateDB) AddLog(log *gethTypes.Log) { + db.lastestView().AddLog(log) +} + +// AddPreimage adds a preimage to the collection of preimages +func (db *StateDB) AddPreimage(hash gethCommon.Hash, data []byte) { + db.lastestView().AddPreimage(hash, data) +} + +// RevertToSnapshot reverts the changes until we reach the given snaptshot +func (db *StateDB) RevertToSnapshot(index int) { + if index > len(db.views) { + db.cachedError = fmt.Errorf("invalid revert") + return + } + db.views = db.views[:index] +} + +// Snapshot takes an snapshot of the state and returns an int +// that can be used later for revert calls. +func (db *StateDB) Snapshot() int { + newView := db.lastestView().NewChildView() + db.views = append(db.views, newView) + return len(db.views) - 1 +} + +// Logs returns the list of logs +// it also update each log with the block and tx info +func (db *StateDB) Logs( + blockHash gethCommon.Hash, + blockNumber uint64, + txHash gethCommon.Hash, + txIndex uint, +) []*gethTypes.Log { + allLogs := make([]*gethTypes.Log, 0) + for _, view := range db.views { + for _, log := range view.Logs() { + log.BlockNumber = blockNumber + log.BlockHash = blockHash + log.TxHash = txHash + log.TxIndex = txIndex + allLogs = append(allLogs, log) + } + } + return allLogs +} + +// Preimages returns a set of preimages +func (db *StateDB) Preimages() map[gethCommon.Hash][]byte { + preImages := make(map[gethCommon.Hash][]byte, 0) + for _, view := range db.views { + for k, v := range view.Preimages() { + preImages[k] = v + } + } + return preImages +} + +// Commit commits state changes back to the underlying +func (db *StateDB) Commit() error { + // return error if any has been acumulated + if db.cachedError != nil { + return wrapError(db.cachedError) + } + + var err error + + // iterate views and collect dirty addresses and slots + addresses := make(map[gethCommon.Address]struct{}) + slots := make(map[types.SlotAddress]struct{}) + for _, view := range db.views { + for key := range view.DirtyAddresses() { + addresses[key] = struct{}{} + } + for key := range view.DirtySlots() { + slots[key] = struct{}{} + } + } + + // sort addresses + sortedAddresses := make([]gethCommon.Address, 0, len(addresses)) + for addr := range addresses { + sortedAddresses = append(sortedAddresses, addr) + } + + sort.Slice(sortedAddresses, + func(i, j int) bool { + return bytes.Compare(sortedAddresses[i][:], sortedAddresses[j][:]) < 0 + }) + + // update accounts + for _, addr := range sortedAddresses { + deleted := false + // first we need to delete accounts + if db.HasSelfDestructed(addr) { + err = db.baseView.DeleteAccount(addr) + if err != nil { + return wrapError(err) + } + deleted = true + } + // then create new ones + // an account might be in a single transaction be deleted and recreated + if db.IsCreated(addr) { + err = db.baseView.CreateAccount( + addr, + db.GetBalance(addr), + db.GetNonce(addr), + db.GetCode(addr), + db.GetCodeHash(addr), + ) + if err != nil { + return wrapError(err) + } + continue + } + if deleted { + continue + } + err = db.baseView.UpdateAccount( + addr, + db.GetBalance(addr), + db.GetNonce(addr), + db.GetCode(addr), + db.GetCodeHash(addr), + ) + if err != nil { + return wrapError(err) + } + } + + // sort slots + sortedSlots := make([]types.SlotAddress, 0, len(slots)) + for slot := range slots { + sortedSlots = append(sortedSlots, slot) + } + sort.Slice(sortedSlots, func(i, j int) bool { + comp := bytes.Compare(sortedSlots[i].Address[:], sortedSlots[j].Address[:]) + if comp == 0 { + return bytes.Compare(sortedSlots[i].Key[:], sortedSlots[j].Key[:]) < 0 + } + return comp < 0 + }) + + // update slots + for _, sk := range sortedSlots { + err = db.baseView.UpdateSlot( + sk, + db.GetState(sk.Address, sk.Key), + ) + if err != nil { + return wrapError(err) + } + } + + // don't purge views yet, people might call the logs etc + err = db.baseView.Commit() + if err != nil { + return wrapError(err) + } + return nil +} + +// Prepare is a highlevel logic that sadly is considered to be part of the +// stateDB interface and not on the layers above. +// based on parameters that are passed it updates accesslists +func (db *StateDB) Prepare(rules gethParams.Rules, sender, coinbase gethCommon.Address, dest *gethCommon.Address, precompiles []gethCommon.Address, txAccesses gethTypes.AccessList) { + if rules.IsBerlin { + db.AddAddressToAccessList(sender) + + if dest != nil { + db.AddAddressToAccessList(*dest) + // If it's a create-tx, the destination will be added inside egethVM.create + } + for _, addr := range precompiles { + db.AddAddressToAccessList(addr) + } + for _, el := range txAccesses { + db.AddAddressToAccessList(el.Address) + for _, key := range el.StorageKeys { + db.AddSlotToAccessList(el.Address, key) + } + } + if rules.IsShanghai { // EIP-3651: warm coinbase + db.AddAddressToAccessList(coinbase) + } + } +} + +// Error returns the memorized database failure occurred earlier. +func (s *StateDB) Error() error { + return wrapError(s.cachedError) +} + +func (db *StateDB) lastestView() *DeltaView { + return db.views[len(db.views)-1] +} + +// set error captures the first non-nil error it is called with. +func (db *StateDB) handleError(err error) { + if err == nil { + return + } + if db.cachedError == nil { + db.cachedError = err + } +} + +func wrapError(err error) error { + if err == nil { + return nil + } + + var atreeUserError *atree.UserError + // if is an atree user error + if stdErrors.As(err, &atreeUserError) { + return types.NewStateError(err) + } + + var atreeFatalError *atree.FatalError + // if is a atree fatal error or + if stdErrors.As(err, &atreeFatalError) { + return types.NewFatalError(err) + } + + // if is fvm fatal error + if errors.IsFailure(err) { + return types.NewFatalError(err) + } + + return types.NewStateError(err) +} diff --git a/fvm/evm/emulator/state/stateDB_test.go b/fvm/evm/emulator/state/stateDB_test.go new file mode 100644 index 00000000000..2d45395a72e --- /dev/null +++ b/fvm/evm/emulator/state/stateDB_test.go @@ -0,0 +1,290 @@ +package state_test + +import ( + "fmt" + "math/big" + "testing" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + gethParams "github.com/ethereum/go-ethereum/params" + "github.com/onflow/atree" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/fvm/evm/emulator/state" + "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/fvm/evm/types" + "github.com/onflow/flow-go/model/flow" +) + +var rootAddr = flow.Address{1, 2, 3, 4, 5, 6, 7, 8} + +func TestStateDB(t *testing.T) { + t.Parallel() + + t.Run("test Empty method", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + addr1 := testutils.RandomCommonAddress(t) + // non-existent account + require.True(t, db.Empty(addr1)) + require.NoError(t, db.Error()) + + db.CreateAccount(addr1) + require.NoError(t, db.Error()) + + require.True(t, db.Empty(addr1)) + require.NoError(t, db.Error()) + + db.AddBalance(addr1, big.NewInt(10)) + require.NoError(t, db.Error()) + + require.False(t, db.Empty(addr1)) + }) + + t.Run("test commit functionality", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + addr1 := testutils.RandomCommonAddress(t) + key1 := testutils.RandomCommonHash(t) + value1 := testutils.RandomCommonHash(t) + + db.CreateAccount(addr1) + require.NoError(t, db.Error()) + + db.AddBalance(addr1, big.NewInt(5)) + require.NoError(t, db.Error()) + + // should have code to be able to set state + db.SetCode(addr1, []byte{1, 2, 3}) + require.NoError(t, db.Error()) + + db.SetState(addr1, key1, value1) + + ret := db.GetState(addr1, key1) + require.Equal(t, value1, ret) + + ret = db.GetCommittedState(addr1, key1) + require.Equal(t, gethCommon.Hash{}, ret) + + err = db.Commit() + require.NoError(t, err) + + ret = db.GetCommittedState(addr1, key1) + require.Equal(t, value1, ret) + + // create a new db + db, err = state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + bal := db.GetBalance(addr1) + require.NoError(t, db.Error()) + require.Equal(t, big.NewInt(5), bal) + + val := db.GetState(addr1, key1) + require.NoError(t, db.Error()) + require.Equal(t, value1, val) + }) + + t.Run("test snapshot and revert functionality", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + addr1 := testutils.RandomCommonAddress(t) + require.False(t, db.Exist(addr1)) + require.NoError(t, db.Error()) + + snapshot1 := db.Snapshot() + require.Equal(t, 1, snapshot1) + + db.CreateAccount(addr1) + require.NoError(t, db.Error()) + + require.True(t, db.Exist(addr1)) + require.NoError(t, db.Error()) + + db.AddBalance(addr1, big.NewInt(5)) + require.NoError(t, db.Error()) + + bal := db.GetBalance(addr1) + require.NoError(t, db.Error()) + require.Equal(t, big.NewInt(5), bal) + + snapshot2 := db.Snapshot() + require.Equal(t, 2, snapshot2) + + db.AddBalance(addr1, big.NewInt(5)) + require.NoError(t, db.Error()) + + bal = db.GetBalance(addr1) + require.NoError(t, db.Error()) + require.Equal(t, big.NewInt(10), bal) + + // revert to snapshot 2 + db.RevertToSnapshot(snapshot2) + require.NoError(t, db.Error()) + + bal = db.GetBalance(addr1) + require.NoError(t, db.Error()) + require.Equal(t, big.NewInt(5), bal) + + // revert to snapshot 1 + db.RevertToSnapshot(snapshot1) + require.NoError(t, db.Error()) + + bal = db.GetBalance(addr1) + require.NoError(t, db.Error()) + require.Equal(t, big.NewInt(0), bal) + + // revert to an invalid snapshot + db.RevertToSnapshot(10) + require.Error(t, db.Error()) + }) + + t.Run("test log functionality", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + logs := []*gethTypes.Log{ + testutils.GetRandomLogFixture(t), + testutils.GetRandomLogFixture(t), + testutils.GetRandomLogFixture(t), + testutils.GetRandomLogFixture(t), + } + + db.AddLog(logs[0]) + db.AddLog(logs[1]) + + _ = db.Snapshot() + + db.AddLog(logs[2]) + db.AddLog(logs[3]) + + snapshot := db.Snapshot() + db.AddLog(testutils.GetRandomLogFixture(t)) + db.RevertToSnapshot(snapshot) + + ret := db.Logs(gethCommon.Hash{}, 1, gethCommon.Hash{}, 1) + require.Equal(t, ret, logs) + }) + + t.Run("test refund functionality", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + require.Equal(t, uint64(0), db.GetRefund()) + db.AddRefund(10) + require.Equal(t, uint64(10), db.GetRefund()) + db.SubRefund(3) + require.Equal(t, uint64(7), db.GetRefund()) + + snap1 := db.Snapshot() + db.AddRefund(10) + require.Equal(t, uint64(17), db.GetRefund()) + + db.RevertToSnapshot(snap1) + require.Equal(t, uint64(7), db.GetRefund()) + }) + + t.Run("test Prepare functionality", func(t *testing.T) { + ledger := testutils.GetSimpleValueStore() + db, err := state.NewStateDB(ledger, rootAddr) + + sender := testutils.RandomCommonAddress(t) + coinbase := testutils.RandomCommonAddress(t) + dest := testutils.RandomCommonAddress(t) + precompiles := []gethCommon.Address{ + testutils.RandomCommonAddress(t), + testutils.RandomCommonAddress(t), + } + + txAccesses := gethTypes.AccessList([]gethTypes.AccessTuple{ + {Address: testutils.RandomCommonAddress(t), + StorageKeys: []gethCommon.Hash{ + testutils.RandomCommonHash(t), + testutils.RandomCommonHash(t), + }, + }, + }) + + rules := gethParams.Rules{ + IsBerlin: true, + IsShanghai: true, + } + + require.NoError(t, err) + db.Prepare(rules, sender, coinbase, &dest, precompiles, txAccesses) + + require.True(t, db.AddressInAccessList(sender)) + require.True(t, db.AddressInAccessList(coinbase)) + require.True(t, db.AddressInAccessList(dest)) + + for _, add := range precompiles { + require.True(t, db.AddressInAccessList(add)) + } + + for _, el := range txAccesses { + for _, key := range el.StorageKeys { + addrFound, slotFound := db.SlotInAccessList(el.Address, key) + require.True(t, addrFound) + require.True(t, slotFound) + } + } + }) + + t.Run("test non-fatal error handling", func(t *testing.T) { + ledger := &testutils.TestValueStore{ + GetValueFunc: func(owner, key []byte) ([]byte, error) { + return nil, nil + }, + SetValueFunc: func(owner, key, value []byte) error { + return atree.NewUserError(fmt.Errorf("key not found")) + }, + AllocateStorageIndexFunc: func(owner []byte) (atree.StorageIndex, error) { + return atree.StorageIndex{}, nil + }, + } + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + db.CreateAccount(testutils.RandomCommonAddress(t)) + + err = db.Commit() + // ret := db.Error() + require.Error(t, err) + // check wrapping + require.True(t, types.IsAStateError(err)) + }) + + t.Run("test fatal error handling", func(t *testing.T) { + ledger := &testutils.TestValueStore{ + GetValueFunc: func(owner, key []byte) ([]byte, error) { + return nil, nil + }, + SetValueFunc: func(owner, key, value []byte) error { + return atree.NewFatalError(fmt.Errorf("key not found")) + }, + AllocateStorageIndexFunc: func(owner []byte) (atree.StorageIndex, error) { + return atree.StorageIndex{}, nil + }, + } + db, err := state.NewStateDB(ledger, rootAddr) + require.NoError(t, err) + + db.CreateAccount(testutils.RandomCommonAddress(t)) + + err = db.Commit() + // ret := db.Error() + require.Error(t, err) + // check wrapping + require.True(t, types.IsAFatalError(err)) + }) + +} diff --git a/fvm/evm/emulator/state_test.go b/fvm/evm/emulator/state/state_growth_test.go similarity index 77% rename from fvm/evm/emulator/state_test.go rename to fvm/evm/emulator/state/state_growth_test.go index a7341ba6c97..b7502728d8b 100644 --- a/fvm/evm/emulator/state_test.go +++ b/fvm/evm/emulator/state/state_growth_test.go @@ -1,4 +1,4 @@ -package emulator_test +package state_test import ( "encoding/binary" @@ -10,15 +10,12 @@ import ( "github.com/onflow/flow-go/utils/io" - "github.com/ethereum/go-ethereum/ethdb" - "github.com/ethereum/go-ethereum/common" - gethRawDB "github.com/ethereum/go-ethereum/core/rawdb" - gethState "github.com/ethereum/go-ethereum/core/state" "github.com/stretchr/testify/require" - "github.com/onflow/flow-go/fvm/evm/emulator/database" + "github.com/onflow/flow-go/fvm/evm/emulator/state" "github.com/onflow/flow-go/fvm/evm/testutils" + "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" ) @@ -36,37 +33,16 @@ const ( type storageTest struct { store *testutils.TestValueStore - db *database.MeteredDatabase - ethDB ethdb.Database - stateDB gethState.Database addressIndex uint64 - hash common.Hash metrics *metrics } func newStorageTest() (*storageTest, error) { simpleStore := testutils.GetSimpleValueStore() - db, err := database.NewMeteredDatabase(simpleStore, flow.Address{0x01}) - if err != nil { - return nil, err - } - - hash, err := db.GetRootHash() - if err != nil { - return nil, err - } - - rawDB := gethRawDB.NewDatabase(db) - stateDB := gethState.NewDatabase(rawDB) - return &storageTest{ store: simpleStore, - db: db, - ethDB: rawDB, - stateDB: stateDB, addressIndex: 100, - hash: hash, metrics: newMetrics(), }, nil } @@ -80,33 +56,21 @@ func (s *storageTest) newAddress() common.Address { // run the provided runner with a newly created state which gets comitted after the runner // is finished. Storage metrics are being recorded with each run. -func (s *storageTest) run(runner func(state *gethState.StateDB)) error { - state, err := gethState.New(s.hash, s.stateDB, nil) +func (s *storageTest) run(runner func(state types.StateDB)) error { + state, err := state.NewStateDB(s.store, flow.Address{0x01}) if err != nil { return err } runner(state) - s.hash, err = state.Commit(true) - if err != nil { - return err - } - - err = state.Database().TrieDB().Commit(s.hash, true) + err = state.Commit() if err != nil { return err } - err = s.db.Commit(s.hash) - if err != nil { - return err - } - - s.db.DropCache() - - s.metrics.add(bytesWrittenMetric, s.db.BytesStored()) - s.metrics.add(bytesReadMetric, s.db.BytesRetrieved()) + s.metrics.add(bytesWrittenMetric, s.store.TotalBytesWritten()) + s.metrics.add(bytesReadMetric, s.store.TotalBytesRead()) s.metrics.add(storageItemsMetric, s.store.TotalStorageItems()) s.metrics.add(storageBytesMetric, s.store.TotalStorageSize()) @@ -168,7 +132,7 @@ func Test_AccountCreations(t *testing.T) { accountChart := "accounts,storage_size" maxAccounts := 50_000 for i := 0; i < maxAccounts; i++ { - err = tester.run(func(state *gethState.StateDB) { + err = tester.run(func(state types.StateDB) { state.AddBalance(tester.newAddress(), big.NewInt(100)) }) require.NoError(t, err) @@ -205,16 +169,19 @@ func Test_AccountContractInteraction(t *testing.T) { interactions := 50000 for i := 0; i < interactions; i++ { - err = tester.run(func(state *gethState.StateDB) { + err = tester.run(func(state types.StateDB) { // create a new account accAddr := tester.newAddress() state.AddBalance(accAddr, big.NewInt(100)) // create a contract contractAddr := tester.newAddress() - state.SetBalance(contractAddr, big.NewInt(int64(i))) + state.AddBalance(contractAddr, big.NewInt(int64(i))) state.SetCode(contractAddr, code) - state.SetStorage(contractAddr, contractState) + + for k, v := range contractState { + state.SetState(contractAddr, k, v) + } // simulate interaction with contract state and account balance for fees state.SetState(contractAddr, common.HexToHash("0x03"), common.HexToHash("0x40")) diff --git a/fvm/evm/evm.go b/fvm/evm/evm.go index a44b8be4552..800130bc86e 100644 --- a/fvm/evm/evm.go +++ b/fvm/evm/evm.go @@ -5,7 +5,6 @@ import ( "github.com/onflow/cadence/runtime/common" evm "github.com/onflow/flow-go/fvm/evm/emulator" - "github.com/onflow/flow-go/fvm/evm/emulator/database" "github.com/onflow/flow-go/fvm/evm/handler" "github.com/onflow/flow-go/fvm/evm/stdlib" "github.com/onflow/flow-go/fvm/evm/types" @@ -31,12 +30,7 @@ func SetupEnvironment( return err } - db, err := database.NewDatabase(backend, evmRootAddress) - if err != nil { - return err - } - - em := evm.NewEmulator(db) + em := evm.NewEmulator(backend, evmRootAddress) bs, err := handler.NewBlockStore(backend, evmRootAddress) if err != nil { diff --git a/fvm/evm/handler/handler.go b/fvm/evm/handler/handler.go index 4ed802e08a8..acbfc1c173e 100644 --- a/fvm/evm/handler/handler.go +++ b/fvm/evm/handler/handler.go @@ -103,7 +103,6 @@ func (h *ContractHandler) Run(rlpEncodedTx []byte, coinbase types.Address) { bp, err := h.blockstore.BlockProposal() handleError(err) - bp.StateRoot = res.StateRootHash txHash := tx.Hash() bp.AppendTxHash(txHash) @@ -301,7 +300,6 @@ func (a *Account) executeAndHandleCall( bp, err := a.fch.blockstore.BlockProposal() handleError(err) bp.AppendTxHash(callHash) - bp.StateRoot = res.StateRootHash if deductSupplyDiff { bp.TotalSupply -= totalSupplyDiff } else { diff --git a/fvm/evm/handler/handler_benchmark_test.go b/fvm/evm/handler/handler_benchmark_test.go index 73f0f0ed59d..6165b26c29d 100644 --- a/fvm/evm/handler/handler_benchmark_test.go +++ b/fvm/evm/handler/handler_benchmark_test.go @@ -21,12 +21,11 @@ func benchmarkStorageGrowth(b *testing.B, accountCount, setupKittyCount int) { backend, rootAddr, func(tc *testutils.TestContract) { - db, handler := SetupHandler(b, backend, rootAddr) - numOfAccounts := 100000 - accounts := make([]types.Account, numOfAccounts) + handler := SetupHandler(b, backend, rootAddr) + accounts := make([]types.Account, accountCount) // setup several of accounts // note that trie growth is the function of number of accounts - for i := 0; i < numOfAccounts; i++ { + for i := 0; i < accountCount; i++ { account := handler.AccountByAddress(handler.AllocateAddress(), true) account.Deposit(types.NewFlowTokenVault(types.Balance(100))) accounts[i] = account @@ -54,12 +53,9 @@ func benchmarkStorageGrowth(b *testing.B, accountCount, setupKittyCount int) { ) require.Equal(b, 2, len(backend.Events())) backend.DropEvents() // this would make things lighter + backend.ResetStats() // reset stats } - // measure the impact of mint after the setup phase - db.ResetReporter() - db.DropCache() - accounts[0].Call( tc.DeployedAt, tc.MakeCallData(b, @@ -73,8 +69,8 @@ func benchmarkStorageGrowth(b *testing.B, accountCount, setupKittyCount int) { types.Balance(0), ) - b.ReportMetric(float64(db.BytesRetrieved()), "bytes_read") - b.ReportMetric(float64(db.BytesStored()), "bytes_written") + b.ReportMetric(float64(backend.TotalBytesRead()), "bytes_read") + b.ReportMetric(float64(backend.TotalBytesWritten()), "bytes_written") b.ReportMetric(float64(backend.TotalStorageSize()), "total_storage_size") }) }) diff --git a/fvm/evm/handler/handler_test.go b/fvm/evm/handler/handler_test.go index db78cf2d827..0acf3d3ceff 100644 --- a/fvm/evm/handler/handler_test.go +++ b/fvm/evm/handler/handler_test.go @@ -23,7 +23,6 @@ import ( "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/fvm/evm/emulator" - "github.com/onflow/flow-go/fvm/evm/emulator/database" "github.com/onflow/flow-go/fvm/evm/handler" "github.com/onflow/flow-go/fvm/evm/testutils" "github.com/onflow/flow-go/fvm/evm/types" @@ -52,7 +51,6 @@ func TestHandler_TransactionRun(t *testing.T) { require.NoError(t, err) result := &types.Result{ - StateRootHash: testutils.RandomCommonHash(t), DeployedContractAddress: types.Address(testutils.RandomAddress(t)), ReturnedValue: testutils.RandomData(t), GasConsumed: testutils.RandomGas(1000), @@ -194,7 +192,7 @@ func TestHandler_TransactionRun(t *testing.T) { testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - _, handler := SetupHandler(t, backend, rootAddr) + handler := SetupHandler(t, backend, rootAddr) eoa := testutils.GetTestEOAAccount(t, testutils.EOATestAccount1KeyHex) @@ -250,7 +248,7 @@ func TestHandler_OpsWithoutEmulator(t *testing.T) { testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - _, handler := SetupHandler(t, backend, rootAddr) + handler := SetupHandler(t, backend, rootAddr) // test call last executed block without initialization b := handler.LastExecutedBlock() @@ -300,7 +298,7 @@ func TestHandler_BridgedAccount(t *testing.T) { testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - _, handler := SetupHandler(t, backend, rootAddr) + handler := SetupHandler(t, backend, rootAddr) foa := handler.AccountByAddress(handler.AllocateAddress(), true) require.NotNil(t, foa) @@ -474,7 +472,7 @@ func TestHandler_BridgedAccount(t *testing.T) { // TODO update this test with events, gas metering, etc testutils.RunWithTestBackend(t, func(backend *testutils.TestBackend) { testutils.RunWithTestFlowEVMRootAddress(t, backend, func(rootAddr flow.Address) { - _, handler := SetupHandler(t, backend, rootAddr) + handler := SetupHandler(t, backend, rootAddr) foa := handler.AccountByAddress(handler.AllocateAddress(), true) require.NotNil(t, foa) @@ -533,18 +531,15 @@ func assertPanic(t *testing.T, check checkError, f func()) { f() } -func SetupHandler(t testing.TB, backend types.Backend, rootAddr flow.Address) (*database.Database, *handler.ContractHandler) { +func SetupHandler(t testing.TB, backend types.Backend, rootAddr flow.Address) *handler.ContractHandler { bs, err := handler.NewBlockStore(backend, rootAddr) require.NoError(t, err) aa, err := handler.NewAddressAllocator(backend, rootAddr) require.NoError(t, err) - db, err := database.NewDatabase(backend, rootAddr) - require.NoError(t, err) - - emulator := emulator.NewEmulator(db) + emulator := emulator.NewEmulator(backend, rootAddr) handler := handler.NewContractHandler(flowTokenAddress, bs, aa, backend, emulator) - return db, handler + return handler } diff --git a/fvm/evm/stdlib/abiOnlyContract.cdc b/fvm/evm/stdlib/abiOnlyContract.cdc new file mode 100644 index 00000000000..45378726215 --- /dev/null +++ b/fvm/evm/stdlib/abiOnlyContract.cdc @@ -0,0 +1,60 @@ +access(all) +contract EVM { + + /// EVMAddress is an EVM-compatible address + access(all) + struct EVMAddress { + + /// Bytes of the address + access(all) + let bytes: [UInt8; 20] + + /// Constructs a new EVM address from the given byte representation + init(bytes: [UInt8; 20]) { + self.bytes = bytes + } + + } + + access(all) + fun encodeABI(_ values: [AnyStruct]): [UInt8] { + return InternalEVM.encodeABI(values) + } + + access(all) + fun decodeABI(types: [Type], data: [UInt8]): [AnyStruct] { + return InternalEVM.decodeABI(types: types, data: data) + } + + access(all) + fun encodeABIWithSignature( + _ signature: String, + _ values: [AnyStruct] + ): [UInt8] { + let methodID = HashAlgorithm.KECCAK_256.hash( + signature.utf8 + ).slice(from: 0, upTo: 4) + let arguments = InternalEVM.encodeABI(values) + + return methodID.concat(arguments) + } + + access(all) + fun decodeABIWithSignature( + _ signature: String, + types: [Type], + data: [UInt8] + ): [AnyStruct] { + let methodID = HashAlgorithm.KECCAK_256.hash( + signature.utf8 + ).slice(from: 0, upTo: 4) + + for byte in methodID { + if byte != data.removeFirst() { + panic("signature mismatch") + } + } + + return InternalEVM.decodeABI(types: types, data: data) + } +} diff --git a/fvm/evm/stdlib/contract.cdc b/fvm/evm/stdlib/contract.cdc index 5b902379f42..e8946f214cf 100644 --- a/fvm/evm/stdlib/contract.cdc +++ b/fvm/evm/stdlib/contract.cdc @@ -157,4 +157,36 @@ contract EVM { fun decodeABI(types: [Type], data: [UInt8]): [AnyStruct] { return InternalEVM.decodeABI(types: types, data: data) } + + access(all) + fun encodeABIWithSignature( + _ signature: String, + _ values: [AnyStruct] + ): [UInt8] { + let methodID = HashAlgorithm.KECCAK_256.hash( + signature.utf8 + ).slice(from: 0, upTo: 4) + let arguments = InternalEVM.encodeABI(values) + + return methodID.concat(arguments) + } + + access(all) + fun decodeABIWithSignature( + _ signature: String, + types: [Type], + data: [UInt8] + ): [AnyStruct] { + let methodID = HashAlgorithm.KECCAK_256.hash( + signature.utf8 + ).slice(from: 0, upTo: 4) + + for byte in methodID { + if byte != data.removeFirst() { + panic("signature mismatch") + } + } + + return InternalEVM.decodeABI(types: types, data: data) + } } diff --git a/fvm/evm/stdlib/contract.go b/fvm/evm/stdlib/contract.go index 112c773650d..a98f7a1a66d 100644 --- a/fvm/evm/stdlib/contract.go +++ b/fvm/evm/stdlib/contract.go @@ -27,9 +27,16 @@ import ( //go:embed contract.cdc var contractCode string +//go:embed abiOnlyContract.cdc +var abiOnlyContractCode string + var flowTokenImportPattern = regexp.MustCompile(`^import "FlowToken"\n`) -func ContractCode(flowTokenAddress flow.Address) []byte { +func ContractCode(flowTokenAddress flow.Address, evmAbiOnly bool) []byte { + if evmAbiOnly { + return []byte(abiOnlyContractCode) + } + return []byte(flowTokenImportPattern.ReplaceAllString( contractCode, fmt.Sprintf("import FlowToken from %s", flowTokenAddress.HexWithPrefix()), diff --git a/fvm/evm/stdlib/contract_test.go b/fvm/evm/stdlib/contract_test.go index 34c7ec6a14c..326a2901ad6 100644 --- a/fvm/evm/stdlib/contract_test.go +++ b/fvm/evm/stdlib/contract_test.go @@ -4,6 +4,7 @@ import ( "encoding/binary" "testing" + "github.com/ethereum/go-ethereum/crypto" "github.com/onflow/cadence" "github.com/onflow/cadence/encoding/json" "github.com/onflow/cadence/runtime" @@ -132,6 +133,7 @@ func deployContracts( runtimeInterface *TestRuntimeInterface, transactionEnvironment runtime.Environment, nextTransactionLocation func() common.TransactionLocation, + evmAbiOnly bool, ) { contractsAddressHex := contractsAddress.Hex() @@ -191,7 +193,7 @@ func deployContracts( }, { name: stdlib.ContractName, - code: stdlib.ContractCode(contractsAddress), + code: stdlib.ContractCode(contractsAddress, evmAbiOnly), }, } @@ -312,6 +314,7 @@ func TestEVMEncodeABI(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -440,6 +443,7 @@ func TestEVMEncodeABIComputation(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -535,6 +539,7 @@ func TestEVMEncodeABIComputationEmptyDynamicVariables(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -639,6 +644,7 @@ func TestEVMEncodeABIComputationDynamicVariablesAboveChunkSize(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -737,6 +743,7 @@ func TestEVMDecodeABI(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -871,6 +878,7 @@ func TestEVMDecodeABIComputation(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1144,6 +1152,7 @@ func TestEVMEncodeDecodeABIRoundtrip(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1222,6 +1231,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1308,6 +1318,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1393,6 +1404,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1479,6 +1491,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1565,6 +1578,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1661,6 +1675,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1747,6 +1762,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1833,6 +1849,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -1919,6 +1936,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -2005,6 +2023,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -2091,6 +2110,7 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -2137,6 +2157,377 @@ func TestEVMEncodeDecodeABIErrors(t *testing.T) { }) } +func TestEVMEncodeABIWithSignature(t *testing.T) { + + t.Parallel() + + handler := &testContractHandler{} + + contractsAddress := flow.BytesToAddress([]byte{0x1}) + + transactionEnvironment := newEVMTransactionEnvironment(handler, contractsAddress) + scriptEnvironment := newEVMScriptEnvironment(handler, contractsAddress) + + rt := runtime.NewInterpreterRuntime(runtime.Config{}) + + script := []byte(` + import EVM from 0x1 + + access(all) + fun main(): [UInt8] { + // bytes for address 0x7A58c0Be72BE218B41C608b7Fe7C5bB630736C71 + let address = EVM.EVMAddress( + bytes: [ + 122, 88, 192, 190, 114, 190, 33, 139, 65, 198, + 8, 183, 254, 124, 91, 182, 48, 115, 108, 113 + ] + ) + + return EVM.encodeABIWithSignature( + "withdraw(address,uint256)", + [address, UInt256(250)] + ) + } + `) + + accountCodes := map[common.Location][]byte{} + var events []cadence.Event + + computation := uint(0) + runtimeInterface := &TestRuntimeInterface{ + Storage: NewTestLedger(nil, nil), + OnGetSigningAccounts: func() ([]runtime.Address, error) { + return []runtime.Address{runtime.Address(contractsAddress)}, nil + }, + OnResolveLocation: SingleIdentifierLocationResolver(t), + OnUpdateAccountContractCode: func(location common.AddressLocation, code []byte) error { + accountCodes[location] = code + return nil + }, + OnGetAccountContractCode: func(location common.AddressLocation) (code []byte, err error) { + code = accountCodes[location] + return code, nil + }, + OnEmitEvent: func(event cadence.Event) error { + events = append(events, event) + return nil + }, + OnDecodeArgument: func(b []byte, t cadence.Type) (cadence.Value, error) { + return json.Decode(nil, b) + }, + OnMeterComputation: func(compKind common.ComputationKind, intensity uint) error { + if compKind == environment.ComputationKindEVMEncodeABI { + computation += intensity + } + return nil + }, + OnHash: func( + data []byte, + tag string, + hashAlgorithm runtime.HashAlgorithm, + ) ([]byte, error) { + return crypto.Keccak256(data), nil + }, + } + + nextTransactionLocation := NewTransactionLocationGenerator() + nextScriptLocation := NewScriptLocationGenerator() + + // Deploy contracts + + deployContracts( + t, + rt, + contractsAddress, + runtimeInterface, + transactionEnvironment, + nextTransactionLocation, + true, + ) + + // Run script + + result, err := rt.ExecuteScript( + runtime.Script{ + Source: script, + Arguments: [][]byte{}, + }, + runtime.Context{ + Interface: runtimeInterface, + Environment: scriptEnvironment, + Location: nextScriptLocation(), + }, + ) + require.NoError(t, err) + + abiBytes := []byte{ + 0xf3, 0xfe, 0xf3, 0xa3, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x7a, 0x58, 0xc0, 0xbe, 0x72, 0xbe, 0x21, 0x8b, 0x41, + 0xc6, 0x8, 0xb7, 0xfe, 0x7c, 0x5b, 0xb6, 0x30, 0x73, 0x6c, 0x71, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0xfa, + } + cdcBytes := make([]cadence.Value, 0) + for _, bt := range abiBytes { + cdcBytes = append(cdcBytes, cadence.UInt8(bt)) + } + encodedABI := cadence.NewArray( + cdcBytes, + ).WithType(cadence.NewVariableSizedArrayType(cadence.UInt8Type)) + + assert.Equal(t, + encodedABI, + result, + ) + // The method ID is a byte array of length 4 + assert.Equal(t, computation+4, uint(len(cdcBytes))) +} + +func TestEVMDecodeABIWithSignature(t *testing.T) { + + t.Parallel() + + handler := &testContractHandler{} + + contractsAddress := flow.BytesToAddress([]byte{0x1}) + + transactionEnvironment := newEVMTransactionEnvironment(handler, contractsAddress) + scriptEnvironment := newEVMScriptEnvironment(handler, contractsAddress) + + rt := runtime.NewInterpreterRuntime(runtime.Config{}) + + script := []byte(` + import EVM from 0x1 + + access(all) + fun main(data: [UInt8]): Bool { + let values = EVM.decodeABIWithSignature( + "withdraw(address,uint256)", + types: [Type(), Type()], + data: data + ) + + // bytes for address 0x7A58c0Be72BE218B41C608b7Fe7C5bB630736C71 + let address = EVM.EVMAddress( + bytes: [ + 122, 88, 192, 190, 114, 190, 33, 139, 65, 198, + 8, 183, 254, 124, 91, 182, 48, 115, 108, 113 + ] + ) + + assert(values.length == 2) + assert((values[0] as! EVM.EVMAddress).bytes == address.bytes) + assert((values[1] as! UInt256) == UInt256(250)) + + return true + } + `) + + accountCodes := map[common.Location][]byte{} + var events []cadence.Event + + computation := uint(0) + runtimeInterface := &TestRuntimeInterface{ + Storage: NewTestLedger(nil, nil), + OnGetSigningAccounts: func() ([]runtime.Address, error) { + return []runtime.Address{runtime.Address(contractsAddress)}, nil + }, + OnResolveLocation: SingleIdentifierLocationResolver(t), + OnUpdateAccountContractCode: func(location common.AddressLocation, code []byte) error { + accountCodes[location] = code + return nil + }, + OnGetAccountContractCode: func(location common.AddressLocation) (code []byte, err error) { + code = accountCodes[location] + return code, nil + }, + OnEmitEvent: func(event cadence.Event) error { + events = append(events, event) + return nil + }, + OnDecodeArgument: func(b []byte, t cadence.Type) (cadence.Value, error) { + return json.Decode(nil, b) + }, + OnMeterComputation: func(compKind common.ComputationKind, intensity uint) error { + if compKind == environment.ComputationKindEVMDecodeABI { + computation += intensity + } + return nil + }, + OnHash: func( + data []byte, + tag string, + hashAlgorithm runtime.HashAlgorithm, + ) ([]byte, error) { + return crypto.Keccak256(data), nil + }, + } + + nextTransactionLocation := NewTransactionLocationGenerator() + nextScriptLocation := NewScriptLocationGenerator() + + // Deploy contracts + + deployContracts( + t, + rt, + contractsAddress, + runtimeInterface, + transactionEnvironment, + nextTransactionLocation, + true, + ) + + // Run script + abiBytes := []byte{ + 0xf3, 0xfe, 0xf3, 0xa3, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x7a, 0x58, 0xc0, 0xbe, 0x72, 0xbe, 0x21, 0x8b, 0x41, + 0xc6, 0x8, 0xb7, 0xfe, 0x7c, 0x5b, 0xb6, 0x30, 0x73, 0x6c, 0x71, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0xfa, + } + cdcBytes := make([]cadence.Value, 0) + for _, bt := range abiBytes { + cdcBytes = append(cdcBytes, cadence.UInt8(bt)) + } + encodedABI := cadence.NewArray( + cdcBytes, + ).WithType(cadence.NewVariableSizedArrayType(cadence.UInt8Type)) + + result, err := rt.ExecuteScript( + runtime.Script{ + Source: script, + Arguments: EncodeArgs([]cadence.Value{ + encodedABI, + }), + }, + runtime.Context{ + Interface: runtimeInterface, + Environment: scriptEnvironment, + Location: nextScriptLocation(), + }, + ) + require.NoError(t, err) + + assert.Equal(t, cadence.NewBool(true), result) + // The method ID is a byte array of length 4 + assert.Equal(t, computation+4, uint(len(cdcBytes))) +} + +func TestEVMDecodeABIWithSignatureMismatch(t *testing.T) { + + t.Parallel() + + handler := &testContractHandler{} + + contractsAddress := flow.BytesToAddress([]byte{0x1}) + + transactionEnvironment := newEVMTransactionEnvironment(handler, contractsAddress) + scriptEnvironment := newEVMScriptEnvironment(handler, contractsAddress) + + rt := runtime.NewInterpreterRuntime(runtime.Config{}) + + script := []byte(` + import EVM from 0x1 + + access(all) + fun main(data: [UInt8]): Bool { + // The data was encoded for the function "withdraw(address,uint256)", + // but we pass a different function signature + let values = EVM.decodeABIWithSignature( + "deposit(uint256, address)", + types: [Type(), Type()], + data: data + ) + + return true + } + `) + + accountCodes := map[common.Location][]byte{} + var events []cadence.Event + + runtimeInterface := &TestRuntimeInterface{ + Storage: NewTestLedger(nil, nil), + OnGetSigningAccounts: func() ([]runtime.Address, error) { + return []runtime.Address{runtime.Address(contractsAddress)}, nil + }, + OnResolveLocation: SingleIdentifierLocationResolver(t), + OnUpdateAccountContractCode: func(location common.AddressLocation, code []byte) error { + accountCodes[location] = code + return nil + }, + OnGetAccountContractCode: func(location common.AddressLocation) (code []byte, err error) { + code = accountCodes[location] + return code, nil + }, + OnEmitEvent: func(event cadence.Event) error { + events = append(events, event) + return nil + }, + OnDecodeArgument: func(b []byte, t cadence.Type) (cadence.Value, error) { + return json.Decode(nil, b) + }, + OnHash: func( + data []byte, + tag string, + hashAlgorithm runtime.HashAlgorithm, + ) ([]byte, error) { + return crypto.Keccak256(data), nil + }, + } + + nextTransactionLocation := NewTransactionLocationGenerator() + nextScriptLocation := NewScriptLocationGenerator() + + // Deploy contracts + + deployContracts( + t, + rt, + contractsAddress, + runtimeInterface, + transactionEnvironment, + nextTransactionLocation, + true, + ) + + // Run script + abiBytes := []byte{ + 0xf3, 0xfe, 0xf3, 0xa3, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x7a, 0x58, 0xc0, 0xbe, 0x72, 0xbe, 0x21, 0x8b, 0x41, + 0xc6, 0x8, 0xb7, 0xfe, 0x7c, 0x5b, 0xb6, 0x30, 0x73, 0x6c, 0x71, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, + 0x0, 0x0, 0xfa, + } + cdcBytes := make([]cadence.Value, 0) + for _, bt := range abiBytes { + cdcBytes = append(cdcBytes, cadence.UInt8(bt)) + } + encodedABI := cadence.NewArray( + cdcBytes, + ).WithType(cadence.NewVariableSizedArrayType(cadence.UInt8Type)) + + _, err := rt.ExecuteScript( + runtime.Script{ + Source: script, + Arguments: EncodeArgs([]cadence.Value{ + encodedABI, + }), + }, + runtime.Context{ + Interface: runtimeInterface, + Environment: scriptEnvironment, + Location: nextScriptLocation(), + }, + ) + require.Error(t, err) + assert.ErrorContains(t, err, "panic: signature mismatch") +} + func TestEVMAddressConstructionAndReturn(t *testing.T) { t.Parallel() @@ -2210,6 +2601,7 @@ func TestEVMAddressConstructionAndReturn(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + true, ) // Run script @@ -2302,6 +2694,7 @@ func TestBalanceConstructionAndReturn(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -2432,6 +2825,7 @@ func TestEVMRun(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -2519,6 +2913,7 @@ func TestEVMCreateBridgedAccount(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -2649,6 +3044,7 @@ func TestBridgedAccountCall(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -2767,6 +3163,7 @@ func TestEVMAddressDeposit(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -2895,6 +3292,7 @@ func TestBridgedAccountWithdraw(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -3008,6 +3406,7 @@ func TestBridgedAccountDeploy(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -3128,6 +3527,7 @@ func TestEVMAccountBalance(t *testing.T) { runtimeInterface, transactionEnvironment, nextTransactionLocation, + false, ) // Run script @@ -3147,3 +3547,111 @@ func TestEVMAccountBalance(t *testing.T) { require.NoError(t, err) require.Equal(t, expectedBalance, actual) } + +func TestEVMAccountBalanceForABIOnlyContract(t *testing.T) { + + t.Parallel() + + contractsAddress := flow.BytesToAddress([]byte{0x1}) + + expectedBalanceValue, err := cadence.NewUFix64FromParts(1, 1337000) + require.NoError(t, err) + + handler := &testContractHandler{ + flowTokenAddress: common.Address(contractsAddress), + accountByAddress: func(fromAddress types.Address, isAuthorized bool) types.Account { + assert.Equal(t, types.Address{1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, fromAddress) + assert.False(t, isAuthorized) + + return &testFlowAccount{ + address: fromAddress, + balance: func() types.Balance { + return types.Balance(expectedBalanceValue) + }, + } + }, + } + + transactionEnvironment := newEVMTransactionEnvironment(handler, contractsAddress) + scriptEnvironment := newEVMScriptEnvironment(handler, contractsAddress) + + rt := runtime.NewInterpreterRuntime(runtime.Config{}) + + script := []byte(` + import EVM from 0x1 + + access(all) + fun main(): EVM.Balance { + let bridgedAccount <- EVM.createBridgedAccount() + let balance = bridgedAccount.balance() + destroy bridgedAccount + return balance + } + `) + + accountCodes := map[common.Location][]byte{} + var events []cadence.Event + + runtimeInterface := &TestRuntimeInterface{ + Storage: NewTestLedger(nil, nil), + OnGetSigningAccounts: func() ([]runtime.Address, error) { + return []runtime.Address{runtime.Address(contractsAddress)}, nil + }, + OnResolveLocation: SingleIdentifierLocationResolver(t), + OnUpdateAccountContractCode: func(location common.AddressLocation, code []byte) error { + accountCodes[location] = code + return nil + }, + OnGetAccountContractCode: func(location common.AddressLocation) (code []byte, err error) { + code = accountCodes[location] + return code, nil + }, + OnEmitEvent: func(event cadence.Event) error { + events = append(events, event) + return nil + }, + OnDecodeArgument: func(b []byte, t cadence.Type) (cadence.Value, error) { + return json.Decode(nil, b) + }, + } + + nextTransactionLocation := NewTransactionLocationGenerator() + nextScriptLocation := NewScriptLocationGenerator() + + // Deploy contracts + + deployContracts( + t, + rt, + contractsAddress, + runtimeInterface, + transactionEnvironment, + nextTransactionLocation, + true, + ) + + // Run script + + _, err = rt.ExecuteScript( + runtime.Script{ + Source: script, + }, + runtime.Context{ + Interface: runtimeInterface, + Environment: scriptEnvironment, + Location: nextScriptLocation(), + }, + ) + require.Error(t, err) + + assert.ErrorContains( + t, + err, + "error: cannot find type in this scope: `EVM.Balance`", + ) + assert.ErrorContains( + t, + err, + "error: value of type `EVM` has no member `createBridgedAccount`", + ) +} diff --git a/fvm/evm/testutils/accounts.go b/fvm/evm/testutils/accounts.go index 34487b8e91f..8df4b712508 100644 --- a/fvm/evm/testutils/accounts.go +++ b/fvm/evm/testutils/accounts.go @@ -16,7 +16,6 @@ import ( "github.com/onflow/atree" "github.com/onflow/flow-go/fvm/evm/emulator" - "github.com/onflow/flow-go/fvm/evm/emulator/database" "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" ) @@ -126,11 +125,7 @@ func FundAndGetEOATestAccount(t testing.TB, led atree.Ledger, flowEVMRootAddress account := GetTestEOAAccount(t, EOATestAccount1KeyHex) // fund account - db, err := database.NewDatabase(led, flowEVMRootAddress) - require.NoError(t, err) - - e := emulator.NewEmulator(db) - require.NoError(t, err) + e := emulator.NewEmulator(led, flowEVMRootAddress) blk, err := e.NewBlockView(types.NewDefaultBlockContext(2)) require.NoError(t, err) diff --git a/fvm/evm/testutils/backend.go b/fvm/evm/testutils/backend.go index 308b46f41e2..fc08d84aad8 100644 --- a/fvm/evm/testutils/backend.go +++ b/fvm/evm/testutils/backend.go @@ -52,24 +52,34 @@ func fullKey(owner, key []byte) string { func GetSimpleValueStore() *TestValueStore { data := make(map[string][]byte) allocator := make(map[string]uint64) - + bytesRead := 0 + bytesWritten := 0 return &TestValueStore{ GetValueFunc: func(owner, key []byte) ([]byte, error) { - return data[fullKey(owner, key)], nil + fk := fullKey(owner, key) + value := data[fk] + bytesRead += len(fk) + len(value) + return value, nil }, SetValueFunc: func(owner, key, value []byte) error { - data[fullKey(owner, key)] = value + fk := fullKey(owner, key) + data[fk] = value + bytesWritten += len(fk) + len(value) return nil }, ValueExistsFunc: func(owner, key []byte) (bool, error) { - return len(data[fullKey(owner, key)]) > 0, nil - + fk := fullKey(owner, key) + value := data[fk] + bytesRead += len(fk) + len(value) + return len(value) > 0, nil }, AllocateStorageIndexFunc: func(owner []byte) (atree.StorageIndex, error) { index := allocator[string(owner)] var data [8]byte allocator[string(owner)] = index + 1 binary.BigEndian.PutUint64(data[:], index) + bytesRead += len(owner) + 8 + bytesWritten += len(owner) + 8 return atree.StorageIndex(data), nil }, TotalStorageSizeFunc: func() int { @@ -82,9 +92,19 @@ func GetSimpleValueStore() *TestValueStore { } return size }, + TotalBytesReadFunc: func() int { + return bytesRead + }, + TotalBytesWrittenFunc: func() int { + return bytesWritten + }, TotalStorageItemsFunc: func() int { return len(maps.Keys(data)) + len(maps.Keys(allocator)) }, + ResetStatsFunc: func() { + bytesRead = 0 + bytesWritten = 0 + }, } } @@ -155,7 +175,10 @@ type TestValueStore struct { ValueExistsFunc func(owner, key []byte) (bool, error) AllocateStorageIndexFunc func(owner []byte) (atree.StorageIndex, error) TotalStorageSizeFunc func() int + TotalBytesReadFunc func() int + TotalBytesWrittenFunc func() int TotalStorageItemsFunc func() int + ResetStatsFunc func() } var _ environment.ValueStore = &TestValueStore{} @@ -188,6 +211,20 @@ func (vs *TestValueStore) AllocateStorageIndex(owner []byte) (atree.StorageIndex return vs.AllocateStorageIndexFunc(owner) } +func (vs *TestValueStore) TotalBytesRead() int { + if vs.TotalBytesReadFunc == nil { + panic("method not set") + } + return vs.TotalBytesReadFunc() +} + +func (vs *TestValueStore) TotalBytesWritten() int { + if vs.TotalBytesWrittenFunc == nil { + panic("method not set") + } + return vs.TotalBytesWrittenFunc() +} + func (vs *TestValueStore) TotalStorageSize() int { if vs.TotalStorageSizeFunc == nil { panic("method not set") @@ -202,6 +239,13 @@ func (vs *TestValueStore) TotalStorageItems() int { return vs.TotalStorageItemsFunc() } +func (vs *TestValueStore) ResetStats() { + if vs.ResetStatsFunc == nil { + panic("method not set") + } + vs.ResetStatsFunc() +} + type testMeter struct { meterComputation func(common.ComputationKind, uint) error hasComputationCapacity func(common.ComputationKind, uint) bool diff --git a/fvm/evm/testutils/contract.go b/fvm/evm/testutils/contract.go index 78316f44cd2..f67cced4c94 100644 --- a/fvm/evm/testutils/contract.go +++ b/fvm/evm/testutils/contract.go @@ -14,7 +14,6 @@ import ( "github.com/onflow/atree" "github.com/onflow/flow-go/fvm/evm/emulator" - "github.com/onflow/flow-go/fvm/evm/emulator/database" "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/model/flow" ) @@ -387,10 +386,7 @@ func RunWithDeployedContract(t testing.TB, tc *TestContract, led atree.Ledger, f func DeployContract(t testing.TB, tc *TestContract, led atree.Ledger, flowEVMRootAddress flow.Address) { // deploy contract - db, err := database.NewDatabase(led, flowEVMRootAddress) - require.NoError(t, err) - - e := emulator.NewEmulator(db) + e := emulator.NewEmulator(led, flowEVMRootAddress) blk, err := e.NewBlockView(types.NewDefaultBlockContext(2)) require.NoError(t, err) diff --git a/fvm/evm/testutils/database.go b/fvm/evm/testutils/database.go deleted file mode 100644 index fcc4ef3d117..00000000000 --- a/fvm/evm/testutils/database.go +++ /dev/null @@ -1,117 +0,0 @@ -package testutils - -import ( - gethCommon "github.com/ethereum/go-ethereum/common" - gethDB "github.com/ethereum/go-ethereum/ethdb" - - "github.com/onflow/flow-go/fvm/evm/types" -) - -type TestDatabase struct { - GetFunc func(key []byte) ([]byte, error) - HasFunc func(key []byte) (bool, error) - PutFunc func(key []byte, value []byte) error - DeleteFunc func(key []byte) error - StatFunc func(property string) (string, error) - NewBatchFunc func() gethDB.Batch - NewBatchWithSizeFunc func(size int) gethDB.Batch - NewIteratorFunc func(prefix []byte, start []byte) gethDB.Iterator - CompactFunc func(start []byte, limit []byte) error - NewSnapshotFunc func() (gethDB.Snapshot, error) - CloseFunc func() error - GetRootHashFunc func() (gethCommon.Hash, error) - CommitFunc func(roothash gethCommon.Hash) error -} - -var _ types.Database = &TestDatabase{} - -func (db *TestDatabase) Get(key []byte) ([]byte, error) { - if db.GetFunc == nil { - panic("method not set") - } - return db.GetFunc(key) -} - -func (db *TestDatabase) Has(key []byte) (bool, error) { - if db.HasFunc == nil { - panic("method not set") - } - return db.HasFunc(key) -} - -func (db *TestDatabase) Put(key []byte, value []byte) error { - if db.PutFunc == nil { - panic("method not set") - } - return db.PutFunc(key, value) -} - -func (db *TestDatabase) Delete(key []byte) error { - if db.DeleteFunc == nil { - panic("method not set") - } - return db.DeleteFunc(key) -} - -func (db *TestDatabase) Commit(root gethCommon.Hash) error { - if db.CommitFunc == nil { - panic("method not set") - } - return db.CommitFunc(root) -} - -func (db *TestDatabase) GetRootHash() (gethCommon.Hash, error) { - if db.GetRootHashFunc == nil { - panic("method not set") - } - return db.GetRootHashFunc() -} - -func (db *TestDatabase) Stat(property string) (string, error) { - if db.StatFunc == nil { - panic("method not set") - } - return db.StatFunc(property) -} - -func (db *TestDatabase) NewBatch() gethDB.Batch { - if db.NewBatchFunc == nil { - panic("method not set") - } - return db.NewBatchFunc() -} - -func (db *TestDatabase) NewBatchWithSize(size int) gethDB.Batch { - if db.NewBatchWithSizeFunc == nil { - panic("method not set") - } - return db.NewBatchWithSizeFunc(size) -} - -func (db *TestDatabase) NewIterator(prefix []byte, start []byte) gethDB.Iterator { - if db.NewIteratorFunc == nil { - panic("method not set") - } - return db.NewIteratorFunc(prefix, start) -} - -func (db *TestDatabase) Compact(start []byte, limit []byte) error { - if db.CompactFunc == nil { - panic("method not set") - } - return db.CompactFunc(start, limit) -} - -func (db *TestDatabase) NewSnapshot() (gethDB.Snapshot, error) { - if db.NewSnapshotFunc == nil { - panic("method not set") - } - return db.NewSnapshotFunc() -} - -func (db *TestDatabase) Close() error { - if db.CloseFunc == nil { - panic("method not set") - } - return db.CloseFunc() -} diff --git a/fvm/evm/types/block.go b/fvm/evm/types/block.go index 6c20c6a4d90..88d7f930502 100644 --- a/fvm/evm/types/block.go +++ b/fvm/evm/types/block.go @@ -18,9 +18,6 @@ type Block struct { // holds the total amount of the native token deposited in the evm side. TotalSupply uint64 - // StateRoot returns the EVM root hash of the state after executing this block - StateRoot gethCommon.Hash - // ReceiptRoot returns the root hash of the receipts emitted in this block ReceiptRoot gethCommon.Hash @@ -52,7 +49,6 @@ func NewBlock(height, uuidIndex, totalSupply uint64, return &Block{ Height: height, TotalSupply: totalSupply, - StateRoot: stateRoot, ReceiptRoot: receiptRoot, TransactionHashes: txHashes, } @@ -69,6 +65,5 @@ func NewBlockFromBytes(encoded []byte) (*Block, error) { var GenesisBlock = &Block{ ParentBlockHash: gethCommon.Hash{}, Height: uint64(0), - StateRoot: gethTypes.EmptyRootHash, ReceiptRoot: gethTypes.EmptyRootHash, } diff --git a/fvm/evm/types/emulator.go b/fvm/evm/types/emulator.go index 1b86e06fe7b..73eef076a5b 100644 --- a/fvm/evm/types/emulator.go +++ b/fvm/evm/types/emulator.go @@ -3,9 +3,7 @@ package types import ( "math/big" - gethCommon "github.com/ethereum/go-ethereum/common" gethTypes "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/ethdb" ) var ( @@ -65,17 +63,3 @@ type Emulator interface { // constructs a new block NewBlockView(ctx BlockContext) (BlockView, error) } - -// Database provides what Emulator needs for storing tries and accounts -// Errors returned by the methods are one of the followings: -// - Fatal error -// - Database error (non-fatal) -type Database interface { - ethdb.KeyValueStore - - // Commit commits the changes - Commit(rootHash gethCommon.Hash) error - - // GetRootHash returns the active root hash - GetRootHash() (gethCommon.Hash, error) -} diff --git a/fvm/evm/types/errors.go b/fvm/evm/types/errors.go index fa008b741a2..3e8385f5331 100644 --- a/fvm/evm/types/errors.go +++ b/fvm/evm/types/errors.go @@ -16,8 +16,9 @@ var ( // left in the context of flow transaction to execute the evm operation. ErrInsufficientComputation = errors.New("insufficient computation") - // unauthorized method call, usually emited when calls are called on EOA accounts + // ErrUnAuthroizedMethodCall method call, usually emited when calls are called on EOA accounts ErrUnAuthroizedMethodCall = errors.New("unauthroized method call") + // ErrInsufficientTotalSupply is returned when flow token // is withdraw request is there but not enough balance is on EVM vault // this should never happen but its a saftey measure to protect Flow against EVM issues. @@ -89,35 +90,35 @@ func IsEVMValidationError(err error) bool { return errors.As(err, &EVMValidationError{}) } -// DatabaseError is a non-fatal error, returned when a database operation +// StateError is a non-fatal error, returned when a state operation // has failed (e.g. reaching storage interaction limit) -type DatabaseError struct { +type StateError struct { err error } -// NewDatabaseError returns a new DatabaseError -func NewDatabaseError(rootCause error) DatabaseError { - return DatabaseError{ +// NewStateError returns a new StateError +func NewStateError(rootCause error) StateError { + return StateError{ err: rootCause, } } // Unwrap unwraps the underlying evm error -func (err DatabaseError) Unwrap() error { +func (err StateError) Unwrap() error { return err.err } -func (err DatabaseError) Error() string { - return fmt.Sprintf("database error: %v", err.err) +func (err StateError) Error() string { + return fmt.Sprintf("state error: %v", err.err) } -// IsADatabaseError returns true if the error or any underlying errors +// IsAStateError returns true if the error or any underlying errors // is of the type EVM validation error -func IsADatabaseError(err error) bool { - return errors.As(err, &DatabaseError{}) +func IsAStateError(err error) bool { + return errors.As(err, &StateError{}) } -// FatalError is user for any error that is not user related and something +// FatalError is used for any error that is not user related and something // unusual has happend. Usually we stop the node when this happens // given it might have a non-deterministic root. type FatalError struct { diff --git a/fvm/evm/types/events.go b/fvm/evm/types/events.go index 494b735838d..97ab10a0c90 100644 --- a/fvm/evm/types/events.go +++ b/fvm/evm/types/events.go @@ -44,7 +44,6 @@ var transactionExecutedEventCadenceType = &cadence.EventType{ cadence.NewField("failed", cadence.BoolType), cadence.NewField("transactionType", cadence.UInt8Type), cadence.NewField("gasConsumed", cadence.UInt64Type), - cadence.NewField("stateRootHash", cadence.StringType), cadence.NewField("deployedContractAddress", cadence.StringType), cadence.NewField("returnedValue", cadence.StringType), cadence.NewField("logs", cadence.StringType), @@ -70,7 +69,6 @@ func (p *TransactionExecutedPayload) CadenceEvent() (cadence.Event, error) { cadence.NewBool(p.Result.Failed), cadence.NewUInt8(p.Result.TxType), cadence.NewUInt64(p.Result.GasConsumed), - cadence.String(p.Result.StateRootHash.String()), cadence.String(hex.EncodeToString(p.Result.DeployedContractAddress.Bytes())), cadence.String(hex.EncodeToString(p.Result.ReturnedValue)), cadence.String(hex.EncodeToString(encodedLogs)), @@ -105,7 +103,6 @@ var blockExecutedEventCadenceType = &cadence.EventType{ cadence.NewField("height", cadence.UInt64Type), cadence.NewField("totalSupply", cadence.UInt64Type), cadence.NewField("parentHash", cadence.StringType), - cadence.NewField("stateRoot", cadence.StringType), cadence.NewField("receiptRoot", cadence.StringType), cadence.NewField( "transactionHashes", @@ -129,7 +126,6 @@ func (p *BlockExecutedEventPayload) CadenceEvent() (cadence.Event, error) { cadence.NewUInt64(p.Block.TotalSupply), cadence.String(p.Block.ReceiptRoot.String()), cadence.String(p.Block.ParentBlockHash.String()), - cadence.String(p.Block.StateRoot.String()), cadence.NewArray(hashes).WithType(cadence.NewVariableSizedArrayType(cadence.StringType)), } diff --git a/fvm/evm/types/result.go b/fvm/evm/types/result.go index 6e4248b2d58..fb7321a5847 100644 --- a/fvm/evm/types/result.go +++ b/fvm/evm/types/result.go @@ -1,7 +1,6 @@ package types import ( - gethCommon "github.com/ethereum/go-ethereum/common" gethTypes "github.com/ethereum/go-ethereum/core/types" ) @@ -20,8 +19,6 @@ type Result struct { TxType uint8 // total gas consumed during an opeartion GasConsumed uint64 - // the root hash of the state after execution - StateRootHash gethCommon.Hash // the address where the contract is deployed (if any) DeployedContractAddress Address // returned value from a function call @@ -35,7 +32,6 @@ type Result struct { func (res *Result) Receipt() *gethTypes.ReceiptForStorage { receipt := &gethTypes.Receipt{ Type: res.TxType, - PostState: res.StateRootHash[:], CumulativeGasUsed: res.GasConsumed, // TODO: update to capture cumulative Logs: res.Logs, ContractAddress: res.DeployedContractAddress.ToCommon(), diff --git a/fvm/evm/types/state.go b/fvm/evm/types/state.go new file mode 100644 index 00000000000..ee31f94fa89 --- /dev/null +++ b/fvm/evm/types/state.go @@ -0,0 +1,140 @@ +package types + +import ( + "math/big" + + gethCommon "github.com/ethereum/go-ethereum/common" + gethTypes "github.com/ethereum/go-ethereum/core/types" + gethVM "github.com/ethereum/go-ethereum/core/vm" +) + +// StateDB acts as the main interface to the EVM runtime +type StateDB interface { + gethVM.StateDB + + // Commit commits the changes + Commit() error + + // Logs collects and prepares logs + Logs( + blockHash gethCommon.Hash, + blockNumber uint64, + txHash gethCommon.Hash, + txIndex uint, + ) []*gethTypes.Log + + // returns a map of preimages + Preimages() map[gethCommon.Hash][]byte +} + +// ReadOnlyView provides a readonly view of the state +type ReadOnlyView interface { + // Exist returns true if the address exist in the state + Exist(gethCommon.Address) (bool, error) + // IsCreated returns true if address has been created in this tx + IsCreated(gethCommon.Address) bool + // HasSelfDestructed returns true if an address has self destructed + // it also returns the balance of address before selfdestruction call + HasSelfDestructed(gethCommon.Address) (bool, *big.Int) + // GetBalance returns the balance of an address + GetBalance(gethCommon.Address) (*big.Int, error) + // GetNonce returns the nonce of an address + GetNonce(gethCommon.Address) (uint64, error) + // GetCode returns the code of an address + GetCode(gethCommon.Address) ([]byte, error) + // GetCodeHash returns the code hash of an address + GetCodeHash(gethCommon.Address) (gethCommon.Hash, error) + // GetCodeSize returns the code size of an address + GetCodeSize(gethCommon.Address) (int, error) + // GetState returns values for an slot in the main storage + GetState(SlotAddress) (gethCommon.Hash, error) + // GetTransientState returns values for an slot transient storage + GetTransientState(SlotAddress) gethCommon.Hash + // GetRefund returns the total amount of (gas) refund + GetRefund() uint64 + // AddressInAccessList checks if an address is in the access list + AddressInAccessList(gethCommon.Address) bool + // SlotInAccessList checks if a slot is in the access list + SlotInAccessList(SlotAddress) (addressOk bool, slotOk bool) +} + +// HotView captures a high-level mutable view of the state +type HotView interface { + ReadOnlyView + + // CreateAccount creates a new account + CreateAccount(gethCommon.Address) error + // SelfDestruct set the flag for destruction of the account after execution + SelfDestruct(gethCommon.Address) error + + // SubBalance subtracts the amount from the balance the given address + SubBalance(gethCommon.Address, *big.Int) error + // AddBalance adds the amount to the balance of the given address + AddBalance(gethCommon.Address, *big.Int) error + // SetNonce sets the nonce for the given address + SetNonce(gethCommon.Address, uint64) error + // SetCode sets the code for the given address + SetCode(gethCommon.Address, []byte) error + + // SetState sets a value for the given slot in the main storage + SetState(SlotAddress, gethCommon.Hash) error + // SetTransientState sets a value for the given slot in the transient storage + SetTransientState(SlotAddress, gethCommon.Hash) + + // AddRefund adds the amount to the total (gas) refund + AddRefund(uint64) error + // SubRefund subtracts the amount from the total (gas) refund + SubRefund(uint64) error + + // AddAddressToAccessList adds an address to the per-transaction access list + AddAddressToAccessList(addr gethCommon.Address) (addressAdded bool) + // AddSlotToAccessList adds a slot to the per-transaction access list + AddSlotToAccessList(SlotAddress) (addressAdded, slotAdded bool) + + // AddLog append a log to the log collection + AddLog(*gethTypes.Log) + // AddPreimage adds a preimage to the list of preimages (input -> hash mapping) + AddPreimage(gethCommon.Hash, []byte) +} + +// BaseView is a low-level mutable view of the state +// baseview is usually updated at the commit calls to the higher level view +type BaseView interface { + ReadOnlyView + + // Creates a new account + CreateAccount( + addr gethCommon.Address, + balance *big.Int, + nonce uint64, + code []byte, + codeHash gethCommon.Hash, + ) error + + // UpdateAccount updates a account + UpdateAccount( + addr gethCommon.Address, + balance *big.Int, + nonce uint64, + code []byte, + codeHash gethCommon.Hash, + ) error + + // DeleteAccount deletes an account + DeleteAccount(addr gethCommon.Address) error + + // UpdateSlot updates the value for the given slot in the main storage + UpdateSlot( + slot SlotAddress, + value gethCommon.Hash, + ) error + + // Commit commits the changes + Commit() error +} + +// SlotAddress captures an address to a storage slot +type SlotAddress struct { + Address gethCommon.Address + Key gethCommon.Hash +} diff --git a/fvm/fvm_blockcontext_test.go b/fvm/fvm_blockcontext_test.go index 04aaafe5805..98b5da33c6e 100644 --- a/fvm/fvm_blockcontext_test.go +++ b/fvm/fvm_blockcontext_test.go @@ -952,6 +952,9 @@ func TestBlockContext_ExecuteTransaction_StorageLimit(t *testing.T) { fvm.WithAccountCreationFee(fvm.DefaultAccountCreationFee), fvm.WithMinimumStorageReservation(fvm.DefaultMinimumStorageReservation), fvm.WithStorageMBPerFLOW(fvm.DefaultStorageMBPerFLOW), + // The evm account has a storage exception, and if we don't bootstrap with evm, + // the first created account will have that address. + fvm.WithSetupEVMEnabled(true), } t.Run("Storing too much data fails", newVMTest().withBootstrapProcedureOptions(bootstrapOptions...). @@ -1862,6 +1865,9 @@ func TestBlockContext_ExecuteTransaction_FailingTransactions(t *testing.T) { fvm.WithAccountCreationFee(fvm.DefaultAccountCreationFee), fvm.WithStorageMBPerFLOW(fvm.DefaultStorageMBPerFLOW), fvm.WithExecutionMemoryLimit(math.MaxUint64), + // The evm account has a storage exception, and if we don't bootstrap with evm, + // the first created account will have that address. + fvm.WithSetupEVMEnabled(true), ).run( func(t *testing.T, vm fvm.VM, chain flow.Chain, ctx fvm.Context, snapshotTree snapshot.SnapshotTree) { ctx.LimitAccountStorage = true // this test requires storage limits to be enforced diff --git a/fvm/fvm_test.go b/fvm/fvm_test.go index ded86ab6c05..431b22e062c 100644 --- a/fvm/fvm_test.go +++ b/fvm/fvm_test.go @@ -8,27 +8,30 @@ import ( "strings" "testing" - "github.com/onflow/flow-go/fvm/evm/stdlib" - "github.com/onflow/cadence" "github.com/onflow/cadence/encoding/ccf" jsoncdc "github.com/onflow/cadence/encoding/json" "github.com/onflow/cadence/runtime" "github.com/onflow/cadence/runtime/common" + cadenceErrors "github.com/onflow/cadence/runtime/errors" "github.com/onflow/cadence/runtime/tests/utils" + "github.com/stretchr/testify/assert" + mockery "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/crypto" - "github.com/onflow/flow-go/engine/execution/testutil" exeUtils "github.com/onflow/flow-go/engine/execution/utils" "github.com/onflow/flow-go/fvm" fvmCrypto "github.com/onflow/flow-go/fvm/crypto" "github.com/onflow/flow-go/fvm/environment" - errors "github.com/onflow/flow-go/fvm/errors" + "github.com/onflow/flow-go/fvm/errors" + "github.com/onflow/flow-go/fvm/evm/stdlib" + "github.com/onflow/flow-go/fvm/evm/types" "github.com/onflow/flow-go/fvm/meter" reusableRuntime "github.com/onflow/flow-go/fvm/runtime" "github.com/onflow/flow-go/fvm/storage/snapshot" + "github.com/onflow/flow-go/fvm/storage/snapshot/mock" "github.com/onflow/flow-go/fvm/storage/testutils" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" @@ -2730,12 +2733,10 @@ func TestTransientNetworkCoreContractAddresses(t *testing.T) { } func TestEVM(t *testing.T) { - t.Run("successful transaction", newVMTest(). withBootstrapProcedureOptions(fvm.WithSetupEVMEnabled(true)). - // we keep this dissabled during bootstrap and later overwrite in the test for test transaction withContextOptions( - fvm.WithEVMEnabled(false), + fvm.WithEVMEnabled(true), fvm.WithCadenceLogging(true), ). run(func( @@ -2772,7 +2773,6 @@ func TestEVM(t *testing.T) { err = testutil.SignTransactionAsServiceAccount(txBody, 0, chain) require.NoError(t, err) - ctx = fvm.NewContextFromParent(ctx, fvm.WithEVMEnabled(true)) _, output, err := vm.Run( ctx, fvm.Transaction(txBody, 0), @@ -2788,4 +2788,147 @@ func TestEVM(t *testing.T) { )) }), ) + + // this test makes sure that only ABI encoding/decoding functionality is + // available through the EVM contract, when bootstraped with `WithEVMABIOnly` + t.Run("with ABI only EVM", newVMTest(). + withBootstrapProcedureOptions( + fvm.WithSetupEVMEnabled(true), + fvm.WithEVMABIOnly(true), + ). + withContextOptions( + fvm.WithEVMEnabled(true), + ). + run(func( + t *testing.T, + vm fvm.VM, + chain flow.Chain, + ctx fvm.Context, + snapshotTree snapshot.SnapshotTree, + ) { + txBody := flow.NewTransactionBody(). + SetScript([]byte(fmt.Sprintf(` + import EVM from %s + + transaction { + execute { + let data = EVM.encodeABI(["John Doe", UInt64(33), false]) + log(data.length) + assert(data.length == 160) + + let acc <- EVM.createBridgedAccount() + destroy acc + } + } + `, chain.ServiceAddress().HexWithPrefix()))). + SetProposalKey(chain.ServiceAddress(), 0, 0). + SetPayer(chain.ServiceAddress()) + + err := testutil.SignTransactionAsServiceAccount(txBody, 0, chain) + require.NoError(t, err) + + _, output, err := vm.Run( + ctx, + fvm.Transaction(txBody, 0), + snapshotTree) + + require.NoError(t, err) + require.Error(t, output.Err) + assert.ErrorContains( + t, + output.Err, + "value of type `EVM` has no member `createBridgedAccount`", + ) + }), + ) + + // this test makes sure the execution error is correctly handled and returned as a correct type + t.Run("execution reverted", newVMTest(). + withBootstrapProcedureOptions(fvm.WithSetupEVMEnabled(true)). + withContextOptions(fvm.WithEVMEnabled(true)). + run(func( + t *testing.T, + vm fvm.VM, + chain flow.Chain, + ctx fvm.Context, + snapshotTree snapshot.SnapshotTree, + ) { + script := fvm.Script([]byte(fmt.Sprintf(` + import EVM from %s + + access(all) fun main() { + let bal = EVM.Balance(flow: 1.0); + let acc <- EVM.createBridgedAccount(); + // withdraw insufficient balance + destroy acc.withdraw(balance: bal); + destroy acc; + } + `, chain.ServiceAddress().HexWithPrefix()))) + + _, output, err := vm.Run(ctx, script, snapshotTree) + + require.NoError(t, err) + require.Error(t, output.Err) + require.True(t, errors.IsEVMError(output.Err)) + + // make sure error is not treated as internal error by Cadence + var internal cadenceErrors.InternalError + require.False(t, errors.As(output.Err, &internal)) + }), + ) + + // this test makes sure the EVM error is correctly returned as an error and has a correct type + // we have implemented a snapshot wrapper to return an error from the EVM + t.Run("internal evm error handling", newVMTest(). + withBootstrapProcedureOptions(fvm.WithSetupEVMEnabled(true)). + withContextOptions(fvm.WithEVMEnabled(true)). + run(func( + t *testing.T, + vm fvm.VM, + chain flow.Chain, + ctx fvm.Context, + snapshotTree snapshot.SnapshotTree, + ) { + + tests := []struct { + err error + errChecker func(error) bool + }{{ + types.ErrNotImplemented, + types.IsAFatalError, + }, { + types.NewStateError(fmt.Errorf("test state error")), + types.IsAStateError, + }} + + for _, e := range tests { + // this mock will return an error we provide with the test once it starts to access address allocator registers + // that is done to make sure the error is coming out of EVM execution + errStorage := &mock.StorageSnapshot{} + errStorage. + On("Get", mockery.AnythingOfType("flow.RegisterID")). + Return(func(id flow.RegisterID) (flow.RegisterValue, error) { + if id.Key == "AddressAllocator" { + return nil, e.err + } + return snapshotTree.Get(id) + }) + + script := fvm.Script([]byte(fmt.Sprintf(` + import EVM from %s + + access(all) fun main() { + destroy <- EVM.createBridgedAccount(); + } + `, chain.ServiceAddress().HexWithPrefix()))) + + _, output, err := vm.Run(ctx, script, errStorage) + + require.NoError(t, output.Err) + require.Error(t, err) + // make sure error it's the right type of error + require.True(t, e.errChecker(err), "error is not of the right type") + } + }), + ) } diff --git a/fvm/storage/snapshot/execution_snapshot.go b/fvm/storage/snapshot/execution_snapshot.go index 420c4ffccb4..9b26e4e07b6 100644 --- a/fvm/storage/snapshot/execution_snapshot.go +++ b/fvm/storage/snapshot/execution_snapshot.go @@ -1,6 +1,8 @@ package snapshot import ( + "strings" + "golang.org/x/exp/slices" "github.com/onflow/flow-go/fvm/meter" @@ -29,9 +31,12 @@ func (snapshot *ExecutionSnapshot) UpdatedRegisters() flow.RegisterEntries { entries = append(entries, flow.RegisterEntry{Key: key, Value: value}) } - slices.SortFunc(entries, func(a, b flow.RegisterEntry) bool { - return (a.Key.Owner < b.Key.Owner) || - (a.Key.Owner == b.Key.Owner && a.Key.Key < b.Key.Key) + slices.SortFunc(entries, func(a, b flow.RegisterEntry) int { + ownerCmp := strings.Compare(a.Key.Owner, b.Key.Owner) + if ownerCmp != 0 { + return ownerCmp + } + return strings.Compare(a.Key.Key, b.Key.Key) }) return entries diff --git a/fvm/storage/snapshot/mock/storage_snapshot.go b/fvm/storage/snapshot/mock/storage_snapshot.go new file mode 100644 index 00000000000..4b9e5c33f26 --- /dev/null +++ b/fvm/storage/snapshot/mock/storage_snapshot.go @@ -0,0 +1,54 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mock + +import ( + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" +) + +// StorageSnapshot is an autogenerated mock type for the StorageSnapshot type +type StorageSnapshot struct { + mock.Mock +} + +// Get provides a mock function with given fields: id +func (_m *StorageSnapshot) Get(id flow.RegisterID) ([]byte, error) { + ret := _m.Called(id) + + var r0 []byte + var r1 error + if rf, ok := ret.Get(0).(func(flow.RegisterID) ([]byte, error)); ok { + return rf(id) + } + if rf, ok := ret.Get(0).(func(flow.RegisterID) []byte); ok { + r0 = rf(id) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + if rf, ok := ret.Get(1).(func(flow.RegisterID) error); ok { + r1 = rf(id) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type mockConstructorTestingTNewStorageSnapshot interface { + mock.TestingT + Cleanup(func()) +} + +// NewStorageSnapshot creates a new instance of StorageSnapshot. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewStorageSnapshot(t mockConstructorTestingTNewStorageSnapshot) *StorageSnapshot { + mock := &StorageSnapshot{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/fvm/transactionInvoker.go b/fvm/transactionInvoker.go index acd4088c4bd..57c0c449cbf 100644 --- a/fvm/transactionInvoker.go +++ b/fvm/transactionInvoker.go @@ -406,6 +406,7 @@ func (executor *transactionExecutor) normalExecution() ( // actual balance, for the purpose of calculating storage capacity, because the payer will have to pay for this tx. executor.txnState.RunWithAllLimitsDisabled(func() { err = executor.CheckStorageLimits( + executor.ctx, executor.env, bodySnapshot, executor.proc.Transaction.Payer, diff --git a/fvm/transactionStorageLimiter.go b/fvm/transactionStorageLimiter.go index 9d504adf7bf..9c98d7ef644 100644 --- a/fvm/transactionStorageLimiter.go +++ b/fvm/transactionStorageLimiter.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/fvm/storage/snapshot" + "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/trace" ) @@ -33,6 +34,7 @@ type TransactionStorageLimiter struct{} // The payers balance is considered to be maxTxFees lower that its actual balance, due to the fact that // the fee deduction step happens after the storage limit check. func (limiter TransactionStorageLimiter) CheckStorageLimits( + ctx Context, env environment.Environment, snapshot *snapshot.ExecutionSnapshot, payer flow.Address, @@ -44,7 +46,7 @@ func (limiter TransactionStorageLimiter) CheckStorageLimits( defer env.StartChildSpan(trace.FVMTransactionStorageUsedCheck).End() - err := limiter.checkStorageLimits(env, snapshot, payer, maxTxFees) + err := limiter.checkStorageLimits(ctx, env, snapshot, payer, maxTxFees) if err != nil { return fmt.Errorf("storage limit check failed: %w", err) } @@ -55,6 +57,7 @@ func (limiter TransactionStorageLimiter) CheckStorageLimits( // storage limit is exceeded. The returned list include addresses of updated // registers (and the payer's address). func (limiter TransactionStorageLimiter) getStorageCheckAddresses( + ctx Context, snapshot *snapshot.ExecutionSnapshot, payer flow.Address, maxTxFees uint64, @@ -71,12 +74,17 @@ func (limiter TransactionStorageLimiter) getStorageCheckAddresses( addresses = append(addresses, payer) } + sc := systemcontracts.SystemContractsForChain(ctx.Chain.ChainID()) for id := range snapshot.WriteSet { address, ok := addressFromRegisterId(id) if !ok { continue } + if limiter.shouldSkipSpecialAddress(ctx, address, sc) { + continue + } + _, ok = dedup[address] if ok { continue @@ -88,10 +96,10 @@ func (limiter TransactionStorageLimiter) getStorageCheckAddresses( slices.SortFunc( addresses, - func(a flow.Address, b flow.Address) bool { + func(a flow.Address, b flow.Address) int { // reverse order to maintain compatibility with previous // implementation. - return bytes.Compare(a[:], b[:]) >= 0 + return bytes.Compare(b[:], a[:]) }) return addresses } @@ -99,12 +107,13 @@ func (limiter TransactionStorageLimiter) getStorageCheckAddresses( // checkStorageLimits checks if the transaction changed the storage of any // address and exceeded the storage limit. func (limiter TransactionStorageLimiter) checkStorageLimits( + ctx Context, env environment.Environment, snapshot *snapshot.ExecutionSnapshot, payer flow.Address, maxTxFees uint64, ) error { - addresses := limiter.getStorageCheckAddresses(snapshot, payer, maxTxFees) + addresses := limiter.getStorageCheckAddresses(ctx, snapshot, payer, maxTxFees) usages := make([]uint64, len(addresses)) @@ -155,3 +164,14 @@ func (limiter TransactionStorageLimiter) checkStorageLimits( return nil } + +// shouldSkipSpecialAddress returns true if the address is a special address where storage +// limits are not enforced. +// This is currently only the EVM storage address. This is a temporary solution. +func (limiter TransactionStorageLimiter) shouldSkipSpecialAddress( + ctx Context, + address flow.Address, + sc *systemcontracts.SystemContracts, +) bool { + return sc.EVM.Address == address +} diff --git a/fvm/transactionStorageLimiter_test.go b/fvm/transactionStorageLimiter_test.go index 45b089ff868..d73701bdaee 100644 --- a/fvm/transactionStorageLimiter_test.go +++ b/fvm/transactionStorageLimiter_test.go @@ -8,9 +8,11 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/fvm" + "github.com/onflow/flow-go/fvm/environment" fvmmock "github.com/onflow/flow-go/fvm/environment/mock" "github.com/onflow/flow-go/fvm/errors" "github.com/onflow/flow-go/fvm/storage/snapshot" + "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/fvm/tracing" "github.com/onflow/flow-go/model/flow" ) @@ -24,10 +26,14 @@ func TestTransactionStorageLimiter(t *testing.T) { }, } + ctx := fvm.Context{ + EnvironmentParams: environment.EnvironmentParams{ + Chain: flow.Emulator.Chain(), + }, + } + t.Run("capacity > storage -> OK", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -40,13 +46,11 @@ func TestTransactionStorageLimiter(t *testing.T) { ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, flow.EmptyAddress, 0) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) require.NoError(t, err, "Transaction with higher capacity than storage used should work") }) t.Run("capacity = storage -> OK", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -59,13 +63,11 @@ func TestTransactionStorageLimiter(t *testing.T) { ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, flow.EmptyAddress, 0) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) require.NoError(t, err, "Transaction with equal capacity than storage used should work") }) t.Run("capacity = storage -> OK (dedup payer)", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -78,13 +80,11 @@ func TestTransactionStorageLimiter(t *testing.T) { ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, owner, 0) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, owner, 0) require.NoError(t, err, "Transaction with equal capacity than storage used should work") }) t.Run("capacity < storage -> Not OK", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -97,13 +97,11 @@ func TestTransactionStorageLimiter(t *testing.T) { ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, flow.EmptyAddress, 0) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) require.Error(t, err, "Transaction with lower capacity than storage used should fail") }) t.Run("capacity > storage -> OK (payer not updated)", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -118,13 +116,11 @@ func TestTransactionStorageLimiter(t *testing.T) { executionSnapshot = &snapshot.ExecutionSnapshot{} d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, owner, 1) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, owner, 1) require.NoError(t, err, "Transaction with higher capacity than storage used should work") }) t.Run("capacity < storage -> Not OK (payer not updated)", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -139,13 +135,11 @@ func TestTransactionStorageLimiter(t *testing.T) { executionSnapshot = &snapshot.ExecutionSnapshot{} d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, owner, 1000) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, owner, 1000) require.Error(t, err, "Transaction with lower capacity than storage used should fail") }) t.Run("if ctx LimitAccountStorage false-> OK", func(t *testing.T) { - chain := flow.Mainnet.Chain() env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(false) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) @@ -159,27 +153,63 @@ func TestTransactionStorageLimiter(t *testing.T) { ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, flow.EmptyAddress, 0) + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) require.NoError(t, err, "Transaction with higher capacity than storage used should work") }) - t.Run("non existing accounts or any other errors on fetching storage used -> Not OK", func(t *testing.T) { - chain := flow.Mainnet.Chain() + t.Run( + "non existing accounts or any other errors on fetching storage used -> Not OK", + func(t *testing.T) { + env := &fvmmock.Environment{} + env.On("LimitAccountStorage").Return(true) + env.On("StartChildSpan", mock.Anything).Return( + tracing.NewMockTracerSpan()) + env.On("GetStorageUsed", mock.Anything). + Return(uint64(0), errors.NewAccountNotFoundError(owner)) + env.On("AccountsStorageCapacity", mock.Anything, mock.Anything, mock.Anything).Return( + cadence.NewArray([]cadence.Value{ + bytesToUFix64(100), + }), + nil, + ) + + d := &fvm.TransactionStorageLimiter{} + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) + require.Error( + t, + err, + "check storage used on non existing account (not general registers) should fail", + ) + }, + ) + t.Run("special account is skipped", func(t *testing.T) { + sc := systemcontracts.SystemContractsForChain(ctx.Chain.ChainID()) + evm := sc.EVM.Address + + executionSnapshot := &snapshot.ExecutionSnapshot{ + WriteSet: map[flow.RegisterID]flow.RegisterValue{ + flow.NewRegisterID(evm, "a"): flow.RegisterValue("foo"), + }, + } + env := &fvmmock.Environment{} - env.On("Chain").Return(chain) env.On("LimitAccountStorage").Return(true) env.On("StartChildSpan", mock.Anything).Return( tracing.NewMockTracerSpan()) - env.On("GetStorageUsed", mock.Anything).Return(uint64(0), errors.NewAccountNotFoundError(owner)) - env.On("AccountsStorageCapacity", mock.Anything, mock.Anything, mock.Anything).Return( - cadence.NewArray([]cadence.Value{ - bytesToUFix64(100), - }), - nil, - ) + env.On("GetStorageUsed", mock.Anything). + Return(uint64(0), errors.NewAccountNotFoundError(owner)) + env.On("AccountsStorageCapacity", mock.Anything, mock.Anything, mock.Anything). + Run(func(args mock.Arguments) { + require.Len(t, args.Get(0).([]flow.Address), 0) + }). + Return( + // since the special account is skipped, the resulting array from AccountsStorageCapacity should be empty + cadence.NewArray([]cadence.Value{}), + nil, + ) d := &fvm.TransactionStorageLimiter{} - err := d.CheckStorageLimits(env, executionSnapshot, flow.EmptyAddress, 0) - require.Error(t, err, "check storage used on non existing account (not general registers) should fail") + err := d.CheckStorageLimits(ctx, env, executionSnapshot, flow.EmptyAddress, 0) + require.NoError(t, err) }) } diff --git a/go.mod b/go.mod index efbffb248bb..afe63d28217 100644 --- a/go.mod +++ b/go.mod @@ -7,14 +7,14 @@ require ( cloud.google.com/go/profiler v0.3.0 cloud.google.com/go/storage v1.30.1 github.com/antihax/optional v1.0.0 - github.com/aws/aws-sdk-go-v2/config v1.18.19 + github.com/aws/aws-sdk-go-v2/config v1.18.45 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 github.com/btcsuite/btcd/btcec/v2 v2.2.1 github.com/davecgh/go-spew v1.1.1 github.com/dgraph-io/badger/v2 v2.2007.4 github.com/ef-ds/deque v1.0.4 - github.com/ethereum/go-ethereum v1.12.0 + github.com/ethereum/go-ethereum v1.13.5 github.com/fxamacker/cbor/v2 v2.4.1-0.20230228173756-c0c9f774e40c github.com/gammazero/workerpool v1.1.2 github.com/gogo/protobuf v1.3.2 @@ -22,10 +22,10 @@ require ( github.com/golang/protobuf v1.5.3 github.com/google/go-cmp v0.5.9 github.com/google/pprof v0.0.0-20230602150820-91b7bce49751 - github.com/google/uuid v1.3.0 - github.com/gorilla/mux v1.8.0 + github.com/google/uuid v1.3.1 + github.com/gorilla/mux v1.8.1 github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 - github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-rc.2 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3 github.com/hashicorp/go-multierror v1.1.1 @@ -51,11 +51,11 @@ require ( github.com/multiformats/go-multiaddr-dns v0.3.1 github.com/multiformats/go-multihash v0.2.3 github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f - github.com/onflow/cadence v1.0.0-preview.2 + github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb github.com/onflow/flow v0.3.4 - github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 - github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 - github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 + github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b + github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b + github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca github.com/onflow/flow-go/crypto v0.25.0 github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d @@ -63,13 +63,13 @@ require ( github.com/pierrec/lz4 v2.6.1+incompatible github.com/pkg/errors v0.9.1 github.com/pkg/profile v1.7.0 - github.com/prometheus/client_golang v1.14.0 + github.com/prometheus/client_golang v1.16.0 github.com/rs/cors v1.8.0 github.com/rs/zerolog v1.29.0 github.com/schollz/progressbar/v3 v3.13.1 github.com/sethvargo/go-retry v0.2.3 github.com/shirou/gopsutil/v3 v3.22.2 - github.com/spf13/cobra v1.6.1 + github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 github.com/spf13/viper v1.15.0 github.com/stretchr/testify v1.8.4 @@ -81,16 +81,16 @@ require ( go.opentelemetry.io/otel/trace v1.16.0 go.uber.org/atomic v1.11.0 go.uber.org/multierr v1.11.0 - golang.org/x/crypto v0.12.0 - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 - golang.org/x/sync v0.3.0 - golang.org/x/sys v0.11.0 - golang.org/x/text v0.12.0 - golang.org/x/time v0.1.0 - golang.org/x/tools v0.9.1 + golang.org/x/crypto v0.17.0 + golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc + golang.org/x/sync v0.5.0 + golang.org/x/sys v0.15.0 + golang.org/x/text v0.14.0 + golang.org/x/time v0.3.0 + golang.org/x/tools v0.16.0 google.golang.org/api v0.126.0 - google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98 - google.golang.org/grpc v1.58.3 + google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d + google.golang.org/grpc v1.59.0 google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 google.golang.org/protobuf v1.31.0 gotest.tools v2.2.0+incompatible @@ -98,43 +98,46 @@ require ( ) require ( - github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 + github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 github.com/coreos/go-semver v0.3.0 github.com/go-playground/validator/v10 v10.14.1 github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb github.com/gorilla/websocket v1.5.0 github.com/hashicorp/golang-lru/v2 v2.0.2 github.com/mitchellh/mapstructure v1.5.0 - github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 + github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 + github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 github.com/onflow/wal v0.0.0-20230529184820-bc9f8244608d github.com/slok/go-http-metrics v0.10.0 github.com/sony/gobreaker v0.5.0 - google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98 + google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc gopkg.in/yaml.v2 v2.4.0 ) require ( - cloud.google.com/go v0.110.4 // indirect - cloud.google.com/go/compute v1.21.0 // indirect + cloud.google.com/go v0.110.7 // indirect + cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/iam v1.1.1 // indirect github.com/DataDog/zstd v1.5.2 // indirect - github.com/SaveTheRbtz/mph v0.1.2 // indirect - github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect - github.com/VictoriaMetrics/fastcache v1.6.0 // indirect - github.com/aws/aws-sdk-go-v2 v1.17.7 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.13.18 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 // indirect + github.com/Microsoft/go-winio v0.6.1 // indirect + github.com/OneOfOne/xxhash v1.2.8 // indirect + github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc // indirect + github.com/StackExchange/wmi v1.2.1 // indirect + github.com/VictoriaMetrics/fastcache v1.12.1 // indirect + github.com/aws/aws-sdk-go-v2 v1.21.2 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.13.43 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 // indirect - github.com/aws/smithy-go v1.13.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 // indirect + github.com/aws/smithy-go v1.15.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.7.0 // indirect @@ -146,18 +149,23 @@ require ( github.com/cockroachdb/errors v1.9.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.3 // indirect + github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/consensys/bavard v0.1.13 // indirect + github.com/consensys/gnark-crypto v0.12.1 // indirect github.com/containerd/cgroups v1.1.0 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/crate-crypto/go-kzg-4844 v0.7.0 // indirect github.com/cskr/pubsub v1.0.2 // indirect github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect github.com/deckarep/golang-set/v2 v2.1.0 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect - github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de // indirect + github.com/dgraph-io/ristretto v0.1.0 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/dustin/go-humanize v1.0.1 // indirect github.com/elastic/gosigar v0.14.2 // indirect + github.com/ethereum/c-kzg-4844 v0.4.0 // indirect github.com/felixge/fgprof v0.9.3 // indirect github.com/flynn/noise v1.0.0 // indirect github.com/francoispqt/gojay v1.2.13 // indirect @@ -178,7 +186,7 @@ require ( github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect github.com/godbus/dbus/v5 v5.1.0 // indirect github.com/gofrs/flock v0.8.1 // indirect - github.com/golang/glog v1.1.0 // indirect + github.com/golang/glog v1.1.2 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/google/gopacket v1.1.19 // indirect github.com/google/s2a-go v0.1.4 // indirect @@ -187,9 +195,9 @@ require ( github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/hcl v1.0.0 // indirect github.com/holiman/bloomfilter/v2 v2.0.3 // indirect - github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c // indirect - github.com/huin/goupnp v1.2.0 // indirect - github.com/inconshreveable/mousetrap v1.0.1 // indirect + github.com/holiman/uint256 v1.2.3 // indirect + github.com/huin/goupnp v1.3.0 // indirect + github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/ipfs/bbloom v0.0.4 // indirect github.com/ipfs/boxo v0.10.0 // indirect github.com/ipfs/go-bitswap v0.9.0 // indirect @@ -240,6 +248,7 @@ require ( github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc // indirect github.com/minio/sha256-simd v1.0.1 // indirect github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db // indirect + github.com/mmcloughlin/addchain v0.4.0 // indirect github.com/mr-tron/base58 v1.2.0 // indirect github.com/multiformats/go-base32 v0.1.0 // indirect github.com/multiformats/go-base36 v0.2.0 // indirect @@ -248,8 +257,8 @@ require ( github.com/multiformats/go-multicodec v0.9.0 // indirect github.com/multiformats/go-multistream v0.4.1 // indirect github.com/multiformats/go-varint v0.0.7 // indirect + github.com/nxadm/tail v1.4.8 // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect - github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 // indirect github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba // indirect github.com/onsi/ginkgo/v2 v2.9.7 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect @@ -260,7 +269,7 @@ require ( github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/prometheus/client_model v0.4.0 // indirect github.com/prometheus/common v0.42.0 // indirect - github.com/prometheus/procfs v0.9.0 // indirect + github.com/prometheus/procfs v0.10.1 // indirect github.com/psiemens/sconfig v0.1.0 // indirect github.com/quic-go/qpack v0.4.0 // indirect github.com/quic-go/qtls-go1-19 v0.3.2 // indirect @@ -277,17 +286,17 @@ require ( github.com/spf13/jwalterweatherman v1.1.0 // indirect github.com/stretchr/objx v0.5.0 // indirect github.com/subosito/gotenv v1.4.2 // indirect + github.com/supranational/blst v0.3.11 // indirect github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c // indirect - github.com/tklauser/go-sysconf v0.3.9 // indirect - github.com/tklauser/numcpus v0.3.0 // indirect + github.com/tklauser/go-sysconf v0.3.12 // indirect + github.com/tklauser/numcpus v0.6.1 // indirect github.com/turbolent/prettier v0.0.0-20220320183459-661cc755135d // indirect github.com/vmihailenco/tagparser v0.1.1 // indirect github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 // indirect github.com/x448/float16 v0.8.4 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect github.com/zeebo/blake3 v0.2.3 // indirect - github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.16.0 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.16.0 // indirect @@ -296,17 +305,17 @@ require ( go.uber.org/dig v1.17.0 // indirect go.uber.org/fx v1.19.2 // indirect go.uber.org/zap v1.24.0 // indirect - golang.org/x/mod v0.10.0 // indirect - golang.org/x/net v0.12.0 // indirect - golang.org/x/oauth2 v0.10.0 // indirect - golang.org/x/term v0.11.0 // indirect + golang.org/x/mod v0.14.0 // indirect + golang.org/x/net v0.19.0 // indirect + golang.org/x/oauth2 v0.11.0 // indirect + golang.org/x/term v0.15.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.13.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect gopkg.in/ini.v1 v1.67.0 // indirect - gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect gopkg.in/yaml.v3 v3.0.1 // indirect lukechampine.com/blake3 v1.2.1 // indirect nhooyr.io/websocket v1.8.7 // indirect + rsc.io/tmplfunc v0.0.3 // indirect ) diff --git a/go.sum b/go.sum index 27809ac9d1c..84a7600db63 100644 --- a/go.sum +++ b/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRY cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= -cloud.google.com/go v0.110.4 h1:1JYyxKMN9hd5dR2MYTPWkGUgcoxVVhg0LKNKEo0qvmk= -cloud.google.com/go v0.110.4/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= +cloud.google.com/go v0.110.7 h1:rJyC7nWRg2jWGZ4wSJ5nY65GTdYJkg0cd/uXb+ACI6o= +cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= @@ -155,8 +155,8 @@ cloud.google.com/go/compute v1.13.0/go.mod h1:5aPTS0cUNMIc1CE546K+Th6weJUNQErARy cloud.google.com/go/compute v1.14.0/go.mod h1:YfLtxrj9sU4Yxv+sXzZkyPjEyPBZfXHUvjxega5vAdo= cloud.google.com/go/compute v1.15.1/go.mod h1:bjjoF/NtFUrkD/urWfdHaKuOPDR5nWIs63rR+SXhcpA= cloud.google.com/go/compute v1.18.0/go.mod h1:1X7yHxec2Ga+Ss6jPyjxRxpu2uu7PLgsOVXvgU0yacs= -cloud.google.com/go/compute v1.21.0 h1:JNBsyXVoOoNJtTQcnEY5uYpZIbeCTYIeDe0Xh1bySMk= -cloud.google.com/go/compute v1.21.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= +cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZEXYonfTBHHFPO/4UU= cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= @@ -564,19 +564,23 @@ github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0 github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= github.com/Kubuxu/go-os-helper v0.0.1/go.mod h1:N8B+I7vPCT80IcP58r50u4+gEEcsZETFUpAzWW2ep1Y= +github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= +github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/OneOfOne/xxhash v1.2.5 h1:zl/OfRA6nftbBK9qTohYBJ5xvw6C/oNKizR7cZGl3cI= github.com/OneOfOne/xxhash v1.2.5/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= -github.com/SaveTheRbtz/mph v0.1.2 h1:5l3W496Up+7BNOVJQnJhzcGBh+wWfxWdmPUAkx3WmaM= -github.com/SaveTheRbtz/mph v0.1.2/go.mod h1:V4+WtKQPe2+dEA5os1WnGsEB0NR9qgqqgIiSt73+sT4= +github.com/OneOfOne/xxhash v1.2.8 h1:31czK/TI9sNkxIKfaUfGlU47BAxQ0ztGgd9vPyqimf8= +github.com/OneOfOne/xxhash v1.2.8/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc h1:DCHzPQOcU/7gwDTWbFQZc5qHMPS1g0xTO56k8NXsv9M= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc/go.mod h1:LJM5a3zcIJ/8TmZwlUczvROEJT8ntOdhdG9jjcR1B0I= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 h1:fLjPD/aNc3UIOA6tDi6QXUemppXK3P9BI7mr2hd6gx8= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/StackExchange/wmi v1.2.1 h1:VIkavFPXSjcnS+O8yTq7NI32k0R5Aj+v39y29VYDOSA= +github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= github.com/VictoriaMetrics/fastcache v1.5.3/go.mod h1:+jv9Ckb+za/P1ZRg/sulP5Ni1v49daAVERr0H3CuscE= -github.com/VictoriaMetrics/fastcache v1.6.0 h1:C/3Oi3EiBCqufydp1neRZkqcwmEiuRT9c3fqvvgKm5o= -github.com/VictoriaMetrics/fastcache v1.6.0/go.mod h1:0qHz5QP0GMX4pfmMA/zt5RgfNuXJrTP0zS7DqpHGGTw= +github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= +github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= @@ -612,49 +616,61 @@ github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/aws/aws-sdk-go-v2 v1.9.0/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4= github.com/aws/aws-sdk-go-v2 v1.17.3/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= -github.com/aws/aws-sdk-go-v2 v1.17.7 h1:CLSjnhJSTSogvqUGhIC6LqFKATMRexcxLZ0i/Nzk9Eg= github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= +github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/config v1.8.0/go.mod h1:w9+nMZ7soXCe5nT46Ri354SNhXDQ6v+V5wqDjnZE+GY= -github.com/aws/aws-sdk-go-v2/config v1.18.19 h1:AqFK6zFNtq4i1EYu+eC7lcKHYnZagMn6SW171la0bGw= github.com/aws/aws-sdk-go-v2/config v1.18.19/go.mod h1:XvTmGMY8d52ougvakOv1RpiTLPz9dlG/OQHsKU/cMmY= +github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= +github.com/aws/aws-sdk-go-v2/config v1.18.45/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.4.0/go.mod h1:dgGR+Qq7Wjcd4AOAW5Rf5Tnv3+x7ed6kETXyS9WCuAY= -github.com/aws/aws-sdk-go-v2/credentials v1.13.18 h1:EQMdtHwz0ILTW1hoP+EwuWhwCG1hD6l3+RWFQABET4c= github.com/aws/aws-sdk-go-v2/credentials v1.13.18/go.mod h1:vnwlwjIe+3XJPBYKu1et30ZPABG3VaXJYr8ryohpIyM= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.5.0/go.mod h1:CpNzHK9VEFUCknu50kkB8z58AH2B5DvPP7ea1LHve/Y= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 h1:gt57MN3liKiyGopcqgNzJb2+d9MJaKT/q1OksHNXVE4= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1/go.mod h1:lfUx8puBRdM5lVVMQlwt2v+ofiG/X6Ms+dy0UkG/kXw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 h1:VGkV9KmhGqOQWnHyi4gLG98kE6OecT42fdrCGFWxJsc= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1/go.mod h1:PLlnMiki//sGnCJiW+aVpvP/C8Kcm8mEj/IVm9+9qk4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.27/go.mod h1:a1/UpzeyBBerajpnP5nGZa9mGzsBn5cOKxm6NWQsvoI= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 h1:sJLYcS+eZn5EeNINGHSCRAwUJMFVqklwkH36Vbyai7M= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.21/go.mod h1:+Gxn8jYn5k9ebfHEqlhrMirFjSW0v0C9fI+KN5vk2kE= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 h1:1mnRASEKnkqsntcxHaysxwgVoUUp5dkiB+l3llKnqyg= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= github.com/aws/aws-sdk-go-v2/internal/ini v1.2.2/go.mod h1:BQV0agm+JEhqR+2RT5e1XTFIDcAAV0eW6z2trp+iduw= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 h1:p5luUImdIqywn6JpQsW3tq5GNOxKmOnEpybzPx+d1lk= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32/go.mod h1:XGhIBZDEgfqmFIugclZ6FU7v75nHhBDtzuB4xB/tEi4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 h1:hze8YsjSh8Wl1rYa1CJpRmXP21BvOBuc76YhW0HsuQ4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 h1:gceOysEWNNwLd6cki65IMBZ4WAM0MwgBQq2n7kejoT8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0/go.mod h1:v8ygadNyATSm6elwJ/4gzJwcFhri9RqS8skgHKiwXPU= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.3.0/go.mod h1:R1KK+vY8AfalhG1AOu5e35pOD2SdoPKQCFLTvnxiohk= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 h1:5LHn8JQ0qvjD9L9JhMtylnkcw7j05GDZqM9Oin6hpr0= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25/go.mod h1:/95IA+0lMnzW6XzqYJRpjjsAbKEORVeO0anQqjd2CNU= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 h1:WWZA/I2K4ptBS1kg0kV1JbBtG/umed0vwHRrmcr9z7k= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 h1:HWsM0YQWX76V6MOp07YuTYacm8k7h69ObJuw7Nck+og= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0/go.mod h1:LKb3cKNQIMh+itGnEpKGcnL/6OIjPZqrtYah1w5f+3o= github.com/aws/aws-sdk-go-v2/service/kms v1.20.1/go.mod h1:13sjgMH7Xu4e46+0BEDhSnNh+cImHSYS5PpBjV3oXcU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 h1:nPLfLPfglacc29Y949sDxpr3X/blaY40s3B85WT2yZU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0/go.mod h1:Iv2aJVtVSm/D22rFoX99cLG4q4uB7tppuCsulGe98k4= github.com/aws/aws-sdk-go-v2/service/sso v1.4.0/go.mod h1:+1fpWnL96DL23aXPpMGbsmKe8jLTEfbjuQoA4WS1VaA= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 h1:5V7DWLBd7wTELVz5bPpwzYy/sikk0gsgZfj40X+l5OI= github.com/aws/aws-sdk-go-v2/service/sso v1.12.6/go.mod h1:Y1VOmit/Fn6Tz1uFAeCO6Q7M2fmfXSCLeL5INVYsLuY= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 h1:B8cauxOH1W1v7rd8RdI/MWnoR4Ze0wIHWrb90qczxj4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 h1:JuPGc7IkOP4AaqcZSIcyqLpFSqBWK32rM9+a1g6u73k= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6/go.mod h1:Lh/bc9XUf8CfOY6Jp5aIkQtN+j1mc+nExc+KXj9jx2s= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 h1:HFiiRkf1SdaAmV3/BHOFZ9DjFynPHj8G/UIO1lQS+fk= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= github.com/aws/aws-sdk-go-v2/service/sts v1.7.0/go.mod h1:0qcSMCyASQPN2sk/1KQLQ2Fh6yq8wm0HSDAimPhzCoM= -github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 h1:bWNgNdRko2x6gqa0blfATqAZKZokPIeM1vfmQt2pnvM= github.com/aws/aws-sdk-go-v2/service/sts v1.18.7/go.mod h1:JuTnSoeePXmMVe9G8NcjjwgOKEfZ4cOjMuT2IBT/2eI= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwFYTCZVhlsSSBvlbU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= +github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -665,7 +681,6 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= @@ -697,7 +712,7 @@ github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtE github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46fmI40EZs= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/bytecodealliance/wasmtime-go/v7 v7.0.0/go.mod h1:bu6fic7trDt20w+LMooX7j3fsOwv4/ln6j8gAdP6vmA= -github.com/c-bata/go-prompt v0.2.5/go.mod h1:vFnjEGDIIA/Lib7giyE4E9c50Lvl8j0S+7FVlAwDAVw= +github.com/c-bata/go-prompt v0.2.6/go.mod h1:/LMAke8wD2FsNu9EXNdHxNLbd9MedkPnCdfpU9wwHfY= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= @@ -736,18 +751,24 @@ github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/datadriven v1.0.3-0.20230801171734-e384cf455877 h1:1MLK4YpFtIEo3ZtMA5C795Wtv5VuUnrXX7mQG+aHg6o= +github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/errors v1.9.1 h1:yFVvsI0VxmRShfawbt/laCIDy/mtTqqnvoNgiy5bEV8= github.com/cockroachdb/errors v1.9.1/go.mod h1:2sxOtL2WIc096WSZqZ5h8fa17rdDq9HZOZLBCor4mBk= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 h1:T+Np/xtzIjYM/P5NAw0e2Rf1FGvzDau1h54MKvx8G7w= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06/go.mod h1:bynZ3gvVyhlvjLI7PT6dmZ7g76xzJ7HpxfjgkzCGz6s= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= +github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= +github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= +github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= @@ -769,7 +790,9 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfc github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/crate-crypto/go-kzg-4844 v0.7.0 h1:C0vgZRk4q4EZ/JgPfzuSoxdCq3C3mOZMBShovmncxvA= +github.com/crate-crypto/go-kzg-4844 v0.7.0/go.mod h1:1kMhvPgI0Ky3yIa+9lFySEBUBXkYxeOi8ZF1sYioxhc= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cskr/pubsub v1.0.2 h1:vlOzMhl6PFn60gRlTQQsIfVwaPB/B/8MziK8FhEPt/0= @@ -809,8 +832,9 @@ github.com/dgraph-io/badger/v2 v2.2007.3/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDm github.com/dgraph-io/badger/v2 v2.2007.4 h1:TRWBQg8UrlUhaFdco01nO2uXwzKS7zd+HVdwV/GHc4o= github.com/dgraph-io/badger/v2 v2.2007.4/go.mod h1:vSw/ax2qojzbN6eXHIx6KPKtCSHJN/Uz0X0VPruTIhk= github.com/dgraph-io/ristretto v0.0.2/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= -github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de h1:t0UHb5vdojIDUqktM6+xJAfScFBsVpXZmqC9dsgJmeA= github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= +github.com/dgraph-io/ristretto v0.1.0 h1:Jv3CGQHp9OjuMBSne1485aDpUkTKEcUqF+jm/LuerPI= +github.com/dgraph-io/ristretto v0.1.0/go.mod h1:fux0lOrBhrVCJd3lcTHsIJhq1T2rokOu6v9Vcb3Q9ug= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= @@ -854,9 +878,11 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/ethereum/c-kzg-4844 v0.4.0 h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R3nlY= +github.com/ethereum/c-kzg-4844 v0.4.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/ethereum/go-ethereum v1.9.13/go.mod h1:qwN9d1GLyDh0N7Ab8bMGd0H9knaji2jOBm2RrMGjXls= -github.com/ethereum/go-ethereum v1.12.0 h1:bdnhLPtqETd4m3mS8BGMNvBTf36bO5bx/hxE2zljOa0= -github.com/ethereum/go-ethereum v1.12.0/go.mod h1:/oo2X/dZLJjf2mJ6YT9wcWxa4nNJDBKDBU6sFIpx1Gs= +github.com/ethereum/go-ethereum v1.13.5 h1:U6TCRciCqZRe4FPXmy1sMGxTfuk8P7u2UoinF3VbaFk= +github.com/ethereum/go-ethereum v1.13.5/go.mod h1:yMTu38GSuyxaYzQMViqNmQ1s3cE84abZexQmTgenWk0= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.3.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= @@ -938,6 +964,7 @@ github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= +github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= @@ -962,8 +989,6 @@ github.com/go-stack/stack v1.8.1 h1:ntEHSVwIt7PNXNpgPmVfMrNhLtgjlmnZha2kOpuRiDw= github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP3XYfe4= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= -github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= -github.com/go-test/deep v1.1.0/go.mod h1:5C2ZWiW0ErCdrYzpqxLbTX7MG14M9iiw8DgHncVwcsE= github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee h1:s+21KNqlpePfkah2I+gwHF8xmJWRjooY+5248k6m4A0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= @@ -971,8 +996,8 @@ github.com/gobwas/pool v0.2.0 h1:QEmUOlnSjWtnpRGHF3SauEiOsy82Cup83Vf2LcMlnc8= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2 h1:CoAavW/wd/kulfZmSIBt6p24n4j7tHgNVCjsfHVNUbo= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/godbus/dbus/v5 v5.0.3/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk= @@ -994,8 +1019,8 @@ github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzq github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= -github.com/golang/glog v1.1.0 h1:/d3pCKDPWNnvIWe0vVUpNP32qc8U3PDVxySP/y360qE= -github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= +github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= +github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -1097,12 +1122,14 @@ github.com/google/pprof v0.0.0-20230602150820-91b7bce49751/go.mod h1:Jh3hGz2jkYa github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.0.0-20220520183353-fd19c99a87aa/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= @@ -1131,8 +1158,8 @@ github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c h1:7lF+Vz0LqiRid github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= -github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.1-0.20190629185528-ae1634f6a989/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= @@ -1147,8 +1174,9 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de github.com/grpc-ecosystem/go-grpc-middleware v1.2.2/go.mod h1:EaizFBKfUKtMIF5iaDEhniwNedqGo9FuLFzppDr3uwI= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2 h1:uxUHSMwWDJ/9jVPHNumRC8WZOi3hrBL22ObVOoLg4ww= github.com/grpc-ecosystem/go-grpc-middleware/providers/zerolog/v2 v2.0.0-rc.2/go.mod h1:BL7w7qd2l/j9jgY6WMhYutfOFQc0I8RTVwtjpnAMoTM= -github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea h1:1Tk1IbruXbunEnaIZEFb+Hpv9BIZti3OxKwKn5wWyKk= github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbfea/go.mod h1:GugMBs30ZSAkckqXEAIEGyYdDH6EgqowG8ppA3Zt+AY= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-rc.2 h1:1aeRCnE2CkKYqyzBu0+B2lgTcZPc3ea2lGpijeHbI1c= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-rc.2/go.mod h1:GhphxcdlaRyAuBSvo6rV71BvQcvB/vuX8ugCyybuS2k= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= @@ -1193,14 +1221,14 @@ github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2p github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iURXE7ZOP9L9hSkA= -github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c h1:DZfsyhDK1hnSS5lH8l+JggqzEleHteTYfutAiVlSUM8= -github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw= +github.com/holiman/uint256 v1.2.3 h1:K8UWO1HUJpRMXBxbmaY1Y8IAMZC/RsKB+ArEnnK4l5o= +github.com/holiman/uint256 v1.2.3/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= github.com/huin/goupnp v0.0.0-20161224104101-679507af18f3/go.mod h1:MZ2ZmwcBpvOoJ22IJsc7va19ZwoheaBk43rKg12SKag= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= -github.com/huin/goupnp v1.2.0 h1:uOKW26NG1hsSSbXIZ1IR7XP9Gjd1U8pnLaCMgntmkmY= -github.com/huin/goupnp v1.2.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= +github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc= +github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/hydrogen18/memlistener v0.0.0-20200120041712-dcc25e7acd91/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= @@ -1211,8 +1239,8 @@ github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/C github.com/improbable-eng/grpc-web v0.15.0 h1:BN+7z6uNXZ1tQGcNAuaU1YjsLTApzkjt2tzCixLaUPQ= github.com/improbable-eng/grpc-web v0.15.0/go.mod h1:1sy9HKV4Jt9aEs9JSnkWlRJPuPtwNr0l57L4f878wP8= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/inconshreveable/mousetrap v1.0.1 h1:U3uMjPSQEBMNp1lFxmllqCPM6P5u/Xq7Pgzkat/bFNc= -github.com/inconshreveable/mousetrap v1.0.1/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= +github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/influxdata/influxdb v1.2.3-0.20180221223340-01288bdb0883/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= github.com/ipfs/bbloom v0.0.1/go.mod h1:oqo8CVWsJFMOZqTglBG4wydCE4IQA/G2/SEofB0rjUI= @@ -1709,7 +1737,6 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.6/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= @@ -1758,6 +1785,9 @@ github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:F github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iPfkHRY= +github.com/mmcloughlin/addchain v0.4.0/go.mod h1:A86O+tHqZLMNO4w6ZZ4FlVQEadcoqkyU72HC5wJ4RlU= +github.com/mmcloughlin/profile v0.1.1/go.mod h1:IhHD7q1ooxgwTgjxQYkACGA77oFTDdFVejUS1/tS/qU= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -1854,8 +1884,9 @@ github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OS github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= @@ -1864,36 +1895,32 @@ github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXW github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= -github.com/onflow/atree v0.5.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= -github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f h1:Z8/PgTqOgOg02MTRpTBYO2k16FE6z4wEOtaC2WBR9Xo= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f/go.mod h1:xvP61FoOs95K7IYdIYRnNcYQGf4nbF/uuJ0tHf4DRuM= -github.com/onflow/cadence v0.39.13-stable-cadence/go.mod h1:SxT8/IEkS1drFj2ofUEK9S6KyJ5GQbrm0LX4EFCp/7Q= -github.com/onflow/cadence v1.0.0-preview.2 h1:EYJiaYHU35AilrvbrfWhYO3thTCeWBX3aHyy5RPeEag= -github.com/onflow/cadence v1.0.0-preview.2/go.mod h1:JmTHN1dOmN5330TkvSC8P50RcteXegxQS5+Aw2zr7+I= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb h1:OpNQ8+ZPBg5DHchnZyiBySz/OQc4uIeptTm7cBfCvOA= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb/go.mod h1:odXGZZ/wGNA5mwT8bC9v8u8EXACHllB2ABSZK65TGL8= github.com/onflow/crypto v0.25.0 h1:BeWbLsh3ZD13Ej+Uky6kg1PL1ZIVBDVX+2MVBNwqddg= +github.com/onflow/crypto v0.25.0/go.mod h1:C8FbaX0x8y+FxWjbkHy0Q4EASCDR9bSPWZqlpCLYyVI= github.com/onflow/flow v0.3.4 h1:FXUWVdYB90f/rjNcY0Owo30gL790tiYff9Pb/sycXYE= github.com/onflow/flow v0.3.4/go.mod h1:lzyAYmbu1HfkZ9cfnL5/sjrrsnJiUU8fRL26CqLP7+c= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 h1:zvLHFxySeg61/dgp/IbvaN+k4BXPuAhBOslrPQjrX9Q= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425/go.mod h1:N+1bEs/159Efg75hSQIkb90FVinxUMxL/6mA3I6dXtQ= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 h1:5dDtY8iItVVvIY+YXbavGDMaVz4Gq7sq4ILF/cZb7/8= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86/go.mod h1:6XIbPht7u7ADszXSHI2APY+OL78XVaUU8+OdgWEElAY= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b h1:ggf5emTr01eHx8J9ME7BYVk6aa3CMg0fdssJL7i89lo= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:uQMLA8dV0qO4By7bcCGNGo2jHfqOkY9RP8H/iCltO+k= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b h1:7AIREWwdF+sQlFblzj3BjMu+X3MBVx0Bn9k3v5fikrk= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:HrqH9MDQF62pYeWmKkVVXUPC3xcgDvkV7gF77sjG9VI= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 h1:MTgrwXkiWwNysYpWGzWjc1n9w1nfXvizmGkSAuEY6jk= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596/go.mod h1:uugR8U8Rlk2Xbn1ne7WWkPIcLReOyyXeQ/6tBg2Lsu8= -github.com/onflow/flow-go-sdk v0.41.7-stable-cadence/go.mod h1:ejVN+bqcsTHVvRpDDJDoBNdmcxUfFMW4IvdTbMeQ/hQ= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 h1:vUVO6m85BiT8c50Oc8YGc3CU+sGqiKW9FZbmiRph2dU= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2/go.mod h1:mbLrR3MkYbi9LH3yasDj1jrR4QTR8vjRLVFCm4jMHn0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca h1:7yG8dYqMzWzTZWJ17dnBdS01UDlOBnf1dd1rWKcFdY0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca/go.mod h1:O5+TK1qs2c1R5X4TEQp4m2c/YhlCjwdW7bsRcUB1U8s= github.com/onflow/flow-go/crypto v0.24.7/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= github.com/onflow/flow-go/crypto v0.25.0 h1:6lmoiAQ3APCF+nV7f4f2AXL3PuDKqQiWqRJXmjrMEq4= github.com/onflow/flow-go/crypto v0.25.0/go.mod h1:OOb2vYcS8AOCajBClhHTJ0NKftFl1RQgTQ0+Vh4nbqk= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 h1:pHblchb5rKrpt1xmeKnMNq3utzBi2bvepHu7z1ZPWjw= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14/go.mod h1:OKA2xWNugqqbaFSmxdb1VWixtuqSdz/VK1MlbdcIUxw= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 h1:UPDAaWUag3epvX+yJ5IrFESQr1P53rDDqw+ShGxg5/k= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991/go.mod h1:CPLntX0UYLgZvvgoHfEwaaeSL3IS/eFzAdfbT0aIhb4= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20221202093946-932d1c70e288/go.mod h1:gQxYqCfkI8lpnKsmIjwtN2mV/N2PIwc1I+RUK4HPIc8= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 h1:+rT+UsfTR39JZO8ht2+4fkaWfHw74SCj1fyz1lWuX8A= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2/go.mod h1:NA2pX2nw8zuaxfKphhKsk00kWLwfd+tv8mS23YXO4Sk= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d h1:QcOAeEyF3iAUHv21LQ12sdcsr0yFrJGoGLyCAzYYtvI= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d/go.mod h1:GCPpiyRoHncdqPj++zPr9ZOYBX4hpJ0pYZRYqSE8VKk= -github.com/onflow/sdks v0.5.0/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba h1:rIehuhO6bj4FkwE4VzwEjX7MoAlOhUJENBJLqDqVxAo= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/wal v0.0.0-20230529184820-bc9f8244608d h1:gAEqYPn3DS83rHIKEpsajnppVD1+zwuYPFyeDVFaQvg= @@ -1960,7 +1987,7 @@ github.com/pkg/profile v1.7.0 h1:hnbDkaNWPCLMO9wGLdBFTIZvzDrDfBM2072E1S9gJkA= github.com/pkg/profile v1.7.0/go.mod h1:8Uer0jas47ZQMJ7VD+OHknK4YDY07LPUC6dEvqDjvNo= github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= -github.com/pkg/term v1.1.0/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= +github.com/pkg/term v1.2.0-beta.2/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/polydawn/refmt v0.0.0-20201211092308-30ac6d18308e/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= @@ -1977,8 +2004,8 @@ github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5Fsn github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= -github.com/prometheus/client_golang v1.14.0 h1:nJdhIvne2eSX/XRAFV9PcvFFRbrjbcTUj0VP62TMhnw= -github.com/prometheus/client_golang v1.14.0/go.mod h1:8vpkKitgIVNcqrRBWh1C4TIUQgYNtG/XQE4E/Zae36Y= +github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= +github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -2007,8 +2034,8 @@ github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+Gx github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.3.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI= -github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= +github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= +github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/prometheus/tsdb v0.6.2-0.20190402121629-4f204dcbc150/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/psiemens/sconfig v0.1.0 h1:xfWqW+TRpih7mXZIqKYTmpRhlZLQ1kbxV8EjllPv76s= @@ -2028,7 +2055,6 @@ github.com/raulk/go-watchdog v1.3.0/go.mod h1:fIvOnLbF0b0ZwkB9YU4mOW9Did//4vPZtD github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.2.1-0.20211004051800-57c86be7915a/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rjeczalik/notify v0.9.1/go.mod h1:rKwnCoCGeuQnwBtTSPL9Dad03Vh2n40ePRrjvIXnJho= @@ -2058,7 +2084,6 @@ github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g= -github.com/schollz/progressbar/v3 v3.8.3/go.mod h1:pWnVCjSBZsT2X3nx9HfRdnCDrpbevliMeoEVhStwHko= github.com/schollz/progressbar/v3 v3.13.1 h1:o8rySDYiQ59Mwzy2FELeHY5ZARXZTVJC7iHD6PEFUiE= github.com/schollz/progressbar/v3 v3.13.1/go.mod h1:xvrbki8kfT1fzWzBT/UZd9L6GA+jdL7HAgq2RFnO6fQ= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= @@ -2130,8 +2155,8 @@ github.com/spf13/cast v1.5.0 h1:rj3WzYc11XZaIZMPKmwP96zkFEnnAmV8s6XbB2aY32w= github.com/spf13/cast v1.5.0/go.mod h1:SpXXQ5YoyJw6s3/6cMTQuxvgRl3PCJiyaX9p6b155UU= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/cobra v1.6.1 h1:o94oiPyS4KD1mPy2fmcYYHHfCxLqYjJOhGsCHFZtEzA= -github.com/spf13/cobra v1.6.1/go.mod h1:IOw/AERYS7UzyrGinqmz6HLUo219MORXGxhbaJUqzrY= +github.com/spf13/cobra v1.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= +github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/jwalterweatherman v1.1.0 h1:ue6voC5bR5F8YxI5S67j9i582FU4Qvo2bmqnqMYADFk= github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= @@ -2170,6 +2195,8 @@ github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXl github.com/subosito/gotenv v1.4.2 h1:X1TuBLAMDFbaTAChgCBLu3DU3UPyELpnF2jjJ2cz/S8= github.com/subosito/gotenv v1.4.2/go.mod h1:ayKnFf/c6rvx/2iiLrJUk1e6plDbT3edrFNGqEflhK0= github.com/supranational/blst v0.3.10/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= +github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbeoEm4= +github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= github.com/syndtr/goleveldb v1.0.1-0.20190923125748-758128399b1d/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= @@ -2178,10 +2205,12 @@ github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cb github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c h1:HelZ2kAFadG0La9d+4htN4HzQ68Bm2iM9qKMSMES6xg= github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c/go.mod h1:JlzghshsemAMDGZLytTFY8C1JQxQPhnatWqNwUXjggo= github.com/tidwall/pretty v1.2.1/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= -github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2biQ= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/turbolent/prettier v0.0.0-20220320183459-661cc755135d h1:5JInRQbk5UBX8JfUvKh2oYTLMVwj3p6n+wapDDm7hko= @@ -2253,7 +2282,6 @@ github.com/zeebo/blake3 v0.2.3 h1:TFoLXsjeXqRNFxSbk35Dk4YtszE/MQQGK10BH4ptoTg= github.com/zeebo/blake3 v0.2.3/go.mod h1:mjJjZpnsyIVtVgTOSpJ9vmRE4wgDeyt2HU3qXvvKCaQ= github.com/zeebo/pcg v1.0.1 h1:lyqfGeWiv4ahac6ttHs+I5hwtH/+1mrhlCtVNQM2kHo= github.com/zeebo/pcg v1.0.1/go.mod h1:09F0S9iiKrwn9rlI5yjLkmrug154/YRW6KnnXVDM/l4= -github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -2359,15 +2387,15 @@ golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210506145944-38f3c27a63bf/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= -golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= -golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2384,9 +2412,8 @@ golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EH golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20200331195152-e8c3332aa8e5/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp v0.0.0-20221110155412-d0897a79cd37/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc h1:ao2WRsKSzW6KuUY9IWPwWahcHCgR0s52IfwutMfEbdM= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -2427,11 +2454,11 @@ golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.0/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.6.0/go.mod h1:4mET923SAdbXp2ki8ey+zGs1SLqsuM2Y0uvdZR/fUNI= golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.10.0 h1:lFO9qtOdlre5W1jxS3r/4szv2/6iXxScdzjoBMXNhYk= -golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.14.0 h1:dGoOF9QVLYng8IHTm7BAyWqCqSheQ5pYWGhzW00YJr0= +golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -2514,8 +2541,10 @@ golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= -golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= -golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= +golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -2546,8 +2575,8 @@ golang.org/x/oauth2 v0.0.0-20221014153046-6fdb5e3db783/go.mod h1:h4gKUeWbJ4rQPri golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= -golang.org/x/oauth2 v0.10.0 h1:zHCpF2Khkwy4mMB4bv0U37YtJdTGW8jI0glAApi0Kh8= -golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= +golang.org/x/oauth2 v0.11.0 h1:vPL4xzxBM4niKCW6g9whtaWVXTJf1U5e4aZxxFx/gbU= +golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk= golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852/go.mod h1:JLpeXjPJfIyPr5TlbXLkXWLhP8nz10XfvxElABhCtcw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -2565,8 +2594,9 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= +golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -2649,7 +2679,6 @@ golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210317225723-c4fcb01b228e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -2667,7 +2696,6 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210910150752-751e447fb3d0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -2698,19 +2726,23 @@ golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= -golang.org/x/term v0.11.0 h1:F9tnn/DA/Im8nCwm+fX+1/eBwi4qFjRT++MhtVC4ZX0= -golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= +golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= +golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2726,16 +2758,19 @@ golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.12.0 h1:k+n5B8goJNdU7hSvEtMUz3d1Q6D/XW4COJSJR6fN0mc= -golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.1.0 h1:xYY+Bajn2a7VBmTM5GikTmnK8ZuX8YgnQCqZpbBNtmA= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -2811,11 +2846,11 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.2.0/go.mod h1:y4OqIKeOV/fWJetJ8bXPU1sEVniLMIyDAZWeHdV+NTA= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= -golang.org/x/tools v0.9.1 h1:8WMNJAz3zrtPmnYC7ISf5dEn3MT0gY7jBJfw27yrrLo= -golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/tools v0.16.0 h1:GO788SKMRunPIBCXiQyo2AaexLstOrVhuAL5YwsckQM= +golang.org/x/tools v0.16.0/go.mod h1:kYVVN6I1mBNoB1OX+noeBjbRk4IUEPa7JJ+TJMEooJ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -3044,14 +3079,14 @@ google.golang.org/genproto v0.0.0-20230222225845-10f96fb3dbec/go.mod h1:3Dl5ZL0q google.golang.org/genproto v0.0.0-20230223222841-637eb2293923/go.mod h1:3Dl5ZL0q0isWJt+FVcfpQyirqemEuLAK/iFvg1UP1Hw= google.golang.org/genproto v0.0.0-20230303212802-e74f57abe488/go.mod h1:TvhZT5f700eVlTNwND1xoEZQeWTB2RY/65kplwl/bFA= google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= -google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98 h1:Z0hjGZePRE0ZBWotvtrwxFNrNE9CUAGtplaDK5NNI/g= -google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98/go.mod h1:S7mY02OqCJTD0E1OiQy1F72PWFB4bZJ87cAtLPYgDR0= -google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98 h1:FmF5cCW94Ij59cfpoLiwTgodWmm60eEV0CjlsVg2fuw= -google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= +google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d h1:VBu5YqKPv6XiJ199exd8Br+Aetz+o08F+PLMnwJQHAY= +google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d h1:DoPTO70H+bcDXcd39vOqb2viZxgqeBeSGtZ55yZU4/Q= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc h1:g3hIDl0jRNd9PPTs2uBzYuaD5mQuwOkZY0vSc0LR32o= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98 h1:bVf09lpb+OJbByTj913DRJioFFAjf/ZGxEz7MajTp2U= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d h1:uvYuEyMHKNt+lT4K3bN6fGswmK8qSvcreM3BwjDh+y4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= @@ -3100,8 +3135,8 @@ google.golang.org/grpc v1.50.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCD google.golang.org/grpc v1.50.1/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsAIPww= google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= -google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= -google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 h1:TLkBREm4nIsEcexnCjgQd5GQWaHcqMzwQV0TX9pq8S0= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0/go.mod h1:DNq5QpG7LJqD2AamLZ7zvKE0DEpVl2BSEVjFycAAjRY= @@ -3143,7 +3178,6 @@ gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce h1:+JknDZhAj8YMt7GC73Ei8pv4MzjDUNPHgQWJdtMAaDU= gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce/go.mod h1:5AcXVHNjg+BDxry382+8OKon8SEWiKktQR07RKPsv1c= gopkg.in/olebedev/go-duktape.v3 v3.0.0-20200316214253-d7b0ff38cac9/go.mod h1:uAJfkITjFhyEEuUfm7bsmCZRbW5WRq8s9EY8HZ6hCns= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= @@ -3227,6 +3261,8 @@ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8 rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= +rsc.io/tmplfunc v0.0.3 h1:53XFQh69AfOa8Tw0Jm7t+GV7KZhOi6jzsCzTtKbMvzU= +rsc.io/tmplfunc v0.0.3/go.mod h1:AG3sTPzElb1Io3Yg4voV9AGZJuleGAwaVRxL9M49PhA= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= diff --git a/insecure/go.mod b/insecure/go.mod index 648631e987b..94a3a1379ff 100644 --- a/insecure/go.mod +++ b/insecure/go.mod @@ -10,7 +10,7 @@ require ( github.com/libp2p/go-libp2p v0.28.1 github.com/libp2p/go-libp2p-pubsub v0.9.3 github.com/multiformats/go-multiaddr-dns v0.3.1 - github.com/onflow/flow-go v0.32.4-0.20231130134727-3c01c7f8966c + github.com/onflow/flow-go v0.33.2-0.20240122190738-254af677b873 github.com/onflow/flow-go/crypto v0.25.0 github.com/rs/zerolog v1.29.0 github.com/spf13/pflag v1.0.5 @@ -28,25 +28,26 @@ require ( cloud.google.com/go/iam v1.1.1 // indirect cloud.google.com/go/storage v1.30.1 // indirect github.com/DataDog/zstd v1.5.2 // indirect - github.com/SaveTheRbtz/mph v0.1.2 // indirect - github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect - github.com/VictoriaMetrics/fastcache v1.6.0 // indirect - github.com/aws/aws-sdk-go-v2 v1.17.7 // indirect - github.com/aws/aws-sdk-go-v2/config v1.18.19 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.13.18 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 // indirect + github.com/Microsoft/go-winio v0.6.1 // indirect + github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc // indirect + github.com/StackExchange/wmi v1.2.1 // indirect + github.com/VictoriaMetrics/fastcache v1.12.1 // indirect + github.com/aws/aws-sdk-go-v2 v1.21.2 // indirect + github.com/aws/aws-sdk-go-v2/config v1.18.45 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.13.43 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 // indirect github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 // indirect github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 // indirect - github.com/aws/smithy-go v1.13.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 // indirect + github.com/aws/smithy-go v1.15.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.7.0 // indirect @@ -57,11 +58,15 @@ require ( github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cockroachdb/errors v1.9.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect - github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 // indirect + github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 // indirect github.com/cockroachdb/redact v1.1.3 // indirect + github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/consensys/bavard v0.1.13 // indirect + github.com/consensys/gnark-crypto v0.12.1 // indirect github.com/containerd/cgroups v1.1.0 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/crate-crypto/go-kzg-4844 v0.7.0 // indirect github.com/cskr/pubsub v1.0.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect @@ -75,7 +80,8 @@ require ( github.com/dustin/go-humanize v1.0.1 // indirect github.com/ef-ds/deque v1.0.4 // indirect github.com/elastic/gosigar v0.14.2 // indirect - github.com/ethereum/go-ethereum v1.12.0 // indirect + github.com/ethereum/c-kzg-4844 v0.4.0 // indirect + github.com/ethereum/go-ethereum v1.13.5 // indirect github.com/flynn/noise v1.0.0 // indirect github.com/francoispqt/gojay v1.2.13 // indirect github.com/fsnotify/fsnotify v1.6.0 // indirect @@ -120,8 +126,8 @@ require ( github.com/hashicorp/golang-lru/v2 v2.0.2 // indirect github.com/hashicorp/hcl v1.0.0 // indirect github.com/holiman/bloomfilter/v2 v2.0.3 // indirect - github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c // indirect - github.com/huin/goupnp v1.2.0 // indirect + github.com/holiman/uint256 v1.2.3 // indirect + github.com/huin/goupnp v1.3.0 // indirect github.com/improbable-eng/grpc-web v0.15.0 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/ipfs/bbloom v0.0.4 // indirect @@ -186,6 +192,7 @@ require ( github.com/minio/sha256-simd v1.0.1 // indirect github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect + github.com/mmcloughlin/addchain v0.4.0 // indirect github.com/mr-tron/base58 v1.2.0 // indirect github.com/multiformats/go-base32 v0.1.0 // indirect github.com/multiformats/go-base36 v0.2.0 // indirect @@ -196,15 +203,14 @@ require ( github.com/multiformats/go-multihash v0.2.3 // indirect github.com/multiformats/go-multistream v0.4.1 // indirect github.com/multiformats/go-varint v0.0.7 // indirect - github.com/nxadm/tail v1.4.8 // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f // indirect - github.com/onflow/cadence v1.0.0-preview.2 // indirect - github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 // indirect - github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 // indirect + github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb // indirect + github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b // indirect + github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b // indirect github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 // indirect - github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 // indirect - github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 // indirect + github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca // indirect + github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 // indirect github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 // indirect github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d // indirect github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba // indirect @@ -247,10 +253,11 @@ require ( github.com/spf13/viper v1.15.0 // indirect github.com/stretchr/objx v0.5.0 // indirect github.com/subosito/gotenv v1.4.2 // indirect + github.com/supranational/blst v0.3.11 // indirect github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c // indirect - github.com/tklauser/go-sysconf v0.3.9 // indirect - github.com/tklauser/numcpus v0.3.0 // indirect + github.com/tklauser/go-sysconf v0.3.12 // indirect + github.com/tklauser/numcpus v0.6.1 // indirect github.com/turbolent/prettier v0.0.0-20220320183459-661cc755135d // indirect github.com/vmihailenco/msgpack v4.0.4+incompatible // indirect github.com/vmihailenco/msgpack/v4 v4.3.11 // indirect @@ -259,7 +266,6 @@ require ( github.com/x448/float16 v0.8.4 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect github.com/zeebo/blake3 v0.2.3 // indirect - github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/otel v1.16.0 // indirect go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.16.0 // indirect @@ -273,17 +279,17 @@ require ( go.uber.org/fx v1.19.2 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect - golang.org/x/crypto v0.12.0 // indirect - golang.org/x/exp v0.0.0-20230321023759-10a507213a29 // indirect - golang.org/x/mod v0.10.0 // indirect - golang.org/x/net v0.14.0 // indirect + golang.org/x/crypto v0.17.0 // indirect + golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc // indirect + golang.org/x/mod v0.14.0 // indirect + golang.org/x/net v0.19.0 // indirect golang.org/x/oauth2 v0.11.0 // indirect - golang.org/x/sync v0.3.0 // indirect - golang.org/x/sys v0.11.0 // indirect - golang.org/x/term v0.11.0 // indirect - golang.org/x/text v0.12.0 // indirect - golang.org/x/time v0.1.0 // indirect - golang.org/x/tools v0.9.1 // indirect + golang.org/x/sync v0.5.0 // indirect + golang.org/x/sys v0.15.0 // indirect + golang.org/x/term v0.15.0 // indirect + golang.org/x/text v0.14.0 // indirect + golang.org/x/time v0.3.0 // indirect + golang.org/x/tools v0.16.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.13.0 // indirect google.golang.org/api v0.126.0 // indirect @@ -293,10 +299,10 @@ require ( google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect - gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect gopkg.in/yaml.v3 v3.0.1 // indirect lukechampine.com/blake3 v1.2.1 // indirect nhooyr.io/websocket v1.8.7 // indirect + rsc.io/tmplfunc v0.0.3 // indirect ) replace github.com/onflow/flow-go => ../ diff --git a/insecure/go.sum b/insecure/go.sum index a8375623723..b54378e5a4b 100644 --- a/insecure/go.sum +++ b/insecure/go.sum @@ -563,19 +563,22 @@ github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0 github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= github.com/Kubuxu/go-os-helper v0.0.1/go.mod h1:N8B+I7vPCT80IcP58r50u4+gEEcsZETFUpAzWW2ep1Y= +github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= +github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/OneOfOne/xxhash v1.2.5 h1:zl/OfRA6nftbBK9qTohYBJ5xvw6C/oNKizR7cZGl3cI= github.com/OneOfOne/xxhash v1.2.5/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= -github.com/SaveTheRbtz/mph v0.1.2 h1:5l3W496Up+7BNOVJQnJhzcGBh+wWfxWdmPUAkx3WmaM= -github.com/SaveTheRbtz/mph v0.1.2/go.mod h1:V4+WtKQPe2+dEA5os1WnGsEB0NR9qgqqgIiSt73+sT4= +github.com/OneOfOne/xxhash v1.2.8 h1:31czK/TI9sNkxIKfaUfGlU47BAxQ0ztGgd9vPyqimf8= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc h1:DCHzPQOcU/7gwDTWbFQZc5qHMPS1g0xTO56k8NXsv9M= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc/go.mod h1:LJM5a3zcIJ/8TmZwlUczvROEJT8ntOdhdG9jjcR1B0I= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 h1:fLjPD/aNc3UIOA6tDi6QXUemppXK3P9BI7mr2hd6gx8= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/StackExchange/wmi v1.2.1 h1:VIkavFPXSjcnS+O8yTq7NI32k0R5Aj+v39y29VYDOSA= +github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= github.com/VictoriaMetrics/fastcache v1.5.3/go.mod h1:+jv9Ckb+za/P1ZRg/sulP5Ni1v49daAVERr0H3CuscE= -github.com/VictoriaMetrics/fastcache v1.6.0 h1:C/3Oi3EiBCqufydp1neRZkqcwmEiuRT9c3fqvvgKm5o= -github.com/VictoriaMetrics/fastcache v1.6.0/go.mod h1:0qHz5QP0GMX4pfmMA/zt5RgfNuXJrTP0zS7DqpHGGTw= +github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= +github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c= @@ -610,49 +613,61 @@ github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/aws/aws-sdk-go-v2 v1.9.0/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4= github.com/aws/aws-sdk-go-v2 v1.17.3/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= -github.com/aws/aws-sdk-go-v2 v1.17.7 h1:CLSjnhJSTSogvqUGhIC6LqFKATMRexcxLZ0i/Nzk9Eg= github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= +github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/config v1.8.0/go.mod h1:w9+nMZ7soXCe5nT46Ri354SNhXDQ6v+V5wqDjnZE+GY= -github.com/aws/aws-sdk-go-v2/config v1.18.19 h1:AqFK6zFNtq4i1EYu+eC7lcKHYnZagMn6SW171la0bGw= github.com/aws/aws-sdk-go-v2/config v1.18.19/go.mod h1:XvTmGMY8d52ougvakOv1RpiTLPz9dlG/OQHsKU/cMmY= +github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= +github.com/aws/aws-sdk-go-v2/config v1.18.45/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.4.0/go.mod h1:dgGR+Qq7Wjcd4AOAW5Rf5Tnv3+x7ed6kETXyS9WCuAY= -github.com/aws/aws-sdk-go-v2/credentials v1.13.18 h1:EQMdtHwz0ILTW1hoP+EwuWhwCG1hD6l3+RWFQABET4c= github.com/aws/aws-sdk-go-v2/credentials v1.13.18/go.mod h1:vnwlwjIe+3XJPBYKu1et30ZPABG3VaXJYr8ryohpIyM= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.5.0/go.mod h1:CpNzHK9VEFUCknu50kkB8z58AH2B5DvPP7ea1LHve/Y= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 h1:gt57MN3liKiyGopcqgNzJb2+d9MJaKT/q1OksHNXVE4= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1/go.mod h1:lfUx8puBRdM5lVVMQlwt2v+ofiG/X6Ms+dy0UkG/kXw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 h1:VGkV9KmhGqOQWnHyi4gLG98kE6OecT42fdrCGFWxJsc= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1/go.mod h1:PLlnMiki//sGnCJiW+aVpvP/C8Kcm8mEj/IVm9+9qk4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.27/go.mod h1:a1/UpzeyBBerajpnP5nGZa9mGzsBn5cOKxm6NWQsvoI= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 h1:sJLYcS+eZn5EeNINGHSCRAwUJMFVqklwkH36Vbyai7M= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.21/go.mod h1:+Gxn8jYn5k9ebfHEqlhrMirFjSW0v0C9fI+KN5vk2kE= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 h1:1mnRASEKnkqsntcxHaysxwgVoUUp5dkiB+l3llKnqyg= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= github.com/aws/aws-sdk-go-v2/internal/ini v1.2.2/go.mod h1:BQV0agm+JEhqR+2RT5e1XTFIDcAAV0eW6z2trp+iduw= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 h1:p5luUImdIqywn6JpQsW3tq5GNOxKmOnEpybzPx+d1lk= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32/go.mod h1:XGhIBZDEgfqmFIugclZ6FU7v75nHhBDtzuB4xB/tEi4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 h1:hze8YsjSh8Wl1rYa1CJpRmXP21BvOBuc76YhW0HsuQ4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 h1:gceOysEWNNwLd6cki65IMBZ4WAM0MwgBQq2n7kejoT8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0/go.mod h1:v8ygadNyATSm6elwJ/4gzJwcFhri9RqS8skgHKiwXPU= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.3.0/go.mod h1:R1KK+vY8AfalhG1AOu5e35pOD2SdoPKQCFLTvnxiohk= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 h1:5LHn8JQ0qvjD9L9JhMtylnkcw7j05GDZqM9Oin6hpr0= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25/go.mod h1:/95IA+0lMnzW6XzqYJRpjjsAbKEORVeO0anQqjd2CNU= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 h1:WWZA/I2K4ptBS1kg0kV1JbBtG/umed0vwHRrmcr9z7k= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 h1:HWsM0YQWX76V6MOp07YuTYacm8k7h69ObJuw7Nck+og= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0/go.mod h1:LKb3cKNQIMh+itGnEpKGcnL/6OIjPZqrtYah1w5f+3o= github.com/aws/aws-sdk-go-v2/service/kms v1.20.1/go.mod h1:13sjgMH7Xu4e46+0BEDhSnNh+cImHSYS5PpBjV3oXcU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 h1:nPLfLPfglacc29Y949sDxpr3X/blaY40s3B85WT2yZU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0/go.mod h1:Iv2aJVtVSm/D22rFoX99cLG4q4uB7tppuCsulGe98k4= github.com/aws/aws-sdk-go-v2/service/sso v1.4.0/go.mod h1:+1fpWnL96DL23aXPpMGbsmKe8jLTEfbjuQoA4WS1VaA= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 h1:5V7DWLBd7wTELVz5bPpwzYy/sikk0gsgZfj40X+l5OI= github.com/aws/aws-sdk-go-v2/service/sso v1.12.6/go.mod h1:Y1VOmit/Fn6Tz1uFAeCO6Q7M2fmfXSCLeL5INVYsLuY= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 h1:B8cauxOH1W1v7rd8RdI/MWnoR4Ze0wIHWrb90qczxj4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 h1:JuPGc7IkOP4AaqcZSIcyqLpFSqBWK32rM9+a1g6u73k= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6/go.mod h1:Lh/bc9XUf8CfOY6Jp5aIkQtN+j1mc+nExc+KXj9jx2s= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 h1:HFiiRkf1SdaAmV3/BHOFZ9DjFynPHj8G/UIO1lQS+fk= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= github.com/aws/aws-sdk-go-v2/service/sts v1.7.0/go.mod h1:0qcSMCyASQPN2sk/1KQLQ2Fh6yq8wm0HSDAimPhzCoM= -github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 h1:bWNgNdRko2x6gqa0blfATqAZKZokPIeM1vfmQt2pnvM= github.com/aws/aws-sdk-go-v2/service/sts v1.18.7/go.mod h1:JuTnSoeePXmMVe9G8NcjjwgOKEfZ4cOjMuT2IBT/2eI= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwFYTCZVhlsSSBvlbU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= +github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -663,7 +678,6 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= @@ -695,7 +709,7 @@ github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtE github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46fmI40EZs= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/bytecodealliance/wasmtime-go/v7 v7.0.0/go.mod h1:bu6fic7trDt20w+LMooX7j3fsOwv4/ln6j8gAdP6vmA= -github.com/c-bata/go-prompt v0.2.5/go.mod h1:vFnjEGDIIA/Lib7giyE4E9c50Lvl8j0S+7FVlAwDAVw= +github.com/c-bata/go-prompt v0.2.6/go.mod h1:/LMAke8wD2FsNu9EXNdHxNLbd9MedkPnCdfpU9wwHfY= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= @@ -734,18 +748,24 @@ github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/datadriven v1.0.3-0.20230801171734-e384cf455877 h1:1MLK4YpFtIEo3ZtMA5C795Wtv5VuUnrXX7mQG+aHg6o= +github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/errors v1.9.1 h1:yFVvsI0VxmRShfawbt/laCIDy/mtTqqnvoNgiy5bEV8= github.com/cockroachdb/errors v1.9.1/go.mod h1:2sxOtL2WIc096WSZqZ5h8fa17rdDq9HZOZLBCor4mBk= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 h1:T+Np/xtzIjYM/P5NAw0e2Rf1FGvzDau1h54MKvx8G7w= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06/go.mod h1:bynZ3gvVyhlvjLI7PT6dmZ7g76xzJ7HpxfjgkzCGz6s= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= +github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= +github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= +github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= @@ -768,6 +788,8 @@ github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwc github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/crate-crypto/go-kzg-4844 v0.7.0 h1:C0vgZRk4q4EZ/JgPfzuSoxdCq3C3mOZMBShovmncxvA= +github.com/crate-crypto/go-kzg-4844 v0.7.0/go.mod h1:1kMhvPgI0Ky3yIa+9lFySEBUBXkYxeOi8ZF1sYioxhc= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cskr/pubsub v1.0.2 h1:vlOzMhl6PFn60gRlTQQsIfVwaPB/B/8MziK8FhEPt/0= @@ -853,9 +875,11 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/ethereum/c-kzg-4844 v0.4.0 h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R3nlY= +github.com/ethereum/c-kzg-4844 v0.4.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/ethereum/go-ethereum v1.9.13/go.mod h1:qwN9d1GLyDh0N7Ab8bMGd0H9knaji2jOBm2RrMGjXls= -github.com/ethereum/go-ethereum v1.12.0 h1:bdnhLPtqETd4m3mS8BGMNvBTf36bO5bx/hxE2zljOa0= -github.com/ethereum/go-ethereum v1.12.0/go.mod h1:/oo2X/dZLJjf2mJ6YT9wcWxa4nNJDBKDBU6sFIpx1Gs= +github.com/ethereum/go-ethereum v1.13.5 h1:U6TCRciCqZRe4FPXmy1sMGxTfuk8P7u2UoinF3VbaFk= +github.com/ethereum/go-ethereum v1.13.5/go.mod h1:yMTu38GSuyxaYzQMViqNmQ1s3cE84abZexQmTgenWk0= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.3.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= @@ -935,6 +959,7 @@ github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= +github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= @@ -959,8 +984,6 @@ github.com/go-stack/stack v1.8.1 h1:ntEHSVwIt7PNXNpgPmVfMrNhLtgjlmnZha2kOpuRiDw= github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP3XYfe4= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= -github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= -github.com/go-test/deep v1.1.0/go.mod h1:5C2ZWiW0ErCdrYzpqxLbTX7MG14M9iiw8DgHncVwcsE= github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee h1:s+21KNqlpePfkah2I+gwHF8xmJWRjooY+5248k6m4A0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= @@ -968,8 +991,8 @@ github.com/gobwas/pool v0.2.0 h1:QEmUOlnSjWtnpRGHF3SauEiOsy82Cup83Vf2LcMlnc8= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2 h1:CoAavW/wd/kulfZmSIBt6p24n4j7tHgNVCjsfHVNUbo= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/godbus/dbus/v5 v5.0.3/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk= @@ -1092,6 +1115,7 @@ github.com/google/pprof v0.0.0-20230602150820-91b7bce49751/go.mod h1:Jh3hGz2jkYa github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -1190,14 +1214,14 @@ github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2p github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iURXE7ZOP9L9hSkA= -github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c h1:DZfsyhDK1hnSS5lH8l+JggqzEleHteTYfutAiVlSUM8= -github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw= +github.com/holiman/uint256 v1.2.3 h1:K8UWO1HUJpRMXBxbmaY1Y8IAMZC/RsKB+ArEnnK4l5o= +github.com/holiman/uint256 v1.2.3/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= github.com/huin/goupnp v0.0.0-20161224104101-679507af18f3/go.mod h1:MZ2ZmwcBpvOoJ22IJsc7va19ZwoheaBk43rKg12SKag= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= -github.com/huin/goupnp v1.2.0 h1:uOKW26NG1hsSSbXIZ1IR7XP9Gjd1U8pnLaCMgntmkmY= -github.com/huin/goupnp v1.2.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= +github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc= +github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/hydrogen18/memlistener v0.0.0-20200120041712-dcc25e7acd91/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= @@ -1704,7 +1728,6 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.6/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= @@ -1753,6 +1776,9 @@ github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:F github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iPfkHRY= +github.com/mmcloughlin/addchain v0.4.0/go.mod h1:A86O+tHqZLMNO4w6ZZ4FlVQEadcoqkyU72HC5wJ4RlU= +github.com/mmcloughlin/profile v0.1.1/go.mod h1:IhHD7q1ooxgwTgjxQYkACGA77oFTDdFVejUS1/tS/qU= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -1849,7 +1875,6 @@ github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs= github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= @@ -1858,34 +1883,30 @@ github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXW github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= -github.com/onflow/atree v0.5.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= -github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f h1:Z8/PgTqOgOg02MTRpTBYO2k16FE6z4wEOtaC2WBR9Xo= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f/go.mod h1:xvP61FoOs95K7IYdIYRnNcYQGf4nbF/uuJ0tHf4DRuM= -github.com/onflow/cadence v0.39.13-stable-cadence/go.mod h1:SxT8/IEkS1drFj2ofUEK9S6KyJ5GQbrm0LX4EFCp/7Q= -github.com/onflow/cadence v1.0.0-preview.2 h1:EYJiaYHU35AilrvbrfWhYO3thTCeWBX3aHyy5RPeEag= -github.com/onflow/cadence v1.0.0-preview.2/go.mod h1:JmTHN1dOmN5330TkvSC8P50RcteXegxQS5+Aw2zr7+I= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb h1:OpNQ8+ZPBg5DHchnZyiBySz/OQc4uIeptTm7cBfCvOA= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb/go.mod h1:odXGZZ/wGNA5mwT8bC9v8u8EXACHllB2ABSZK65TGL8= github.com/onflow/crypto v0.25.0 h1:BeWbLsh3ZD13Ej+Uky6kg1PL1ZIVBDVX+2MVBNwqddg= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 h1:zvLHFxySeg61/dgp/IbvaN+k4BXPuAhBOslrPQjrX9Q= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425/go.mod h1:N+1bEs/159Efg75hSQIkb90FVinxUMxL/6mA3I6dXtQ= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 h1:5dDtY8iItVVvIY+YXbavGDMaVz4Gq7sq4ILF/cZb7/8= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86/go.mod h1:6XIbPht7u7ADszXSHI2APY+OL78XVaUU8+OdgWEElAY= +github.com/onflow/crypto v0.25.0/go.mod h1:C8FbaX0x8y+FxWjbkHy0Q4EASCDR9bSPWZqlpCLYyVI= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b h1:ggf5emTr01eHx8J9ME7BYVk6aa3CMg0fdssJL7i89lo= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:uQMLA8dV0qO4By7bcCGNGo2jHfqOkY9RP8H/iCltO+k= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b h1:7AIREWwdF+sQlFblzj3BjMu+X3MBVx0Bn9k3v5fikrk= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:HrqH9MDQF62pYeWmKkVVXUPC3xcgDvkV7gF77sjG9VI= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 h1:MTgrwXkiWwNysYpWGzWjc1n9w1nfXvizmGkSAuEY6jk= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596/go.mod h1:uugR8U8Rlk2Xbn1ne7WWkPIcLReOyyXeQ/6tBg2Lsu8= -github.com/onflow/flow-go-sdk v0.41.7-stable-cadence/go.mod h1:ejVN+bqcsTHVvRpDDJDoBNdmcxUfFMW4IvdTbMeQ/hQ= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 h1:vUVO6m85BiT8c50Oc8YGc3CU+sGqiKW9FZbmiRph2dU= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2/go.mod h1:mbLrR3MkYbi9LH3yasDj1jrR4QTR8vjRLVFCm4jMHn0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca h1:7yG8dYqMzWzTZWJ17dnBdS01UDlOBnf1dd1rWKcFdY0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca/go.mod h1:O5+TK1qs2c1R5X4TEQp4m2c/YhlCjwdW7bsRcUB1U8s= github.com/onflow/flow-go/crypto v0.24.7/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= github.com/onflow/flow-go/crypto v0.25.0 h1:6lmoiAQ3APCF+nV7f4f2AXL3PuDKqQiWqRJXmjrMEq4= github.com/onflow/flow-go/crypto v0.25.0/go.mod h1:OOb2vYcS8AOCajBClhHTJ0NKftFl1RQgTQ0+Vh4nbqk= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 h1:pHblchb5rKrpt1xmeKnMNq3utzBi2bvepHu7z1ZPWjw= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14/go.mod h1:OKA2xWNugqqbaFSmxdb1VWixtuqSdz/VK1MlbdcIUxw= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 h1:UPDAaWUag3epvX+yJ5IrFESQr1P53rDDqw+ShGxg5/k= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991/go.mod h1:CPLntX0UYLgZvvgoHfEwaaeSL3IS/eFzAdfbT0aIhb4= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20221202093946-932d1c70e288/go.mod h1:gQxYqCfkI8lpnKsmIjwtN2mV/N2PIwc1I+RUK4HPIc8= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 h1:+rT+UsfTR39JZO8ht2+4fkaWfHw74SCj1fyz1lWuX8A= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2/go.mod h1:NA2pX2nw8zuaxfKphhKsk00kWLwfd+tv8mS23YXO4Sk= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d h1:QcOAeEyF3iAUHv21LQ12sdcsr0yFrJGoGLyCAzYYtvI= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d/go.mod h1:GCPpiyRoHncdqPj++zPr9ZOYBX4hpJ0pYZRYqSE8VKk= -github.com/onflow/sdks v0.5.0/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba h1:rIehuhO6bj4FkwE4VzwEjX7MoAlOhUJENBJLqDqVxAo= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/wal v0.0.0-20230529184820-bc9f8244608d h1:gAEqYPn3DS83rHIKEpsajnppVD1+zwuYPFyeDVFaQvg= @@ -1950,7 +1971,7 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= -github.com/pkg/term v1.1.0/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= +github.com/pkg/term v1.2.0-beta.2/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/polydawn/refmt v0.0.0-20201211092308-30ac6d18308e/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= @@ -2018,7 +2039,6 @@ github.com/raulk/go-watchdog v1.3.0/go.mod h1:fIvOnLbF0b0ZwkB9YU4mOW9Did//4vPZtD github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.2.1-0.20211004051800-57c86be7915a/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rjeczalik/notify v0.9.1/go.mod h1:rKwnCoCGeuQnwBtTSPL9Dad03Vh2n40ePRrjvIXnJho= @@ -2048,7 +2068,6 @@ github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g= -github.com/schollz/progressbar/v3 v3.8.3/go.mod h1:pWnVCjSBZsT2X3nx9HfRdnCDrpbevliMeoEVhStwHko= github.com/schollz/progressbar/v3 v3.13.1 h1:o8rySDYiQ59Mwzy2FELeHY5ZARXZTVJC7iHD6PEFUiE= github.com/schollz/progressbar/v3 v3.13.1/go.mod h1:xvrbki8kfT1fzWzBT/UZd9L6GA+jdL7HAgq2RFnO6fQ= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= @@ -2160,6 +2179,8 @@ github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXl github.com/subosito/gotenv v1.4.2 h1:X1TuBLAMDFbaTAChgCBLu3DU3UPyELpnF2jjJ2cz/S8= github.com/subosito/gotenv v1.4.2/go.mod h1:ayKnFf/c6rvx/2iiLrJUk1e6plDbT3edrFNGqEflhK0= github.com/supranational/blst v0.3.10/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= +github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbeoEm4= +github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= github.com/syndtr/goleveldb v1.0.1-0.20190923125748-758128399b1d/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= @@ -2168,10 +2189,12 @@ github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cb github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c h1:HelZ2kAFadG0La9d+4htN4HzQ68Bm2iM9qKMSMES6xg= github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c/go.mod h1:JlzghshsemAMDGZLytTFY8C1JQxQPhnatWqNwUXjggo= github.com/tidwall/pretty v1.2.1/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= -github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= -github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2biQ= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/turbolent/prettier v0.0.0-20220320183459-661cc755135d h1:5JInRQbk5UBX8JfUvKh2oYTLMVwj3p6n+wapDDm7hko= @@ -2245,7 +2268,6 @@ github.com/zeebo/blake3 v0.2.3 h1:TFoLXsjeXqRNFxSbk35Dk4YtszE/MQQGK10BH4ptoTg= github.com/zeebo/blake3 v0.2.3/go.mod h1:mjJjZpnsyIVtVgTOSpJ9vmRE4wgDeyt2HU3qXvvKCaQ= github.com/zeebo/pcg v1.0.1 h1:lyqfGeWiv4ahac6ttHs+I5hwtH/+1mrhlCtVNQM2kHo= github.com/zeebo/pcg v1.0.1/go.mod h1:09F0S9iiKrwn9rlI5yjLkmrug154/YRW6KnnXVDM/l4= -github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -2351,15 +2373,15 @@ golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210506145944-38f3c27a63bf/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= -golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= -golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2376,9 +2398,8 @@ golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EH golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20200331195152-e8c3332aa8e5/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp v0.0.0-20221110155412-d0897a79cd37/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29 h1:ooxPy7fPvB4kwsA2h+iBNHkAbp/4JxTSwCmvdjEYmug= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc h1:ao2WRsKSzW6KuUY9IWPwWahcHCgR0s52IfwutMfEbdM= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -2419,11 +2440,11 @@ golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.0/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.6.0/go.mod h1:4mET923SAdbXp2ki8ey+zGs1SLqsuM2Y0uvdZR/fUNI= golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.10.0 h1:lFO9qtOdlre5W1jxS3r/4szv2/6iXxScdzjoBMXNhYk= -golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.14.0 h1:dGoOF9QVLYng8IHTm7BAyWqCqSheQ5pYWGhzW00YJr0= +golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -2506,8 +2527,10 @@ golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= -golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= -golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= +golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -2557,8 +2580,9 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= +golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -2641,7 +2665,6 @@ golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210317225723-c4fcb01b228e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -2659,7 +2682,6 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210910150752-751e447fb3d0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -2690,19 +2712,23 @@ golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= -golang.org/x/term v0.11.0 h1:F9tnn/DA/Im8nCwm+fX+1/eBwi4qFjRT++MhtVC4ZX0= -golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= +golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= +golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2718,16 +2744,19 @@ golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.12.0 h1:k+n5B8goJNdU7hSvEtMUz3d1Q6D/XW4COJSJR6fN0mc= -golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.1.0 h1:xYY+Bajn2a7VBmTM5GikTmnK8ZuX8YgnQCqZpbBNtmA= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -2803,11 +2832,11 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.2.0/go.mod h1:y4OqIKeOV/fWJetJ8bXPU1sEVniLMIyDAZWeHdV+NTA= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= -golang.org/x/tools v0.9.1 h1:8WMNJAz3zrtPmnYC7ISf5dEn3MT0gY7jBJfw27yrrLo= -golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/tools v0.16.0 h1:GO788SKMRunPIBCXiQyo2AaexLstOrVhuAL5YwsckQM= +golang.org/x/tools v0.16.0/go.mod h1:kYVVN6I1mBNoB1OX+noeBjbRk4IUEPa7JJ+TJMEooJ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -3134,7 +3163,6 @@ gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce h1:+JknDZhAj8YMt7GC73Ei8pv4MzjDUNPHgQWJdtMAaDU= gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce/go.mod h1:5AcXVHNjg+BDxry382+8OKon8SEWiKktQR07RKPsv1c= gopkg.in/olebedev/go-duktape.v3 v3.0.0-20200316214253-d7b0ff38cac9/go.mod h1:uAJfkITjFhyEEuUfm7bsmCZRbW5WRq8s9EY8HZ6hCns= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= @@ -3217,6 +3245,8 @@ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8 rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= +rsc.io/tmplfunc v0.0.3 h1:53XFQh69AfOa8Tw0Jm7t+GV7KZhOi6jzsCzTtKbMvzU= +rsc.io/tmplfunc v0.0.3/go.mod h1:AG3sTPzElb1Io3Yg4voV9AGZJuleGAwaVRxL9M49PhA= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/metrics_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/metrics_inspector_test.go deleted file mode 100644 index baafcc70f87..00000000000 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/metrics_inspector_test.go +++ /dev/null @@ -1,87 +0,0 @@ -package rpc_inspector - -import ( - "context" - "testing" - "time" - - pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "go.uber.org/atomic" - - "github.com/onflow/flow-go/insecure/corruptlibp2p" - "github.com/onflow/flow-go/insecure/internal" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/network/channels" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/inspector" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" - p2ptest "github.com/onflow/flow-go/network/p2p/test" - "github.com/onflow/flow-go/utils/unittest" -) - -// TestMetricsInspector_ObserveRPC ensures that the gossipsub rpc metrics inspector observes metrics for control messages as expected. -func TestMetricsInspector_ObserveRPC(t *testing.T) { - role := flow.RoleConsensus - sporkID := unittest.IdentifierFixture() - idProvider := unittest.NewUpdatableIDProvider(flow.IdentityList{}) - spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) - ctx, cancel := context.WithCancel(context.Background()) - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - - messageCount := 100 - controlMessageCount := 5 - - metricsObservedCount := atomic.NewInt64(0) - mockMetricsObserver := mockp2p.NewGossipSubControlMetricsObserver(t) - mockMetricsObserver.On("ObserveRPC", mock.Anything, mock.Anything). - Run(func(args mock.Arguments) { - peerID, ok := args.Get(0).(peer.ID) - require.True(t, ok) - require.Equal(t, spammer.SpammerNode.ID(), peerID) - rpc, ok := args.Get(1).(*pubsub.RPC) - require.True(t, ok) - // there are some default rpc messages exchanged between the nodes on startup - // we can ignore those rpc messages not configured directly by this test - if len(rpc.GetControl().GetPrune()) != 100 { - return - } - require.True(t, messageCount == len(rpc.GetControl().GetPrune())) - require.True(t, messageCount == len(rpc.GetControl().GetGraft())) - require.True(t, messageCount == len(rpc.GetControl().GetIhave())) - metricsObservedCount.Inc() - }) - metricsInspector := inspector.NewControlMsgMetricsInspector(unittest.Logger(), mockMetricsObserver, 2) - corruptInspectorFunc := corruptlibp2p.CorruptInspectorFunc(metricsInspector) - victimNode, victimIdentity := p2ptest.NodeFixture( - t, - sporkID, - t.Name(), - idProvider, - p2ptest.WithRole(role), - internal.WithCorruptGossipSub(corruptlibp2p.CorruptGossipSubFactory(), - corruptlibp2p.CorruptGossipSubConfigFactoryWithInspector(corruptInspectorFunc)), - ) - idProvider.SetIdentities(flow.IdentityList{&victimIdentity, &spammer.SpammerId}) - metricsInspector.Start(signalerCtx) - nodes := []p2p.LibP2PNode{victimNode, spammer.SpammerNode} - startNodesAndEnsureConnected(t, signalerCtx, nodes, sporkID) - spammer.Start(t) - defer stopComponents(t, cancel, nodes, metricsInspector) - // prepare to spam - generate control messages - ctlMsgs := spammer.GenerateCtlMessages(controlMessageCount, - p2ptest.WithGraft(messageCount, channels.PushBlocks.String()), - p2ptest.WithPrune(messageCount, channels.PushBlocks.String()), - p2ptest.WithIHave(messageCount, 1000, channels.PushBlocks.String())) - - // start spamming the victim peer - spammer.SpamControlMessage(t, victimNode, ctlMsgs) - - // eventually we should process each spammed control message and observe metrics for them - require.Eventually(t, func() bool { - return metricsObservedCount.Load() == int64(controlMessageCount) - }, 5*time.Second, 10*time.Millisecond, "did not observe metrics for all control messages on time") -} diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go index b66b6259535..6a3e168f0f4 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go @@ -28,7 +28,6 @@ import ( "github.com/onflow/flow-go/network/p2p/inspector/validation" p2pmsg "github.com/onflow/flow-go/network/p2p/message" mockp2p "github.com/onflow/flow-go/network/p2p/mock" - "github.com/onflow/flow-go/network/p2p/scoring" p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/utils/unittest" ) @@ -1024,7 +1023,7 @@ func TestGossipSubSpamMitigationIntegration(t *testing.T) { require.NoError(t, err) // set the scoring parameters to be more aggressive to speed up the test cfg.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 100 * time.Millisecond - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond victimNode, victimId := p2ptest.NodeFixture(t, sporkID, t.Name(), @@ -1105,11 +1104,11 @@ func TestGossipSubSpamMitigationIntegration(t *testing.T) { spammer.SpamControlMessage(t, victimNode, pruneCtlMsgsWithMalformedTopic) spammer.SpamControlMessage(t, victimNode, pruneCtlMsgsInvalidSporkIDTopic) spammer.SpamControlMessage(t, victimNode, pruneCtlMsgsDuplicateTopic) - + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds // wait for three GossipSub heartbeat intervals to ensure that the victim node has penalized the spammer node. require.Eventually(t, func() bool { score, ok := victimNode.PeerScoreExposer().GetScore(spammer.SpammerNode.ID()) - return ok && score < 2*scoring.DefaultGraylistThreshold + return ok && score < 2*scoreOptParameters.Graylist }, 5*time.Second, 100*time.Millisecond, "expected victim node to penalize spammer node") // now we expect the detection and mitigation to kick in and the victim node to disconnect from the spammer node. diff --git a/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go b/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go index e11951cc7dd..8d6cfec2cef 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go @@ -18,7 +18,6 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/scoring" p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/utils/unittest" ) @@ -55,7 +54,7 @@ func TestGossipSubIHaveBrokenPromises_Below_Threshold(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) // we override some of the default scoring parameters in order to speed up the test in a time-efficient manner. - blockTopicOverrideParams := scoring.DefaultTopicScoreParams() + blockTopicOverrideParams := defaultTopicScoreParams(t) blockTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. // we disable invalid message delivery parameters, as the way we implement spammer, when it spams ihave messages, it does not sign them. Hence, without decaying the invalid message deliveries, // the node would be penalized for invalid message delivery way sooner than it can mount an ihave broken-promises spam attack. @@ -65,9 +64,15 @@ func TestGossipSubIHaveBrokenPromises_Below_Threshold(t *testing.T) { conf, err := config.DefaultConfig() require.NoError(t, err) // we override the decay interval to 1 second so that the score is updated within 1 second intervals. - conf.NetworkConfig.GossipSub.ScoringParameters.DecayInterval = 1 * time.Second + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval = 1 * time.Second // score tracer interval is set to 500 milliseconds to speed up the test, it should be shorter than the heartbeat interval (1 second) of gossipsub to catch the score updates in time. conf.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 500 * time.Millisecond + + // relaxing the scoring parameters to fit the test scenario. + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyDecay = 0.99 + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyThreshold = 10 + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyWeight = -1 + victimNode, victimIdentity := p2ptest.NodeFixture( t, sporkId, @@ -127,26 +132,28 @@ func TestGossipSubIHaveBrokenPromises_Below_Threshold(t *testing.T) { // Also, the internal heartbeat of GossipSub is 1 second, hence, there is no need to have ticks shorter than 500 milliseconds. }, 10*time.Second, 500*time.Millisecond) + scoreParams := conf.NetworkConfig.GossipSub.ScoringParameters + spammerScore, ok := victimNode.PeerScoreExposer().GetScore(spammer.SpammerNode.ID()) require.True(t, ok, "sanity check failed, we should have a score for the spammer node") // since spammer is not yet considered to be penalized, its score must be greater than the gossipsub health thresholds. require.Greaterf(t, spammerScore, - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, "sanity check failed, the score of the spammer node must be greater than gossip threshold: %f, actual: %f", - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, spammerScore) require.Greaterf(t, spammerScore, - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, "sanity check failed, the score of the spammer node must be greater than publish threshold: %f, actual: %f", - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, spammerScore) require.Greaterf(t, spammerScore, - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, "sanity check failed, the score of the spammer node must be greater than graylist threshold: %f, actual: %f", - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, spammerScore) // eventually, after a heartbeat the spammer behavioral counter must be decayed @@ -204,19 +211,25 @@ func TestGossipSubIHaveBrokenPromises_Above_Threshold(t *testing.T) { conf.NetworkConfig.GossipSub.RpcInspector.Validation.IHave.MaxSampleSize = 10000 conf.NetworkConfig.GossipSub.RpcInspector.Validation.IHave.MaxMessageIDSampleSize = 10000 // we override the decay interval to 1 second so that the score is updated within 1 second intervals. - conf.NetworkConfig.GossipSub.ScoringParameters.DecayInterval = 1 * time.Second + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval = 1 * time.Second // score tracer interval is set to 500 milliseconds to speed up the test, it should be shorter than the heartbeat interval (1 second) of gossipsub to catch the score updates in time. conf.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 500 * time.Millisecond ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) // we override some of the default scoring parameters in order to speed up the test in a time-efficient manner. - blockTopicOverrideParams := scoring.DefaultTopicScoreParams() + blockTopicOverrideParams := defaultTopicScoreParams(t) blockTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. // we disable invalid message delivery parameters, as the way we implement spammer, when it spams ihave messages, it does not sign them. Hence, without decaying the invalid message deliveries, // the node would be penalized for invalid message delivery way sooner than it can mount an ihave broken-promises spam attack. blockTopicOverrideParams.InvalidMessageDeliveriesWeight = 0.0 blockTopicOverrideParams.InvalidMessageDeliveriesDecay = 0.0 + + // relaxing the scoring parameters to fit the test scenario. + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyDecay = 0.99 + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyThreshold = 10 + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyWeight = -1 + victimNode, victimIdentity := p2ptest.NodeFixture( t, sporkId, @@ -307,33 +320,35 @@ func TestGossipSubIHaveBrokenPromises_Above_Threshold(t *testing.T) { // Also, the internal heartbeat of GossipSub is 1 second, hence, there is no need to have ticks shorter than 500 milliseconds. }, 10*time.Second, 500*time.Millisecond) + scoreParams := conf.NetworkConfig.GossipSub.ScoringParameters + spammerScore, ok := victimNode.PeerScoreExposer().GetScore(spammer.SpammerNode.ID()) require.True(t, ok, "sanity check failed, we should have a score for the spammer node") // with the second round of the attack, the spammer is about 10 broken promises above the threshold (total ~20 broken promises, but the first 10 are not counted). // we expect the score to be dropped to initScore - 10 * 10 * 0.01 * scoring.MaxAppSpecificReward, however, instead of 10, we consider 5 about the threshold, to account for decays. require.LessOrEqual(t, spammerScore, - initScore-5*5*0.01*scoring.MaxAppSpecificReward, + initScore-5*5*0.01*scoreParams.PeerScoring.Protocol.AppSpecificScore.MaxAppSpecificReward, "sanity check failed, the score of the spammer node must be less than the initial score minus 8 * 8 * 0.01 * scoring.MaxAppSpecificReward: %f, actual: %f", - initScore-5*5*0.1*scoring.MaxAppSpecificReward, + initScore-5*5*0.1*scoreParams.PeerScoring.Protocol.AppSpecificScore.MaxAppSpecificReward, spammerScore) require.Greaterf(t, spammerScore, - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, "sanity check failed, the score of the spammer node must be greater than gossip threshold: %f, actual: %f", - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, spammerScore) require.Greaterf(t, spammerScore, - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, "sanity check failed, the score of the spammer node must be greater than publish threshold: %f, actual: %f", - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, spammerScore) require.Greaterf(t, spammerScore, - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, "sanity check failed, the score of the spammer node must be greater than graylist threshold: %f, actual: %f", - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, spammerScore) // since the spammer score is above the gossip, graylist and publish thresholds, it should be still able to exchange messages with victim. @@ -368,21 +383,21 @@ func TestGossipSubIHaveBrokenPromises_Above_Threshold(t *testing.T) { // victim will not exchange messages with it anymore, and also that it will be graylisted meaning all incoming and outgoing RPCs to and from the spammer will be dropped by the victim. require.Lessf(t, spammerScore, - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, "sanity check failed, the score of the spammer node must be less than gossip threshold: %f, actual: %f", - scoring.DefaultGossipThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Gossip, spammerScore) require.Lessf(t, spammerScore, - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, "sanity check failed, the score of the spammer node must be less than publish threshold: %f, actual: %f", - scoring.DefaultPublishThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Publish, spammerScore) require.Lessf(t, spammerScore, - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, "sanity check failed, the score of the spammer node must be less than graylist threshold: %f, actual: %f", - scoring.DefaultGraylistThreshold, + scoreParams.PeerScoring.Internal.Thresholds.Graylist, spammerScore) // since the spammer score is below the gossip, graylist and publish thresholds, it should not be able to exchange messages with victim anymore. diff --git a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go index 0197f219960..902b698aa2d 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go @@ -18,7 +18,6 @@ import ( "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/scoring" p2ptest "github.com/onflow/flow-go/network/p2p/test" validator "github.com/onflow/flow-go/network/validator/pubsub" "github.com/onflow/flow-go/utils/unittest" @@ -142,6 +141,8 @@ func testGossipSubInvalidMessageDeliveryScoring(t *testing.T, spamMsgFactory fun spamMsgFactory(spammer.SpammerNode.ID(), victimNode.ID(), blockTopic)) } + scoreParams := cfg.NetworkConfig.GossipSub.ScoringParameters + // wait for at most 3 seconds for the victim node to penalize the spammer node. // Each heartbeat is 1 second, so 3 heartbeats should be enough to penalize the spammer node. // Ideally, we should wait for 1 heartbeat, but the score may not be updated immediately after the heartbeat. @@ -150,15 +151,15 @@ func testGossipSubInvalidMessageDeliveryScoring(t *testing.T, spamMsgFactory fun if !ok { return false } - if spammerScore >= scoring.DefaultGossipThreshold { + if spammerScore >= scoreParams.PeerScoring.Internal.Thresholds.Gossip { // ensure the score is low enough so that no gossip is routed by victim node to spammer node. return false } - if spammerScore >= scoring.DefaultPublishThreshold { + if spammerScore >= scoreParams.PeerScoring.Internal.Thresholds.Publish { // ensure the score is low enough so that non of the published messages of the victim node are routed to the spammer node. return false } - if spammerScore >= scoring.DefaultGraylistThreshold { + if spammerScore >= scoreParams.PeerScoring.Internal.Thresholds.Graylist { // ensure the score is low enough so that the victim node does not accept RPC messages from the spammer node. return false } @@ -209,13 +210,13 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_SingleTopic(t *testing.T) { blockTopic := channels.TopicFromChannel(channels.PushBlocks, sporkId) // we override some of the default scoring parameters in order to speed up the test in a time-efficient manner. - blockTopicOverrideParams := scoring.DefaultTopicScoreParams() + blockTopicOverrideParams := defaultTopicScoreParams(t) blockTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. conf, err := config.DefaultConfig() require.NoError(t, err) // we override the decay interval to 1 second so that the score is updated within 1 second intervals. - conf.NetworkConfig.GossipSub.ScoringParameters.DecayInterval = 1 * time.Second + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval = 1 * time.Second conf.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 1 * time.Second thisNode, thisId := p2ptest.NodeFixture( // this node is the one that will be penalizing the under-performer node. t, @@ -256,6 +257,8 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_SingleTopic(t *testing.T) { return unittest.ProposalFixture() }) + scoreParams := conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol + // Also initially the under-performing node should have a score that is at least equal to the MaxAppSpecificReward. // The reason is in our scoring system, we reward the staked nodes by MaxAppSpecificReward, and the under-performing node is considered staked // as it is in the id provider of thisNode. @@ -264,7 +267,7 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_SingleTopic(t *testing.T) { if !ok { return false } - if underPerformingNodeScore < scoring.MaxAppSpecificReward { + if underPerformingNodeScore < scoreParams.AppSpecificScore.MaxAppSpecificReward { // ensure the score is high enough so that gossip is routed by victim node to spammer node. return false } @@ -279,17 +282,17 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_SingleTopic(t *testing.T) { if !ok { return false } - if underPerformingNodeScore > 0.96*scoring.MaxAppSpecificReward { // score must be penalized by -0.05 * MaxAppSpecificReward. + if underPerformingNodeScore > 0.96*scoreParams.AppSpecificScore.MaxAppSpecificReward { // score must be penalized by -0.05 * MaxAppSpecificReward. // 0.96 is to account for floating point errors. return false } - if underPerformingNodeScore < scoring.DefaultGossipThreshold { // even the node is slightly penalized, it should still be able to gossip with this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Gossip { // even the node is slightly penalized, it should still be able to gossip with this node. return false } - if underPerformingNodeScore < scoring.DefaultPublishThreshold { // even the node is slightly penalized, it should still be able to publish to this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Publish { // even the node is slightly penalized, it should still be able to publish to this node. return false } - if underPerformingNodeScore < scoring.DefaultGraylistThreshold { // even the node is slightly penalized, it should still be able to establish rpc connection with this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Graylist { // even the node is slightly penalized, it should still be able to establish rpc connection with this node. return false } @@ -317,15 +320,15 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_TwoTopics(t *testing.T) { dkgTopic := channels.TopicFromChannel(channels.DKGCommittee, sporkId) // we override some of the default scoring parameters in order to speed up the test in a time-efficient manner. - blockTopicOverrideParams := scoring.DefaultTopicScoreParams() + blockTopicOverrideParams := defaultTopicScoreParams(t) blockTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. - dkgTopicOverrideParams := scoring.DefaultTopicScoreParams() + dkgTopicOverrideParams := defaultTopicScoreParams(t) dkgTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. conf, err := config.DefaultConfig() require.NoError(t, err) // we override the decay interval to 1 second so that the score is updated within 1 second intervals. - conf.NetworkConfig.GossipSub.ScoringParameters.DecayInterval = 1 * time.Second + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval = 1 * time.Second conf.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 1 * time.Second thisNode, thisId := p2ptest.NodeFixture( // this node is the one that will be penalizing the under-performer node. t, @@ -370,6 +373,8 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_TwoTopics(t *testing.T) { } } + scoreParams := conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // Initially the under-performing node should have a score that is at least equal to the MaxAppSpecificReward. // The reason is in our scoring system, we reward the staked nodes by MaxAppSpecificReward, and the under-performing node is considered staked // as it is in the id provider of thisNode. @@ -378,7 +383,7 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_TwoTopics(t *testing.T) { if !ok { return false } - if underPerformingNodeScore < scoring.MaxAppSpecificReward { + if underPerformingNodeScore < scoreParams.MaxAppSpecificReward { // ensure the score is high enough so that gossip is routed by victim node to spammer node. return false } @@ -394,17 +399,17 @@ func TestGossipSubMeshDeliveryScoring_UnderDelivery_TwoTopics(t *testing.T) { if !ok { return false } - if underPerformingNodeScore > 0.91*scoring.MaxAppSpecificReward { // score must be penalized by ~ 2 * -0.05 * MaxAppSpecificReward. + if underPerformingNodeScore > 0.91*scoreParams.MaxAppSpecificReward { // score must be penalized by ~ 2 * -0.05 * MaxAppSpecificReward. // 0.91 is to account for the floating point errors. return false } - if underPerformingNodeScore < scoring.DefaultGossipThreshold { // even the node is slightly penalized, it should still be able to gossip with this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Gossip { // even the node is slightly penalized, it should still be able to gossip with this node. return false } - if underPerformingNodeScore < scoring.DefaultPublishThreshold { // even the node is slightly penalized, it should still be able to publish to this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Publish { // even the node is slightly penalized, it should still be able to publish to this node. return false } - if underPerformingNodeScore < scoring.DefaultGraylistThreshold { // even the node is slightly penalized, it should still be able to establish rpc connection with this node. + if underPerformingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Graylist { // even the node is slightly penalized, it should still be able to establish rpc connection with this node. return false } @@ -437,9 +442,9 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { conf, err := config.DefaultConfig() require.NoError(t, err) // we override the decay interval to 1 second so that the score is updated within 1 second intervals. - conf.NetworkConfig.GossipSub.ScoringParameters.DecayInterval = 1 * time.Second + conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval = 1 * time.Second conf.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 1 * time.Second - blockTopicOverrideParams := scoring.DefaultTopicScoreParams() + blockTopicOverrideParams := defaultTopicScoreParams(t) blockTopicOverrideParams.MeshMessageDeliveriesActivation = 1 * time.Second // we start observing the mesh message deliveries after 1 second of the node startup. thisNode, thisId := p2ptest.NodeFixture( // this node is the one that will be penalizing the under-performer node. t, @@ -480,6 +485,8 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { return unittest.ProposalFixture() }) + scoreParams := conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // Initially the replaying node should have a score that is at least equal to the MaxAppSpecificReward. // The reason is in our scoring system, we reward the staked nodes by MaxAppSpecificReward, and initially every node is considered staked // as it is in the id provider of thisNode. @@ -489,7 +496,7 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { if !ok { return false } - if replayingNodeScore < scoring.MaxAppSpecificReward { + if replayingNodeScore < scoreParams.MaxAppSpecificReward { // ensure the score is high enough so that gossip is routed by victim node to spammer node. return false } @@ -516,7 +523,7 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { if !ok { return false } - if replayingNodeScore < scoring.MaxAppSpecificReward { + if replayingNodeScore < scoreParams.MaxAppSpecificReward { // ensure the score is high enough so that gossip is routed by victim node to spammer node. return false } @@ -556,22 +563,22 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { return false } - if replayingNodeScore >= scoring.MaxAppSpecificReward { + if replayingNodeScore >= scoreParams.MaxAppSpecificReward { // node must be penalized for just replaying the same messages. return false } // following if-statements check that even though the node is penalized, it is not penalized too much, and // can still participate in the network. We don't desire to disallow list a node for just under-performing. - if replayingNodeScore < scoring.DefaultGossipThreshold { + if replayingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Gossip { return false } - if replayingNodeScore < scoring.DefaultPublishThreshold { + if replayingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Publish { return false } - if replayingNodeScore < scoring.DefaultGraylistThreshold { + if replayingNodeScore < conf.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Graylist { return false } @@ -584,3 +591,24 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { return unittest.ProposalFixture() }) } + +// defaultTopicScoreParams returns the default score params for topics. +func defaultTopicScoreParams(t *testing.T) *pubsub.TopicScoreParams { + defaultConfig, err := config.DefaultConfig() + require.NoError(t, err) + topicScoreParams := defaultConfig.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters + p := &pubsub.TopicScoreParams{ + TopicWeight: topicScoreParams.TopicWeight, + SkipAtomicValidation: topicScoreParams.SkipAtomicValidation, + InvalidMessageDeliveriesWeight: topicScoreParams.InvalidMessageDeliveriesWeight, + InvalidMessageDeliveriesDecay: topicScoreParams.InvalidMessageDeliveriesDecay, + TimeInMeshQuantum: topicScoreParams.TimeInMeshQuantum, + MeshMessageDeliveriesWeight: topicScoreParams.MeshDeliveriesWeight, + MeshMessageDeliveriesDecay: topicScoreParams.MeshMessageDeliveriesDecay, + MeshMessageDeliveriesCap: topicScoreParams.MeshMessageDeliveriesCap, + MeshMessageDeliveriesThreshold: topicScoreParams.MeshMessageDeliveryThreshold, + MeshMessageDeliveriesWindow: topicScoreParams.MeshMessageDeliveriesWindow, + MeshMessageDeliveriesActivation: topicScoreParams.MeshMessageDeliveryActivation, + } + return p +} diff --git a/integration/benchnet2/Makefile b/integration/benchnet2/Makefile index b7911fdc0f9..53473ffa590 100644 --- a/integration/benchnet2/Makefile +++ b/integration/benchnet2/Makefile @@ -4,8 +4,8 @@ CONFIGURATION_BUCKET := flow-benchnet-automation # default values that callers can override when calling target ACCESS = 1 -COLLECTION = 6 -VALID_COLLECTION := $(shell test $(COLLECTION) -ge 6; echo $$?) +COLLECTION = 2 +VALID_COLLECTION := $(shell test $(COLLECTION) -ge 2; echo $$?) CONSENSUS = 2 VALID_CONSENSUS := $(shell test $(CONSENSUS) -ge 2; echo $$?) EXECUTION = 2 @@ -19,7 +19,7 @@ ifeq ($(strip $(VALID_EXECUTION)), 1) else ifeq ($(strip $(VALID_CONSENSUS)), 1) $(error Number of Consensus nodes should be no less than 2) else ifeq ($(strip $(VALID_COLLECTION)), 1) - $(error Number of Collection nodes should be no less than 6) + $(error Number of Collection nodes should be no less than 2) else ifeq ($(strip $(NETWORK_ID)),) $(error NETWORK_ID cannot be empty) else ifeq ($(strip $(NAMESPACE)),) @@ -64,11 +64,15 @@ clean-gen-helm: rm -f values.yml rm -f template-data.json +download-values-file: + gsutil cp gs://${CONFIGURATION_BUCKET}/${NETWORK_ID}/values.yml . + upload-bootstrap: tar -cvf ${NETWORK_ID}.tar -C ./bootstrap . gsutil cp ${NETWORK_ID}.tar gs://${CONFIGURATION_BUCKET}/${NETWORK_ID}.tar + gsutil cp values.yml gs://${CONFIGURATION_BUCKET}/${NETWORK_ID}/values.yml -helm-deploy: +helm-deploy: download-values-file helm upgrade --install -f ./values.yml ${NETWORK_ID} ./flow --set ingress.enabled=true --set networkId="${NETWORK_ID}" --set owner="${OWNER}" --set configurationBucket="${CONFIGURATION_BUCKET}" --debug --namespace ${NAMESPACE} --wait k8s-delete: @@ -77,6 +81,7 @@ k8s-delete: delete-configuration: gsutil rm gs://${CONFIGURATION_BUCKET}/${NETWORK_ID}.tar + gsutil rm gs://${CONFIGURATION_BUCKET}/${NETWORK_ID}/values.yml k8s-pod-health: validate kubectl get pods --namespace ${NAMESPACE} diff --git a/integration/benchnet2/automate/templates/helm-values-all-nodes.yml b/integration/benchnet2/automate/templates/helm-values-all-nodes.yml index 211ec7ca1fd..141f0edda24 100644 --- a/integration/benchnet2/automate/templates/helm-values-all-nodes.yml +++ b/integration/benchnet2/automate/templates/helm-values-all-nodes.yml @@ -10,7 +10,7 @@ access: limits: cpu: "800m" memory: "10Gi" - storage: 1G + storage: 2G nodes: {{- range $val := .}}{{if eq ($val.role) ("access")}} {{$val.name}}: @@ -38,7 +38,7 @@ collection: limits: cpu: "800m" memory: "10Gi" - storage: 1G + storage: 2G nodes: {{- range $val := .}}{{if eq ($val.role) ("collection")}} {{$val.name}}: @@ -85,7 +85,7 @@ execution: limits: cpu: "800m" memory: "10Gi" - storage: 10G + storage: 50G nodes: {{- range $val := .}}{{if eq ($val.role) ("execution")}} {{$val.name}}: diff --git a/integration/dkg/dkg_emulator_suite.go b/integration/dkg/dkg_emulator_suite.go index c70a6c7efc4..7b8b3986333 100644 --- a/integration/dkg/dkg_emulator_suite.go +++ b/integration/dkg/dkg_emulator_suite.go @@ -460,12 +460,6 @@ func (s *EmulatorSuite) initEngines(node *node, ids flow.IdentityList) { controllerFactoryLogger = zerolog.New(os.Stdout).Hook(hook) } - // create a config with no delays for tests - config := dkg.ControllerConfig{ - BaseStartDelay: 0, - BaseHandleFirstBroadcastDelay: 0, - } - // the reactor engine reacts to new views being finalized and drives the // DKG protocol reactorEngine := dkgeng.NewReactorEngine( @@ -478,7 +472,6 @@ func (s *EmulatorSuite) initEngines(node *node, ids flow.IdentityList) { core.Me, []module.DKGContractClient{node.dkgContractClient}, brokerTunnel, - config, ), viewsObserver, ) diff --git a/integration/dkg/dkg_emulator_test.go b/integration/dkg/dkg_emulator_test.go index 8d349bd8899..c7e248a3c9d 100644 --- a/integration/dkg/dkg_emulator_test.go +++ b/integration/dkg/dkg_emulator_test.go @@ -51,7 +51,7 @@ func (s *EmulatorSuite) runTest(goodNodes int, emulatorProblems bool) { DKGPhase3FinalView: 250, FinalView: 300, Participants: s.netIDs, - RandomSource: []byte("random bytes for seed"), + RandomSource: unittest.EpochSetupRandomSourceFixture(), } // create the EpochSetup that will trigger the next DKG run with all the @@ -59,7 +59,7 @@ func (s *EmulatorSuite) runTest(goodNodes int, emulatorProblems bool) { nextEpochSetup := flow.EpochSetup{ Counter: currentCounter + 1, Participants: s.netIDs, - RandomSource: []byte("random bytes for seed"), + RandomSource: unittest.EpochSetupRandomSourceFixture(), FirstView: 301, FinalView: 600, } diff --git a/integration/dkg/dkg_whiteboard_test.go b/integration/dkg/dkg_whiteboard_test.go index a7b00fa1172..6b2085ffc68 100644 --- a/integration/dkg/dkg_whiteboard_test.go +++ b/integration/dkg/dkg_whiteboard_test.go @@ -138,12 +138,6 @@ func createNode( }) controllerFactoryLogger := zerolog.New(os.Stdout).Hook(hook) - // create a config with no delays for tests - config := dkg.ControllerConfig{ - BaseStartDelay: 0, - BaseHandleFirstBroadcastDelay: 0, - } - // the reactor engine reacts to new views being finalized and drives the // DKG protocol reactorEngine := dkgeng.NewReactorEngine( @@ -156,7 +150,6 @@ func createNode( core.Me, []module.DKGContractClient{NewWhiteboardClient(id.NodeID, whiteboard)}, brokerTunnel, - config, ), viewsObserver, ) @@ -228,7 +221,7 @@ func TestWithWhiteboard(t *testing.T) { DKGPhase3FinalView: 250, FinalView: 300, Participants: conIdentities, - RandomSource: []byte("random bytes for seed"), + RandomSource: unittest.EpochSetupRandomSourceFixture(), } // create the EpochSetup that will trigger the next DKG run with all the @@ -236,7 +229,7 @@ func TestWithWhiteboard(t *testing.T) { nextEpochSetup := flow.EpochSetup{ Counter: currentCounter + 1, Participants: conIdentities, - RandomSource: []byte("random bytes for seed"), + RandomSource: unittest.EpochSetupRandomSourceFixture(), } nodes, _ := createNodes( diff --git a/integration/go.mod b/integration/go.mod index 34e467b8536..310487232fd 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -19,12 +19,12 @@ require ( github.com/ipfs/go-datastore v0.6.0 github.com/ipfs/go-ds-badger2 v0.1.3 github.com/ipfs/go-ipfs-blockstore v1.3.0 - github.com/onflow/cadence v1.0.0-preview.2 - github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 - github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 - github.com/onflow/flow-emulator v0.59.1-0.20231213222325-7896fff7cacb - github.com/onflow/flow-go v0.32.4-0.20231213192043-8399b2afb801 - github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 + github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb + github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b + github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b + github.com/onflow/flow-emulator v0.59.1-0.20240122200325-58ef35ed4aed + github.com/onflow/flow-go v0.33.2-0.20240122202431-1d676bfe7460 + github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca github.com/onflow/flow-go/crypto v0.25.0 github.com/onflow/flow-go/insecure v0.0.0-00010101000000-000000000000 github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 @@ -36,7 +36,7 @@ require ( github.com/stretchr/testify v1.8.4 go.einride.tech/pid v0.1.0 go.uber.org/atomic v1.11.0 - golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df + golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc golang.org/x/sync v0.5.0 google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 @@ -52,29 +52,29 @@ require ( github.com/DataDog/zstd v1.5.2 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4 // indirect - github.com/SaveTheRbtz/mph v0.1.2 // indirect + github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc // indirect github.com/StackExchange/wmi v1.2.1 // indirect github.com/VictoriaMetrics/fastcache v1.12.1 // indirect github.com/acomagu/bufpipe v1.0.3 // indirect github.com/andybalholm/brotli v1.0.4 // indirect github.com/apache/arrow/go/v12 v12.0.0 // indirect github.com/apache/thrift v0.16.0 // indirect - github.com/aws/aws-sdk-go-v2 v1.17.7 // indirect - github.com/aws/aws-sdk-go-v2/config v1.18.19 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.13.18 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 // indirect + github.com/aws/aws-sdk-go-v2 v1.21.2 // indirect + github.com/aws/aws-sdk-go-v2/config v1.18.45 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.13.43 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 // indirect github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 // indirect github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 // indirect - github.com/aws/smithy-go v1.13.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 // indirect + github.com/aws/smithy-go v1.15.0 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.7.0 // indirect @@ -86,11 +86,15 @@ require ( github.com/cloudflare/circl v1.1.0 // indirect github.com/cockroachdb/errors v1.9.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect - github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 // indirect + github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 // indirect github.com/cockroachdb/redact v1.1.3 // indirect + github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/consensys/bavard v0.1.13 // indirect + github.com/consensys/gnark-crypto v0.12.1 // indirect github.com/containerd/cgroups v1.1.0 // indirect github.com/containerd/fifo v1.1.0 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/crate-crypto/go-kzg-4844 v0.7.0 // indirect github.com/cskr/pubsub v1.0.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect @@ -109,7 +113,8 @@ require ( github.com/ef-ds/deque v1.0.4 // indirect github.com/elastic/gosigar v0.14.2 // indirect github.com/emirpasic/gods v1.18.1 // indirect - github.com/ethereum/go-ethereum v1.12.0 // indirect + github.com/ethereum/c-kzg-4844 v0.4.0 // indirect + github.com/ethereum/go-ethereum v1.13.5 // indirect github.com/flynn/noise v1.0.0 // indirect github.com/francoispqt/gojay v1.2.13 // indirect github.com/fsnotify/fsnotify v1.6.0 // indirect @@ -134,7 +139,7 @@ require ( github.com/go-redis/redis/v8 v8.11.5 // indirect github.com/go-stack/stack v1.8.1 // indirect github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect - github.com/goccy/go-json v0.9.11 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/godbus/dbus/v5 v5.1.0 // indirect github.com/gofrs/flock v0.8.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect @@ -162,7 +167,7 @@ require ( github.com/hashicorp/hcl v1.0.0 // indirect github.com/holiman/bloomfilter/v2 v2.0.3 // indirect github.com/holiman/uint256 v1.2.3 // indirect - github.com/huin/goupnp v1.2.0 // indirect + github.com/huin/goupnp v1.3.0 // indirect github.com/imdario/mergo v0.3.13 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/ipfs/bbloom v0.0.4 // indirect @@ -231,6 +236,7 @@ require ( github.com/minio/sha256-simd v1.0.1 // indirect github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect + github.com/mmcloughlin/addchain v0.4.0 // indirect github.com/moby/term v0.5.0 // indirect github.com/morikuni/aec v1.0.0 // indirect github.com/mr-tron/base58 v1.2.0 // indirect @@ -247,7 +253,7 @@ require ( github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f // indirect github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 // indirect - github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 // indirect + github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 // indirect github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d // indirect github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba // indirect github.com/onflow/wal v0.0.0-20230529184820-bc9f8244608d // indirect @@ -296,6 +302,7 @@ require ( github.com/spf13/viper v1.15.0 // indirect github.com/stretchr/objx v0.5.0 // indirect github.com/subosito/gotenv v1.4.2 // indirect + github.com/supranational/blst v0.3.11 // indirect github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect github.com/texttheater/golang-levenshtein/levenshtein v0.0.0-20200805054039-cae8b0eaed6c // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect @@ -324,7 +331,7 @@ require ( go.uber.org/fx v1.19.2 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect - golang.org/x/crypto v0.16.0 // indirect + golang.org/x/crypto v0.17.0 // indirect golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.19.0 // indirect golang.org/x/oauth2 v0.11.0 // indirect @@ -342,7 +349,6 @@ require ( google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect - gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect gopkg.in/warnings.v0 v0.1.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect lukechampine.com/blake3 v1.2.1 // indirect @@ -350,6 +356,7 @@ require ( modernc.org/mathutil v1.5.0 // indirect modernc.org/memory v1.5.0 // indirect modernc.org/sqlite v1.21.1 // indirect + rsc.io/tmplfunc v0.0.3 // indirect ) replace github.com/onflow/flow-go => ../ diff --git a/integration/go.sum b/integration/go.sum index b3e5629e63f..d4ae7f9291b 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -579,8 +579,8 @@ github.com/OneOfOne/xxhash v1.2.5/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdII github.com/OneOfOne/xxhash v1.2.8 h1:31czK/TI9sNkxIKfaUfGlU47BAxQ0ztGgd9vPyqimf8= github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4 h1:ra2OtmuW0AE5csawV4YXMNGNQQXvLRps3z2Z59OPO+I= github.com/ProtonMail/go-crypto v0.0.0-20221026131551-cf6655e29de4/go.mod h1:UBYPn8k0D56RtnR8RFQMjmh4KrZzWJ5o7Z9SYjossQ8= -github.com/SaveTheRbtz/mph v0.1.2 h1:5l3W496Up+7BNOVJQnJhzcGBh+wWfxWdmPUAkx3WmaM= -github.com/SaveTheRbtz/mph v0.1.2/go.mod h1:V4+WtKQPe2+dEA5os1WnGsEB0NR9qgqqgIiSt73+sT4= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc h1:DCHzPQOcU/7gwDTWbFQZc5qHMPS1g0xTO56k8NXsv9M= +github.com/SaveTheRbtz/mph v0.1.1-0.20240117162131-4166ec7869bc/go.mod h1:LJM5a3zcIJ/8TmZwlUczvROEJT8ntOdhdG9jjcR1B0I= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= @@ -636,49 +636,61 @@ github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/aws/aws-sdk-go-v2 v1.9.0/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4= github.com/aws/aws-sdk-go-v2 v1.17.3/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= -github.com/aws/aws-sdk-go-v2 v1.17.7 h1:CLSjnhJSTSogvqUGhIC6LqFKATMRexcxLZ0i/Nzk9Eg= github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go-v2 v1.21.2 h1:+LXZ0sgo8quN9UOKXXzAWRT3FWd4NxeXWOZom9pE7GA= +github.com/aws/aws-sdk-go-v2 v1.21.2/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/config v1.8.0/go.mod h1:w9+nMZ7soXCe5nT46Ri354SNhXDQ6v+V5wqDjnZE+GY= -github.com/aws/aws-sdk-go-v2/config v1.18.19 h1:AqFK6zFNtq4i1EYu+eC7lcKHYnZagMn6SW171la0bGw= github.com/aws/aws-sdk-go-v2/config v1.18.19/go.mod h1:XvTmGMY8d52ougvakOv1RpiTLPz9dlG/OQHsKU/cMmY= +github.com/aws/aws-sdk-go-v2/config v1.18.45 h1:Aka9bI7n8ysuwPeFdm77nfbyHCAKQ3z9ghB3S/38zes= +github.com/aws/aws-sdk-go-v2/config v1.18.45/go.mod h1:ZwDUgFnQgsazQTnWfeLWk5GjeqTQTL8lMkoE1UXzxdE= github.com/aws/aws-sdk-go-v2/credentials v1.4.0/go.mod h1:dgGR+Qq7Wjcd4AOAW5Rf5Tnv3+x7ed6kETXyS9WCuAY= -github.com/aws/aws-sdk-go-v2/credentials v1.13.18 h1:EQMdtHwz0ILTW1hoP+EwuWhwCG1hD6l3+RWFQABET4c= github.com/aws/aws-sdk-go-v2/credentials v1.13.18/go.mod h1:vnwlwjIe+3XJPBYKu1et30ZPABG3VaXJYr8ryohpIyM= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43 h1:LU8vo40zBlo3R7bAvBVy/ku4nxGEyZe9N8MqAeFTzF8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.43/go.mod h1:zWJBz1Yf1ZtX5NGax9ZdNjhhI4rgjfgsyk6vTY1yfVg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.5.0/go.mod h1:CpNzHK9VEFUCknu50kkB8z58AH2B5DvPP7ea1LHve/Y= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1 h1:gt57MN3liKiyGopcqgNzJb2+d9MJaKT/q1OksHNXVE4= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.1/go.mod h1:lfUx8puBRdM5lVVMQlwt2v+ofiG/X6Ms+dy0UkG/kXw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13 h1:PIktER+hwIG286DqXyvVENjgLTAwGgoeriLDD5C+YlQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.13/go.mod h1:f/Ib/qYjhV2/qdsf79H3QP/eRE4AkVyEf6sk7XfZ1tg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1 h1:VGkV9KmhGqOQWnHyi4gLG98kE6OecT42fdrCGFWxJsc= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.5.1/go.mod h1:PLlnMiki//sGnCJiW+aVpvP/C8Kcm8mEj/IVm9+9qk4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.27/go.mod h1:a1/UpzeyBBerajpnP5nGZa9mGzsBn5cOKxm6NWQsvoI= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31 h1:sJLYcS+eZn5EeNINGHSCRAwUJMFVqklwkH36Vbyai7M= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43 h1:nFBQlGtkbPzp/NjZLuFxRqmT91rLJkgvsEQs68h962Y= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.43/go.mod h1:auo+PiyLl0n1l8A0e8RIeR8tOzYPfZZH/JNlrJ8igTQ= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.21/go.mod h1:+Gxn8jYn5k9ebfHEqlhrMirFjSW0v0C9fI+KN5vk2kE= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25 h1:1mnRASEKnkqsntcxHaysxwgVoUUp5dkiB+l3llKnqyg= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37 h1:JRVhO25+r3ar2mKGP7E0LDl8K9/G36gjlqca5iQbaqc= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.37/go.mod h1:Qe+2KtKml+FEsQF/DHmDV+xjtche/hwoF75EG4UlHW8= github.com/aws/aws-sdk-go-v2/internal/ini v1.2.2/go.mod h1:BQV0agm+JEhqR+2RT5e1XTFIDcAAV0eW6z2trp+iduw= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32 h1:p5luUImdIqywn6JpQsW3tq5GNOxKmOnEpybzPx+d1lk= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.32/go.mod h1:XGhIBZDEgfqmFIugclZ6FU7v75nHhBDtzuB4xB/tEi4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45 h1:hze8YsjSh8Wl1rYa1CJpRmXP21BvOBuc76YhW0HsuQ4= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.45/go.mod h1:lD5M20o09/LCuQ2mE62Mb/iSdSlCNuj6H5ci7tW7OsE= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 h1:gceOysEWNNwLd6cki65IMBZ4WAM0MwgBQq2n7kejoT8= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0/go.mod h1:v8ygadNyATSm6elwJ/4gzJwcFhri9RqS8skgHKiwXPU= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.3.0/go.mod h1:R1KK+vY8AfalhG1AOu5e35pOD2SdoPKQCFLTvnxiohk= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25 h1:5LHn8JQ0qvjD9L9JhMtylnkcw7j05GDZqM9Oin6hpr0= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.25/go.mod h1:/95IA+0lMnzW6XzqYJRpjjsAbKEORVeO0anQqjd2CNU= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37 h1:WWZA/I2K4ptBS1kg0kV1JbBtG/umed0vwHRrmcr9z7k= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.37/go.mod h1:vBmDnwWXWxNPFRMmG2m/3MKOe+xEcMDo1tanpaWCcck= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0 h1:HWsM0YQWX76V6MOp07YuTYacm8k7h69ObJuw7Nck+og= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.7.0/go.mod h1:LKb3cKNQIMh+itGnEpKGcnL/6OIjPZqrtYah1w5f+3o= github.com/aws/aws-sdk-go-v2/service/kms v1.20.1/go.mod h1:13sjgMH7Xu4e46+0BEDhSnNh+cImHSYS5PpBjV3oXcU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0 h1:nPLfLPfglacc29Y949sDxpr3X/blaY40s3B85WT2yZU= github.com/aws/aws-sdk-go-v2/service/s3 v1.15.0/go.mod h1:Iv2aJVtVSm/D22rFoX99cLG4q4uB7tppuCsulGe98k4= github.com/aws/aws-sdk-go-v2/service/sso v1.4.0/go.mod h1:+1fpWnL96DL23aXPpMGbsmKe8jLTEfbjuQoA4WS1VaA= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.6 h1:5V7DWLBd7wTELVz5bPpwzYy/sikk0gsgZfj40X+l5OI= github.com/aws/aws-sdk-go-v2/service/sso v1.12.6/go.mod h1:Y1VOmit/Fn6Tz1uFAeCO6Q7M2fmfXSCLeL5INVYsLuY= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6 h1:B8cauxOH1W1v7rd8RdI/MWnoR4Ze0wIHWrb90qczxj4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2 h1:JuPGc7IkOP4AaqcZSIcyqLpFSqBWK32rM9+a1g6u73k= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.2/go.mod h1:gsL4keucRCgW+xA85ALBpRFfdSLH4kHOVSnLMSuBECo= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.6/go.mod h1:Lh/bc9XUf8CfOY6Jp5aIkQtN+j1mc+nExc+KXj9jx2s= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3 h1:HFiiRkf1SdaAmV3/BHOFZ9DjFynPHj8G/UIO1lQS+fk= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.3/go.mod h1:a7bHA82fyUXOm+ZSWKU6PIoBxrjSprdLoM8xPYvzYVg= github.com/aws/aws-sdk-go-v2/service/sts v1.7.0/go.mod h1:0qcSMCyASQPN2sk/1KQLQ2Fh6yq8wm0HSDAimPhzCoM= -github.com/aws/aws-sdk-go-v2/service/sts v1.18.7 h1:bWNgNdRko2x6gqa0blfATqAZKZokPIeM1vfmQt2pnvM= github.com/aws/aws-sdk-go-v2/service/sts v1.18.7/go.mod h1:JuTnSoeePXmMVe9G8NcjjwgOKEfZ4cOjMuT2IBT/2eI= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2 h1:0BkLfgeDjfZnZ+MhB3ONb01u9pwFYTCZVhlsSSBvlbU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.2/go.mod h1:Eows6e1uQEsc4ZaHANmsPRzAKcVDrcmjjWiih2+HUUQ= github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= +github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -689,7 +701,6 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bits-and-blooms/bitset v1.2.2/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.5.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bits-and-blooms/bitset v1.7.0 h1:YjAGVd3XmtK9ktAbX8Zg2g2PwLIMjGREZJHlV4j7NEo= github.com/bits-and-blooms/bitset v1.7.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= @@ -722,7 +733,7 @@ github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46f github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/bwesterb/go-ristretto v1.2.0/go.mod h1:fUIoIZaG73pV5biE2Blr2xEzDoMj7NFEuV9ekS419A0= github.com/bytecodealliance/wasmtime-go/v7 v7.0.0/go.mod h1:bu6fic7trDt20w+LMooX7j3fsOwv4/ln6j8gAdP6vmA= -github.com/c-bata/go-prompt v0.2.5/go.mod h1:vFnjEGDIIA/Lib7giyE4E9c50Lvl8j0S+7FVlAwDAVw= +github.com/c-bata/go-prompt v0.2.6/go.mod h1:/LMAke8wD2FsNu9EXNdHxNLbd9MedkPnCdfpU9wwHfY= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= @@ -762,18 +773,24 @@ github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/datadriven v1.0.3-0.20230801171734-e384cf455877 h1:1MLK4YpFtIEo3ZtMA5C795Wtv5VuUnrXX7mQG+aHg6o= +github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/errors v1.9.1 h1:yFVvsI0VxmRShfawbt/laCIDy/mtTqqnvoNgiy5bEV8= github.com/cockroachdb/errors v1.9.1/go.mod h1:2sxOtL2WIc096WSZqZ5h8fa17rdDq9HZOZLBCor4mBk= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06 h1:T+Np/xtzIjYM/P5NAw0e2Rf1FGvzDau1h54MKvx8G7w= -github.com/cockroachdb/pebble v0.0.0-20230906160148-46873a6a7a06/go.mod h1:bynZ3gvVyhlvjLI7PT6dmZ7g76xzJ7HpxfjgkzCGz6s= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= +github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= +github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= +github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM= github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw= @@ -799,6 +816,8 @@ github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:ma github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= +github.com/crate-crypto/go-kzg-4844 v0.7.0 h1:C0vgZRk4q4EZ/JgPfzuSoxdCq3C3mOZMBShovmncxvA= +github.com/crate-crypto/go-kzg-4844 v0.7.0/go.mod h1:1kMhvPgI0Ky3yIa+9lFySEBUBXkYxeOi8ZF1sYioxhc= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= @@ -906,9 +925,11 @@ github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7 github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/ethereum/c-kzg-4844 v0.4.0 h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R3nlY= +github.com/ethereum/c-kzg-4844 v0.4.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/ethereum/go-ethereum v1.9.13/go.mod h1:qwN9d1GLyDh0N7Ab8bMGd0H9knaji2jOBm2RrMGjXls= -github.com/ethereum/go-ethereum v1.12.0 h1:bdnhLPtqETd4m3mS8BGMNvBTf36bO5bx/hxE2zljOa0= -github.com/ethereum/go-ethereum v1.12.0/go.mod h1:/oo2X/dZLJjf2mJ6YT9wcWxa4nNJDBKDBU6sFIpx1Gs= +github.com/ethereum/go-ethereum v1.13.5 h1:U6TCRciCqZRe4FPXmy1sMGxTfuk8P7u2UoinF3VbaFk= +github.com/ethereum/go-ethereum v1.13.5/go.mod h1:yMTu38GSuyxaYzQMViqNmQ1s3cE84abZexQmTgenWk0= github.com/fanliao/go-promise v0.0.0-20141029170127-1890db352a72/go.mod h1:PjfxuH4FZdUyfMdtBio2lsRr1AKEaVPwelzuHuh8Lqc= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.3.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= @@ -1018,15 +1039,14 @@ github.com/go-stack/stack v1.8.1 h1:ntEHSVwIt7PNXNpgPmVfMrNhLtgjlmnZha2kOpuRiDw= github.com/go-stack/stack v1.8.1/go.mod h1:dcoOX6HbPZSZptuspn9bctJ+N/CnF5gGygcUP3XYfe4= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI= github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls= -github.com/go-test/deep v1.0.5/go.mod h1:QV8Hv/iy04NyLBxAdO9njL0iVPN1S4d/A3NVv1V36o8= -github.com/go-test/deep v1.1.0/go.mod h1:5C2ZWiW0ErCdrYzpqxLbTX7MG14M9iiw8DgHncVwcsE= github.com/go-yaml/yaml v2.1.0+incompatible h1:RYi2hDdss1u4YE7GwixGzWwVo47T8UQwnTLB6vQiq+o= github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus/v5 v5.0.3/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk= @@ -1150,6 +1170,7 @@ github.com/google/pprof v0.0.0-20230602150820-91b7bce49751/go.mod h1:Jh3hGz2jkYa github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -1256,8 +1277,8 @@ github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmK github.com/hugelgupf/socketpair v0.0.0-20190730060125-05d35a94e714/go.mod h1:2Goc3h8EklBH5mspfHFxBnEoURQCGzQQH1ga9Myjvis= github.com/huin/goupnp v0.0.0-20161224104101-679507af18f3/go.mod h1:MZ2ZmwcBpvOoJ22IJsc7va19ZwoheaBk43rKg12SKag= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= -github.com/huin/goupnp v1.2.0 h1:uOKW26NG1hsSSbXIZ1IR7XP9Gjd1U8pnLaCMgntmkmY= -github.com/huin/goupnp v1.2.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= +github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc= +github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8= github.com/huin/goutil v0.0.0-20170803182201-1ca381bf3150/go.mod h1:PpLOETDnJ0o3iZrZfqZzyLl6l7F3c6L1oWn7OICBi6o= github.com/hydrogen18/memlistener v0.0.0-20200120041712-dcc25e7acd91/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= @@ -1779,7 +1800,6 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.6/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-runewidth v0.0.14 h1:+xnbZSEeDbOIg5/mE6JF0w6n9duR1l3/WmbinWVwUuU= github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= @@ -1836,6 +1856,9 @@ github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:F github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iPfkHRY= +github.com/mmcloughlin/addchain v0.4.0/go.mod h1:A86O+tHqZLMNO4w6ZZ4FlVQEadcoqkyU72HC5wJ4RlU= +github.com/mmcloughlin/profile v0.1.1/go.mod h1:IhHD7q1ooxgwTgjxQYkACGA77oFTDdFVejUS1/tS/qU= github.com/moby/sys/mountinfo v0.6.2/go.mod h1:IJb6JQeOklcdMU9F5xQ8ZALD+CUr5VlGpwtX+VE0rpI= github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= @@ -1942,36 +1965,32 @@ github.com/olekukonko/tablewriter v0.0.1/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXW github.com/olekukonko/tablewriter v0.0.2-0.20190409134802-7e037d187b0c/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= -github.com/onflow/atree v0.5.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= -github.com/onflow/atree v0.6.0/go.mod h1:gBHU0M05qCbv9NN0kijLWMgC47gHVNBIp4KmsVFi0tc= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f h1:Z8/PgTqOgOg02MTRpTBYO2k16FE6z4wEOtaC2WBR9Xo= github.com/onflow/atree v0.6.1-0.20230711151834-86040b30171f/go.mod h1:xvP61FoOs95K7IYdIYRnNcYQGf4nbF/uuJ0tHf4DRuM= -github.com/onflow/cadence v0.39.13-stable-cadence/go.mod h1:SxT8/IEkS1drFj2ofUEK9S6KyJ5GQbrm0LX4EFCp/7Q= -github.com/onflow/cadence v1.0.0-preview.2 h1:EYJiaYHU35AilrvbrfWhYO3thTCeWBX3aHyy5RPeEag= -github.com/onflow/cadence v1.0.0-preview.2/go.mod h1:JmTHN1dOmN5330TkvSC8P50RcteXegxQS5+Aw2zr7+I= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb h1:OpNQ8+ZPBg5DHchnZyiBySz/OQc4uIeptTm7cBfCvOA= +github.com/onflow/cadence v1.0.0-preview.2.0.20240122125204-5ce1f36f95bb/go.mod h1:odXGZZ/wGNA5mwT8bC9v8u8EXACHllB2ABSZK65TGL8= github.com/onflow/crypto v0.25.0 h1:BeWbLsh3ZD13Ej+Uky6kg1PL1ZIVBDVX+2MVBNwqddg= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425 h1:zvLHFxySeg61/dgp/IbvaN+k4BXPuAhBOslrPQjrX9Q= -github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20231212203043-37cbe453d425/go.mod h1:N+1bEs/159Efg75hSQIkb90FVinxUMxL/6mA3I6dXtQ= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86 h1:5dDtY8iItVVvIY+YXbavGDMaVz4Gq7sq4ILF/cZb7/8= -github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20231204202154-f8dfacb39d86/go.mod h1:6XIbPht7u7ADszXSHI2APY+OL78XVaUU8+OdgWEElAY= -github.com/onflow/flow-emulator v0.59.1-0.20231213222325-7896fff7cacb h1:nA795oIS0QjOqriXyKjZq6WX/qNOeLnXcgvZtoOFjmU= -github.com/onflow/flow-emulator v0.59.1-0.20231213222325-7896fff7cacb/go.mod h1:PlkJg0YStLlk0qwgpMaqs9jsOgHzFCHxS8tGI25vwMg= +github.com/onflow/crypto v0.25.0/go.mod h1:C8FbaX0x8y+FxWjbkHy0Q4EASCDR9bSPWZqlpCLYyVI= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b h1:ggf5emTr01eHx8J9ME7BYVk6aa3CMg0fdssJL7i89lo= +github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:uQMLA8dV0qO4By7bcCGNGo2jHfqOkY9RP8H/iCltO+k= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b h1:7AIREWwdF+sQlFblzj3BjMu+X3MBVx0Bn9k3v5fikrk= +github.com/onflow/flow-core-contracts/lib/go/templates v0.15.1-0.20240122220432-a5993d06ce1b/go.mod h1:HrqH9MDQF62pYeWmKkVVXUPC3xcgDvkV7gF77sjG9VI= +github.com/onflow/flow-emulator v0.59.1-0.20240122200325-58ef35ed4aed h1:ybV/+STwazQhJrTuWklYWcuZdWFpti5b0EkTE2MIe7Q= +github.com/onflow/flow-emulator v0.59.1-0.20240122200325-58ef35ed4aed/go.mod h1:UfcYYcaMMFHvSBA78+goDf2K/eNDrDoBlqjN+JPZyJE= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596 h1:MTgrwXkiWwNysYpWGzWjc1n9w1nfXvizmGkSAuEY6jk= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20231212194336-a2802ba36596/go.mod h1:uugR8U8Rlk2Xbn1ne7WWkPIcLReOyyXeQ/6tBg2Lsu8= -github.com/onflow/flow-go-sdk v0.41.7-stable-cadence/go.mod h1:ejVN+bqcsTHVvRpDDJDoBNdmcxUfFMW4IvdTbMeQ/hQ= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2 h1:vUVO6m85BiT8c50Oc8YGc3CU+sGqiKW9FZbmiRph2dU= -github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2/go.mod h1:mbLrR3MkYbi9LH3yasDj1jrR4QTR8vjRLVFCm4jMHn0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca h1:7yG8dYqMzWzTZWJ17dnBdS01UDlOBnf1dd1rWKcFdY0= +github.com/onflow/flow-go-sdk v0.44.0-stable-cadence.2.0.20240122164005-147ad40664ca/go.mod h1:O5+TK1qs2c1R5X4TEQp4m2c/YhlCjwdW7bsRcUB1U8s= github.com/onflow/flow-go/crypto v0.24.7/go.mod h1:fqCzkIBBMRRkciVrvW21rECKq1oD7Q6u+bCI78lfNX0= github.com/onflow/flow-go/crypto v0.25.0 h1:6lmoiAQ3APCF+nV7f4f2AXL3PuDKqQiWqRJXmjrMEq4= github.com/onflow/flow-go/crypto v0.25.0/go.mod h1:OOb2vYcS8AOCajBClhHTJ0NKftFl1RQgTQ0+Vh4nbqk= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14 h1:pHblchb5rKrpt1xmeKnMNq3utzBi2bvepHu7z1ZPWjw= -github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20231213195450-0b951b342b14/go.mod h1:OKA2xWNugqqbaFSmxdb1VWixtuqSdz/VK1MlbdcIUxw= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991 h1:UPDAaWUag3epvX+yJ5IrFESQr1P53rDDqw+ShGxg5/k= +github.com/onflow/flow-nft/lib/go/contracts v1.1.1-0.20240122215824-10d8a31d1991/go.mod h1:CPLntX0UYLgZvvgoHfEwaaeSL3IS/eFzAdfbT0aIhb4= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20221202093946-932d1c70e288/go.mod h1:gQxYqCfkI8lpnKsmIjwtN2mV/N2PIwc1I+RUK4HPIc8= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2 h1:+rT+UsfTR39JZO8ht2+4fkaWfHw74SCj1fyz1lWuX8A= github.com/onflow/flow/protobuf/go/flow v0.3.2-0.20231213135419-ae911cc351a2/go.mod h1:NA2pX2nw8zuaxfKphhKsk00kWLwfd+tv8mS23YXO4Sk= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d h1:QcOAeEyF3iAUHv21LQ12sdcsr0yFrJGoGLyCAzYYtvI= github.com/onflow/go-bitswap v0.0.0-20230703214630-6d3db958c73d/go.mod h1:GCPpiyRoHncdqPj++zPr9ZOYBX4hpJ0pYZRYqSE8VKk= -github.com/onflow/sdks v0.5.0/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba h1:rIehuhO6bj4FkwE4VzwEjX7MoAlOhUJENBJLqDqVxAo= github.com/onflow/sdks v0.5.1-0.20230912225508-b35402f12bba/go.mod h1:F0dj0EyHC55kknLkeD10js4mo14yTdMotnWMslPirrU= github.com/onflow/wal v0.0.0-20230529184820-bc9f8244608d h1:gAEqYPn3DS83rHIKEpsajnppVD1+zwuYPFyeDVFaQvg= @@ -2045,7 +2064,7 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= -github.com/pkg/term v1.1.0/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= +github.com/pkg/term v1.2.0-beta.2/go.mod h1:E25nymQcrSllhX42Ok8MRm1+hyBdHY0dCeiKZ9jpNGw= github.com/plus3it/gorecurcopy v0.0.1 h1:H7AgvM0N/uIo7o1PQRlewEGQ92BNr7DqbPy5lnR3uJI= github.com/plus3it/gorecurcopy v0.0.1/go.mod h1:NvVTm4RX68A1vQbHmHunDO4OtBLVroT6CrsiqAzNyJA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -2120,7 +2139,6 @@ github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qq github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.2.1-0.20211004051800-57c86be7915a/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis= github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rjeczalik/notify v0.9.1/go.mod h1:rKwnCoCGeuQnwBtTSPL9Dad03Vh2n40ePRrjvIXnJho= @@ -2149,7 +2167,6 @@ github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g= -github.com/schollz/progressbar/v3 v3.8.3/go.mod h1:pWnVCjSBZsT2X3nx9HfRdnCDrpbevliMeoEVhStwHko= github.com/schollz/progressbar/v3 v3.13.1 h1:o8rySDYiQ59Mwzy2FELeHY5ZARXZTVJC7iHD6PEFUiE= github.com/schollz/progressbar/v3 v3.13.1/go.mod h1:xvrbki8kfT1fzWzBT/UZd9L6GA+jdL7HAgq2RFnO6fQ= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= @@ -2268,6 +2285,8 @@ github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXl github.com/subosito/gotenv v1.4.2 h1:X1TuBLAMDFbaTAChgCBLu3DU3UPyELpnF2jjJ2cz/S8= github.com/subosito/gotenv v1.4.2/go.mod h1:ayKnFf/c6rvx/2iiLrJUk1e6plDbT3edrFNGqEflhK0= github.com/supranational/blst v0.3.10/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= +github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbeoEm4= +github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= github.com/syndtr/goleveldb v1.0.1-0.20190923125748-758128399b1d/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= @@ -2468,7 +2487,6 @@ golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210506145944-38f3c27a63bf/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= -golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= @@ -2477,9 +2495,10 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220826181053-bd7e27e6170d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= -golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= -golang.org/x/crypto v0.16.0 h1:mMMrFzRSCF0GvB7Ne27XVtVAaXLrPmgPC7/v0tkwHaY= +golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2495,10 +2514,8 @@ golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp v0.0.0-20221110155412-d0897a79cd37/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= -golang.org/x/exp v0.0.0-20230321023759-10a507213a29/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= -golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df h1:UA2aFVmmsIlefxMk29Dp2juaUSth8Pyn3Tq5Y5mJGME= -golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc h1:ao2WRsKSzW6KuUY9IWPwWahcHCgR0s52IfwutMfEbdM= +golang.org/x/exp v0.0.0-20240103183307-be819d1f06fc/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -2542,6 +2559,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.6.0/go.mod h1:4mET923SAdbXp2ki8ey+zGs1SLqsuM2Y0uvdZR/fUNI= golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.14.0 h1:dGoOF9QVLYng8IHTm7BAyWqCqSheQ5pYWGhzW00YJr0= golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180719180050-a680a1efc54d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -2628,6 +2646,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -2783,7 +2802,6 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210910150752-751e447fb3d0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -2825,7 +2843,6 @@ golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20220722155259-a9ba230a4035/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -2834,6 +2851,7 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2852,6 +2870,7 @@ golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2942,6 +2961,7 @@ golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc golang.org/x/tools v0.2.0/go.mod h1:y4OqIKeOV/fWJetJ8bXPU1sEVniLMIyDAZWeHdV+NTA= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.16.0 h1:GO788SKMRunPIBCXiQyo2AaexLstOrVhuAL5YwsckQM= golang.org/x/tools v0.16.0/go.mod h1:kYVVN6I1mBNoB1OX+noeBjbRk4IUEPa7JJ+TJMEooJ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -3267,7 +3287,6 @@ gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce h1:+JknDZhAj8YMt7GC73Ei8pv4MzjDUNPHgQWJdtMAaDU= gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce/go.mod h1:5AcXVHNjg+BDxry382+8OKon8SEWiKktQR07RKPsv1c= gopkg.in/olebedev/go-duktape.v3 v3.0.0-20200316214253-d7b0ff38cac9/go.mod h1:uAJfkITjFhyEEuUfm7bsmCZRbW5WRq8s9EY8HZ6hCns= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= @@ -3360,6 +3379,8 @@ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8 rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= +rsc.io/tmplfunc v0.0.3 h1:53XFQh69AfOa8Tw0Jm7t+GV7KZhOi6jzsCzTtKbMvzU= +rsc.io/tmplfunc v0.0.3/go.mod h1:AG3sTPzElb1Io3Yg4voV9AGZJuleGAwaVRxL9M49PhA= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= diff --git a/integration/testnet/network.go b/integration/testnet/network.go index e47e94da3a3..267cae3cb72 100644 --- a/integration/testnet/network.go +++ b/integration/testnet/network.go @@ -42,7 +42,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/factory" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/epochs" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/network/p2p/keyutils" @@ -1043,7 +1042,7 @@ func BootstrapNetwork(networkConf NetworkConfig, bootstrapDir string, chainID fl // IMPORTANT: we must use this ordering when writing the DKG keys as // this ordering defines the DKG participant's indices - stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), order.Canonical) + stakedNodeInfos := bootstrap.Sort(toNodeInfos(stakedConfs), flow.Canonical) dkg, err := runBeaconKG(stakedConfs) if err != nil { @@ -1310,7 +1309,7 @@ func runBeaconKG(confs []ContainerConfig) (dkgmod.DKGData, error) { func setupClusterGenesisBlockQCs(nClusters uint, epochCounter uint64, confs []ContainerConfig) ([]*cluster.Block, flow.AssignmentList, []*flow.QuorumCertificate, error) { participantsUnsorted := toParticipants(confs) - participants := participantsUnsorted.Sort(order.Canonical) + participants := participantsUnsorted.Sort(flow.Canonical) collectors := participants.Filter(filter.HasRole(flow.RoleCollection)) assignments := unittest.ClusterAssignment(nClusters, collectors) clusters, err := factory.NewClusterList(assignments, collectors) @@ -1343,7 +1342,7 @@ func setupClusterGenesisBlockQCs(nClusters uint, epochCounter uint64, confs []Co } // must order in canonical ordering otherwise decoding signer indices from cluster QC would fail - clusterCommittee := bootstrap.ToIdentityList(clusterNodeInfos).Sort(order.Canonical) + clusterCommittee := bootstrap.ToIdentityList(clusterNodeInfos).Sort(flow.Canonical) qc, err := run.GenerateClusterRootQC(clusterNodeInfos, clusterCommittee, block) if err != nil { return nil, nil, nil, fmt.Errorf("fail to generate cluster root QC with clusterNodeInfos %v, %w", diff --git a/integration/testnet/node_config.go b/integration/testnet/node_config.go index bf1469a0c20..011d280ac2a 100644 --- a/integration/testnet/node_config.go +++ b/integration/testnet/node_config.go @@ -149,3 +149,10 @@ func WithAdditionalFlag(flag string) func(config *NodeConfig) { func WithAdditionalFlagf(format string, a ...any) func(config *NodeConfig) { return WithAdditionalFlag(fmt.Sprintf(format, a...)) } + +// WithMetricsServer exposes the metrics server +func WithMetricsServer() func(config *NodeConfig) { + return func(config *NodeConfig) { + config.EnableMetricsServer = true + } +} diff --git a/integration/tests/access/cohort1/access_api_test.go b/integration/tests/access/cohort1/access_api_test.go index b7cc1093e0d..036c0a837f8 100644 --- a/integration/tests/access/cohort1/access_api_test.go +++ b/integration/tests/access/cohort1/access_api_test.go @@ -72,7 +72,7 @@ func (s *AccessAPISuite) SetupTest() { flow.RoleAccess, testnet.WithLogLevel(zerolog.FatalLevel), // make sure test continues to test as expected if the default config changes - testnet.WithAdditionalFlagf("--script-execution-mode=%s", backend.ScriptExecutionModeExecutionNodesOnly), + testnet.WithAdditionalFlagf("--script-execution-mode=%s", backend.IndexQueryModeExecutionNodesOnly), ) indexingAccessConfig := testnet.NewNodeConfig( @@ -83,7 +83,7 @@ func (s *AccessAPISuite) SetupTest() { testnet.WithAdditionalFlag("--execution-data-retry-delay=1s"), testnet.WithAdditionalFlag("--execution-data-indexing-enabled=true"), testnet.WithAdditionalFlagf("--execution-state-dir=%s", testnet.DefaultExecutionStateDir), - testnet.WithAdditionalFlagf("--script-execution-mode=%s", backend.ScriptExecutionModeLocalOnly), + testnet.WithAdditionalFlagf("--script-execution-mode=%s", backend.IndexQueryModeLocalOnly), ) consensusConfigs := []func(config *testnet.NodeConfig){ diff --git a/integration/tests/access/cohort2/observer_test.go b/integration/tests/access/cohort2/observer_test.go index cd217b63379..fa853c4dc4a 100644 --- a/integration/tests/access/cohort2/observer_test.go +++ b/integration/tests/access/cohort2/observer_test.go @@ -402,7 +402,7 @@ func (s *ObserverSuite) getRestEndpoints() []RestEndpointTest { block := unittest.BlockFixture() executionResult := unittest.ExecutionResultFixture() collection := unittest.CollectionFixture(2) - eventType := "A.0123456789abcdef.flow.event" + eventType := unittest.EventTypeFixture(flow.Localnet) return []RestEndpointTest{ { diff --git a/integration/tests/access/cohort3/collection_indexing_test.go b/integration/tests/access/cohort3/collection_indexing_test.go new file mode 100644 index 00000000000..dcf23e175ab --- /dev/null +++ b/integration/tests/access/cohort3/collection_indexing_test.go @@ -0,0 +1,120 @@ +package cohort3 + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/suite" + + "github.com/onflow/flow-go/integration/testnet" + "github.com/onflow/flow-go/model/flow" +) + +// This suite tests collection syncing using the ingestion engine and the indexer. + +const lastFullBlockMetric = "access_ingestion_last_full_finalized_block_height" + +func TestCollectionIndexing(t *testing.T) { + suite.Run(t, new(CollectionIndexingSuite)) +} + +type CollectionIndexingSuite struct { + suite.Suite + net *testnet.FlowNetwork + + cancel context.CancelFunc +} + +func (s *CollectionIndexingSuite) SetupTest() { + // access_1 is not running the indexer, so all collections are indexed using the ingestion engine + defaultAccessOpts := []func(config *testnet.NodeConfig){ + testnet.WithLogLevel(zerolog.FatalLevel), + testnet.WithAdditionalFlag("--execution-data-sync-enabled=true"), + testnet.WithAdditionalFlagf("--execution-data-dir=%s", testnet.DefaultExecutionDataServiceDir), + testnet.WithAdditionalFlagf("--execution-state-dir=%s", testnet.DefaultExecutionStateDir), + testnet.WithMetricsServer(), + } + // access_2 is running the indexer, so all collections are indexed using the indexer + testANOpts := append(defaultAccessOpts, + testnet.WithLogLevel(zerolog.DebugLevel), + testnet.WithAdditionalFlag("--execution-data-indexing-enabled=true"), + ) + + nodeConfigs := []testnet.NodeConfig{ + testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleCollection, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleExecution, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleConsensus, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleVerification, testnet.WithLogLevel(zerolog.FatalLevel)), + testnet.NewNodeConfig(flow.RoleAccess, defaultAccessOpts...), + testnet.NewNodeConfig(flow.RoleAccess, testANOpts...), + } + + // prepare the network + conf := testnet.NewNetworkConfig("access_collection_indexing_test", nodeConfigs) + s.net = testnet.PrepareFlowNetwork(s.T(), conf, flow.Localnet) + + // start the network + ctx, cancel := context.WithCancel(context.Background()) + s.cancel = cancel + + s.net.Start(ctx) +} + +func (s *CollectionIndexingSuite) TearDownTest() { + if s.net != nil { + s.net.Remove() + s.net = nil + } + if s.cancel != nil { + s.cancel() + s.cancel = nil + } +} + +func (s *CollectionIndexingSuite) Test() { + // start the network with access_2 disconnected. + // this simulates it falling behind on syncing collections + access2 := s.net.ContainerByName("access_2") + s.Require().NoError(access2.Disconnect()) + + // wait for access_1 to sync collections + targetBlockCount := uint64(50) + s.Eventually(func() bool { + value, err := s.getLastFullHeight("access_1") + s.T().Logf("access_1 last full height: %d", value) + return err == nil && value > targetBlockCount + }, 60*time.Second, 1*time.Second) + + // stop the collection nodes + // this will prevent access_2 from syncing collections from the network + s.Require().NoError(s.net.ContainerByName("collection_1").Pause()) + s.Require().NoError(s.net.ContainerByName("collection_2").Pause()) + + // now start access_2, and wait for it to catch up with collections + s.Require().NoError(access2.Connect()) + + s.Eventually(func() bool { + value, err := s.getLastFullHeight("access_2") + s.T().Logf("access_2 last full height: %d", value) + return err == nil && value > targetBlockCount + }, 60*time.Second, 1*time.Second) +} + +func (s *CollectionIndexingSuite) getLastFullHeight(containerName string) (uint64, error) { + node := s.net.ContainerByName(containerName) + metricsURL := fmt.Sprintf("http://0.0.0.0:%s/metrics", node.Port(testnet.MetricsPort)) + values := s.net.GetMetricFromContainer(s.T(), containerName, metricsURL, lastFullBlockMetric) + + if len(values) == 0 { + return 0, fmt.Errorf("no values found") + } + + return uint64(values[0].GetGauge().GetValue()), nil +} diff --git a/integration/tests/access/cohort3/consensus_follower_test.go b/integration/tests/access/cohort3/consensus_follower_test.go index d227a09ad2e..ac2bf3b5800 100644 --- a/integration/tests/access/cohort3/consensus_follower_test.go +++ b/integration/tests/access/cohort3/consensus_follower_test.go @@ -117,7 +117,6 @@ func (s *ConsensusFollowerSuite) TestReceiveBlocks() { func (s *ConsensusFollowerSuite) buildNetworkConfig() { // staked access node - unittest.IdentityFixture() s.stakedID = unittest.IdentifierFixture() stakedConfig := testnet.NewNodeConfig( flow.RoleAccess, diff --git a/integration/tests/access/cohort3/rest_state_stream_test.go b/integration/tests/access/cohort3/rest_state_stream_test.go index ad6eab9d8b5..6e22e7d0390 100644 --- a/integration/tests/access/cohort3/rest_state_stream_test.go +++ b/integration/tests/access/cohort3/rest_state_stream_test.go @@ -60,8 +60,10 @@ func (s *RestStateStreamSuite) SetupTest() { flow.RoleAccess, testnet.WithLogLevel(zerolog.InfoLevel), testnet.WithAdditionalFlag("--execution-data-sync-enabled=true"), - testnet.WithAdditionalFlag(fmt.Sprintf("--execution-data-dir=%s", testnet.DefaultExecutionDataServiceDir)), + testnet.WithAdditionalFlagf("--execution-data-dir=%s", testnet.DefaultExecutionDataServiceDir), testnet.WithAdditionalFlag("--execution-data-retry-delay=1s"), + testnet.WithAdditionalFlag("--execution-data-indexing-enabled=true"), + testnet.WithAdditionalFlagf("--execution-state-dir=%s", testnet.DefaultExecutionStateDir), ) // add the ghost (access) node config @@ -176,15 +178,20 @@ func (s *RestStateStreamSuite) requireEvents(receivedEventsResponse []*backend.E for eventType, receivedEventList := range receivedEventMap { // get events by block id and event type - response, err := MakeApiRequest(grpcClient.GetEventsForBlockIDs, grpcCtx, - &accessproto.GetEventsForBlockIDsRequest{BlockIds: [][]byte{convert.IdentifierToMessage(receivedEventResponse.BlockID)}, - Type: string(eventType)}) + response, err := MakeApiRequest( + grpcClient.GetEventsForBlockIDs, + grpcCtx, + &accessproto.GetEventsForBlockIDsRequest{ + BlockIds: [][]byte{convert.IdentifierToMessage(receivedEventResponse.BlockID)}, + Type: string(eventType), + }, + ) require.NoError(s.T(), err) require.Equal(s.T(), 1, len(response.Results), "expect to get 1 result") expectedEventsResult := response.Results[0] require.Equal(s.T(), expectedEventsResult.BlockHeight, receivedEventResponse.Height, "expect the same block height") - require.Equal(s.T(), len(expectedEventsResult.Events), len(receivedEventList), "expect the same count of events") + require.Equal(s.T(), len(expectedEventsResult.Events), len(receivedEventList), "expect the same count of events: want: %+v, got: %+v", expectedEventsResult.Events, receivedEventList) for i, event := range receivedEventList { require.Equal(s.T(), expectedEventsResult.Events[i].EventIndex, event.EventIndex, "expect the same event index") diff --git a/ledger/trie.go b/ledger/trie.go index 17f2ba1a232..a46b90780b2 100644 --- a/ledger/trie.go +++ b/ledger/trie.go @@ -319,6 +319,16 @@ func (p *Payload) Key() (Key, error) { return *k, nil } +// EncodedKey returns payload key. +// CAUTION: do not modify returned encoded key +// because it shares underlying data with payload key. +func (p *Payload) EncodedKey() []byte { + if p == nil { + return nil + } + return p.encKey +} + // Value returns payload value. // CAUTION: do not modify returned value because it shares underlying data with payload value. func (p *Payload) Value() Value { diff --git a/model/bootstrap/node_info.go b/model/bootstrap/node_info.go index 62a33f6f442..0c5f8b7169b 100644 --- a/model/bootstrap/node_info.go +++ b/model/bootstrap/node_info.go @@ -4,9 +4,10 @@ package bootstrap import ( "encoding/json" "fmt" - "sort" "strings" + "golang.org/x/exp/slices" + sdk "github.com/onflow/flow-go-sdk" sdkcrypto "github.com/onflow/flow-go-sdk/crypto" @@ -402,12 +403,14 @@ func FilterByRole(nodes []NodeInfo, role flow.Role) []NodeInfo { return filtered } -// Sort sorts the NodeInfo list using the given ordering. +// Sort sorts the NodeInfo list using the given order. +// +// The sorted list is returned and the original list is untouched. func Sort(nodes []NodeInfo, order flow.IdentityOrder) []NodeInfo { dup := make([]NodeInfo, len(nodes)) copy(dup, nodes) - sort.Slice(dup, func(i, j int) bool { - return order(dup[i].Identity(), dup[j].Identity()) + slices.SortFunc(dup, func(i, j NodeInfo) int { + return order(i.Identity(), j.Identity()) }) return dup } diff --git a/model/bootstrap/node_info_test.go b/model/bootstrap/node_info_test.go index 39294de5f69..b00f6cd986a 100644 --- a/model/bootstrap/node_info_test.go +++ b/model/bootstrap/node_info_test.go @@ -9,14 +9,36 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/bootstrap" - "github.com/onflow/flow-go/model/flow/order" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" ) -func TestSort(t *testing.T) { +func TestIdentityListCanonical(t *testing.T) { nodes := unittest.NodeInfosFixture(20) - nodes = bootstrap.Sort(nodes, order.Canonical) - require.True(t, bootstrap.ToIdentityList(nodes).Sorted(order.Canonical)) + // make sure the list is not sorted + nodes[0].NodeID[0], nodes[1].NodeID[0] = 2, 1 + require.False(t, flow.IsIdentifierCanonical(nodes[0].NodeID, nodes[1].NodeID)) + ids := bootstrap.ToIdentityList(nodes) + assert.False(t, flow.IsIdentityListCanonical(ids)) + + // make a copy of the original list of nodes + nodesCopy := make([]bootstrap.NodeInfo, len(nodes)) + copy(nodesCopy, nodes) + + sortedNodes := bootstrap.Sort(nodes, flow.Canonical) + sortedIds := bootstrap.ToIdentityList(sortedNodes) + require.True(t, flow.IsIdentityListCanonical(sortedIds)) + // make sure original list didn't change + assert.Equal(t, nodesCopy, nodes) + + // check `IsIdentityListCanonical` detects order equality in a sorted list + nodes[1] = nodes[10] // add a duplication + copy(nodesCopy, nodes) + sortedNodes = bootstrap.Sort(nodes, flow.Canonical) + sortedIds = bootstrap.ToIdentityList(sortedNodes) + assert.False(t, flow.IsIdentityListCanonical(sortedIds)) + // make sure original list didn't change + assert.Equal(t, nodesCopy, nodes) } func TestNodeConfigEncodingJSON(t *testing.T) { diff --git a/model/convert/service_event.go b/model/convert/service_event.go index 669daca5a84..21bf853a956 100644 --- a/model/convert/service_event.go +++ b/model/convert/service_event.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/assignment" - "github.com/onflow/flow-go/model/flow/order" ) // ServiceEvent converts a service event encoded as the generic flow.Event @@ -199,22 +198,25 @@ func convertServiceEventEpochSetup(event flow.Event) (*flow.ServiceEvent, error) DKGPhase3FinalView: uint64(dkgPhase3FinalView), } - // Cadence's unsafeRandom().toString() produces a string of variable length. - // Here we pad it with enough 0s to meet the required length. - paddedRandomSrcHex := fmt.Sprintf( - "%0*s", - 2*flow.EpochSetupRandomSourceLength, - string(randomSrcHex), - ) - setup.RandomSource, err = hex.DecodeString(paddedRandomSrcHex) + // random source from the event must be a hex string + // containing exactly 128 bits (equivalent to 16 bytes or 32 hex characters) + setup.RandomSource, err = hex.DecodeString(string(randomSrcHex)) if err != nil { return nil, fmt.Errorf( "could not decode random source hex (%v): %w", - paddedRandomSrcHex, + randomSrcHex, err, ) } + if len(setup.RandomSource) != flow.EpochSetupRandomSourceLength { + return nil, fmt.Errorf( + "random source in epoch setup event must be of (%d) bytes, got (%d)", + flow.EpochSetupRandomSourceLength, + len(setup.RandomSource), + ) + } + // parse cluster assignments setup.Assignments, err = convertClusterAssignments(cdcClusters.Values) if err != nil { @@ -631,7 +633,7 @@ func convertParticipants(cdcParticipants []cadence.Value) (flow.IdentityList, er participants = append(participants, identity) } - participants = participants.Sort(order.Canonical) + participants = participants.Sort(flow.Canonical) return participants, nil } diff --git a/model/convert/service_event_test.go b/model/convert/service_event_test.go index 9c50a98d1c3..9afd3a32499 100644 --- a/model/convert/service_event_test.go +++ b/model/convert/service_event_test.go @@ -4,11 +4,10 @@ import ( "fmt" "testing" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/onflow/cadence" "github.com/onflow/cadence/encoding/ccf" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/onflow/flow-go/fvm/systemcontracts" "github.com/onflow/flow-go/model/convert" @@ -39,6 +38,55 @@ func TestEventConversion(t *testing.T) { }, ) + t.Run( + "epoch setup with random source with leading zeroes", func(t *testing.T) { + + fixture, _ := unittest.EpochSetupFixtureByChainID(chainID) + // all zero source to cover all cases of endiannesses + randomSource := make([]byte, flow.EpochSetupRandomSourceLength) + // update the random source in event fixture + fixture.Payload = unittest.EpochSetupFixtureCCF(randomSource) + + // convert Cadence types to Go types + event, err := convert.ServiceEvent(chainID, fixture) + require.NoError(t, err) + require.NotNil(t, event) + + // cast event type to epoch setup + _, ok := event.Event.(*flow.EpochSetup) + require.True(t, ok) + }, + ) + + t.Run( + "epoch setup with short random source", func(t *testing.T) { + + fixture, _ := unittest.EpochSetupFixtureByChainID(chainID) + // update the random source in event fixture + randomSource := unittest.EpochSetupRandomSourceFixture() + fixture.Payload = unittest.EpochSetupFixtureCCF(randomSource[:flow.EpochSetupRandomSourceLength-1]) + + // convert Cadence types to Go types + event, err := convert.ServiceEvent(chainID, fixture) + require.Error(t, err) + require.Nil(t, event) + }, + ) + + t.Run( + "epoch setup with non-hex random source", func(t *testing.T) { + + fixture, _ := unittest.EpochSetupFixtureByChainID(chainID) + // update the random source in event fixture + fixture.Payload = unittest.EpochSetupCCFWithNonHexRandomSource() + + // convert Cadence types to Go types + event, err := convert.ServiceEvent(chainID, fixture) + require.Error(t, err) + require.Nil(t, event) + }, + ) + t.Run( "epoch commit", func(t *testing.T) { diff --git a/engine/access/state_stream/event.go b/model/events/parse.go similarity index 65% rename from engine/access/state_stream/event.go rename to model/events/parse.go index c88c78c9a66..8c7eebfdf4f 100644 --- a/engine/access/state_stream/event.go +++ b/model/events/parse.go @@ -1,4 +1,4 @@ -package state_stream +package events import ( "fmt" @@ -57,3 +57,24 @@ func ParseEvent(eventType flow.EventType) (*ParsedEvent, error) { return nil, fmt.Errorf("invalid event type: %s", eventType) } + +// ValidateEvent validates an event type is properly formed and for the correct network, and returns +// a parsed event. If the event type is invalid, an error is returned. +func ValidateEvent(eventType flow.EventType, chain flow.Chain) (*ParsedEvent, error) { + parsed, err := ParseEvent(eventType) + if err != nil { + return nil, err + } + + // only account type events have an address field + if parsed.Type != AccountEventType { + return parsed, nil + } + + contractAddress := flow.HexToAddress(parsed.Address) + if !chain.IsValid(contractAddress) { + return nil, fmt.Errorf("invalid event contract address") + } + + return parsed, nil +} diff --git a/model/events/parse_test.go b/model/events/parse_test.go new file mode 100644 index 00000000000..053bbd3ec92 --- /dev/null +++ b/model/events/parse_test.go @@ -0,0 +1,149 @@ +package events_test + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/model/events" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" +) + +func TestParseEvent(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + eventType flow.EventType + expected events.ParsedEvent + }{ + { + name: "flow event", + eventType: "flow.AccountCreated", + expected: events.ParsedEvent{ + Type: events.ProtocolEventType, + EventType: "flow.AccountCreated", + Contract: "flow", + ContractName: "flow", + Name: "AccountCreated", + }, + }, + { + name: "account event", + eventType: "A.0000000000000001.Contract1.EventA", + expected: events.ParsedEvent{ + Type: events.AccountEventType, + EventType: "A.0000000000000001.Contract1.EventA", + Address: "0000000000000001", + Contract: "A.0000000000000001.Contract1", + ContractName: "Contract1", + Name: "EventA", + }, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + event, err := events.ParseEvent(test.eventType) + require.NoError(t, err) + + assert.Equal(t, test.expected.Type, event.Type) + assert.Equal(t, test.expected.EventType, event.EventType) + assert.Equal(t, test.expected.Address, event.Address) + assert.Equal(t, test.expected.Contract, event.Contract) + assert.Equal(t, test.expected.Name, event.Name) + }) + } +} + +func TestParseEvent_Invalid(t *testing.T) { + t.Parallel() + + eventTypes := []flow.EventType{ + "", // not enough parts + "invalid", // not enough parts + "invalid.event", // invalid first part + "B.0000000000000001.invalid.event", // invalid first part + "flow", // incorrect number of parts for protocol event + "flow.invalid.event", // incorrect number of parts for protocol event + "A.0000000000000001.invalid", // incorrect number of parts for account event + "A.0000000000000001.invalid.a.b", // incorrect number of parts for account event + + } + + for _, eventType := range eventTypes { + _, err := events.ParseEvent(eventType) + assert.Error(t, err, "expected error for event type: %s", eventType) + } +} + +func TestValidateEvent(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + eventType flow.EventType + expected events.ParsedEvent + }{ + { + name: "flow event", + eventType: "flow.AccountCreated", + expected: events.ParsedEvent{ + Type: events.ProtocolEventType, + EventType: "flow.AccountCreated", + Contract: "flow", + ContractName: "flow", + Name: "AccountCreated", + }, + }, + { + name: "account event", + eventType: "A.0000000000000001.Contract1.EventA", + expected: events.ParsedEvent{ + Type: events.AccountEventType, + EventType: "A.0000000000000001.Contract1.EventA", + Address: "0000000000000001", + Contract: "A.0000000000000001.Contract1", + ContractName: "Contract1", + Name: "EventA", + }, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + event, err := events.ValidateEvent(test.eventType, flow.MonotonicEmulator.Chain()) + require.NoError(t, err) + + assert.Equal(t, test.expected.Type, event.Type) + assert.Equal(t, test.expected.EventType, event.EventType) + assert.Equal(t, test.expected.Address, event.Address) + assert.Equal(t, test.expected.Contract, event.Contract) + assert.Equal(t, test.expected.Name, event.Name) + }) + } +} + +func TestValidateEvent_Invalid(t *testing.T) { + t.Parallel() + + eventTypes := []flow.EventType{ + "", // not enough parts + "invalid", // not enough parts + "invalid.event", // invalid first part + "B.0000000000000001.invalid.event", // invalid first part + "flow", // incorrect number of parts for protocol event + "flow.invalid.event", // incorrect number of parts for protocol event + "A.0000000000000001.invalid", // incorrect number of parts for account event + "A.0000000000000001.invalid.a.b", // incorrect number of parts for account event + flow.EventType(fmt.Sprintf("A.%s.Contract1.EventA", unittest.RandomAddressFixture())), // address from wrong chain + } + + for _, eventType := range eventTypes { + _, err := events.ValidateEvent(eventType, flow.MonotonicEmulator.Chain()) + assert.Error(t, err, "expected error for event type: %s", eventType) + } +} diff --git a/model/flow/assignment/sort.go b/model/flow/assignment/sort.go index 3b135d91152..8e590a86089 100644 --- a/model/flow/assignment/sort.go +++ b/model/flow/assignment/sort.go @@ -2,7 +2,6 @@ package assignment import ( "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" ) // FromIdentifierLists creates a `flow.AssignmentList` with canonical ordering from @@ -11,7 +10,7 @@ func FromIdentifierLists(identifierLists []flow.IdentifierList) flow.AssignmentL assignments := make(flow.AssignmentList, 0, len(identifierLists)) // in place sort to order the assignment in canonical order for _, identities := range identifierLists { - assignment := flow.IdentifierList(identities).Sort(order.IdentifierCanonical) + assignment := flow.IdentifierList(identities).Sort(flow.IdentifierCanonical) assignments = append(assignments, assignment) } return assignments diff --git a/model/flow/factory/cluster_list.go b/model/flow/factory/cluster_list.go index 29bf374ac23..9ff7e0c7464 100644 --- a/model/flow/factory/cluster_list.go +++ b/model/flow/factory/cluster_list.go @@ -4,14 +4,17 @@ import ( "fmt" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" ) // NewClusterList creates a new cluster list based on the given cluster assignment // and the provided list of identities. -// The caller must ensure each assignment contains identities ordered in canonical order, so that -// each cluster in the returned cluster list is ordered in canonical order as well. If not, -// an error will be returned. +// +// The caller must ensure the following prerequisites: +// - each assignment contains identities ordered in canonical order +// - every collector has a unique NodeID, i.e. there are no two elements in `collectors` with the same NodeID +// +// These prerequisites ensures that each cluster in the returned cluster list is ordered in canonical order as well. +// This function checks that the prerequisites are satisfied and errors otherwise. func NewClusterList(assignments flow.AssignmentList, collectors flow.IdentityList) (flow.ClusterList, error) { // build a lookup for all the identities by node identifier @@ -28,7 +31,7 @@ func NewClusterList(assignments flow.AssignmentList, collectors flow.IdentityLis for i, participants := range assignments { cluster := make(flow.IdentityList, 0, len(participants)) if len(participants) == 0 { - return nil, fmt.Errorf("particpants in assignment list is empty, cluster index %v", i) + return nil, fmt.Errorf("participants in assignment list is empty, cluster index %v", i) } // Check assignments is sorted in canonical order @@ -43,8 +46,8 @@ func NewClusterList(assignments flow.AssignmentList, collectors flow.IdentityLis delete(lookup, participantID) if i > 0 { - if !order.IdentifierCanonical(prev, participantID) { - return nil, fmt.Errorf("the assignments is not sorted in canonical order in cluster index %v, prev %v, next %v", + if !flow.IsIdentifierCanonical(prev, participantID) { + return nil, fmt.Errorf("the assignments is not sorted in canonical order or there are duplicates in cluster index %v, prev %v, next %v", i, prev, participantID) } } diff --git a/model/flow/identifier.go b/model/flow/identifier.go index e205e74a716..5ac883bd9de 100644 --- a/model/flow/identifier.go +++ b/model/flow/identifier.go @@ -26,8 +26,16 @@ type Identifier [IdentifierLen]byte // IdentifierFilter is a filter on identifiers. type IdentifierFilter func(Identifier) bool -// IdentifierOrder is a sort for identifier -type IdentifierOrder func(Identifier, Identifier) bool +// IdentifierOrder is an order function for identifiers. +// +// It defines a strict weak ordering between identifiers. +// It returns a negative number if the first identifier is "strictly less" than the second, +// a positive number if the second identifier is "strictly less" than the first, +// and zero if the two identifiers are equal. +// +// `IdentifierOrder` can be used to sort identifiers with +// https://pkg.go.dev/golang.org/x/exp/slices#SortFunc. +type IdentifierOrder func(Identifier, Identifier) int var ( // ZeroID is the lowest value in the 32-byte ID space. diff --git a/model/flow/identifierList.go b/model/flow/identifierList.go index 1cf3e0263a8..29eb02de735 100644 --- a/model/flow/identifierList.go +++ b/model/flow/identifierList.go @@ -1,16 +1,13 @@ package flow import ( - "bytes" - "sort" - - "github.com/rs/zerolog/log" + "golang.org/x/exp/slices" ) // IdentifierList defines a sortable list of identifiers type IdentifierList []Identifier -// Len returns length of the IdentiferList in the number of stored identifiers. +// Len returns length of the IdentifierList in the number of stored identifiers. // It satisfies the sort.Interface making the IdentifierList sortable. func (il IdentifierList) Len() int { return len(il) @@ -29,16 +26,7 @@ func (il IdentifierList) Lookup() map[Identifier]struct{} { // Otherwise it returns true. // It satisfies the sort.Interface making the IdentifierList sortable. func (il IdentifierList) Less(i, j int) bool { - // bytes package already implements Comparable for []byte. - switch bytes.Compare(il[i][:], il[j][:]) { - case -1: - return true - case 0, 1: - return false - default: - log.Error().Msg("not fail-able with `bytes.Comparable` bounded [-1, 1].") - return false - } + return IsIdentifierCanonical(il[i], il[j]) } // Swap swaps the element i and j in the IdentifierList. @@ -120,22 +108,9 @@ IDLoop: return dup } +// Sort returns a sorted _copy_ of the IdentifierList, leaving the original invariant. func (il IdentifierList) Sort(less IdentifierOrder) IdentifierList { dup := il.Copy() - sort.Slice(dup, func(i int, j int) bool { - return less(dup[i], dup[j]) - }) + slices.SortFunc(dup, less) return dup } - -// Sorted returns whether the list is sorted by the input ordering. -func (il IdentifierList) Sorted(less IdentifierOrder) bool { - for i := 0; i < len(il)-1; i++ { - a := il[i] - b := il[i+1] - if !less(a, b) { - return false - } - } - return true -} diff --git a/model/flow/identifierList_test.go b/model/flow/identifierList_test.go index 7e18b6ee921..166b6dcbd5a 100644 --- a/model/flow/identifierList_test.go +++ b/model/flow/identifierList_test.go @@ -12,6 +12,14 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +// Test the canonical ordering of identity and identifier match +func TestCanonicalOrderingMatch(t *testing.T) { + identities := unittest.IdentityListFixture(100) + require.Equal(t, + identities.Sort(flow.Canonical).NodeIDs(), + identities.NodeIDs().Sort(flow.IdentifierCanonical)) +} + // TestIdentifierListSort tests the IdentityList against its implemented sort interface // it generates and sorts a list of ids, and then evaluates sorting in ascending order func TestIdentifierListSort(t *testing.T) { diff --git a/model/flow/identifier_order.go b/model/flow/identifier_order.go new file mode 100644 index 00000000000..af258f531bc --- /dev/null +++ b/model/flow/identifier_order.go @@ -0,0 +1,53 @@ +package flow + +import ( + "bytes" +) + +// IdentifierCanonical is a function that defines a weak strict ordering "<" for identifiers. +// It returns: +// - a strict negative number if id1 < id2 +// - a strict positive number if id2 < id1 +// - zero if id1 and id2 are equal +// +// By definition, two Identifiers (id1, id2) are in canonical order if id1 is lexicographically +// _strictly_ smaller than id2. The strictness is important, meaning that duplicates do not +// satisfy canonical ordering (order is irreflexive). Hence, only a returned strictly negative +// value means the pair is in canonical order. +// Use `IsIdentifierCanonical` for canonical order checks. +// +// The current function is based on the identifiers bytes lexicographic comparison. +// Example: +// +// IdentifierCanonical(Identifier{1}, Identifier{2}) // -1 +// IdentifierCanonical(Identifier{2}, Identifier{1}) // 1 +// IdentifierCanonical(Identifier{1}, Identifier{1}) // 0 +// IdentifierCanonical(Identifier{0, 1}, Identifier{0, 2}) // -1 +func IdentifierCanonical(id1 Identifier, id2 Identifier) int { + return bytes.Compare(id1[:], id2[:]) +} + +// IsCanonical returns true if and only if the given identifiers are in canonical order. +// +// By convention, two identifiers (i1, i2) are in canonical order if i1's bytes +// are lexicographically _strictly_ smaller than i2's bytes. +// +// The strictness is important, meaning that the canonical order +// is irreflexive ((i,i) isn't in canonical order). +func IsIdentifierCanonical(i1, i2 Identifier) bool { + return IdentifierCanonical(i1, i2) < 0 +} + +// IsIdentityListCanonical returns true if and only if the given list is +// _strictly_ sorted with regards to the canonical order. +// +// The strictness is important here, meaning that a list with 2 equal identifiers +// isn't considered well sorted. +func IsIdentifierListCanonical(il IdentifierList) bool { + for i := 0; i < len(il)-1; i++ { + if !IsIdentifierCanonical(il[i], il[i+1]) { + return false + } + } + return true +} diff --git a/model/flow/identity.go b/model/flow/identity.go index 975baa556e9..5e5a8c9194d 100644 --- a/model/flow/identity.go +++ b/model/flow/identity.go @@ -1,7 +1,6 @@ package flow import ( - "bytes" "encoding/json" "fmt" "io" @@ -301,8 +300,16 @@ func (iy *Identity) EqualTo(other *Identity) bool { // IdentityFilter is a filter on identities. type IdentityFilter func(*Identity) bool -// IdentityOrder is a sort for identities. -type IdentityOrder func(*Identity, *Identity) bool +// IdentityOrder is an order function for identities. +// +// It defines a strict weak ordering between identities. +// It returns a negative number if the first identity is "strictly less" than the second, +// a positive number if the second identity is "strictly less" than the first, +// and zero if the two identities are equal. +// +// `IdentityOrder` can be used to sort identities with +// https://pkg.go.dev/golang.org/x/exp/slices#SortFunc. +type IdentityOrder func(*Identity, *Identity) int // IdentityMapFunc is a modifier function for map operations for identities. // Identities are COPIED from the source slice. @@ -388,11 +395,6 @@ func (il IdentityList) Sort(less IdentityOrder) IdentityList { return dup } -// Sorted returns whether the list is sorted by the input ordering. -func (il IdentityList) Sorted(less IdentityOrder) bool { - return slices.IsSortedFunc(il, less) -} - // NodeIDs returns the NodeIDs of the nodes in the list. func (il IdentityList) NodeIDs() IdentifierList { nodeIDs := make([]Identifier, 0, len(il)) @@ -521,7 +523,7 @@ func (il IdentityList) SamplePct(pct float64) (IdentityList, error) { // Union returns a new identity list containing every identity that occurs in // either `il`, or `other`, or both. There are no duplicates in the output, // where duplicates are identities with the same node ID. -// The returned IdentityList is sorted +// The returned IdentityList is sorted canonically. func (il IdentityList) Union(other IdentityList) IdentityList { maxLen := len(il) + len(other) @@ -537,10 +539,7 @@ func (il IdentityList) Union(other IdentityList) IdentityList { } } - slices.SortFunc(union, func(a, b *Identity) bool { - return bytes.Compare(a.NodeID[:], b.NodeID[:]) < 0 - }) - + slices.SortFunc(union, Canonical) return union } @@ -564,9 +563,7 @@ func (il IdentityList) Exists(target *Identity) bool { // target: value to search for // CAUTION: The identity list MUST be sorted prior to calling this method func (il IdentityList) IdentifierExists(target Identifier) bool { - _, ok := slices.BinarySearchFunc(il, &Identity{NodeID: target}, func(a, b *Identity) int { - return bytes.Compare(a.NodeID[:], b.NodeID[:]) - }) + _, ok := slices.BinarySearchFunc(il, &Identity{NodeID: target}, Canonical) return ok } diff --git a/model/flow/identity_order.go b/model/flow/identity_order.go new file mode 100644 index 00000000000..17930d79d82 --- /dev/null +++ b/model/flow/identity_order.go @@ -0,0 +1,62 @@ +// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED + +package flow + +// Canonical is a function that defines a weak strict ordering "<" for identities. +// It returns: +// - a strict negative number if id1 < id2 +// - a strict positive number if id2 < id1 +// - zero if id1 and id2 are equal +// +// By definition, two identities (id1, id2) are in canonical order if id1's NodeID is lexicographically +// _strictly_ smaller than id2's NodeID. The strictness is important, meaning that identities +// with equal NodeIDs do not satisfy canonical ordering (order is irreflexive). +// Hence, only a returned strictly negative value means the pair is in canonical order. +// Use `IsCanonical` for canonical order checks. +// +// The current function is based on the identifiers bytes lexicographic comparison. +func Canonical(identity1 *Identity, identity2 *Identity) int { + return IdentifierCanonical(identity1.NodeID, identity2.NodeID) +} + +// IsCanonical returns true if and only if the given Identities are in canonical order. +// +// By convention, two Identities (i1, i2) are in canonical order if i1's NodeID bytes +// are lexicographically _strictly_ smaller than i2's NodeID bytes. +// +// The strictness is important, meaning that two identities with the same +// NodeID do not satisfy the canonical order. +// This also implies that the canonical order is irreflexive ((i,i) isn't in canonical order). +func IsCanonical(i1, i2 *Identity) bool { + return Canonical(i1, i2) < 0 +} + +// ByReferenceOrder return a function for sorting identities based on the order +// of the given nodeIDs +func ByReferenceOrder(nodeIDs []Identifier) func(*Identity, *Identity) int { + indices := make(map[Identifier]int) + for index, nodeID := range nodeIDs { + _, ok := indices[nodeID] + if ok { + panic("should never order by reference order with duplicate node IDs") + } + indices[nodeID] = index + } + return func(identity1 *Identity, identity2 *Identity) int { + return indices[identity1.NodeID] - indices[identity2.NodeID] + } +} + +// IsIdentityListCanonical returns true if and only if the given IdentityList is +// _strictly_ sorted with regards to the canonical order. +// +// The strictness is important here, meaning that a list with 2 successive entities +// with equal NodeID isn't considered well sorted. +func IsIdentityListCanonical(il IdentityList) bool { + for i := 0; i < len(il)-1; i++ { + if !IsCanonical(il[i], il[i+1]) { + return false + } + } + return true +} diff --git a/model/flow/identity_test.go b/model/flow/identity_test.go index 849db712d7d..f702f29318e 100644 --- a/model/flow/identity_test.go +++ b/model/flow/identity_test.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/encodable" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/utils/unittest" ) @@ -103,7 +102,7 @@ func TestIdentityList_Exists(t *testing.T) { il2 := unittest.IdentityListFixture(1) // sort the first list - il1 = il1.Sort(order.Canonical) + il1 = il1.Sort(flow.Canonical) for i := 0; i < 10; i++ { assert.True(t, il1.Exists(il1[i])) @@ -118,7 +117,7 @@ func TestIdentityList_IdentifierExists(t *testing.T) { il2 := unittest.IdentityListFixture(1) // sort the first list - il1 = il1.Sort(order.Canonical) + il1 = il1.Sort(flow.Canonical) for i := 0; i < 10; i++ { assert.True(t, il1.IdentifierExists(il1[i].NodeID)) @@ -243,12 +242,18 @@ func TestIdentity_ID(t *testing.T) { func TestIdentity_Sort(t *testing.T) { il := unittest.IdentityListFixture(20) - random, err := il.Shuffle() - require.NoError(t, err) - assert.False(t, random.Sorted(order.Canonical)) - - canonical := il.Sort(order.Canonical) - assert.True(t, canonical.Sorted(order.Canonical)) + // make sure the list is not sorted + il[0].NodeID[0], il[1].NodeID[0] = 2, 1 + require.False(t, flow.IsCanonical(il[0], il[1])) + assert.False(t, flow.IsIdentityListCanonical(il)) + + canonical := il.Sort(flow.Canonical) + assert.True(t, flow.IsIdentityListCanonical(canonical)) + + // check `IsIdentityListCanonical` detects order equality in a sorted list + il[1] = il[10] // add a duplication + canonical = il.Sort(flow.Canonical) + assert.False(t, flow.IsIdentityListCanonical(canonical)) } func TestIdentity_EqualTo(t *testing.T) { diff --git a/model/flow/order/identifier.go b/model/flow/order/identifier.go deleted file mode 100644 index 0102005b1b8..00000000000 --- a/model/flow/order/identifier.go +++ /dev/null @@ -1,13 +0,0 @@ -package order - -import ( - "bytes" - - "github.com/onflow/flow-go/model/flow" -) - -// IdentifierCanonical is a function for sorting IdentifierList into -// canonical order -func IdentifierCanonical(id1 flow.Identifier, id2 flow.Identifier) bool { - return bytes.Compare(id1[:], id2[:]) < 0 -} diff --git a/model/flow/order/identity.go b/model/flow/order/identity.go deleted file mode 100644 index 5b78c7a3dd4..00000000000 --- a/model/flow/order/identity.go +++ /dev/null @@ -1,47 +0,0 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - -package order - -import ( - "github.com/onflow/flow-go/model/flow" -) - -// Canonical represents the canonical ordering for identity lists. -func Canonical(identity1 *flow.Identity, identity2 *flow.Identity) bool { - return IdentifierCanonical(identity1.NodeID, identity2.NodeID) -} - -// ByReferenceOrder return a function for sorting identities based on the order -// of the given nodeIDs -func ByReferenceOrder(nodeIDs []flow.Identifier) func(*flow.Identity, *flow.Identity) bool { - indices := make(map[flow.Identifier]uint) - for index, nodeID := range nodeIDs { - _, ok := indices[nodeID] - if ok { - panic("should never order by reference order with duplicate node IDs") - } - indices[nodeID] = uint(index) - } - return func(identity1 *flow.Identity, identity2 *flow.Identity) bool { - return indices[identity1.NodeID] < indices[identity2.NodeID] - } -} - -// IdentityListCanonical takes a list of identities and -// check if it's ordered in canonical order. -func IdentityListCanonical(identities flow.IdentityList) bool { - if len(identities) == 0 { - return true - } - - prev := identities[0].ID() - for i := 1; i < len(identities); i++ { - id := identities[i].ID() - if !IdentifierCanonical(prev, id) { - return false - } - prev = id - } - - return true -} diff --git a/model/flow/order/identity_test.go b/model/flow/order/identity_test.go deleted file mode 100644 index 2c79b61ab4a..00000000000 --- a/model/flow/order/identity_test.go +++ /dev/null @@ -1,18 +0,0 @@ -// (c) 2019 Dapper Labs - ALL RIGHTS RESERVED - -package order_test - -import ( - "testing" - - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow/order" - "github.com/onflow/flow-go/utils/unittest" -) - -// Test the canonical ordering of identity and identifier match -func TestCanonicalOrderingMatch(t *testing.T) { - identities := unittest.IdentityListFixture(100) - require.Equal(t, identities.Sort(order.Canonical).NodeIDs(), identities.NodeIDs().Sort(order.IdentifierCanonical)) -} diff --git a/model/flow/sealing_segment.go b/model/flow/sealing_segment.go index ea96d69fb64..dffd5a9cef5 100644 --- a/model/flow/sealing_segment.go +++ b/model/flow/sealing_segment.go @@ -315,8 +315,8 @@ func (builder *SealingSegmentBuilder) SealingSegment() (*SealingSegment, error) // SealingSegment must store extra blocks in ascending order, builder stores them in descending. // Apply a sort to reverse the slice and use correct ordering. - slices.SortFunc(builder.extraBlocks, func(lhs, rhs *Block) bool { - return lhs.Header.Height < rhs.Header.Height + slices.SortFunc(builder.extraBlocks, func(lhs, rhs *Block) int { + return int(lhs.Header.Height) - int(rhs.Header.Height) }) return &SealingSegment{ diff --git a/model/flow/transaction.go b/model/flow/transaction.go index 95556ff4c05..fe9e06dafb5 100644 --- a/model/flow/transaction.go +++ b/model/flow/transaction.go @@ -2,7 +2,8 @@ package flow import ( "fmt" - "sort" + + "golang.org/x/exp/slices" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/crypto/hash" @@ -306,7 +307,7 @@ func (tb *TransactionBody) AddPayloadSignature(address Address, keyID uint64, si s := tb.createSignature(address, keyID, sig) tb.PayloadSignatures = append(tb.PayloadSignatures, s) - sort.Slice(tb.PayloadSignatures, compareSignatures(tb.PayloadSignatures)) + slices.SortFunc(tb.PayloadSignatures, compareSignatures) return tb } @@ -316,7 +317,7 @@ func (tb *TransactionBody) AddEnvelopeSignature(address Address, keyID uint64, s s := tb.createSignature(address, keyID, sig) tb.EnvelopeSignatures = append(tb.EnvelopeSignatures, s) - sort.Slice(tb.EnvelopeSignatures, compareSignatures(tb.EnvelopeSignatures)) + slices.SortFunc(tb.EnvelopeSignatures, compareSignatures) return tb } @@ -488,17 +489,12 @@ func (s TransactionSignature) canonicalForm() interface{} { } } -func compareSignatures(signatures []TransactionSignature) func(i, j int) bool { - return func(i, j int) bool { - sigA := signatures[i] - sigB := signatures[j] - - if sigA.SignerIndex == sigB.SignerIndex { - return sigA.KeyIndex < sigB.KeyIndex - } - - return sigA.SignerIndex < sigB.SignerIndex +func compareSignatures(sigA, sigB TransactionSignature) int { + if sigA.SignerIndex == sigB.SignerIndex { + return int(sigA.KeyIndex) - int(sigB.KeyIndex) } + + return sigA.SignerIndex - sigB.SignerIndex } type signaturesList []TransactionSignature diff --git a/model/verification/chunkDataPackRequest_test.go b/model/verification/chunkDataPackRequest_test.go index cb64b7ec502..6bb6bfe419e 100644 --- a/model/verification/chunkDataPackRequest_test.go +++ b/model/verification/chunkDataPackRequest_test.go @@ -8,7 +8,6 @@ import ( "github.com/stretchr/testify/require" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/model/verification" "github.com/onflow/flow-go/utils/unittest" ) @@ -56,7 +55,7 @@ func TestChunkDataPackRequestList_UniqueRequestInfo(t *testing.T) { return bytes.Compare(thisChunkIDReqInfo.Disagrees[p][:], thisChunkIDReqInfo.Disagrees[q][:]) < 0 }) - thisChunkIDReqInfo.Targets = thisChunkIDReqInfo.Targets.Sort(order.Canonical) + thisChunkIDReqInfo.Targets = thisChunkIDReqInfo.Targets.Sort(flow.Canonical) require.Equal(t, thisChunkIDReqInfo.Agrees, thisReq1.Agrees.Union(thisReq2.Agrees)) require.Equal(t, thisChunkIDReqInfo.Disagrees, thisReq1.Disagrees.Union(thisReq2.Disagrees)) diff --git a/module/dkg/controller.go b/module/dkg/controller.go index ae4b54ecb38..0607abbda7b 100644 --- a/module/dkg/controller.go +++ b/module/dkg/controller.go @@ -2,68 +2,15 @@ package dkg import ( "fmt" - "math" "sync" - "time" "github.com/rs/zerolog" "github.com/onflow/flow-go/crypto" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/utils/rand" ) -const ( - - // DefaultBaseStartDelay is the default base delay to use when introducing - // random delay to the DKG start process. See preStartDelay for details. - DefaultBaseStartDelay = 500 * time.Microsecond - - // DefaultBaseHandleFirstBroadcastDelay is the default base to use when - // introducing random delay to processing the first DKG broadcast message. - // See preHandleFirstBroadcastDelay for details. - // - // For a 150-node DKG, we observe a cost of ~2.5s per message to process - // broadcast messages during phase 1, for a total of ~6m of total CPU time. - // We would like to target spreading this cost over a 30 minute period. - // With the default value for DefaultHandleSubsequentBroadcastDelay, this - // results in processing all phase 1 messages in 6m+6m=12m, so for a maximum - // total processing time of 30m, we sample the initial delay from [0,18m]. - // We use 50ms as the default because 50ms*150^2 = 18.75m - // - DefaultBaseHandleFirstBroadcastDelay = 50 * time.Millisecond - - // DefaultHandleSubsequentBroadcastDelay is the default delay to use before - // processing all DKG broadcasts after the first. - DefaultHandleSubsequentBroadcastDelay = 2500 * time.Millisecond -) - -// ControllerConfig defines configuration for the DKG Controller. These define -// how the DKG controller introduces delays to expensive DKG computations. -// -// We introduce delays for two reasons: -// 1. Avoid running long-running expensive DKG computations consecutively. -// 2. Avoid synchronizing expensive DKG computations across the DKG committee. -// -// Delays introduced prior to DKG start and prior to processing the FIRST broadcast -// message are sampled uniformly from [0,m), where m=b*n^2 -// -// b = base delay (from config) -// n = size of DKG committee -// -// Delays introduced prior to processing subsequent broadcast messages are constant. -type ControllerConfig struct { - // BaseStartDelay determines the maximum delay before starting the DKG. - BaseStartDelay time.Duration - // BaseHandleFirstBroadcastDelay determines the maximum delay before handling - // the first broadcast message. - BaseHandleFirstBroadcastDelay time.Duration - // HandleSubsequentBroadcastDelay determines the constant delay before handling - // all broadcast messages following the first. - HandleSubsequentBroadcastDelay time.Duration -} - // Controller implements the DKGController interface. It controls the execution // of a Joint Feldman DKG instance. A new Controller must be instantiated for // every epoch. @@ -101,8 +48,7 @@ type Controller struct { // artifactsLock protects access to artifacts artifactsLock sync.Mutex - config ControllerConfig - once *sync.Once + once *sync.Once } // NewController instantiates a new Joint Feldman DKG controller. @@ -112,7 +58,6 @@ func NewController( dkg crypto.DKGState, seed []byte, broker module.DKGBroker, - config ControllerConfig, ) *Controller { logger := log.With(). @@ -130,7 +75,6 @@ func NewController( endCh: make(chan struct{}), shutdownCh: make(chan struct{}), once: new(sync.Once), - config: config, } } @@ -293,32 +237,6 @@ func (c *Controller) doBackgroundWork() { case msg := <-broadcastMsgCh: - // before processing a broadcast message during phase 1, sleep for a - // random delay to avoid synchronizing this expensive operation across - // all consensus nodes - state := c.GetState() - if state == Phase1 { - - // introduce a large, uniformly sampled delay prior to processing - // the first message - isFirstMessage := false - c.once.Do(func() { - isFirstMessage = true - delay, err := c.preHandleFirstBroadcastDelay() - if err != nil { - c.log.Err(err).Msg("pre handle first broadcast delay failed") - } - c.log.Info().Msgf("sleeping for %s before processing first phase 1 broadcast message", delay) - time.Sleep(delay) - }) - - if !isFirstMessage { - // introduce a constant delay for all subsequent messages - c.log.Debug().Msgf("sleeping for %s before processing subsequent phase 1 broadcast message", c.config.HandleSubsequentBroadcastDelay) - time.Sleep(c.config.HandleSubsequentBroadcastDelay) - } - } - c.dkgLock.Lock() err := c.dkg.HandleBroadcastMsg(int(msg.CommitteeMemberIndex), msg.Data) c.dkgLock.Unlock() @@ -338,17 +256,8 @@ func (c *Controller) start() error { return fmt.Errorf("cannot execute start routine in state %s", state) } - // before starting the DKG, sleep for a random delay to avoid synchronizing - // this expensive operation across all consensus nodes - delay, err := c.preStartDelay() - if err != nil { - return fmt.Errorf("pre start delay failed: %w", err) - } - c.log.Debug().Msgf("sleeping for %s before starting DKG", delay) - time.Sleep(delay) - c.dkgLock.Lock() - err = c.dkg.Start(c.seed) + err := c.dkg.Start(c.seed) c.dkgLock.Unlock() if err != nil { return fmt.Errorf("Error starting DKG: %w", err) @@ -423,56 +332,3 @@ func (c *Controller) phase3() error { } } } - -// preStartDelay returns a duration to delay prior to starting the DKG process. -// This prevents synchronization of the DKG starting (an expensive operation) -// across the network, which can impact finalization. -func (c *Controller) preStartDelay() (time.Duration, error) { - return computePreprocessingDelay(c.config.BaseStartDelay, c.dkg.Size()) -} - -// preHandleFirstBroadcastDelay returns a duration to delay prior to handling -// the first broadcast message. This delay is used only during phase 1 of the DKG. -// This prevents synchronization of processing verification vectors (an -// expensive operation) across the network, which can impact finalization. -func (c *Controller) preHandleFirstBroadcastDelay() (time.Duration, error) { - return computePreprocessingDelay(c.config.BaseHandleFirstBroadcastDelay, c.dkg.Size()) -} - -// computePreprocessingDelay computes a random delay to introduce before an -// expensive operation. -// -// The maximum delay is m=b*n^2 where: -// * b is a configurable base delay -// * n is the size of the DKG committee -func computePreprocessingDelay(baseDelay time.Duration, dkgSize int) (time.Duration, error) { - - maxDelay := computePreprocessingDelayMax(baseDelay, dkgSize) - if maxDelay <= 0 { - return 0, nil - } - // select delay from [0,m) - r, err := rand.Uint64n(uint64(maxDelay.Nanoseconds())) - if err != nil { - return time.Duration(0), fmt.Errorf("delay generation failed %w", err) - } - return time.Duration(r), nil -} - -// computePreprocessingDelayMax computes the maximum dely for computePreprocessingDelay. -func computePreprocessingDelayMax(baseDelay time.Duration, dkgSize int) time.Duration { - // sanity checks - if baseDelay < 0 { - baseDelay = 0 - } - if dkgSize < 0 { - dkgSize = 0 - } - - // m=b*n^2 - maxDelay := time.Duration(math.Pow(float64(dkgSize), 2)) * baseDelay - if maxDelay <= 0 { - return 0 - } - return maxDelay -} diff --git a/module/dkg/controller_factory.go b/module/dkg/controller_factory.go index ae12219706e..21563e02038 100644 --- a/module/dkg/controller_factory.go +++ b/module/dkg/controller_factory.go @@ -21,7 +21,6 @@ type ControllerFactory struct { me module.Local dkgContractClients []module.DKGContractClient tunnel *BrokerTunnel - config ControllerConfig } // NewControllerFactory creates a new factory that generates Controllers with @@ -31,14 +30,13 @@ func NewControllerFactory( me module.Local, dkgContractClients []module.DKGContractClient, tunnel *BrokerTunnel, - config ControllerConfig) *ControllerFactory { +) *ControllerFactory { return &ControllerFactory{ log: log, me: me, dkgContractClients: dkgContractClients, tunnel: tunnel, - config: config, } } @@ -77,7 +75,6 @@ func (f *ControllerFactory) Create( dkg, seed, broker, - f.config, ) return controller, nil diff --git a/module/dkg/controller_test.go b/module/dkg/controller_test.go index 3d9d1676a6a..b24f6a8719e 100644 --- a/module/dkg/controller_test.go +++ b/module/dkg/controller_test.go @@ -7,7 +7,6 @@ import ( "time" "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/onflow/flow-go/crypto" @@ -253,20 +252,12 @@ func initNodes(t *testing.T, n int, phase1Duration, phase2Duration, phase3Durati dkg, err := crypto.NewJointFeldman(n, signature.RandomBeaconThreshold(n), i, broker) require.NoError(t, err) - // create a config with no delays for tests - config := ControllerConfig{ - BaseStartDelay: 0, - BaseHandleFirstBroadcastDelay: 0, - HandleSubsequentBroadcastDelay: 0, - } - controller := NewController( logger, "dkg_test", dkg, seed, broker, - config, ) require.NoError(t, err) @@ -329,62 +320,3 @@ func checkArtifacts(t *testing.T, nodes []*node, totalNodes int) { } } } - -func TestDelay(t *testing.T) { - - t.Run("should return 0 delay for <=0 inputs", func(t *testing.T) { - delay, err := computePreprocessingDelay(0, 100) - require.NoError(t, err) - assert.Equal(t, delay, time.Duration(0)) - delay, err = computePreprocessingDelay(time.Hour, 0) - require.NoError(t, err) - assert.Equal(t, delay, time.Duration(0)) - delay, err = computePreprocessingDelay(time.Millisecond, -1) - require.NoError(t, err) - assert.Equal(t, delay, time.Duration(0)) - delay, err = computePreprocessingDelay(-time.Millisecond, 100) - require.NoError(t, err) - assert.Equal(t, delay, time.Duration(0)) - }) - - // NOTE: this is a probabilistic test. It will (extremely infrequently) fail. - t.Run("should return different values for same inputs", func(t *testing.T) { - d1, err := computePreprocessingDelay(time.Hour, 100) - require.NoError(t, err) - d2, err := computePreprocessingDelay(time.Hour, 100) - require.NoError(t, err) - assert.NotEqual(t, d1, d2) - }) - - t.Run("should return values in expected range", func(t *testing.T) { - baseDelay := time.Second - dkgSize := 100 - minDelay := time.Duration(0) - // m=b*n^2 - expectedMaxDelay := time.Duration(int64(baseDelay) * int64(dkgSize) * int64(dkgSize)) - - maxDelay := computePreprocessingDelayMax(baseDelay, dkgSize) - assert.Equal(t, expectedMaxDelay, maxDelay) - - delay, err := computePreprocessingDelay(baseDelay, dkgSize) - require.NoError(t, err) - assert.LessOrEqual(t, minDelay, delay) - assert.GreaterOrEqual(t, expectedMaxDelay, delay) - }) - - t.Run("should return values in expected range for defaults", func(t *testing.T) { - baseDelay := DefaultBaseHandleFirstBroadcastDelay - dkgSize := 150 - minDelay := time.Duration(0) - // m=b*n^2 - expectedMaxDelay := time.Duration(int64(baseDelay) * int64(dkgSize) * int64(dkgSize)) - - maxDelay := computePreprocessingDelayMax(baseDelay, dkgSize) - assert.Equal(t, expectedMaxDelay, maxDelay) - - delay, err := computePreprocessingDelay(baseDelay, dkgSize) - require.NoError(t, err) - assert.LessOrEqual(t, minDelay, delay) - assert.GreaterOrEqual(t, expectedMaxDelay, delay) - }) -} diff --git a/module/execution/scripts_test.go b/module/execution/scripts_test.go index fc360a9a39a..ccec36d1b11 100644 --- a/module/execution/scripts_test.go +++ b/module/execution/scripts_test.go @@ -156,7 +156,7 @@ func (s *scriptTestSuite) SetupTest() { s.Require().NoError(err) s.registerIndex = pebbleRegisters - index, err := indexer.New(logger, metrics.NewNoopCollector(), nil, s.registerIndex, headers, nil, nil) + index, err := indexer.New(logger, metrics.NewNoopCollector(), nil, s.registerIndex, headers, nil, nil, func(originID flow.Identifier, entity flow.Entity) {}) s.Require().NoError(err) scripts, err := NewScripts( diff --git a/module/finalizedreader/finalizedreader.go b/module/finalizedreader/finalizedreader.go index 01b6e4ec5ce..19012407d30 100644 --- a/module/finalizedreader/finalizedreader.go +++ b/module/finalizedreader/finalizedreader.go @@ -32,12 +32,12 @@ func (r *FinalizedReader) FinalizedBlockIDAtHeight(height uint64) (flow.Identifi return flow.ZeroID, fmt.Errorf("height not finalized (%v): %w", height, storage.ErrNotFound) } - header, err := r.headers.ByHeight(height) + finalizedID, err := r.headers.BlockIDByHeight(height) if err != nil { return flow.ZeroID, err } - return header.ID(), nil + return finalizedID, nil } // BlockFinalized implements the protocol.Consumer interface, which allows FinalizedReader diff --git a/module/finalizer/consensus/finalizer.go b/module/finalizer/consensus/finalizer.go index d0f8bdda796..b5fd97de564 100644 --- a/module/finalizer/consensus/finalizer.go +++ b/module/finalizer/consensus/finalizer.go @@ -75,12 +75,12 @@ func (f *Finalizer) MakeFinal(blockID flow.Identifier) error { } if pending.Height <= finalized { - dup, err := f.headers.ByHeight(pending.Height) + dupID, err := f.headers.BlockIDByHeight(pending.Height) if err != nil { return fmt.Errorf("could not retrieve finalized equivalent: %w", err) } - if dup.ID() != blockID { - return fmt.Errorf("cannot finalize pending block conflicting with finalized state (height: %d, pending: %x, finalized: %x)", pending.Height, blockID, dup.ID()) + if dupID != blockID { + return fmt.Errorf("cannot finalize pending block conflicting with finalized state (height: %d, pending: %x, finalized: %x)", pending.Height, blockID, dupID) } return nil } diff --git a/module/mempool/README.md b/module/mempool/README.md new file mode 100644 index 00000000000..ac7d6efadbf --- /dev/null +++ b/module/mempool/README.md @@ -0,0 +1,26 @@ +# The `mempool` module + +The `mempool` module provides mempool implementations for the Flow blockchain, which +are in-memory data structures that are tasked with storing the `flow.Entity` objects. +`flow.Entity` objects are the fundamental data model of the Flow blockchain, and +every Flow primitives such as transactions, blocks, and collections are represented +as `flow.Entity` objects. + +Each mempool implementation is tasked for storing a specific type of `flow.Entity`. +As a convention, all mempools are built on top of the `stdmap.Backend` struct, which +provides a thread-safe cache implementation for storing and retrieving `flow.Entity` objects. +The primary responsibility of the `stdmap.Backend` struct is to provide thread-safety for its underlying +data model (i.e., `mempool.Backdata`) that is tasked with maintaining the actual `flow.Entity` objects. + +At the moment, the `mempool` module provides two implementations for the `mempool.Backdata`: +- `backdata.Backdata`: a map implementation for storing `flow.Entity` objects using native Go `map`s. +- `herocache.Cache`: a cache implementation for storing `flow.Entity` objects, which is a heap-optimized + cache implementation that is aims on minimizing the memory footprint of the mempool on the heap and + reducing the GC pressure. + +Note-1: by design the `mempool.Backdata` interface is **not thread-safe**. Therefore, it is the responsibility +of the `stdmap.Backend` struct to provide thread-safety for its underlying `mempool.Backdata` implementation. + +Note-2: The `herocache.Cache` implementation is several orders of magnitude faster than the `backdata.Backdata` on +high-throughput workloads. For the read or write-heavy workloads, the `herocache.Cache` implementation is recommended as +the underlying `mempool.Backdata` implementation. diff --git a/module/mempool/backData.go b/module/mempool/backData.go index dbee603299a..cb61654dce6 100644 --- a/module/mempool/backData.go +++ b/module/mempool/backData.go @@ -23,6 +23,28 @@ type BackData interface { // Returns a bool which indicates whether the entity was updated as well as the updated entity. Adjust(entityID flow.Identifier, f func(flow.Entity) flow.Entity) (flow.Entity, bool) + // AdjustWithInit adjusts the entity using the given function if the given identifier can be found. When the + // entity is not found, it initializes the entity using the given init function and then applies the adjust function. + // Args: + // - entityID: the identifier of the entity to adjust. + // - adjust: the function that adjusts the entity. + // - init: the function that initializes the entity when it is not found. + // Returns: + // - the adjusted entity. + // + // - a bool which indicates whether the entity was adjusted. + AdjustWithInit(entityID flow.Identifier, adjust func(flow.Entity) flow.Entity, init func() flow.Entity) (flow.Entity, bool) + + // GetWithInit returns the given entity from the backdata. If the entity does not exist, it creates a new entity + // using the factory function and stores it in the backdata. + // Args: + // - entityID: the identifier of the entity to get. + // - init: the function that initializes the entity when it is not found. + // Returns: + // - the entity. + // - a bool which indicates whether the entity was found (or created). + GetWithInit(entityID flow.Identifier, init func() flow.Entity) (flow.Entity, bool) + // ByID returns the given entity from the backdata. ByID(entityID flow.Identifier) (flow.Entity, bool) diff --git a/module/mempool/herocache/backdata/cache.go b/module/mempool/herocache/backdata/cache.go index b498022c825..ec7a915dea3 100644 --- a/module/mempool/herocache/backdata/cache.go +++ b/module/mempool/herocache/backdata/cache.go @@ -59,6 +59,7 @@ type slotBucket struct { } // Cache implements an array-based generic memory pool backed by a fixed total array. +// Note that this implementation is NOT thread-safe, and the higher-level Backend is responsible for concurrency management. type Cache struct { logger zerolog.Logger collector module.HeroCacheMetrics @@ -203,6 +204,45 @@ func (c *Cache) Adjust(entityID flow.Identifier, f func(flow.Entity) flow.Entity return newEntity, true } +// AdjustWithInit adjusts the entity using the given function if the given identifier can be found. When the +// entity is not found, it initializes the entity using the given init function and then applies the adjust function. +// Args: +// - entityID: the identifier of the entity to adjust. +// - adjust: the function that adjusts the entity. +// - init: the function that initializes the entity when it is not found. +// Returns: +// - the adjusted entity. +// +// - a bool which indicates whether the entity was adjusted. +func (c *Cache) AdjustWithInit(entityID flow.Identifier, adjust func(flow.Entity) flow.Entity, init func() flow.Entity) (flow.Entity, bool) { + defer c.logTelemetry() + + if c.Has(entityID) { + return c.Adjust(entityID, adjust) + } + c.put(entityID, init()) + return c.Adjust(entityID, adjust) +} + +// GetWithInit returns the given entity from the backdata. If the entity does not exist, it creates a new entity +// using the factory function and stores it in the backdata. +// Args: +// - entityID: the identifier of the entity to get. +// - init: the function that initializes the entity when it is not found. +// Returns: +// - the entity. +// +// - a bool which indicates whether the entity was found (or created). +func (c *Cache) GetWithInit(entityID flow.Identifier, init func() flow.Entity) (flow.Entity, bool) { + defer c.logTelemetry() + + if c.Has(entityID) { + return c.ByID(entityID) + } + c.put(entityID, init()) + return c.ByID(entityID) +} + // ByID returns the given entity from the backdata. func (c *Cache) ByID(entityID flow.Identifier) (flow.Entity, bool) { defer c.logTelemetry() diff --git a/module/mempool/herocache/backdata/cache_test.go b/module/mempool/herocache/backdata/cache_test.go index bf1d7b3c60e..fe7a3975284 100644 --- a/module/mempool/herocache/backdata/cache_test.go +++ b/module/mempool/herocache/backdata/cache_test.go @@ -126,6 +126,160 @@ func TestArrayBackData_Adjust(t *testing.T) { require.Equal(t, bd.Size(), uint(limit)) } +// TestArrayBackData_AdjustWitInit evaluates that AdjustWithInit method. It should initialize and then adjust the value of +// non-existing entity while preserving the integrity of BackData on just adjusting the value of existing entity. +func TestArrayBackData_AdjustWitInit(t *testing.T) { + limit := 100_000 + + bd := NewCache(uint32(limit), + 8, + heropool.LRUEjection, + unittest.Logger(), + metrics.NewNoopCollector()) + + entities := unittest.EntityListFixture(uint(limit)) + for _, e := range entities { + adjustedEntity, adjusted := bd.AdjustWithInit(e.ID(), func(entity flow.Entity) flow.Entity { + // adjust logic, increments the nonce of the entity + mockEntity, ok := entity.(*unittest.MockEntity) + require.True(t, ok) + mockEntity.Nonce++ + return mockEntity + }, func() flow.Entity { + return e // initialize with the entity + }) + require.True(t, adjusted) + require.Equal(t, e.ID(), adjustedEntity.ID()) + require.Equal(t, uint64(1), adjustedEntity.(*unittest.MockEntity).Nonce) + } + + // picks a random entity from BackData and adjusts its identifier to a new one. + entityIndex := rand.Int() % limit + // checking integrity of retrieving entity + oldEntity, ok := bd.ByID(entities[entityIndex].ID()) + require.True(t, ok) + oldEntityID := oldEntity.ID() + require.Equal(t, entities[entityIndex].ID(), oldEntityID) + require.Equal(t, entities[entityIndex], oldEntity) + + // picks a new identifier for the entity and makes sure it is different than its current one. + newEntityID := unittest.IdentifierFixture() + require.NotEqual(t, oldEntityID, newEntityID) + + // adjusts old entity to a new entity with a new identifier + newEntity, ok := bd.Adjust(oldEntity.ID(), func(entity flow.Entity) flow.Entity { + mockEntity, ok := entity.(*unittest.MockEntity) + require.True(t, ok) + // oldEntity must be passed to func parameter of adjust. + require.Equal(t, oldEntityID, mockEntity.ID()) + require.Equal(t, oldEntity, mockEntity) + + // adjust logic, adjsuts the nonce of the entity + return &unittest.MockEntity{Identifier: newEntityID, Nonce: 2} + }) + + // adjustment must be successful, and identifier must be updated. + require.True(t, ok) + require.Equal(t, newEntityID, newEntity.ID()) + require.Equal(t, uint64(2), newEntity.(*unittest.MockEntity).Nonce) + newMockEntity, ok := newEntity.(*unittest.MockEntity) + require.True(t, ok) + + // replaces new entity in the original reference list and + // retrieves all. + entities[entityIndex] = newMockEntity + testRetrievableFrom(t, bd, entities, 0) + + // re-adjusting old entity must fail, since its identifier must no longer exist + entity, ok := bd.Adjust(oldEntityID, func(entity flow.Entity) flow.Entity { + require.Fail(t, "function must not be invoked on a non-existing entity") + return entity + }) + require.False(t, ok) + require.Nil(t, entity) + + // similarly, retrieving old entity must fail + entity, ok = bd.ByID(oldEntityID) + require.False(t, ok) + require.Nil(t, entity) + + ok = bd.Has(oldEntityID) + require.False(t, ok) +} + +// TestArrayBackData_GetWithInit evaluates that GetWithInit method. It should initialize and then retrieve the value of +// non-existing entity while preserving the integrity of BackData on just retrieving the value of existing entity. +func TestArrayBackData_GetWithInit(t *testing.T) { + limit := 1000 + + bd := NewCache(uint32(limit), 8, heropool.LRUEjection, unittest.Logger(), metrics.NewNoopCollector()) + + entities := unittest.EntityListFixture(uint(limit)) + + // GetWithInit + for _, e := range entities { + // all entities must be initialized retrieved successfully + actual, ok := bd.GetWithInit(e.ID(), func() flow.Entity { + return e // initialize with the entity + }) + require.True(t, ok) + require.Equal(t, e, actual) + } + + // All + all := bd.All() + require.Equal(t, len(entities), len(all)) + for _, expected := range entities { + actual, ok := bd.ByID(expected.ID()) + require.True(t, ok) + require.Equal(t, expected, actual) + } + + // Identifiers + ids := bd.Identifiers() + require.Equal(t, len(entities), len(ids)) + for _, id := range ids { + require.True(t, bd.Has(id)) + } + + // Entities + actualEntities := bd.Entities() + require.Equal(t, len(entities), len(actualEntities)) + require.ElementsMatch(t, entities, actualEntities) + + // Adjust + for _, e := range entities { + // all entities must be adjusted successfully + actual, ok := bd.Adjust(e.ID(), func(entity flow.Entity) flow.Entity { + // increment nonce of the entity + entity.(*unittest.MockEntity).Nonce++ + return entity + }) + require.True(t, ok) + require.Equal(t, e, actual) + } + + // ByID; should return the latest version of the entity + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := bd.ByID(e.ID()) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } + + // GetWithInit; should return the latest version of the entity, than increment the nonce + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := bd.GetWithInit(e.ID(), func() flow.Entity { + require.Fail(t, "should not be called") // entity has already been initialized + return e + }) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + } +} + // TestArrayBackData_WriteHeavy evaluates correctness of Cache under the writing and retrieving // a heavy load of entities up to its limit. All data must be written successfully and then retrievable. func TestArrayBackData_WriteHeavy(t *testing.T) { diff --git a/module/mempool/mock/back_data.go b/module/mempool/mock/back_data.go index 68661aa9c23..a72f0d6da1f 100644 --- a/module/mempool/mock/back_data.go +++ b/module/mempool/mock/back_data.go @@ -53,6 +53,32 @@ func (_m *BackData) Adjust(entityID flow.Identifier, f func(flow.Entity) flow.En return r0, r1 } +// AdjustWithInit provides a mock function with given fields: entityID, adjust, init +func (_m *BackData) AdjustWithInit(entityID flow.Identifier, adjust func(flow.Entity) flow.Entity, init func() flow.Entity) (flow.Entity, bool) { + ret := _m.Called(entityID, adjust, init) + + var r0 flow.Entity + var r1 bool + if rf, ok := ret.Get(0).(func(flow.Identifier, func(flow.Entity) flow.Entity, func() flow.Entity) (flow.Entity, bool)); ok { + return rf(entityID, adjust, init) + } + if rf, ok := ret.Get(0).(func(flow.Identifier, func(flow.Entity) flow.Entity, func() flow.Entity) flow.Entity); ok { + r0 = rf(entityID, adjust, init) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.Entity) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier, func(flow.Entity) flow.Entity, func() flow.Entity) bool); ok { + r1 = rf(entityID, adjust, init) + } else { + r1 = ret.Get(1).(bool) + } + + return r0, r1 +} + // All provides a mock function with given fields: func (_m *BackData) All() map[flow.Identifier]flow.Entity { ret := _m.Called() @@ -116,6 +142,32 @@ func (_m *BackData) Entities() []flow.Entity { return r0 } +// GetWithInit provides a mock function with given fields: entityID, init +func (_m *BackData) GetWithInit(entityID flow.Identifier, init func() flow.Entity) (flow.Entity, bool) { + ret := _m.Called(entityID, init) + + var r0 flow.Entity + var r1 bool + if rf, ok := ret.Get(0).(func(flow.Identifier, func() flow.Entity) (flow.Entity, bool)); ok { + return rf(entityID, init) + } + if rf, ok := ret.Get(0).(func(flow.Identifier, func() flow.Entity) flow.Entity); ok { + r0 = rf(entityID, init) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.Entity) + } + } + + if rf, ok := ret.Get(1).(func(flow.Identifier, func() flow.Entity) bool); ok { + r1 = rf(entityID, init) + } else { + r1 = ret.Get(1).(bool) + } + + return r0, r1 +} + // Has provides a mock function with given fields: entityID func (_m *BackData) Has(entityID flow.Identifier) bool { ret := _m.Called(entityID) diff --git a/module/mempool/stdmap/backDataHeapBenchmark_test.go b/module/mempool/stdmap/backDataHeapBenchmark_test.go index 1a3fdbc7e17..367f983120a 100644 --- a/module/mempool/stdmap/backDataHeapBenchmark_test.go +++ b/module/mempool/stdmap/backDataHeapBenchmark_test.go @@ -3,6 +3,7 @@ package stdmap_test import ( "runtime" "runtime/debug" + "sync" "testing" "time" @@ -110,6 +111,11 @@ func testAddEntities(t testing.TB, limit uint, b *stdmap.Backend, entities []*un type baselineLRU struct { c *lru.Cache // used to incorporate an LRU cache limit int + + // atomicAdjustMutex is used to synchronize concurrent access to the + // underlying LRU cache. This is needed because hashicorp LRU does not + // provide thread-safety for atomic adjust-with-init or get-with-init operations. + atomicAdjustMutex sync.Mutex } func newBaselineLRU(limit int) *baselineLRU { @@ -167,6 +173,40 @@ func (b *baselineLRU) Adjust(entityID flow.Identifier, f func(flow.Entity) flow. return newEntity, true } +// AdjustWithInit will adjust the value item using the given function if the given key can be found. +// If the key is not found, the init function will be called to create a new value. +// Returns a bool which indicates whether the value was updated as well as the updated value and +// a bool indicating whether the value was initialized. +// Note: this is a benchmark helper, hence, the adjust-with-init provides serializability w.r.t other concurrent adjust-with-init or get-with-init operations, +// and does not provide serializability w.r.t concurrent add, adjust or get operations. +func (b *baselineLRU) AdjustWithInit(entityID flow.Identifier, adjust func(flow.Entity) flow.Entity, init func() flow.Entity) (flow.Entity, bool) { + b.atomicAdjustMutex.Lock() + defer b.atomicAdjustMutex.Unlock() + + if b.Has(entityID) { + return b.Adjust(entityID, adjust) + } + added := b.Add(entityID, init()) + if !added { + return nil, false + } + return b.Adjust(entityID, adjust) +} + +// GetWithInit will retrieve the value item if the given key can be found. +// If the key is not found, the init function will be called to create a new value. +// Returns a bool which indicates whether the entity was found (or created). +func (b *baselineLRU) GetWithInit(entityID flow.Identifier, init func() flow.Entity) (flow.Entity, bool) { + newE := init() + e, ok, _ := b.c.PeekOrAdd(entityID, newE) + if !ok { + // if the entity was not found, it means that the new entity was added to the cache. + return newE, true + } + // if the entity was found, it means that the new entity was not added to the cache. + return e.(flow.Entity), true +} + // ByID returns the given item from the pool. func (b *baselineLRU) ByID(entityID flow.Identifier) (flow.Entity, bool) { e, ok := b.c.Get(entityID) @@ -182,12 +222,12 @@ func (b *baselineLRU) ByID(entityID flow.Identifier) (flow.Entity, bool) { } // Size will return the total of the backend. -func (b baselineLRU) Size() uint { +func (b *baselineLRU) Size() uint { return uint(b.c.Len()) } // All returns all entities from the pool. -func (b baselineLRU) All() map[flow.Identifier]flow.Entity { +func (b *baselineLRU) All() map[flow.Identifier]flow.Entity { all := make(map[flow.Identifier]flow.Entity) for _, entityID := range b.c.Keys() { id, ok := entityID.(flow.Identifier) @@ -205,7 +245,7 @@ func (b baselineLRU) All() map[flow.Identifier]flow.Entity { return all } -func (b baselineLRU) Identifiers() flow.IdentifierList { +func (b *baselineLRU) Identifiers() flow.IdentifierList { ids := make(flow.IdentifierList, b.c.Len()) entityIds := b.c.Keys() total := len(entityIds) @@ -219,7 +259,7 @@ func (b baselineLRU) Identifiers() flow.IdentifierList { return ids } -func (b baselineLRU) Entities() []flow.Entity { +func (b *baselineLRU) Entities() []flow.Entity { entities := make([]flow.Entity, b.c.Len()) entityIds := b.c.Keys() total := len(entityIds) diff --git a/module/mempool/stdmap/backdata/mapBackData.go b/module/mempool/stdmap/backdata/mapBackData.go index 887b6fca335..24c34d79eb0 100644 --- a/module/mempool/stdmap/backdata/mapBackData.go +++ b/module/mempool/stdmap/backdata/mapBackData.go @@ -5,6 +5,7 @@ import ( ) // MapBackData implements a map-based generic memory BackData backed by a Go map. +// Note that this implementation is NOT thread-safe, and the higher-level Backend is responsible for concurrency management. type MapBackData struct { // NOTE: as a BackData implementation, MapBackData must be non-blocking. // Concurrency management is done by overlay Backend. @@ -19,7 +20,7 @@ func NewMapBackData() *MapBackData { } // Has checks if backdata already contains the entity with the given identifier. -func (b MapBackData) Has(entityID flow.Identifier) bool { +func (b *MapBackData) Has(entityID flow.Identifier) bool { _, exists := b.entities[entityID] return exists } @@ -59,8 +60,42 @@ func (b *MapBackData) Adjust(entityID flow.Identifier, f func(flow.Entity) flow. return newentity, true } +// AdjustWithInit adjusts the entity using the given function if the given identifier can be found. When the +// entity is not found, it initializes the entity using the given init function and then applies the adjust function. +// Args: +// - entityID: the identifier of the entity to adjust. +// - adjust: the function that adjusts the entity. +// - init: the function that initializes the entity when it is not found. +// Returns: +// - the adjusted entity. +// +// - a bool which indicates whether the entity was adjusted. +func (b *MapBackData) AdjustWithInit(entityID flow.Identifier, adjust func(flow.Entity) flow.Entity, init func() flow.Entity) (flow.Entity, bool) { + if b.Has(entityID) { + return b.Adjust(entityID, adjust) + } + b.Add(entityID, init()) + return b.Adjust(entityID, adjust) +} + +// GetWithInit returns the given entity from the backdata. If the entity does not exist, it creates a new entity +// using the factory function and stores it in the backdata. +// Args: +// - entityID: the identifier of the entity to get. +// - init: the function that initializes the entity when it is not found. +// Returns: +// - the entity. +// - a bool which indicates whether the entity was found (or created). +func (b *MapBackData) GetWithInit(entityID flow.Identifier, init func() flow.Entity) (flow.Entity, bool) { + if b.Has(entityID) { + return b.ByID(entityID) + } + b.Add(entityID, init()) + return b.ByID(entityID) +} + // ByID returns the given entity from the backdata. -func (b MapBackData) ByID(entityID flow.Identifier) (flow.Entity, bool) { +func (b *MapBackData) ByID(entityID flow.Identifier) (flow.Entity, bool) { entity, exists := b.entities[entityID] if !exists { return nil, false @@ -69,12 +104,12 @@ func (b MapBackData) ByID(entityID flow.Identifier) (flow.Entity, bool) { } // Size returns the size of the backdata, i.e., total number of stored (entityId, entity) -func (b MapBackData) Size() uint { +func (b *MapBackData) Size() uint { return uint(len(b.entities)) } // All returns all entities stored in the backdata. -func (b MapBackData) All() map[flow.Identifier]flow.Entity { +func (b *MapBackData) All() map[flow.Identifier]flow.Entity { entities := make(map[flow.Identifier]flow.Entity) for entityID, entity := range b.entities { entities[entityID] = entity @@ -83,7 +118,7 @@ func (b MapBackData) All() map[flow.Identifier]flow.Entity { } // Identifiers returns the list of identifiers of entities stored in the backdata. -func (b MapBackData) Identifiers() flow.IdentifierList { +func (b *MapBackData) Identifiers() flow.IdentifierList { ids := make(flow.IdentifierList, len(b.entities)) i := 0 for entityID := range b.entities { @@ -94,7 +129,7 @@ func (b MapBackData) Identifiers() flow.IdentifierList { } // Entities returns the list of entities stored in the backdata. -func (b MapBackData) Entities() []flow.Entity { +func (b *MapBackData) Entities() []flow.Entity { entities := make([]flow.Entity, len(b.entities)) i := 0 for _, entity := range b.entities { diff --git a/module/mempool/stdmap/backdata/mapBackData_test.go b/module/mempool/stdmap/backdata/mapBackData_test.go index 7e5858d65ac..a5a13b70bb8 100644 --- a/module/mempool/stdmap/backdata/mapBackData_test.go +++ b/module/mempool/stdmap/backdata/mapBackData_test.go @@ -5,6 +5,7 @@ import ( "github.com/stretchr/testify/require" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/utils/unittest" ) @@ -47,3 +48,140 @@ func TestMapBackData_StoreAnd(t *testing.T) { require.Equal(t, len(entities), len(actualEntities)) require.ElementsMatch(t, entities, actualEntities) } + +// TestMapBackData_AdjustWithInit tests the AdjustWithInit method of the MapBackData. +// Note that as the backdata is not inherently thread-safe, this test is not concurrent. +func TestMapBackData_AdjustWithInit(t *testing.T) { + backData := NewMapBackData() + entities := unittest.EntityListFixture(100) + ids := flow.GetIDs(entities) + + // AdjustWithInit + for _, e := range entities { + // all entities must be adjusted successfully + actual, ok := backData.AdjustWithInit(e.ID(), func(entity flow.Entity) flow.Entity { + // increment nonce of the entity + entity.(*unittest.MockEntity).Nonce++ + return entity + }, func() flow.Entity { + return e + }) + require.True(t, ok) + require.Equal(t, e, actual) + } + + // All + all := backData.All() + require.Equal(t, len(entities), len(all)) + for _, expected := range entities { + actual, ok := backData.ByID(expected.ID()) + require.True(t, ok) + require.Equal(t, expected.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } + + // Identifiers + retriedIds := backData.Identifiers() + require.Equal(t, len(entities), len(retriedIds)) + require.ElementsMatch(t, ids, retriedIds) + for _, id := range retriedIds { + require.True(t, backData.Has(id)) + } + + // Entities + actualEntities := backData.Entities() + require.Equal(t, len(entities), len(actualEntities)) + require.ElementsMatch(t, entities, actualEntities) + + // ByID + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := backData.ByID(e.ID()) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } + + // GetWithInit + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := backData.GetWithInit(e.ID(), func() flow.Entity { + require.Fail(t, "should not be called") // entity has already been initialized + return e + }) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } +} + +// TestMapBackData_GetWithInit tests the GetWithInit method of the MapBackData. +// Note that as the backdata is not inherently thread-safe, this test is not concurrent. +func TestMapBackData_GetWithInit(t *testing.T) { + backData := NewMapBackData() + entities := unittest.EntityListFixture(100) + + // GetWithInit + for _, e := range entities { + // all entities must be initialized retrieved successfully + actual, ok := backData.GetWithInit(e.ID(), func() flow.Entity { + return e // initialize with the entity + }) + require.True(t, ok) + require.Equal(t, e, actual) + } + + // All + all := backData.All() + require.Equal(t, len(entities), len(all)) + for _, expected := range entities { + actual, ok := backData.ByID(expected.ID()) + require.True(t, ok) + require.Equal(t, expected, actual) + } + + // Identifiers + ids := backData.Identifiers() + require.Equal(t, len(entities), len(ids)) + for _, id := range ids { + require.True(t, backData.Has(id)) + } + + // Entities + actualEntities := backData.Entities() + require.Equal(t, len(entities), len(actualEntities)) + require.ElementsMatch(t, entities, actualEntities) + + // Adjust + for _, e := range entities { + // all entities must be adjusted successfully + actual, ok := backData.Adjust(e.ID(), func(entity flow.Entity) flow.Entity { + // increment nonce of the entity + entity.(*unittest.MockEntity).Nonce++ + return entity + }) + require.True(t, ok) + require.Equal(t, e, actual) + } + + // ByID; should return the latest version of the entity + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := backData.ByID(e.ID()) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } + + // GetWithInit; should return the latest version of the entity, than increment the nonce + for _, e := range entities { + // all entities must be retrieved successfully + actual, ok := backData.GetWithInit(e.ID(), func() flow.Entity { + require.Fail(t, "should not be called") // entity has already been initialized + return e + }) + require.True(t, ok) + require.Equal(t, e.ID(), actual.ID()) + require.Equal(t, uint64(1), actual.(*unittest.MockEntity).Nonce) + } +} diff --git a/module/mempool/stdmap/backend.go b/module/mempool/stdmap/backend.go index fb42e5297d5..f7dfc7de323 100644 --- a/module/mempool/stdmap/backend.go +++ b/module/mempool/stdmap/backend.go @@ -12,7 +12,7 @@ import ( _ "github.com/onflow/flow-go/utils/binstat" ) -// Backend provides synchronized access to a backdata +// Backend is a wrapper around the backdata that provides concurrency-safe operations. type Backend struct { sync.RWMutex backData mempool.BackData @@ -40,12 +40,12 @@ func NewBackend(options ...OptionFunc) *Backend { // Has checks if we already contain the item with the given hash. func (b *Backend) Has(entityID flow.Identifier) bool { - //bs1 := binstat.EnterTime(binstat.BinStdmap + ".r_lock.(Backend)Has") + // bs1 := binstat.EnterTime(binstat.BinStdmap + ".r_lock.(Backend)Has") b.RLock() - //binstat.Leave(bs1) + // binstat.Leave(bs1) - //bs2 := binstat.EnterTime(binstat.BinStdmap + ".inlock.(Backend)Has") - //defer binstat.Leave(bs2) + // bs2 := binstat.EnterTime(binstat.BinStdmap + ".inlock.(Backend)Has") + // defer binstat.Leave(bs2) defer b.RUnlock() has := b.backData.Has(entityID) return has @@ -53,16 +53,16 @@ func (b *Backend) Has(entityID flow.Identifier) bool { // Add adds the given item to the pool. func (b *Backend) Add(entity flow.Entity) bool { - //bs0 := binstat.EnterTime(binstat.BinStdmap + ".< 0 { + for _, chunk := range data.ChunkExecutionDatas[0 : len(data.ChunkExecutionDatas)-1] { + c.collectionHandler(flow.ZeroID, chunk.Collection) + indexedCount++ + } + } + + lg.Debug(). + Int("collection_count", indexedCount). + Dur("duration_ms", time.Since(start)). + Msg("indexed collections") + + return nil + }) + g.Go(func() error { start := time.Now() @@ -203,7 +230,7 @@ func (c *IndexerCore) IndexBlockData(data *execution_data.BlockExecutionDataEnti return fmt.Errorf("failed to index block data at height %d: %w", block.Height, err) } - c.metrics.BlockIndexed(block.Height, time.Since(start), registerCount, eventCount, resultCount) + c.metrics.BlockIndexed(block.Height, time.Since(start), eventCount, registerCount, resultCount) lg.Debug(). Dur("duration_ms", time.Since(start)). Msg("indexed block data") diff --git a/module/state_synchronization/indexer/indexer_core_test.go b/module/state_synchronization/indexer/indexer_core_test.go index e4de36f892d..73c0174105f 100644 --- a/module/state_synchronization/indexer/indexer_core_test.go +++ b/module/state_synchronization/indexer/indexer_core_test.go @@ -14,6 +14,7 @@ import ( mocks "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + "github.com/onflow/flow-go/engine/common/requester" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/ledger/common/pathfinder" @@ -28,21 +29,24 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) +var noopHandlerFunc requester.HandleFunc = func(originID flow.Identifier, entity flow.Entity) {} + type indexCoreTest struct { - t *testing.T - indexer *IndexerCore - registers *storagemock.RegisterIndex - events *storagemock.Events - results *storagemock.LightTransactionResults - headers *storagemock.Headers - ctx context.Context - blocks []*flow.Block - data *execution_data.BlockExecutionDataEntity - lastHeightStore func(t *testing.T) uint64 - firstHeightStore func(t *testing.T) uint64 - registersStore func(t *testing.T, entries flow.RegisterEntries, height uint64) error - eventsStore func(t *testing.T, ID flow.Identifier, events []flow.EventsList) error - registersGet func(t *testing.T, IDs flow.RegisterID, height uint64) (flow.RegisterValue, error) + t *testing.T + indexer *IndexerCore + registers *storagemock.RegisterIndex + events *storagemock.Events + results *storagemock.LightTransactionResults + headers *storagemock.Headers + ctx context.Context + blocks []*flow.Block + collectionHandler requester.HandleFunc + data *execution_data.BlockExecutionDataEntity + lastHeightStore func(t *testing.T) uint64 + firstHeightStore func(t *testing.T) uint64 + registersStore func(t *testing.T, entries flow.RegisterEntries, height uint64) error + eventsStore func(t *testing.T, ID flow.Identifier, events []flow.EventsList) error + registersGet func(t *testing.T, IDs flow.RegisterID, height uint64) (flow.RegisterValue, error) } func newIndexCoreTest( @@ -59,6 +63,13 @@ func newIndexCoreTest( ctx: context.Background(), data: exeData, headers: newBlockHeadersStorage(blocks).(*storagemock.Headers), // convert it back to mock type for tests + collectionHandler: func(originID flow.Identifier, entity flow.Entity) { + // collectionHandler is always called. by default, assert the value passed was empty + // to enforce the test writer handles the collection when it's tested. + // this will never happen in production. + assert.Equal(t, flow.ZeroID, originID) + assert.Nil(t, entity) + }, } } @@ -85,6 +96,7 @@ func (i *indexCoreTest) setLastHeight(f func(t *testing.T) uint64) *indexCoreTes }) return i } + func (i *indexCoreTest) useDefaultHeights() *indexCoreTest { i.registers. On("FirstHeight"). @@ -137,6 +149,11 @@ func (i *indexCoreTest) setGetRegisters(f func(t *testing.T, ID flow.RegisterID, return i } +func (i *indexCoreTest) setOnCollection(fn requester.HandleFunc) *indexCoreTest { + i.collectionHandler = fn + return i +} + func (i *indexCoreTest) useDefaultEvents() *indexCoreTest { i.events. On("BatchStore", mock.AnythingOfType("flow.Identifier"), mock.AnythingOfType("[]flow.EventsList"), mock.Anything). @@ -160,7 +177,11 @@ func (i *indexCoreTest) initIndexer() *indexCoreTest { i.useDefaultHeights() - indexer, err := New(zerolog.New(os.Stdout), metrics.NewNoopCollector(), db, i.registers, i.headers, i.events, i.results) + onCollection := func(originID flow.Identifier, entity flow.Entity) { + i.collectionHandler(originID, entity) + } + + indexer, err := New(zerolog.New(os.Stdout), metrics.NewNoopCollector(), db, i.registers, i.headers, i.events, i.results, onCollection) require.NoError(i.t, err) i.indexer = indexer return i @@ -340,9 +361,55 @@ func TestExecutionState_IndexBlockData(t *testing.T) { assert.NoError(t, err) }) + t.Run("Index Collections", func(t *testing.T) { + expectedCollections := unittest.CollectionListFixture(2) + ed := &execution_data.BlockExecutionData{ + BlockID: block.ID(), + ChunkExecutionDatas: []*execution_data.ChunkExecutionData{ + {Collection: expectedCollections[0]}, + {Collection: expectedCollections[1]}, + }, + } + execData := execution_data.NewBlockExecutionDataEntity(block.ID(), ed) + collectionsHandled := 0 + err := newIndexCoreTest(t, blocks, execData). + initIndexer(). + // make sure an empty set of events were stored + setStoreEvents(func(t *testing.T, actualBlockID flow.Identifier, actualEvents []flow.EventsList) error { + assert.Equal(t, block.ID(), actualBlockID) + require.Len(t, actualEvents, 1) + require.Len(t, actualEvents[0], 0) + return nil + }). + // make sure an empty set of transaction results were stored + setStoreTransactionResults(func(t *testing.T, actualBlockID flow.Identifier, actualResults []flow.LightTransactionResult) error { + assert.Equal(t, block.ID(), actualBlockID) + require.Len(t, actualResults, 0) + return nil + }). + setOnCollection(func(_ flow.Identifier, entity flow.Entity) { + require.Less(t, collectionsHandled, len(expectedCollections), "more collections handled than expected") + + actual, ok := entity.(*flow.Collection) + require.True(t, ok) + assert.Equal(t, expectedCollections[collectionsHandled], actual) + collectionsHandled++ + }). + // make sure an empty set of register entries was stored + setStoreRegisters(func(t *testing.T, entries flow.RegisterEntries, height uint64) error { + assert.Equal(t, height, block.Header.Height) + assert.Equal(t, 0, entries.Len()) + return nil + }). + runIndexBlockData() + + assert.NoError(t, err) + }) + t.Run("Index AllTheThings", func(t *testing.T) { expectedEvents := unittest.EventsFixture(20) expectedResults := unittest.LightTransactionResultsFixture(20) + expectedCollections := unittest.CollectionListFixture(2) expectedTries := []*ledger.TrieUpdate{trieUpdateFixture(t), trieUpdateFixture(t)} expectedPayloads := make([]*ledger.Payload, 0) for _, trie := range expectedTries { @@ -353,11 +420,13 @@ func TestExecutionState_IndexBlockData(t *testing.T) { BlockID: block.ID(), ChunkExecutionDatas: []*execution_data.ChunkExecutionData{ { + Collection: expectedCollections[0], Events: expectedEvents[:10], TransactionResults: expectedResults[:10], TrieUpdate: expectedTries[0], }, { + Collection: expectedCollections[1], TransactionResults: expectedResults[10:], Events: expectedEvents[10:], TrieUpdate: expectedTries[1], @@ -365,7 +434,7 @@ func TestExecutionState_IndexBlockData(t *testing.T) { }, } execData := execution_data.NewBlockExecutionDataEntity(block.ID(), ed) - + collectionsHandled := 0 err := newIndexCoreTest(t, blocks, execData). initIndexer(). // make sure all events are stored at once in order @@ -387,6 +456,14 @@ func TestExecutionState_IndexBlockData(t *testing.T) { } return nil }). + setOnCollection(func(_ flow.Identifier, entity flow.Entity) { + require.Less(t, collectionsHandled, len(expectedCollections), "more collections handled than expected") + + actual, ok := entity.(*flow.Collection) + require.True(t, ok) + assert.Equal(t, expectedCollections[collectionsHandled], actual) + collectionsHandled++ + }). // make sure update registers match in length and are same as block data ledger payloads setStoreRegisters(func(t *testing.T, entries flow.RegisterEntries, actualHeight uint64) error { assert.Equal(t, actualHeight, block.Header.Height) @@ -557,7 +634,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { // this test makes sure index values for a single register are correctly updated and always last value is returned t.Run("Single Index Value Changes", func(t *testing.T) { pebbleStorage.RunWithRegistersStorageAtInitialHeights(t, 0, 0, func(registers *pebbleStorage.Registers) { - index, err := New(logger, metrics, db, registers, nil, nil, nil) + index, err := New(logger, metrics, db, registers, nil, nil, nil, noopHandlerFunc) require.NoError(t, err) values := [][]byte{[]byte("1"), []byte("1"), []byte("2"), []byte("3"), []byte("4")} @@ -578,7 +655,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { // up to the specification script executor requires t.Run("Missing Register", func(t *testing.T) { pebbleStorage.RunWithRegistersStorageAtInitialHeights(t, 0, 0, func(registers *pebbleStorage.Registers) { - index, err := New(logger, metrics, db, registers, nil, nil, nil) + index, err := New(logger, metrics, db, registers, nil, nil, nil, noopHandlerFunc) require.NoError(t, err) value, err := index.RegisterValue(registerID, 0) @@ -592,7 +669,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { // e.g. we index A{h(1) -> X}, A{h(2) -> Y}, when we request h(4) we get value Y t.Run("Single Index Value At Later Heights", func(t *testing.T) { pebbleStorage.RunWithRegistersStorageAtInitialHeights(t, 0, 0, func(registers *pebbleStorage.Registers) { - index, err := New(logger, metrics, db, registers, nil, nil, nil) + index, err := New(logger, metrics, db, registers, nil, nil, nil, noopHandlerFunc) require.NoError(t, err) storeValues := [][]byte{[]byte("1"), []byte("2")} @@ -623,7 +700,7 @@ func TestIndexerIntegration_StoreAndGet(t *testing.T) { // this test makes sure we correctly handle weird payloads t.Run("Empty and Nil Payloads", func(t *testing.T) { pebbleStorage.RunWithRegistersStorageAtInitialHeights(t, 0, 0, func(registers *pebbleStorage.Registers) { - index, err := New(logger, metrics, db, registers, nil, nil, nil) + index, err := New(logger, metrics, db, registers, nil, nil, nil, noopHandlerFunc) require.NoError(t, err) require.NoError(t, index.indexRegisters(map[ledger.Path]*ledger.Payload{}, 1)) diff --git a/network/alsp/cache.go b/network/alsp/cache.go index eeab8fee302..be75043cfa0 100644 --- a/network/alsp/cache.go +++ b/network/alsp/cache.go @@ -12,7 +12,7 @@ type SpamRecordCache interface { // Returns the Penalty value of the record after the adjustment. // It returns an error if the adjustFunc returns an error or if the record does not exist. // Assuming that adjust is always called when the record exists, the error is irrecoverable and indicates a bug. - Adjust(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) + AdjustWithInit(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) // Identities returns the list of identities of the nodes that have a spam record in the cache. Identities() []flow.Identifier diff --git a/network/alsp/internal/cache.go b/network/alsp/internal/cache.go index 6bc6f361593..321bdb3baf7 100644 --- a/network/alsp/internal/cache.go +++ b/network/alsp/internal/cache.go @@ -14,8 +14,6 @@ import ( "github.com/onflow/flow-go/network/alsp/model" ) -var ErrSpamRecordNotFound = fmt.Errorf("spam record not found") - // SpamRecordCache is a cache that stores spam records at the protocol layer for ALSP. type SpamRecordCache struct { recordFactory model.SpamRecordFactoryFunc // recordFactory is a factory function that creates a new spam record. @@ -39,11 +37,7 @@ var _ alsp.SpamRecordCache = (*SpamRecordCache)(nil) func NewSpamRecordCache(sizeLimit uint32, logger zerolog.Logger, collector module.HeroCacheMetrics, recordFactory model.SpamRecordFactoryFunc) *SpamRecordCache { backData := herocache.NewCache(sizeLimit, herocache.DefaultOversizeFactor, - // this cache is supposed to keep the spam record for the authorized (staked) nodes. Since the number of such nodes is - // expected to be small, we do not eject any records from the cache. The cache size must be large enough to hold all - // the spam records of the authorized nodes. Also, this cache is keeping at most one record per origin id, so the - // size of the cache must be at least the number of authorized nodes. - heropool.NoEjection, + heropool.LRUEjection, logger.With().Str("mempool", "aslp-spam-records").Logger(), collector) @@ -53,73 +47,21 @@ func NewSpamRecordCache(sizeLimit uint32, logger zerolog.Logger, collector modul } } -// init initializes the spam record cache for the given origin id if it does not exist. -// Returns true if the record is initialized, false otherwise (i.e., the record already exists). -// Args: -// - originId: the origin id of the spam record. -// Returns: -// - true if the record is initialized, false otherwise (i.e., the record already exists). -// Note that if Init is called multiple times for the same origin id, the record is initialized only once, and the -// subsequent calls return false and do not change the record (i.e., the record is not re-initialized). -func (s *SpamRecordCache) init(originId flow.Identifier) bool { - return s.c.Add(ProtocolSpamRecordEntity{s.recordFactory(originId)}) -} - -// Adjust applies the given adjust function to the spam record of the given origin id. +// AdjustWithInit applies the given adjust function to the spam record of the given origin id. // Returns the Penalty value of the record after the adjustment. // It returns an error if the adjustFunc returns an error or if the record does not exist. -// Note that if the record the Adjust is called when the record does not exist, the record is initialized and the -// adjust function is applied to the initialized record again. In this case, the adjust function should not return an error. +// Note that if the record does not exist, the record is initialized and the +// adjust function is applied to the initialized record again. // Args: // - originId: the origin id of the spam record. // - adjustFunc: the function that adjusts the spam record. // Returns: // - Penalty value of the record after the adjustment. // - error any returned error should be considered as an irrecoverable error and indicates a bug. -func (s *SpamRecordCache) Adjust(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) { - // first, we try to optimistically adjust the record assuming that the record already exists. - penalty, err := s.adjust(originId, adjustFunc) - - switch { - case err == ErrSpamRecordNotFound: - // if the record does not exist, we initialize the record and try to adjust it again. - // Note: there is an edge case where the record is initialized by another goroutine between the two calls. - // In this case, the init function is invoked twice, but it is not a problem because the underlying - // cache is thread-safe. Hence, we do not need to synchronize the two calls. In such cases, one of the - // two calls returns false, and the other call returns true. We do not care which call returns false, hence, - // we ignore the return value of the init function. - _ = s.init(originId) - // as the record is initialized, the adjust function should not return an error, and any returned error - // is an irrecoverable error and indicates a bug. - return s.adjust(originId, adjustFunc) - case err != nil: - // if the adjust function returns an unexpected error on the first attempt, we return the error directly. - return 0, err - default: - // if the adjust function returns no error, we return the penalty value. - return penalty, nil - } -} - -// adjust applies the given adjust function to the spam record of the given origin id. -// Returns the Penalty value of the record after the adjustment. -// It returns an error if the adjustFunc returns an error or if the record does not exist. -// Args: -// - originId: the origin id of the spam record. -// - adjustFunc: the function that adjusts the spam record. -// Returns: -// - Penalty value of the record after the adjustment. -// - error if the adjustFunc returns an error or if the record does not exist (ErrSpamRecordNotFound). Except the ErrSpamRecordNotFound, -// any other error should be treated as an irrecoverable error and indicates a bug. -func (s *SpamRecordCache) adjust(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) { +func (s *SpamRecordCache) AdjustWithInit(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) { var rErr error - adjustedEntity, adjusted := s.c.Adjust(originId, func(entity flow.Entity) flow.Entity { - record, ok := entity.(ProtocolSpamRecordEntity) - if !ok { - // sanity check - // This should never happen, because the cache only contains ProtocolSpamRecordEntity entities. - panic(fmt.Sprintf("invalid entity type, expected ProtocolSpamRecordEntity type, got: %T", entity)) - } + wrapAdjustFunc := func(entity flow.Entity) flow.Entity { + record := mustBeProtocolSpamRecordEntity(entity) // Adjust the record. adjustedRecord, err := adjustFunc(record.ProtocolSpamRecord) @@ -130,17 +72,22 @@ func (s *SpamRecordCache) adjust(originId flow.Identifier, adjustFunc model.Reco // Return the adjusted record. return ProtocolSpamRecordEntity{adjustedRecord} - }) + } + initFunc := func() flow.Entity { + return ProtocolSpamRecordEntity{s.recordFactory(originId)} + } + adjustedEntity, adjusted := s.c.AdjustWithInit(originId, wrapAdjustFunc, initFunc) if rErr != nil { return 0, fmt.Errorf("failed to adjust record: %w", rErr) } if !adjusted { - return 0, ErrSpamRecordNotFound + return 0, fmt.Errorf("adjustment failed for origin id %s", originId) } - return adjustedEntity.(ProtocolSpamRecordEntity).Penalty, nil + record := mustBeProtocolSpamRecordEntity(adjustedEntity) + return record.Penalty, nil } // Get returns the spam record of the given origin id. @@ -156,12 +103,7 @@ func (s *SpamRecordCache) Get(originId flow.Identifier) (*model.ProtocolSpamReco return nil, false } - record, ok := entity.(ProtocolSpamRecordEntity) - if !ok { - // sanity check - // This should never happen, because the cache only contains ProtocolSpamRecordEntity entities. - panic(fmt.Sprintf("invalid entity type, expected ProtocolSpamRecordEntity type, got: %T", entity)) - } + record := mustBeProtocolSpamRecordEntity(entity) // return a copy of the record (we do not want the caller to modify the record). return &model.ProtocolSpamRecord{ @@ -192,3 +134,19 @@ func (s *SpamRecordCache) Remove(originId flow.Identifier) bool { func (s *SpamRecordCache) Size() uint { return s.c.Size() } + +// mustBeProtocolSpamRecordEntity returns the given entity as a ProtocolSpamRecordEntity. +// It panics if the given entity is not a ProtocolSpamRecordEntity. +// Args: +// - entity: the entity to be converted. +// Returns: +// - ProtocolSpamRecordEntity, the converted entity. +func mustBeProtocolSpamRecordEntity(entity flow.Entity) ProtocolSpamRecordEntity { + record, ok := entity.(ProtocolSpamRecordEntity) + if !ok { + // sanity check + // This should never happen, because the cache only contains ProtocolSpamRecordEntity entities. + panic(fmt.Sprintf("invalid entity type, expected ProtocolSpamRecordEntity type, got: %T", entity)) + } + return record +} diff --git a/network/alsp/internal/cache_test.go b/network/alsp/internal/cache_test.go index 22efd456a8e..4837bd6bd1c 100644 --- a/network/alsp/internal/cache_test.go +++ b/network/alsp/internal/cache_test.go @@ -73,7 +73,7 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { originID2 := unittest.IdentifierFixture() // adjusting a spam record for an origin ID that does not exist in the cache should initialize the record. - initializedPenalty, err := cache.Adjust(originID1, adjustFuncIncrement) + initializedPenalty, err := cache.AdjustWithInit(originID1, adjustFuncIncrement) require.NoError(t, err, "expected no error") require.Equal(t, float64(1), initializedPenalty, "expected initialized penalty to be 1") @@ -86,7 +86,7 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { // adjusting a spam record for an origin ID that already exists in the cache should not initialize the record, // but should apply the adjust function to the existing record. - initializedPenalty, err = cache.Adjust(originID1, adjustFuncIncrement) + initializedPenalty, err = cache.AdjustWithInit(originID1, adjustFuncIncrement) require.NoError(t, err, "expected no error") require.Equal(t, float64(2), initializedPenalty, "expected initialized penalty to be 2") record1Again, ok := cache.Get(originID1) @@ -98,7 +98,7 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { // adjusting a spam record for a different origin ID should initialize the record. // this is to ensure that the record factory is called only once. - initializedPenalty, err = cache.Adjust(originID2, adjustFuncIncrement) + initializedPenalty, err = cache.AdjustWithInit(originID2, adjustFuncIncrement) require.NoError(t, err, "expected no error") require.Equal(t, float64(1), initializedPenalty, "expected initialized penalty to be 1") record2, ok := cache.Get(originID2) @@ -131,10 +131,10 @@ func TestSpamRecordCache_Adjust_Error(t *testing.T) { originID2 := unittest.IdentifierFixture() // initialize spam records for originID1 and originID2 - penalty, err := cache.Adjust(originID1, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID1, adjustFnNoOp) require.NoError(t, err, "expected no error") require.Equal(t, 0.0, penalty, "expected penalty to be 0") - penalty, err = cache.Adjust(originID2, adjustFnNoOp) + penalty, err = cache.AdjustWithInit(originID2, adjustFnNoOp) require.NoError(t, err, "expected no error") require.Equal(t, 0.0, penalty, "expected penalty to be 0") @@ -143,7 +143,7 @@ func TestSpamRecordCache_Adjust_Error(t *testing.T) { record.Penalty -= 10 return record, nil } - penalty, err = cache.Adjust(originID1, adjustFunc) + penalty, err = cache.AdjustWithInit(originID1, adjustFunc) require.NoError(t, err) require.Equal(t, -10.0, penalty) @@ -156,7 +156,7 @@ func TestSpamRecordCache_Adjust_Error(t *testing.T) { adjustFuncError := func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { return record, errors.New("adjustment error") } - _, err = cache.Adjust(originID1, adjustFuncError) + _, err = cache.AdjustWithInit(originID1, adjustFuncError) require.Error(t, err) // even though the adjustFunc returned an error, the record should be intact. @@ -189,11 +189,11 @@ func TestSpamRecordCache_Identities(t *testing.T) { originID3 := unittest.IdentifierFixture() // initialize spam records for a few origin IDs - _, err := cache.Adjust(originID1, adjustFnNoOp) + _, err := cache.AdjustWithInit(originID1, adjustFnNoOp) require.NoError(t, err) - _, err = cache.Adjust(originID2, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID2, adjustFnNoOp) require.NoError(t, err) - _, err = cache.Adjust(originID3, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID3, adjustFnNoOp) require.NoError(t, err) // check if the Identities method returns the correct set of origin IDs @@ -235,11 +235,11 @@ func TestSpamRecordCache_Remove(t *testing.T) { originID3 := unittest.IdentifierFixture() // initialize spam records for a few origin IDs - _, err := cache.Adjust(originID1, adjustFnNoOp) + _, err := cache.AdjustWithInit(originID1, adjustFnNoOp) require.NoError(t, err) - _, err = cache.Adjust(originID2, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID2, adjustFnNoOp) require.NoError(t, err) - _, err = cache.Adjust(originID3, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID3, adjustFnNoOp) require.NoError(t, err) // remove originID1 and check if the record is removed @@ -280,14 +280,14 @@ func TestSpamRecordCache_EdgeCasesAndInvalidInputs(t *testing.T) { // 1. initializing a spam record multiple times originID1 := unittest.IdentifierFixture() - _, err := cache.Adjust(originID1, adjustFnNoOp) + _, err := cache.AdjustWithInit(originID1, adjustFnNoOp) require.NoError(t, err) - _, err = cache.Adjust(originID1, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID1, adjustFnNoOp) require.NoError(t, err) // 2. Test adjusting a non-existent spam record originID2 := unittest.IdentifierFixture() - initialPenalty, err := cache.Adjust(originID2, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + initialPenalty, err := cache.AdjustWithInit(originID2, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { record.Penalty -= 10 return record, nil }) @@ -296,7 +296,7 @@ func TestSpamRecordCache_EdgeCasesAndInvalidInputs(t *testing.T) { // 3. Test removing a spam record multiple times originID3 := unittest.IdentifierFixture() - _, err = cache.Adjust(originID3, adjustFnNoOp) + _, err = cache.AdjustWithInit(originID3, adjustFnNoOp) require.NoError(t, err) require.True(t, cache.Remove(originID3)) require.False(t, cache.Remove(originID3)) @@ -328,7 +328,7 @@ func TestSpamRecordCache_ConcurrentInitialization(t *testing.T) { for _, originID := range originIDs { go func(id flow.Identifier) { defer wg.Done() - penalty, err := cache.Adjust(id, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(id, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) }(originID) @@ -376,7 +376,7 @@ func TestSpamRecordCache_ConcurrentSameRecordAdjust(t *testing.T) { for i := 0; i < concurrentAttempts; i++ { go func() { defer wg.Done() - penalty, err := cache.Adjust(originID, adjustFn) + penalty, err := cache.AdjustWithInit(originID, adjustFn) require.NoError(t, err) require.Less(t, penalty, 0.0) // penalty should be negative }() @@ -415,7 +415,7 @@ func TestSpamRecordCache_ConcurrentRemoval(t *testing.T) { originIDs := unittest.IdentifierListFixture(10) for _, originID := range originIDs { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -464,7 +464,7 @@ func TestSpamRecordCache_ConcurrentUpdatesAndReads(t *testing.T) { originIDs := unittest.IdentifierListFixture(10) for _, originID := range originIDs { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -481,7 +481,7 @@ func TestSpamRecordCache_ConcurrentUpdatesAndReads(t *testing.T) { // adjust spam records concurrently go func(id flow.Identifier) { defer wg.Done() - _, err := cache.Adjust(id, adjustFunc) + _, err := cache.AdjustWithInit(id, adjustFunc) require.NoError(t, err) }(originID) @@ -529,7 +529,7 @@ func TestSpamRecordCache_ConcurrentInitAndRemove(t *testing.T) { originIDsToRemove := originIDs[10:] for _, originID := range originIDsToRemove { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -542,7 +542,7 @@ func TestSpamRecordCache_ConcurrentInitAndRemove(t *testing.T) { originID := originID // capture range variable go func() { defer wg.Done() - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) }() @@ -597,7 +597,7 @@ func TestSpamRecordCache_ConcurrentInitRemoveAdjust(t *testing.T) { originIDsToAdjust := originIDs[20:] for _, originID := range originIDsToRemove { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -615,7 +615,7 @@ func TestSpamRecordCache_ConcurrentInitRemoveAdjust(t *testing.T) { originID := originID // capture range variable go func() { defer wg.Done() - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) }() @@ -633,7 +633,7 @@ func TestSpamRecordCache_ConcurrentInitRemoveAdjust(t *testing.T) { for _, originID := range originIDsToAdjust { go func(id flow.Identifier) { defer wg.Done() - _, _ = cache.Adjust(id, adjustFunc) + _, _ = cache.AdjustWithInit(id, adjustFunc) }(originID) } @@ -668,13 +668,13 @@ func TestSpamRecordCache_ConcurrentInitRemoveAndAdjust(t *testing.T) { originIDsToAdjust := originIDs[20:] for _, originID := range originIDsToRemove { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } for _, originID := range originIDsToAdjust { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -687,7 +687,7 @@ func TestSpamRecordCache_ConcurrentInitRemoveAndAdjust(t *testing.T) { originID := originID go func() { defer wg.Done() - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) }() @@ -707,7 +707,7 @@ func TestSpamRecordCache_ConcurrentInitRemoveAndAdjust(t *testing.T) { originID := originID go func() { defer wg.Done() - _, err := cache.Adjust(originID, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + _, err := cache.AdjustWithInit(originID, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { record.Penalty -= 1 return record, nil }) @@ -763,7 +763,7 @@ func TestSpamRecordCache_ConcurrentIdentitiesAndOperations(t *testing.T) { originIDsToRemove := originIDs[10:20] for _, originID := range originIDsToRemove { - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) } @@ -776,7 +776,7 @@ func TestSpamRecordCache_ConcurrentIdentitiesAndOperations(t *testing.T) { originID := originID go func() { defer wg.Done() - penalty, err := cache.Adjust(originID, adjustFnNoOp) + penalty, err := cache.AdjustWithInit(originID, adjustFnNoOp) require.NoError(t, err) require.Equal(t, float64(0), penalty) retrieved, ok := cache.Get(originID) diff --git a/network/alsp/manager/manager.go b/network/alsp/manager/manager.go index f29fbc694b4..10d9be5b866 100644 --- a/network/alsp/manager/manager.go +++ b/network/alsp/manager/manager.go @@ -300,7 +300,7 @@ func (m *MisbehaviorReportManager) onHeartbeat() error { for _, id := range allIds { m.logger.Trace().Hex("identifier", logging.ID(id)).Msg("onHeartbeat - looping through spam records") - penalty, err := m.cache.Adjust(id, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + penalty, err := m.cache.AdjustWithInit(id, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { if record.Penalty > 0 { // sanity check; this should never happen. return record, fmt.Errorf("illegal state: spam record %x has positive penalty %f", id, record.Penalty) @@ -423,7 +423,7 @@ func (m *MisbehaviorReportManager) processMisbehaviorReport(report internal.Repo // a spam record for the peer first and then applies the penalty. In other words, Adjust uses an optimistic update by // first assuming that the spam record exists and then initializing it if it does not exist. In this way, we avoid // acquiring the lock twice per misbehavior report, reducing the contention on the lock and improving the performance. - updatedPenalty, err := m.cache.Adjust(report.OriginId, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + updatedPenalty, err := m.cache.AdjustWithInit(report.OriginId, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { if report.Penalty > 0 { // this should never happen, unless there is a bug in the misbehavior report handling logic. // we should crash the node in this case to prevent further misbehavior reports from being lost and fix the bug. diff --git a/network/alsp/manager/manager_test.go b/network/alsp/manager/manager_test.go index 28fd69cbf80..1f6b90c2cfe 100644 --- a/network/alsp/manager/manager_test.go +++ b/network/alsp/manager/manager_test.go @@ -1602,7 +1602,7 @@ func TestDecayMisbehaviorPenalty_DecayToZero_AllowListing(t *testing.T) { // simulates a disallow-listed peer in cache. originId := unittest.IdentifierFixture() - penalty, err := cache.Adjust(originId, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + penalty, err := cache.AdjustWithInit(originId, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { record.Penalty = -10 // set the penalty to -10 to simulate that the penalty has already been decayed for a while. record.CutoffCounter = 1 record.DisallowListed = true diff --git a/network/alsp/mock/spam_record_cache.go b/network/alsp/mock/spam_record_cache.go index ecc9f4ae1a5..f9c7a6952bd 100644 --- a/network/alsp/mock/spam_record_cache.go +++ b/network/alsp/mock/spam_record_cache.go @@ -14,8 +14,8 @@ type SpamRecordCache struct { mock.Mock } -// Adjust provides a mock function with given fields: originId, adjustFunc -func (_m *SpamRecordCache) Adjust(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) { +// AdjustWithInit provides a mock function with given fields: originId, adjustFunc +func (_m *SpamRecordCache) AdjustWithInit(originId flow.Identifier, adjustFunc model.RecordAdjustFunc) (float64, error) { ret := _m.Called(originId, adjustFunc) var r0 float64 diff --git a/network/cache/rcvcache.go b/network/cache/rcvcache.go index bdab2ad894a..f929a18a523 100644 --- a/network/cache/rcvcache.go +++ b/network/cache/rcvcache.go @@ -61,6 +61,6 @@ func (r *ReceiveCache) Add(eventID []byte) bool { return r.c.Add(receiveCacheEntry{eventID: flow.HashToID(eventID)}) // ignore eviction status } -func (r ReceiveCache) Size() uint { +func (r *ReceiveCache) Size() uint { return r.c.Size() } diff --git a/network/netconf/flags.go b/network/netconf/flags.go index 7ac4f69f2c6..7c8f8ca5d90 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -88,8 +88,6 @@ func AllFlagNames() []string { BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ClusterPrefixedMessageConfigKey, p2pconfig.TrackerCacheSizeKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ClusterPrefixedMessageConfigKey, p2pconfig.TrackerCacheDecayKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ClusterPrefixedMessageConfigKey, p2pconfig.HardThresholdKey), - BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.MetricsConfigKey, p2pconfig.NumberOfWorkersKey), - BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.MetricsConfigKey, p2pconfig.CacheSizeKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.NotificationCacheSizeKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MaxSampleSizeKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MaxMessageIDSampleSizeKey), @@ -103,14 +101,60 @@ func AllFlagNames() []string { BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.MessageErrorThresholdKey), BuildFlagName(gossipsubKey, p2pconfig.SubscriptionProviderKey, p2pconfig.UpdateIntervalKey), BuildFlagName(gossipsubKey, p2pconfig.SubscriptionProviderKey, p2pconfig.CacheSizeKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateWorkerNumKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.CacheSizeKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.PenaltyDecaySlowdownThresholdKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayRateReductionFactorKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.PenaltyDecayEvaluationPeriodKey), - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.DecayIntervalKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.AppSpecificScoreWeightKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.DecayIntervalKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.DecayToZeroKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.SkipAtomicValidationKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.InvalidMessageDeliveriesWeightKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.InvalidMessageDeliveriesDecayKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.TimeInMeshQuantumKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.TopicWeightKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesDecayKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesCapKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveryThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshDeliveriesWeightKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesWindowKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveryActivationKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.GossipThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.PublishThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.GraylistThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.AcceptPXThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.OpportunisticGraftThresholdKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyWeightKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyDecayKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.MaxDebugLogsKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MaxAppSpecificKey, p2pconfig.PenaltyKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MinAppSpecificKey, p2pconfig.PenaltyKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.UnknownIdentityKey, p2pconfig.PenaltyKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.InvalidSubscriptionKey, p2pconfig.PenaltyKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MaxAppSpecificKey, p2pconfig.RewardKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.StakedIdentityKey, p2pconfig.RewardKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.StartupSilenceDurationKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateWorkerNumKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.CacheSizeKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.PenaltyDecaySlowdownThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.DecayRateReductionFactorKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.PenaltyDecayEvaluationPeriodKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.MinimumSpamPenaltyDecayFactorKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.MaximumSpamPenaltyDecayFactorKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.SkipDecayThresholdKey), + + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.GraftMisbehaviourKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.PruneMisbehaviourKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.IHaveMisbehaviourKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.IWantMisbehaviourKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.PublishMisbehaviourKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.ClusterPrefixedReductionFactorKey), } for _, scope := range []string{systemScope, transientScope, protocolScope, peerScope, peerProtocolScope} { @@ -201,12 +245,6 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ClusterPrefixedMessageConfigKey, p2pconfig.HardThresholdKey), config.GossipSub.RpcInspector.Validation.ClusterPrefixedMessage.HardThreshold, "the maximum number of cluster-prefixed control messages allowed to be processed when the active cluster id is unset or a mismatch is detected, exceeding this threshold will result in node penalization by gossipsub.") - // gossipsub RPC control message metrics observer inspector configuration - flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.MetricsConfigKey, p2pconfig.NumberOfWorkersKey), - config.GossipSub.RpcInspector.Metrics.NumberOfWorkers, - "number of workers for gossipsub RPC metrics inspector queue.") - flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.MetricsConfigKey, p2pconfig.CacheSizeKey), config.GossipSub.RpcInspector.Metrics.CacheSize, - "cache size for gossipsub RPC metrics inspector events worker pool.") // networking event notifications flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.NotificationCacheSizeKey), config.GossipSub.RpcInspector.NotificationCacheSize, "cache size for notification events from gossipsub rpc inspector") @@ -225,17 +263,6 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { "base probability of creating a misbehavior report for a range request message") flags.Float32(alspSyncEngineSyncRequestProb, config.AlspConfig.SyncEngine.SyncRequestProb, "probability of creating a misbehavior report for a sync request message") - flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.PenaltyDecaySlowdownThresholdKey), - config.GossipSub.ScoringParameters.SpamRecordCache.PenaltyDecaySlowdownThreshold, - fmt.Sprintf("the penalty level at which the decay rate is reduced by --%s", - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayRateReductionFactorKey))) - flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayRateReductionFactorKey), - config.GossipSub.ScoringParameters.SpamRecordCache.DecayRateReductionFactor, - fmt.Sprintf("defines the value by which the decay rate is decreased every time the penalty is below the --%s", - BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.PenaltyDecaySlowdownThresholdKey))) - flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.PenaltyDecayEvaluationPeriodKey), - config.GossipSub.ScoringParameters.SpamRecordCache.PenaltyDecayEvaluationPeriod, - "defines the period at which the decay for a spam record is okay to be adjusted.") flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MaxSampleSizeKey), config.GossipSub.RpcInspector.Validation.IHave.MaxSampleSize, "max number of ihaves to sample when performing validation") @@ -272,21 +299,155 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.SubscriptionProviderKey, p2pconfig.CacheSizeKey), config.GossipSub.SubscriptionProvider.CacheSize, "size of the cache that keeps the list of topics each peer has subscribed to, recommended size is 10x the number of authorized nodes") - flags.Int(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateWorkerNumKey), - config.GossipSub.ScoringParameters.AppSpecificScore.ScoreUpdateWorkerNum, + + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.DecayIntervalKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.DecayInterval, + "interval at which the counters associated with a peer behavior in GossipSub system are decayed, recommended value is one minute") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.AppSpecificScoreWeightKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.AppSpecificScoreWeight, + "the weight for app-specific scores") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.DecayToZeroKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.DecayToZero, + "the maximum value below which a peer scoring counter is reset to zero") + + flags.Bool(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.SkipAtomicValidationKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.SkipAtomicValidation, + "the default value for the skip atomic validation flag for topics") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.InvalidMessageDeliveriesWeightKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.InvalidMessageDeliveriesWeight, + "this value is applied to the square of the number of invalid message deliveries on a topic") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.InvalidMessageDeliveriesDecayKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.InvalidMessageDeliveriesDecay, + "the decay factor used to decay the number of invalid message deliveries") + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.TimeInMeshQuantumKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.TimeInMeshQuantum, + "the time in mesh quantum for the GossipSub scoring system") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.TopicWeightKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.TopicWeight, + "the weight of a topic in the GossipSub scoring system") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesDecayKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesDecay, + "this is applied to the number of actual message deliveries in a topic mesh at each decay interval (i.e., DecayInterval)") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesCapKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesCap, + "The maximum number of actual message deliveries in a topic mesh that is used to calculate the score of a peer in that topic mesh") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveryThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshMessageDeliveryThreshold, + "The threshold for the number of actual message deliveries in a topic mesh that is used to calculate the score of a peer in that topic mesh") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshDeliveriesWeightKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshDeliveriesWeight, + "the weight for applying penalty when a peer is under-performing in a topic mesh") + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveriesWindowKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesWindow, + "the window size is time interval that we count a delivery of an already seen message towards the score of a peer in a topic mesh. The delivery is counted by GossipSub only if the previous sender of the message is different from the current sender") + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.TopicKey, p2pconfig.MeshMessageDeliveryActivationKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.TopicParameters.MeshMessageDeliveryActivation, + "the time interval that we wait for a new peer that joins a topic mesh till start counting the number of actual message deliveries of that peer in that topic mesh") + + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.GossipThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Gossip, + "the threshold when a peer's penalty drops below this threshold, no gossip is emitted towards that peer and gossip from that peer is ignored") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.PublishThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Publish, + "the threshold when a peer's penalty drops below this threshold, self-published messages are not propagated towards this peer") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.GraylistThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.Graylist, + "the threshold when a peer's penalty drops below this threshold, the peer is graylisted, i.e., incoming RPCs from the peer are ignored") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.AcceptPXThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.AcceptPX, + "the threshold when a peer sends us PX information with a prune, we only accept it and connect to the supplied peers if the originating peer's penalty exceeds this threshold") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.ThresholdsKey, p2pconfig.OpportunisticGraftThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Thresholds.OpportunisticGraft, + "the threshold when the median peer penalty in the mesh drops below this value, the peer may select more peers with penalty above the median to opportunistically graft on the mesh") + + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyThresholdKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyThreshold, + "the threshold when the behavior of a peer is considered as bad by GossipSub") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyWeightKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyWeight, + "the weight for applying penalty when a peer misbehavior goes beyond the threshold") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.InternalKey, p2pconfig.BehaviourKey, p2pconfig.BehaviourPenaltyDecayKey), + config.GossipSub.ScoringParameters.PeerScoring.Internal.Behaviour.PenaltyDecay, + "the decay interval for the misbehavior counter of a peer. The misbehavior counter is incremented by GossipSub for iHave broken promises or the GRAFT flooding attacks (i.e., each GRAFT received from a remote peer while that peer is on a PRUNE backoff)") + + flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.MaxDebugLogsKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.MaxDebugLogs, + "the max number of debug/trace log events per second. Logs emitted above this threshold are dropped") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MaxAppSpecificKey, p2pconfig.PenaltyKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.MaxAppSpecificPenalty, + "the maximum penalty for sever offenses that we apply to a remote node score") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MinAppSpecificKey, p2pconfig.PenaltyKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.MinAppSpecificPenalty, + "the minimum penalty for sever offenses that we apply to a remote node score") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.UnknownIdentityKey, p2pconfig.PenaltyKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.UnknownIdentityPenalty, + "the penalty for unknown identity. It is applied to the peer's score when the peer is not in the identity list") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.InvalidSubscriptionKey, p2pconfig.PenaltyKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.InvalidSubscriptionPenalty, + "the penalty for invalid subscription. It is applied to the peer's score when the peer subscribes to a topic that it is not authorized to subscribe to") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.MaxAppSpecificKey, p2pconfig.RewardKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.MaxAppSpecificReward, + "the reward for well-behaving staked peers") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.StakedIdentityKey, p2pconfig.RewardKey), + config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.StakedIdentityReward, + "the reward for staking peers") + + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.StartupSilenceDurationKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.StartupSilenceDuration, + "the duration of time, after the node startup, during which the scoring registry remains inactive before penalizing nodes.") + flags.Int(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateWorkerNumKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreUpdateWorkerNum, "number of workers for the app specific score update worker pool") - flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), - config.GossipSub.ScoringParameters.AppSpecificScore.ScoreUpdateRequestQueueSize, + flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreUpdateRequestQueueSize, "size of the app specific score update worker pool queue") - flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), - config.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL, + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL, "time to live for app specific scores; when expired a new request will be sent to the score update worker pool; till then the expired score will be used") - flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.SpamRecordCacheKey, p2pconfig.CacheSizeKey), - config.GossipSub.ScoringParameters.SpamRecordCache.CacheSize, + + flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.CacheSizeKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.CacheSize, "size of the spam record cache, recommended size is 10x the number of authorized nodes") - flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.DecayIntervalKey), - config.GossipSub.ScoringParameters.DecayInterval, - "interval at which the counters associated with a peer behavior in GossipSub system are decayed, recommended value is one minute") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.PenaltyDecaySlowdownThresholdKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.PenaltyDecaySlowdownThreshold, + fmt.Sprintf("the penalty level at which the decay rate is reduced by --%s", + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.DecayRateReductionFactorKey))) + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.DecayRateReductionFactorKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.DecayRateReductionFactor, + fmt.Sprintf("defines the value by which the decay rate is decreased every time the penalty is below the --%s", + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.PenaltyDecaySlowdownThresholdKey))) + flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.PenaltyDecayEvaluationPeriodKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.PenaltyDecayEvaluationPeriod, + "defines the period at which the decay for a spam record is okay to be adjusted.") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.MinimumSpamPenaltyDecayFactorKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor, + "the minimum speed at which the spam penalty value of a peer is decayed") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.MaximumSpamPenaltyDecayFactorKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor, + "the maximum rate at which the spam penalty value of a peer decays") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.DecayKey, p2pconfig.SkipDecayThresholdKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.SkipDecayThreshold, + "the threshold for which when the negative penalty is above this value, the decay function will not be called") + + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.GraftMisbehaviourKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.GraftMisbehaviour, + "the penalty applied to the application specific penalty when a peer conducts a graft misbehaviour") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.PruneMisbehaviourKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.PruneMisbehaviour, + "the penalty applied to the application specific penalty when a peer conducts a prune misbehaviour") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.IHaveMisbehaviourKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.IHaveMisbehaviour, + "the penalty applied to the application specific penalty when a peer conducts a iHave misbehaviour") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.IWantMisbehaviourKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.IWantMisbehaviour, + "the penalty applied to the application specific penalty when a peer conducts a iWant misbehaviour") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.PublishMisbehaviourKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.PublishMisbehaviour, + "the penalty applied to the application specific penalty when a peer conducts a rpc publish message misbehaviour") + flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.MisbehaviourPenaltiesKey, p2pconfig.ClusterPrefixedReductionFactorKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.MisbehaviourPenalties.ClusterPrefixedReductionFactor, + "the factor used to reduce the penalty for control message misbehaviours on cluster prefixed topics") + } // LoadLibP2PResourceManagerFlags loads all CLI flags for the libp2p resource manager configuration on the provided pflag set. @@ -362,6 +523,7 @@ func SetAliases(conf *viper.Viper) error { if !ok { return fmt.Errorf("invalid network configuration missing configuration key flag name %s check config file and cli flags", flagName) } + conf.RegisterAlias(fullKey, flagName) } return nil diff --git a/network/p2p/builder/gossipsub/gossipSubBuilder.go b/network/p2p/builder/gossipsub/gossipSubBuilder.go index 283af4b1fcd..7c0b7508cb4 100644 --- a/network/p2p/builder/gossipsub/gossipSubBuilder.go +++ b/network/p2p/builder/gossipsub/gossipSubBuilder.go @@ -20,7 +20,6 @@ import ( inspectorbuilder "github.com/onflow/flow-go/network/p2p/builder/inspector" p2pconfig "github.com/onflow/flow-go/network/p2p/config" "github.com/onflow/flow-go/network/p2p/distributor" - "github.com/onflow/flow-go/network/p2p/inspector" "github.com/onflow/flow-go/network/p2p/inspector/validation" p2pnode "github.com/onflow/flow-go/network/p2p/node" "github.com/onflow/flow-go/network/p2p/scoring" @@ -207,6 +206,7 @@ func defaultGossipSubAdapterConfig() p2p.GossipSubAdapterConfigFunc { // defaultInspectorSuite returns the default inspector suite factory function. It is used to create the default inspector suite. // Inspector suite is utilized to inspect the incoming gossipsub rpc messages from different perspectives. // Note: always use the default inspector suite factory function to create the inspector suite (unless you know what you are doing). +// todo: this function can be simplified. func defaultInspectorSuite(rpcTracker p2p.RpcControlTracking) p2p.GossipSubRpcInspectorSuiteFactoryFunc { return func(ctx irrecoverable.SignalerContext, logger zerolog.Logger, @@ -217,13 +217,7 @@ func defaultInspectorSuite(rpcTracker p2p.RpcControlTracking) p2p.GossipSubRpcIn networkType network.NetworkingType, idProvider module.IdentityProvider, topicProvider func() p2p.TopicProvider) (p2p.GossipSubInspectorSuite, error) { - metricsInspector := inspector.NewControlMsgMetricsInspector(logger, - p2pnode.NewGossipSubControlMessageMetrics(gossipSubMetrics, logger), - inspectorCfg.Metrics.NumberOfWorkers, - []queue.HeroStoreConfigOption{ - queue.WithHeroStoreSizeLimit(inspectorCfg.Metrics.CacheSize), - queue.WithHeroStoreCollector(metrics.GossipSubRPCMetricsObserverInspectorQueueMetricFactory(heroCacheMetricsFactory, networkType)), - }...) + notificationDistributor := distributor.DefaultGossipSubInspectorNotificationDistributor(logger, []queue.HeroStoreConfigOption{ queue.WithHeroStoreSizeLimit(inspectorCfg.NotificationCacheSize), queue.WithHeroStoreCollector(metrics.RpcInspectorNotificationQueueMetricFactory(heroCacheMetricsFactory, networkType))}...) @@ -244,7 +238,7 @@ func defaultInspectorSuite(rpcTracker p2p.RpcControlTracking) p2p.GossipSubRpcIn if err != nil { return nil, fmt.Errorf("failed to create new control message valiadation inspector: %w", err) } - return inspectorbuilder.NewGossipSubInspectorSuite(metricsInspector, rpcValidationInspector, notificationDistributor), nil + return inspectorbuilder.NewGossipSubInspectorSuite(rpcValidationInspector, notificationDistributor), nil } } diff --git a/network/p2p/builder/inspector/suite.go b/network/p2p/builder/inspector/suite.go index 8fe6a1c4547..b1b35d8bc2c 100644 --- a/network/p2p/builder/inspector/suite.go +++ b/network/p2p/builder/inspector/suite.go @@ -8,7 +8,6 @@ import ( "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/inspector" "github.com/onflow/flow-go/network/p2p/inspector/validation" ) @@ -21,6 +20,7 @@ type GossipSubInspectorSuite struct { ctrlMsgInspectDistributor p2p.GossipSubInspectorNotifDistributor } +// TODO: this can be simplified as there is no more need for the aggregated inspector. var _ p2p.GossipSubInspectorSuite = (*GossipSubInspectorSuite)(nil) // NewGossipSubInspectorSuite creates a new GossipSubInspectorSuite. @@ -36,10 +36,10 @@ var _ p2p.GossipSubInspectorSuite = (*GossipSubInspectorSuite)(nil) // regarding gossipsub control messages is detected. // Returns: // - the new GossipSubInspectorSuite. -func NewGossipSubInspectorSuite(metricsInspector *inspector.ControlMsgMetricsInspector, +func NewGossipSubInspectorSuite( validationInspector *validation.ControlMsgValidationInspector, ctrlMsgInspectDistributor p2p.GossipSubInspectorNotifDistributor) *GossipSubInspectorSuite { - inspectors := []p2p.GossipSubRPCInspector{metricsInspector, validationInspector} + inspectors := []p2p.GossipSubRPCInspector{validationInspector} s := &GossipSubInspectorSuite{ ctrlMsgInspectDistributor: ctrlMsgInspectDistributor, validationInspector: validationInspector, diff --git a/network/p2p/cache.go b/network/p2p/cache.go index bc87f0d80e8..226617c7a6d 100644 --- a/network/p2p/cache.go +++ b/network/p2p/cache.go @@ -36,15 +36,6 @@ type UpdateFunction func(record GossipSubSpamRecord) GossipSubSpamRecord // // Implementation must be thread-safe. type GossipSubSpamRecordCache interface { - // Add adds the GossipSubSpamRecord of a peer to the cache. - // Args: - // - peerID: the peer ID of the peer in the GossipSub protocol. - // - record: the GossipSubSpamRecord of the peer. - // - // Returns: - // - bool: true if the record was added successfully, false otherwise. - Add(peerId peer.ID, record GossipSubSpamRecord) bool - // Get returns the GossipSubSpamRecord of a peer from the cache. // Args: // - peerID: the peer ID of the peer in the GossipSub protocol. @@ -54,14 +45,16 @@ type GossipSubSpamRecordCache interface { // - bool: true if the record was retrieved successfully, false otherwise. Get(peerID peer.ID) (*GossipSubSpamRecord, error, bool) - // Update updates the GossipSub spam penalty of a peer in the cache using the given adjust function. + // Adjust updates the GossipSub spam penalty of a peer in the cache. If the peer does not have a record in the cache, a new record is created. + // The order of the pre-processing functions is the same as the order in which they were added to the cache. // Args: // - peerID: the peer ID of the peer in the GossipSub protocol. - // - adjustFn: the adjust function to be applied to the record. + // - updateFn: the update function to be applied to the record. // Returns: // - *GossipSubSpamRecord: the updated record. // - error on failure to update the record. The returned error is irrecoverable and indicates an exception. - Update(peerID peer.ID, updateFunc UpdateFunction) (*GossipSubSpamRecord, error) + // Note that if any of the pre-processing functions returns an error, the record is reverted to its original state (prior to applying the update function). + Adjust(peerID peer.ID, updateFunc UpdateFunction) (*GossipSubSpamRecord, error) // Has returns true if the cache contains the GossipSubSpamRecord of the given peer. // Args: @@ -87,7 +80,7 @@ type GossipSubApplicationSpecificScoreCache interface { // - bool: true if the score was retrieved successfully, false otherwise. Get(peerID peer.ID) (float64, time.Time, bool) - // Add adds the application specific score of a peer to the cache. + // AdjustWithInit adds the application specific score of a peer to the cache. // If the peer already has a score in the cache, the score is updated. // Args: // - peerID: the peer ID of the peer in the GossipSub protocol. @@ -95,7 +88,7 @@ type GossipSubApplicationSpecificScoreCache interface { // - time: the time at which the score was last updated. // Returns: // - error on failure to add the score. The returned error is irrecoverable and indicates an exception. - Add(peerID peer.ID, score float64, time time.Time) error + AdjustWithInit(peerID peer.ID, score float64, time time.Time) error } // GossipSubSpamRecord represents spam record of a peer in the GossipSub protocol. diff --git a/network/p2p/cache/gossipsub_spam_records.go b/network/p2p/cache/gossipsub_spam_records.go index 19f73bd246a..2b99666a49b 100644 --- a/network/p2p/cache/gossipsub_spam_records.go +++ b/network/p2p/cache/gossipsub_spam_records.go @@ -30,6 +30,9 @@ type GossipSubSpamRecordCache struct { // Primary use case is to perform decay operations on the record before reading or updating it. In this way, a // record is only decayed when it is read or updated without the need to explicitly iterating over the cache. preprocessFns []PreprocessorFunc + + // initFn is a function that is called to initialize a new record in the cache. + initFn func() p2p.GossipSubSpamRecord } var _ p2p.GossipSubSpamRecordCache = (*GossipSubSpamRecordCache)(nil) @@ -63,41 +66,21 @@ type PreprocessorFunc func(record p2p.GossipSubSpamRecord, lastUpdated time.Time func NewGossipSubSpamRecordCache(sizeLimit uint32, logger zerolog.Logger, collector module.HeroCacheMetrics, + initFn func() p2p.GossipSubSpamRecord, prFns ...PreprocessorFunc) *GossipSubSpamRecordCache { backData := herocache.NewCache(sizeLimit, herocache.DefaultOversizeFactor, - // we should not evict any record from the cache, - // eviction will open the node to spam attacks by malicious peers to erase their application specific penalty. - heropool.NoEjection, + heropool.LRUEjection, logger.With().Str("mempool", "gossipsub-app-Penalty-cache").Logger(), collector) return &GossipSubSpamRecordCache{ c: stdmap.NewBackend(stdmap.WithBackData(backData)), preprocessFns: prFns, + initFn: initFn, } } -// Add adds the GossipSubSpamRecord of a peer to the cache. -// Args: -// - peerID: the peer ID of the peer in the GossipSub protocol. -// - record: the GossipSubSpamRecord of the peer. -// -// Returns: -// - bool: true if the record was added successfully, false otherwise. -// Note that a record is added successfully if the cache has enough space to store the record and no record exists for the peer in the cache. -// In other words, the entries are deduplicated by the peer ID. -func (a *GossipSubSpamRecordCache) Add(peerId peer.ID, record p2p.GossipSubSpamRecord) bool { - entityId := flow.HashToID([]byte(peerId)) // HeroCache uses hash of peer.ID as the unique identifier of the record. - return a.c.Add(gossipsubSpamRecordEntity{ - entityId: entityId, - peerID: peerId, - lastUpdated: time.Now(), - GossipSubSpamRecord: record, - }) -} - -// Update updates the GossipSub spam penalty of a peer in the cache. It assumes that a record already exists for the peer in the cache. -// It first reads the record from the cache, applies the pre-processing functions to the record, and then applies the update function to the record. +// Adjust updates the GossipSub spam penalty of a peer in the cache. If the peer does not have a record in the cache, a new record is created. // The order of the pre-processing functions is the same as the order in which they were added to the cache. // Args: // - peerID: the peer ID of the peer in the GossipSub protocol. @@ -106,16 +89,12 @@ func (a *GossipSubSpamRecordCache) Add(peerId peer.ID, record p2p.GossipSubSpamR // - *GossipSubSpamRecord: the updated record. // - error on failure to update the record. The returned error is irrecoverable and indicates an exception. // Note that if any of the pre-processing functions returns an error, the record is reverted to its original state (prior to applying the update function). -func (a *GossipSubSpamRecordCache) Update(peerID peer.ID, updateFn p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error) { - // HeroCache uses flow.Identifier for keys, so reformat of the peer.ID - entityId := flow.HashToID([]byte(peerID)) - if !a.c.Has(entityId) { - return nil, fmt.Errorf("could not update spam records for peer %s, record not found", peerID.String()) - } +func (a *GossipSubSpamRecordCache) Adjust(peerID peer.ID, updateFn p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error) { + entityId := entityIdOf(peerID) var err error - record, updated := a.c.Adjust(entityId, func(entry flow.Entity) flow.Entity { - e := entry.(gossipsubSpamRecordEntity) + adjustFunc := func(entity flow.Entity) flow.Entity { + e := entity.(gossipsubSpamRecordEntity) currentRecord := e.GossipSubSpamRecord // apply the pre-processing functions to the record. @@ -134,16 +113,25 @@ func (a *GossipSubSpamRecordCache) Update(peerID peer.ID, updateFn p2p.UpdateFun e.lastUpdated = time.Now() } return e - }) + } + + initFunc := func() flow.Entity { + return gossipsubSpamRecordEntity{ + entityId: entityId, + peerID: peerID, + GossipSubSpamRecord: a.initFn(), + } + } + + adjustedEntity, adjusted := a.c.AdjustWithInit(entityId, adjustFunc, initFunc) if err != nil { - return nil, fmt.Errorf("could not update spam records for peer %s, error: %w", peerID.String(), err) + return nil, fmt.Errorf("error while applying pre-processing functions to cache record for peer %s: %w", p2plogging.PeerId(peerID), err) } - if !updated { - // this happens when the underlying HeroCache fails to update the record. - return nil, fmt.Errorf("internal cache error for updating %s", peerID.String()) + if !adjusted { + return nil, fmt.Errorf("could not adjust cache record for peer %s", p2plogging.PeerId(peerID)) } - r := record.(gossipsubSpamRecordEntity).GossipSubSpamRecord + r := adjustedEntity.(gossipsubSpamRecordEntity).GossipSubSpamRecord return &r, nil } @@ -153,7 +141,7 @@ func (a *GossipSubSpamRecordCache) Update(peerID peer.ID, updateFn p2p.UpdateFun // Returns: // - true if the gossipsub spam record of the peer is found in the cache, false otherwise. func (a *GossipSubSpamRecordCache) Has(peerID peer.ID) bool { - entityId := flow.HashToID([]byte(peerID)) // HeroCache uses hash of peer.ID as the unique identifier of the record. + entityId := entityIdOf(peerID) return a.c.Has(entityId) } @@ -168,14 +156,14 @@ func (a *GossipSubSpamRecordCache) Has(peerID peer.ID) bool { // the caller is advised to crash the node. // - true if the record is found in the cache, false otherwise. func (a *GossipSubSpamRecordCache) Get(peerID peer.ID) (*p2p.GossipSubSpamRecord, error, bool) { - entityId := flow.HashToID([]byte(peerID)) // HeroCache uses hash of peer.ID as the unique identifier of the record. + entityId := entityIdOf(peerID) if !a.c.Has(entityId) { return nil, nil, false } var err error - record, updated := a.c.Adjust(entityId, func(entry flow.Entity) flow.Entity { - e := entry.(gossipsubSpamRecordEntity) + record, updated := a.c.Adjust(entityId, func(entity flow.Entity) flow.Entity { + e := mustBeGossipSubSpamRecordEntity(entity) currentRecord := e.GossipSubSpamRecord for _, apply := range a.preprocessFns { @@ -197,7 +185,7 @@ func (a *GossipSubSpamRecordCache) Get(peerID peer.ID) (*p2p.GossipSubSpamRecord return nil, fmt.Errorf("could not decay cache record for peer %s", p2plogging.PeerId(peerID)), false } - r := record.(gossipsubSpamRecordEntity).GossipSubSpamRecord + r := mustBeGossipSubSpamRecordEntity(record).GossipSubSpamRecord return &r, nil, true } @@ -227,3 +215,31 @@ func (a gossipsubSpamRecordEntity) ID() flow.Identifier { func (a gossipsubSpamRecordEntity) Checksum() flow.Identifier { return a.entityId } + +// entityIdOf converts a peer ID to a flow ID by taking the hash of the peer ID. +// This is used to convert the peer ID in a notion that is compatible with HeroCache. +// This is not a protocol-level conversion, and is only used internally by the cache, MUST NOT be exposed outside the cache. +// Args: +// - peerId: the peer ID of the peer in the GossipSub protocol. +// Returns: +// - flow.Identifier: the flow ID of the peer. +func entityIdOf(peerId peer.ID) flow.Identifier { + return flow.MakeID(peerId) +} + +// mustBeGossipSubSpamRecordEntity converts a flow.Entity to a gossipsubSpamRecordEntity. +// This is used to convert the flow.Entity returned by HeroCache to a gossipsubSpamRecordEntity. +// If the conversion fails, it panics. +// Args: +// - entity: the flow.Entity to be converted. +// Returns: +// - gossipsubSpamRecordEntity: the converted gossipsubSpamRecordEntity. +func mustBeGossipSubSpamRecordEntity(entity flow.Entity) gossipsubSpamRecordEntity { + record, ok := entity.(gossipsubSpamRecordEntity) + if !ok { + // sanity check + // This should never happen, because the cache only contains gossipsubSpamRecordEntity entities. + panic(fmt.Sprintf("invalid entity type, expected gossipsubSpamRecordEntity type, got: %T", entity)) + } + return record +} diff --git a/network/p2p/cache/gossipsub_spam_records_test.go b/network/p2p/cache/gossipsub_spam_records_test.go index 166776b93ba..bee5ecdc26e 100644 --- a/network/p2p/cache/gossipsub_spam_records_test.go +++ b/network/p2p/cache/gossipsub_spam_records_test.go @@ -7,7 +7,6 @@ import ( "time" "github.com/libp2p/go-libp2p/core/peer" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -21,58 +20,65 @@ import ( // adding a new record to the cache. func TestGossipSubSpamRecordCache_Add(t *testing.T) { // create a new instance of GossipSubSpamRecordCache. - cache := netcache.NewGossipSubSpamRecordCache(100, unittest.Logger(), metrics.NewNoopCollector()) + cache := netcache.NewGossipSubSpamRecordCache(100, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) - // tests adding a new record to the cache. - require.True(t, cache.Add("peer0", p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) + adjustedEntity, err := cache.Adjust("peer0", func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Decay = 0.1 + record.Penalty = 0.5 - // tests updating an existing record in the cache. - require.False(t, cache.Add("peer0", p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) + return record + }) + require.NoError(t, err) + require.Equal(t, 0.1, adjustedEntity.Decay) + require.Equal(t, 0.5, adjustedEntity.Penalty) // makes the cache full. - for i := 1; i < 100; i++ { - require.True(t, cache.Add(peer.ID(fmt.Sprintf("peer%d", i)), p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) - } + for i := 1; i <= 100; i++ { + adjustedEntity, err := cache.Adjust(peer.ID(fmt.Sprintf("peer%d", i)), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Decay = 0.1 + record.Penalty = 0.5 - // adding a new record to the cache should fail. - require.False(t, cache.Add("peer101", p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) + return record + }) + + require.NoError(t, err) + require.Equal(t, 0.1, adjustedEntity.Decay) + } // retrieving an existing record should work. - for i := 0; i < 100; i++ { + for i := 1; i <= 100; i++ { record, err, ok := cache.Get(peer.ID(fmt.Sprintf("peer%d", i))) - require.True(t, ok) + require.True(t, ok, fmt.Sprintf("record for peer%d should exist", i)) require.NoError(t, err) require.Equal(t, 0.1, record.Decay) require.Equal(t, 0.5, record.Penalty) } - // yet attempting on adding an existing record should fail. - require.False(t, cache.Add("peer1", p2p.GossipSubSpamRecord{ - Decay: 0.2, - Penalty: 0.8, - })) + // since cache is LRU, the first record should be evicted. + _, err, ok := cache.Get("peer0") + require.False(t, ok) + require.NoError(t, err) } -// TestGossipSubSpamRecordCache_Concurrent_Add tests if the cache can be added and retrieved concurrently. -// It updates the cache with a number of records concurrently and then checks if the cache -// can retrieve all records. -func TestGossipSubSpamRecordCache_Concurrent_Add(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector()) +// TestGossipSubSpamRecordCache_Concurrent_Adjust tests if the cache can be adjusted and retrieved concurrently. +// It adjusts the cache with a number of records concurrently and then checks if the cache can retrieve all records. +func TestGossipSubSpamRecordCache_Concurrent_Adjust(t *testing.T) { + cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) - // defines the number of records to update. + // defines the number of records to be adjusted. numRecords := 100 // uses a wait group to wait for all goroutines to finish. @@ -84,15 +90,20 @@ func TestGossipSubSpamRecordCache_Concurrent_Add(t *testing.T) { go func(num int) { defer wg.Done() peerID := fmt.Sprintf("peer%d", num) - added := cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1 * float64(num), - Penalty: float64(num), + adjustedEntity, err := cache.Adjust(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Decay = 0.1 * float64(num) + record.Penalty = float64(num) + + return record }) - require.True(t, added) + + require.NoError(t, err) + require.Equal(t, 0.1*float64(num), adjustedEntity.Decay) + require.Equal(t, float64(num), adjustedEntity.Penalty) }(i) } - unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "could not update all records concurrently on time") + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "could not adjust all records concurrently on time") // checks if the cache can retrieve all records. for i := 0; i < numRecords; i++ { @@ -110,96 +121,51 @@ func TestGossipSubSpamRecordCache_Concurrent_Add(t *testing.T) { } } -// TestGossipSubSpamRecordCache_Update tests the Update method of the GossipSubSpamRecordCache. It tests if the cache can update -// the penalty of an existing record and fail to update the penalty of a non-existing record. -func TestGossipSubSpamRecordCache_Update(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector()) +// TestGossipSubSpamRecordCache_Adjust tests the Adjust method of the GossipSubSpamRecordCache. It tests if the cache can adjust +// the penalty of an existing record and add a new record. +func TestGossipSubSpamRecordCache_Adjust(t *testing.T) { + cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) peerID := "peer1" - // tests updateing the penalty of an existing record. - require.True(t, cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) - record, err := cache.Update(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + // test adjusting a non-existing record. + record, err := cache.Adjust(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { record.Penalty = 0.7 return record }) require.NoError(t, err) - require.Equal(t, 0.7, record.Penalty) // checks if the penalty is updateed correctly. + require.Equal(t, 0.7, record.Penalty) // checks if the penalty is adjusted correctly. - // tests updating the penalty of a non-existing record. - record, err = cache.Update(peer.ID("peer2"), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { - require.Fail(t, "the function should not be called for a non-existing record") + // test adjusting an existing record. + record, err = cache.Adjust(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 0.8 return record }) - require.Error(t, err) - require.Nil(t, record) -} - -// TestGossipSubSpamRecordCache_Concurrent_Update tests if the cache can be updated concurrently. It updates the cache -// with a number of records concurrently and then checks if the cache can retrieve all records. -func TestGossipSubSpamRecordCache_Concurrent_Update(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector()) - - // defines the number of records to update. - numRecords := 100 - - // adds all records to the cache, sequentially. - for i := 0; i < numRecords; i++ { - peerID := fmt.Sprintf("peer%d", i) - err := cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1 * float64(i), - Penalty: float64(i), - }) - require.True(t, err) - } - - // uses a wait group to wait for all goroutines to finish. - var wg sync.WaitGroup - wg.Add(numRecords) - - // updates the records concurrently. - for i := 0; i < numRecords; i++ { - go func(num int) { - defer wg.Done() - peerID := fmt.Sprintf("peer%d", num) - _, err := cache.Update(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { - record.Penalty = 0.7 * float64(num) - record.Decay = 0.1 * float64(num) - return record - }) - require.NoError(t, err) - }(i) - } - - unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "could not update all records concurrently on time") - - // checks if the cache can retrieve all records. - for i := 0; i < numRecords; i++ { - peerID := fmt.Sprintf("peer%d", i) - record, err, found := cache.Get(peer.ID(peerID)) - require.True(t, found) - require.NoError(t, err) - - expectedPenalty := 0.7 * float64(i) - require.Equal(t, expectedPenalty, record.Penalty, - "Get() returned incorrect Penalty for record %s: expected %f, got %f", peerID, expectedPenalty, record.Penalty) - expectedDecay := 0.1 * float64(i) - require.Equal(t, expectedDecay, record.Decay, - "Get() returned incorrect Decay for record %s: expected %f, got %f", peerID, expectedDecay, record.Decay) - } + require.NoError(t, err) + require.Equal(t, 0.8, record.Penalty) // checks if the penalty is adjusted correctly. } -// TestGossipSubSpamRecordCache_Update_With_Preprocess tests Update method of the GossipSubSpamRecordCache when the cache +// TestGossipSubSpamRecordCache_Adjust_With_Preprocess tests Adjust method of the GossipSubSpamRecordCache when the cache // has preprocessor functions. -// It tests when the cache has preprocessor functions, all preprocessor functions are called prior to the update function. +// It tests when the cache has preprocessor functions, all preprocessor functions are called prior to the adjust function. // Also, it tests if the pre-processor functions are called in the order they are added. -func TestGossipSubSpamRecordCache_Update_With_Preprocess(t *testing.T) { +func TestGossipSubSpamRecordCache_Adjust_With_Preprocess(t *testing.T) { cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector(), + func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }, func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { record.Penalty += 1.5 return record, nil @@ -209,14 +175,19 @@ func TestGossipSubSpamRecordCache_Update_With_Preprocess(t *testing.T) { }) peerID := "peer1" - // adds a record to the cache. - require.True(t, cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) + + // test adjusting a non-existing record. + record, err := cache.Adjust(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 0.5 + record.Decay = 0.1 + return record + }) + require.NoError(t, err) + require.Equal(t, 0.5, record.Penalty) // checks if the penalty is adjusted correctly. + require.Equal(t, 0.1, record.Decay) // checks if the decay is adjusted correctly. // tests updating the penalty of an existing record. - record, err := cache.Update(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record, err = cache.Adjust(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { record.Penalty += 0.7 return record }) @@ -225,46 +196,61 @@ func TestGossipSubSpamRecordCache_Update_With_Preprocess(t *testing.T) { require.Equal(t, 0.1, record.Decay) // checks if the decay is not changed. } -// TestGossipSubSpamRecordCache_Update_Preprocess_Error tests the Update method of the GossipSubSpamRecordCache. -// It tests if any of the preprocessor functions returns an error, the update function effect +// TestGossipSubSpamRecordCache_Adjust_Preprocess_Error tests the Adjust method of the GossipSubSpamRecordCache. +// It tests if any of the preprocessor functions returns an error, the Adjust function effect // is reverted, and the error is returned. -func TestGossipSubSpamRecordCache_Update_Preprocess_Error(t *testing.T) { - secondPreprocessorCalled := false +func TestGossipSubSpamRecordCache_Adjust_Preprocess_Error(t *testing.T) { + secondPreprocessorCalled := 0 cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector(), + func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }, // the first preprocessor function does not return an error. func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { return record, nil }, - // the second preprocessor function returns an error on the first call and nil on the second call onwards. + // the second preprocessor function returns an error on the second call, and does not return an error on any other call. + // this means that adjustment should be successful on the first call, and should fail on the second call. func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { - if !secondPreprocessorCalled { - secondPreprocessorCalled = true - return record, fmt.Errorf("error") + secondPreprocessorCalled++ + if secondPreprocessorCalled == 2 { + return record, fmt.Errorf("some error") } return record, nil + }) - peerID := "peer1" - // adds a record to the cache. - require.True(t, cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, - })) + peerID := unittest.PeerIdFixture(t) - // tests updating the penalty of an existing record. - record, err := cache.Update(peer.ID(peerID), func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + // tests adjusting the penalty of a non-existing record; the record should be initiated and the penalty should be adjusted. + record, err := cache.Adjust(peerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 0.5 + record.Decay = 0.1 + return record + }) + require.NoError(t, err) + require.NotNil(t, record) + require.Equal(t, 0.5, record.Penalty) // checks if the penalty is not changed. + require.Equal(t, 0.1, record.Decay) // checks if the decay is not changed. + + // tests adjusting the penalty of an existing record. + record, err = cache.Adjust(peerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { record.Penalty = 0.7 return record }) - // since the second preprocessor function returns an error, the update function effect should be reverted. + // since the second preprocessor function returns an error, the adjust function effect should be reverted. // the error should be returned. require.Error(t, err) require.Nil(t, record) // checks if the record is not changed. - record, err, found := cache.Get(peer.ID(peerID)) + record, err, found := cache.Get(peerID) require.True(t, found) require.NoError(t, err) require.Equal(t, 0.5, record.Penalty) // checks if the penalty is not changed. @@ -272,22 +258,33 @@ func TestGossipSubSpamRecordCache_Update_Preprocess_Error(t *testing.T) { } // TestGossipSubSpamRecordCache_ByValue tests if the cache stores the GossipSubSpamRecord by value. -// It updates the cache with a record and then modifies the record externally. +// It adjusts the cache with a record and then modifies the record externally. // It then checks if the record in the cache is still the original record. // This is a desired behavior that is guaranteed by the underlying HeroCache library. -// In other words, we don't desire the records to be externally mutable after they are added to the cache (unless by a subsequent call to Update). +// In other words, we don't desire the records to be externally mutable after they are added to the cache (unless by a subsequent call to Adjust). func TestGossipSubSpamRecordCache_ByValue(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector()) + cache := netcache.NewGossipSubSpamRecordCache(200, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) - peerID := "peer1" - added := cache.Add(peer.ID(peerID), p2p.GossipSubSpamRecord{ - Decay: 0.1, - Penalty: 0.5, + peerID := unittest.PeerIdFixture(t) + // adjusts a non-existing record, which should initiate the record. + record, err := cache.Adjust(peerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 0.5 + record.Decay = 0.1 + return record }) - require.True(t, added) + require.NoError(t, err) + require.NotNil(t, record) + require.Equal(t, 0.5, record.Penalty) // checks if the penalty is not changed. + require.Equal(t, 0.1, record.Decay) // checks if the decay is not changed. // get the record from the cache - record, err, found := cache.Get(peer.ID(peerID)) + record, err, found := cache.Get(peerID) require.True(t, found) require.NoError(t, err) @@ -296,7 +293,7 @@ func TestGossipSubSpamRecordCache_ByValue(t *testing.T) { record.Penalty = 0.8 // get the record from the cache again - record, err, found = cache.Get(peer.ID(peerID)) + record, err, found = cache.Get(peerID) require.True(t, found) require.NoError(t, err) @@ -305,10 +302,16 @@ func TestGossipSubSpamRecordCache_ByValue(t *testing.T) { require.Equal(t, 0.5, record.Penalty) } -// TestGossipSubSpamRecordCache_Get_With_Preprocessors tests if the cache applies the preprocessors to the records -// before returning them. +// TestGossipSubSpamRecordCache_Get_With_Preprocessors tests if the cache applies the preprocessors to the records before returning them. func TestGossipSubSpamRecordCache_Get_With_Preprocessors(t *testing.T) { cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector(), + func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }, // first preprocessor: adds 1 to the penalty. func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { record.Penalty++ @@ -321,22 +324,24 @@ func TestGossipSubSpamRecordCache_Get_With_Preprocessors(t *testing.T) { }, ) - record := p2p.GossipSubSpamRecord{ - Decay: 0.5, - Penalty: 1, - } - added := cache.Add("peerA", record) - assert.True(t, added) + peerId := unittest.PeerIdFixture(t) + adjustedRecord, err := cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 1 + record.Decay = 0.5 + return record + }) + require.NoError(t, err) + require.Equal(t, 1.0, adjustedRecord.Penalty) - // verifies that the preprocessors were called and the record was updated accordingly. - cachedRecord, err, ok := cache.Get("peerA") - assert.NoError(t, err) - assert.True(t, ok) + // verifies that the preprocessors were called and the record was adjusted accordingly. + cachedRecord, err, ok := cache.Get(peerId) + require.NoError(t, err) + require.True(t, ok) // expected penalty is 4: the first preprocessor adds 1 to the penalty and the second preprocessor multiplies the penalty by 2. // (1 + 1) * 2 = 4 - assert.Equal(t, 4.0, cachedRecord.Penalty) // penalty should be updated - assert.Equal(t, 0.5, cachedRecord.Decay) // decay should not be modified + require.Equal(t, 4.0, cachedRecord.Penalty) // penalty should be adjusted + require.Equal(t, 0.5, cachedRecord.Decay) // decay should not be modified } // TestGossipSubSpamRecordCache_Get_Preprocessor_Error tests if the cache returns an error if one of the preprocessors returns an error upon a Get. @@ -349,15 +354,22 @@ func TestGossipSubSpamRecordCache_Get_Preprocessor_Error(t *testing.T) { thirdPreprocessorCalledCount := 0 cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector(), + func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }, // first preprocessor: adds 1 to the penalty. func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { record.Penalty++ return record, nil }, - // second preprocessor: multiplies the penalty by 2 (this preprocessor returns an error on the second call) + // second preprocessor: multiplies the penalty by 2 (this preprocessor returns an error on the third call and forward) func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { secondPreprocessorCalledCount++ - if secondPreprocessorCalledCount < 2 { + if secondPreprocessorCalledCount < 3 { // on the first call, the preprocessor is successful return record, nil } else { @@ -365,109 +377,153 @@ func TestGossipSubSpamRecordCache_Get_Preprocessor_Error(t *testing.T) { return p2p.GossipSubSpamRecord{}, fmt.Errorf("error in preprocessor") } }, - // since second preprocessor returns an error on the second call, the third preprocessor should not be called more than once.. + // since second preprocessor returns an error on the second call, the third preprocessor should not be called more than once. func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { thirdPreprocessorCalledCount++ - require.Less(t, secondPreprocessorCalledCount, 2) + require.Less(t, secondPreprocessorCalledCount, 3) return record, nil }, ) - record := p2p.GossipSubSpamRecord{ - Decay: 0.5, - Penalty: 1, - } - added := cache.Add("peerA", record) - assert.True(t, added) + peerId := unittest.PeerIdFixture(t) + adjustedRecord, err := cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 1 + record.Decay = 0.5 + return record + }) + require.NoError(t, err) + require.Equal(t, 1.0, adjustedRecord.Penalty) + require.Equal(t, 0.5, adjustedRecord.Decay) - // verifies that the preprocessors were called and the penalty was updated accordingly. - cachedRecord, err, ok := cache.Get("peerA") + // verifies that the preprocessors were called and the penalty was adjusted accordingly. + cachedRecord, err, ok := cache.Get(peerId) require.NoError(t, err) - assert.True(t, ok) - assert.Equal(t, 2.0, cachedRecord.Penalty) // penalty should be updated by the first preprocessor (1 + 1 = 2) - assert.Equal(t, 0.5, cachedRecord.Decay) + require.True(t, ok) + require.Equal(t, 2.0, cachedRecord.Penalty) // penalty should be adjusted by the first preprocessor (1 + 1 = 2) + require.Equal(t, 0.5, cachedRecord.Decay) // query the cache again that should trigger the second preprocessor to return an error. - cachedRecord, err, ok = cache.Get("peerA") + cachedRecord, err, ok = cache.Get(peerId) require.Error(t, err) - assert.False(t, ok) - assert.Nil(t, cachedRecord) + require.False(t, ok) + require.Nil(t, cachedRecord) - // verifies that the third preprocessor was not called. - assert.Equal(t, 1, thirdPreprocessorCalledCount) - // verifies that the second preprocessor was called only twice (one success, and one failure). - assert.Equal(t, 2, secondPreprocessorCalledCount) + // verifies that the third preprocessor was called only twice (two success calls). + require.Equal(t, 2, thirdPreprocessorCalledCount) + // verifies that the second preprocessor was called three times (two success calls and one failure call). + require.Equal(t, 3, secondPreprocessorCalledCount) } // TestGossipSubSpamRecordCache_Get_Without_Preprocessors tests when no preprocessors are provided to the cache constructor // that the cache returns the original record without any modifications. func TestGossipSubSpamRecordCache_Get_Without_Preprocessors(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector()) + cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) - record := p2p.GossipSubSpamRecord{ - Decay: 0.5, - Penalty: 1, - } - added := cache.Add("peerA", record) - assert.True(t, added) - - // verifies that no preprocessors were called and the record was not updated. - cachedRecord, err, ok := cache.Get("peerA") - assert.NoError(t, err) - assert.True(t, ok) - assert.Equal(t, 1.0, cachedRecord.Penalty) - assert.Equal(t, 0.5, cachedRecord.Decay) + peerId := unittest.PeerIdFixture(t) + adjustedRecord, err := cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 1 + record.Decay = 0.5 + return record + }) + require.NoError(t, err) + require.Equal(t, 1.0, adjustedRecord.Penalty) + require.Equal(t, 0.5, adjustedRecord.Decay) + + // verifies that no preprocessors were called and the record was not adjusted. + cachedRecord, err, ok := cache.Get(peerId) + require.NoError(t, err) + require.True(t, ok) + require.Equal(t, 1.0, cachedRecord.Penalty) + require.Equal(t, 0.5, cachedRecord.Decay) } -// TestGossipSubSpamRecordCache_Duplicate_Add_Sequential tests if the cache returns false when a duplicate record is added to the cache. +// TestGossipSubSpamRecordCache_Duplicate_Adjust_Sequential tests if the cache returns false when a duplicate record is added to the cache. // This test evaluates that the cache de-duplicates the records based on their peer id and not content, and hence // each peer id can only be added once to the cache. -func TestGossipSubSpamRecordCache_Duplicate_Add_Sequential(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector()) +func TestGossipSubSpamRecordCache_Duplicate_Adjust_Sequential(t *testing.T) { + cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) - record := p2p.GossipSubSpamRecord{ - Decay: 0.5, - Penalty: 1, - } - added := cache.Add("peerA", record) - assert.True(t, added) + peerId := unittest.PeerIdFixture(t) + adjustedRecord, err := cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 1 + record.Decay = 0.5 + return record + }) + require.NoError(t, err) + require.Equal(t, 1.0, adjustedRecord.Penalty) + require.Equal(t, 0.5, adjustedRecord.Decay) - // verifies that the cache returns false when a duplicate record is added. - added = cache.Add("peerA", record) - assert.False(t, added) + // duplicate adjust should return the same record. + adjustedRecord, err = cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 1 + record.Decay = 0.5 + return record + }) + require.NoError(t, err) + require.Equal(t, 1.0, adjustedRecord.Penalty) + require.Equal(t, 0.5, adjustedRecord.Decay) // verifies that the cache deduplicates the records based on their peer id and not content. - record.Penalty = 2 - added = cache.Add("peerA", record) - assert.False(t, added) + adjustedRecord, err = cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = 3 + record.Decay = 2 + return record + }) + require.NoError(t, err) + require.Equal(t, 3.0, adjustedRecord.Penalty) + require.Equal(t, 2.0, adjustedRecord.Decay) } -// TestGossipSubSpamRecordCache_Duplicate_Add_Concurrent tests if the cache returns false when a duplicate record is added to the cache. -// Test is the concurrent version of TestAppScoreCache_DuplicateAdd_Sequential. -func TestGossipSubSpamRecordCache_Duplicate_Add_Concurrent(t *testing.T) { - cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector()) +// TestGossipSubSpamRecordCache_Duplicate_Adjust_Concurrent tests if the cache returns false when a duplicate record is added to the cache. +// Test is the concurrent version of TestAppScoreCache_Duplicate_Adjust_Sequential. +func TestGossipSubSpamRecordCache_Duplicate_Adjust_Concurrent(t *testing.T) { + cache := netcache.NewGossipSubSpamRecordCache(10, unittest.Logger(), metrics.NewNoopCollector(), func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: 0, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } + }) successAdd := atomic.Int32{} successAdd.Store(0) record1 := p2p.GossipSubSpamRecord{ - Decay: 0.5, + Decay: 1, Penalty: 1, } record2 := p2p.GossipSubSpamRecord{ - Decay: 0.5, + Decay: 1, Penalty: 2, } wg := sync.WaitGroup{} // wait group to wait for all goroutines to finish. wg.Add(2) + peerId := unittest.PeerIdFixture(t) // adds a record to the cache concurrently. - add := func(record p2p.GossipSubSpamRecord) { - added := cache.Add("peerA", record) - if added { - successAdd.Inc() - } + add := func(newRecord p2p.GossipSubSpamRecord) { + _, err := cache.Adjust(peerId, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record.Penalty = newRecord.Penalty + record.Decay = newRecord.Decay + record.LastDecayAdjustment = newRecord.LastDecayAdjustment + return record + }) + require.NoError(t, err) + successAdd.Inc() + wg.Done() } @@ -476,6 +532,12 @@ func TestGossipSubSpamRecordCache_Duplicate_Add_Concurrent(t *testing.T) { unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "could not add records to the cache") - // verifies that only one of the records was added to the cache. - assert.Equal(t, int32(1), successAdd.Load()) + // verifies that both of the records was added to the cache. + require.Equal(t, int32(2), successAdd.Load()) + + // verifies that the record is adjusted to one of the records. + cachedRecord, err, ok := cache.Get(peerId) + require.NoError(t, err) + require.True(t, ok) + require.True(t, cachedRecord.Penalty == 1 && cachedRecord.Decay == 1 || cachedRecord.Penalty == 2 && cachedRecord.Decay == 1) } diff --git a/network/p2p/config/gossipsub.go b/network/p2p/config/gossipsub.go index 95b6f48e8f6..31b69dd221b 100644 --- a/network/p2p/config/gossipsub.go +++ b/network/p2p/config/gossipsub.go @@ -79,62 +79,14 @@ type GossipSubParameters struct { } const ( - AppSpecificScoreRegistryKey = "app-specific-score" - SpamRecordCacheKey = "spam-record-cache" - DecayIntervalKey = "decay-interval" + DecayIntervalKey = "decay-interval" ) // ScoringParameters are the parameters for the score option. // Parameters are "numerical values" that are used to compute or build components that compute the score of a peer in GossipSub system. type ScoringParameters struct { - AppSpecificScore AppSpecificScoreParameters `validate:"required" mapstructure:"app-specific-score"` - SpamRecordCache SpamRecordCacheParameters `validate:"required" mapstructure:"spam-record-cache"` - // DecayInterval is the interval at which the counters associated with a peer behavior in GossipSub system are decayed. - DecayInterval time.Duration `validate:"gt=0s" mapstructure:"decay-interval"` -} - -const ( - ScoreUpdateWorkerNumKey = "score-update-worker-num" - ScoreUpdateRequestQueueSizeKey = "score-update-request-queue-size" - ScoreTTLKey = "score-ttl" -) - -// AppSpecificScoreParameters is the parameters for the GossipSubAppSpecificScoreRegistry. -// Parameters are "numerical values" that are used to compute or build components that compute or maintain the application specific score of peers. -type AppSpecificScoreParameters struct { - // ScoreUpdateWorkerNum is the number of workers in the worker pool for handling the application specific score update of peers in a non-blocking way. - ScoreUpdateWorkerNum int `validate:"gt=0" mapstructure:"score-update-worker-num"` - - // ScoreUpdateRequestQueueSize is the size of the worker pool for handling the application specific score update of peers in a non-blocking way. - ScoreUpdateRequestQueueSize uint32 `validate:"gt=0" mapstructure:"score-update-request-queue-size"` - - // ScoreTTL is the time to live of the application specific score of a peer; the registry keeps a cached copy of the - // application specific score of a peer for this duration. When the duration expires, the application specific score - // of the peer is updated asynchronously. As long as the update is in progress, the cached copy of the application - // specific score of the peer is used even if it is expired. - ScoreTTL time.Duration `validate:"required" mapstructure:"score-ttl"` -} - -const ( - PenaltyDecaySlowdownThresholdKey = "penalty-decay-slowdown-threshold" - DecayRateReductionFactorKey = "penalty-decay-rate-reduction-factor" - PenaltyDecayEvaluationPeriodKey = "penalty-decay-evaluation-period" -) - -type SpamRecordCacheParameters struct { - // CacheSize is size of the cache used to store the spam records of peers. - // The spam records are used to penalize peers that send invalid messages. - CacheSize uint32 `validate:"gt=0" mapstructure:"cache-size"` - - // PenaltyDecaySlowdownThreshold defines the penalty level which the decay rate is reduced by `DecayRateReductionFactor` every time the penalty of a node falls below the threshold, thereby slowing down the decay process. - // This mechanism ensures that malicious nodes experience longer decay periods, while honest nodes benefit from quicker decay. - PenaltyDecaySlowdownThreshold float64 `validate:"lt=0" mapstructure:"penalty-decay-slowdown-threshold"` - - // DecayRateReductionFactor defines the value by which the decay rate is decreased every time the penalty is below the PenaltyDecaySlowdownThreshold. A reduced decay rate extends the time it takes for penalties to diminish. - DecayRateReductionFactor float64 `validate:"gt=0,lt=1" mapstructure:"penalty-decay-rate-reduction-factor"` - - // PenaltyDecayEvaluationPeriod defines the interval at which the decay for a spam record is okay to be adjusted. - PenaltyDecayEvaluationPeriod time.Duration `validate:"gt=0" mapstructure:"penalty-decay-evaluation-period"` + PeerScoring PeerScoringParameters `validate:"required" mapstructure:"peer-scoring"` + ScoringRegistryParameters ScoringRegistryParameters `validate:"required" mapstructure:"scoring-registry"` } // SubscriptionProviderParameters keys. diff --git a/network/p2p/config/gossipsub_rpc_inspectors.go b/network/p2p/config/gossipsub_rpc_inspectors.go index ddceb12d869..829cb33ca61 100644 --- a/network/p2p/config/gossipsub_rpc_inspectors.go +++ b/network/p2p/config/gossipsub_rpc_inspectors.go @@ -11,8 +11,6 @@ const ( type RpcInspectorParameters struct { // RpcValidationInspector control message validation inspector validation configuration and limits. Validation RpcValidationInspector `mapstructure:"validation"` - // RpcMetricsInspectorConfigs control message metrics inspector configuration. - Metrics RpcMetricsInspectorConfigs `mapstructure:"metrics"` // NotificationCacheSize size of the queue for notifications about invalid RPC messages. NotificationCacheSize uint32 `mapstructure:"notification-cache-size"` } @@ -105,11 +103,3 @@ type ClusterPrefixedMessageInspectionParameters struct { const ( NumberOfWorkersKey = "workers" ) - -// RpcMetricsInspectorConfigs contains the "numerical values" for the gossipsub RPC control message metrics inspectors parameters. -type RpcMetricsInspectorConfigs struct { - // NumberOfWorkers number of worker pool workers. - NumberOfWorkers int `validate:"gte=1" mapstructure:"workers"` - // CacheSize size of the queue used by worker pool for the control message metrics inspector. - CacheSize uint32 `validate:"gt=0" mapstructure:"cache-size"` -} diff --git a/network/p2p/config/peer_scoring.go b/network/p2p/config/peer_scoring.go new file mode 100644 index 00000000000..25b70e77687 --- /dev/null +++ b/network/p2p/config/peer_scoring.go @@ -0,0 +1,233 @@ +package p2pconfig + +import "time" + +const ( + PeerScoringKey = "peer-scoring" + InternalKey = "internal" + ProtocolKey = "protocol" +) + +// PeerScoringParameters encapsulates the parameters of the GossipSub scoring system. +type PeerScoringParameters struct { + // Internal is the internal parameters of the GossipSub scoring system that are hosted by + // the GossipSub system, and are not exposed to the Flow protocol. + // The internal parameters are hosted by the GossipSub system. + Internal InternalGossipSubScoreParams `validate:"required" mapstructure:"internal"` + // Protocol is the protocol parameters of the peer scoring system that is hosted by the Flow protocol. + Protocol ProtocolLevelGossipSubScoreParams `validate:"required" mapstructure:"protocol"` +} + +const ( + AppSpecificScoreWeightKey = "app-specific-score-weight" + DecayToZeroKey = "decay-to-zero" + ThresholdsKey = "thresholds" + BehaviourKey = "behaviour" + TopicKey = "topic" +) + +type InternalGossipSubScoreParams struct { + // AppSpecificScoreWeight is the weight for app-specific scores. It is used to scale the app-specific + // scores to the same range as the other scores. At the current version, we don't distinguish between the app-specific + // scores and the other scores, so we set it to 1. + AppSpecificScoreWeight float64 `validate:"gt=0,lte=1" mapstructure:"app-specific-score-weight"` + // DecayInterval is the decay interval for the overall score of a peer at the GossipSub scoring + // system. We set it to 1 minute so that it is not too short so that a malicious node can recover from a penalty + // and is not too long so that a well-behaved node can't recover from a penalty. + DecayInterval time.Duration `validate:"gte=1m" mapstructure:"decay-interval"` + // DecayToZero is the decay to zero for the overall score of a peer at the GossipSub scoring system. + // It defines the maximum value below which a peer scoring counter is reset to zero. + // This is to prevent the counter from decaying to a very small value. + // The value is 0.01, which means that a counter will be reset to zero if it decays to 0.01. + // When a counter hits the DecayToZero threshold, it means that the peer did not exhibit the behavior + // for a long time, and we can reset the counter. + DecayToZero float64 `validate:"required" mapstructure:"decay-to-zero"` + TopicParameters TopicScoringParameters `validate:"required" mapstructure:"topic"` + Thresholds InternalScoringThresholds `validate:"required" mapstructure:"thresholds"` + Behaviour InternalScoringBehavioural `validate:"required" mapstructure:"behaviour"` +} + +const ( + MaxDebugLogsKey = "max-debug-logs" + AppSpecificKey = "application-specific" +) + +type ProtocolLevelGossipSubScoreParams struct { + MaxDebugLogs uint32 `validate:"lte=50" mapstructure:"max-debug-logs"` + AppSpecificScore ApplicationSpecificScoreParameters `validate:"required" mapstructure:"application-specific"` +} + +const ( + MaxAppSpecificKey = "max-app-specific" + MinAppSpecificKey = "min-app-specific" + UnknownIdentityKey = "unknown-identity" + InvalidSubscriptionKey = "invalid-subscription" + StakedIdentityKey = "staked-identity" + RewardKey = "reward" + PenaltyKey = "penalty" +) + +type ApplicationSpecificScoreParameters struct { + // MaxAppSpecificPenalty the maximum penalty for sever offenses that we apply to a remote node score. The score + // mechanism of GossipSub in Flow is designed in a way that all other infractions are penalized with a fraction of + // this value. We have also set the other parameters such as DefaultGraylistThreshold, DefaultGossipThreshold and DefaultPublishThreshold to + // be a bit higher than this, i.e., MaxAppSpecificPenalty + 1. This ensures that a node with a score of MaxAppSpecificPenalty + // will be graylisted (i.e., all incoming and outgoing RPCs are rejected) and will not be able to publish or gossip any messages. + MaxAppSpecificPenalty float64 `validate:"lt=0" mapstructure:"max-app-specific-penalty"` + // MinAppSpecificPenalty the minimum penalty for sever offenses that we apply to a remote node score. + MinAppSpecificPenalty float64 `validate:"lt=0" mapstructure:"min-app-specific-penalty"` + // UnknownIdentityPenalty is the penalty for unknown identity. It is applied to the peer's score when + // the peer is not in the identity list. + UnknownIdentityPenalty float64 `validate:"lt=0" mapstructure:"unknown-identity-penalty"` + // InvalidSubscriptionPenalty is the penalty for invalid subscription. It is applied to the peer's score when + // the peer subscribes to a topic that it is not authorized to subscribe to. + InvalidSubscriptionPenalty float64 `validate:"lt=0" mapstructure:"invalid-subscription-penalty"` + // MaxAppSpecificReward is the reward for well-behaving staked peers. If a peer does not have + // any misbehavior record, e.g., invalid subscription, invalid message, etc., it will be rewarded with this score. + MaxAppSpecificReward float64 `validate:"gt=0" mapstructure:"max-app-specific-reward"` + // StakedIdentityReward is the reward for staking peers. It is applied to the peer's score when + // the peer does not have any misbehavior record, e.g., invalid subscription, invalid message, etc. + // The purpose is to reward the staking peers for their contribution to the network and prioritize them in neighbor selection. + StakedIdentityReward float64 `validate:"gt=0" mapstructure:"staked-identity-reward"` +} + +const ( + GossipThresholdKey = "gossip" + PublishThresholdKey = "publish" + GraylistThresholdKey = "graylist" + AcceptPXThresholdKey = "accept-px" + OpportunisticGraftThresholdKey = "opportunistic-graft" +) + +// InternalScoringThresholds score option threshold configuration parameters. +type InternalScoringThresholds struct { + // Gossip when a peer's penalty drops below this threshold, + // no gossip is emitted towards that peer and gossip from that peer is ignored. + Gossip float64 `validate:"lt=0" mapstructure:"gossip"` + // Publish when a peer's penalty drops below this threshold, + // self-published messages are not propagated towards this peer. + Publish float64 `validate:"lt=0" mapstructure:"publish"` + // Graylist when a peer's penalty drops below this threshold, the peer is graylisted, i.e., + // incoming RPCs from the peer are ignored. + Graylist float64 `validate:"lt=0" mapstructure:"graylist"` + // AcceptPX when a peer sends us PX information with a prune, we only accept it and connect to the supplied + // peers if the originating peer's penalty exceeds this threshold. + AcceptPX float64 `validate:"gt=0" mapstructure:"accept-px"` + // OpportunisticGraft when the median peer penalty in the mesh drops below this value, + // the peer may select more peers with penalty above the median to opportunistically graft on the mesh. + OpportunisticGraft float64 `validate:"gt=0" mapstructure:"opportunistic-graft"` +} + +const ( + BehaviourPenaltyThresholdKey = "penalty-threshold" + BehaviourPenaltyWeightKey = "penalty-weight" + BehaviourPenaltyDecayKey = "penalty-decay" +) + +// InternalScoringBehavioural score option behaviour configuration parameters. +type InternalScoringBehavioural struct { + // PenaltyThreshold is the threshold when the behavior of a peer is considered as bad by GossipSub. + // Currently, the misbehavior is defined as advertising an iHave without responding to the iWants (iHave broken promises), as well as attempting + // on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh + // for a while, and the remote peer keep attempting on GRAFT (aka GRAFT flood). + // When the misbehavior counter of a peer goes beyond this threshold, the peer is penalized by BehaviorPenaltyWeight (see below) for the excess misbehavior. + // + // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + // For iHave broken promises, the gossipsub scoring works as follows: + // It samples ONLY A SINGLE iHave out of the entire RPC. + // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + // + // The counter is also decayed by (0.99) every decay interval (DecayInterval) i.e., every minute. + // Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through + // the ALSP system). + PenaltyThreshold float64 `validate:"gt=0" mapstructure:"penalty-threshold"` + // PenaltyWeight is the weight for applying penalty when a peer misbehavior goes beyond the threshold. + // Misbehavior of a peer at gossipsub layer is defined as advertising an iHave without responding to the iWants (broken promises), as well as attempting + // on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh + // This is detected by the GossipSub scoring system, and the peer is penalized by BehaviorPenaltyWeight. + // + // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + // For iHave broken promises, the gossipsub scoring works as follows: + // It samples ONLY A SINGLE iHave out of the entire RPC. + // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + PenaltyWeight float64 `validate:"lt=0" mapstructure:"penalty-weight"` + // PenaltyDecay is the decay interval for the misbehavior counter of a peer. The misbehavior counter is + // incremented by GossipSub for iHave broken promises or the GRAFT flooding attacks (i.e., each GRAFT received from a remote peer while that peer is on a PRUNE backoff). + // + // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. + // For iHave broken promises, the gossipsub scoring works as follows: + // It samples ONLY A SINGLE iHave out of the entire RPC. + // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. + // This means that regardless of how many iHave broken promises an RPC contains, the misbehavior counter is incremented by 1. + // That is why we decay the misbehavior counter very slow, as this counter indicates a severe misbehavior. + // The misbehavior counter is decayed per decay interval (i.e., DecayInterval = 1 minute) by GossipSub. + // + // Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through + // the ALSP system that is based on the engines report). + PenaltyDecay float64 `validate:"gt=0,lt=1" mapstructure:"penalty-decay"` +} + +const ( + SkipAtomicValidationKey = "skip-atomic-validation" + InvalidMessageDeliveriesWeightKey = "invalid-message-deliveries-weight" + InvalidMessageDeliveriesDecayKey = "invalid-message-deliveries-decay" + TimeInMeshQuantumKey = "time-in-mesh-quantum" + TopicWeightKey = "topic-weight" + MeshMessageDeliveriesDecayKey = "mesh-message-deliveries-decay" + MeshMessageDeliveriesCapKey = "mesh-message-deliveries-cap" + MeshMessageDeliveryThresholdKey = "mesh-message-deliveries-threshold" + MeshDeliveriesWeightKey = "mesh-deliveries-weight" + MeshMessageDeliveriesWindowKey = "mesh-message-deliveries-window" + MeshMessageDeliveryActivationKey = "mesh-message-delivery-activation" +) + +// TopicScoringParameters score option topic validation configuration parameters. +type TopicScoringParameters struct { + // SkipAtomicValidation is the value for the skip atomic validation flag for topics. + // If set it to true, the gossipsub parameter validation will not fail if we leave some of the + // topic parameters at their values, i.e., zero. + SkipAtomicValidation bool `validate:"required" mapstructure:"skip-atomic-validation"` + // InvalidMessageDeliveriesWeight this value is applied to the square of the number of invalid message deliveries on a topic. + // It is used to penalize peers that send invalid messages. By an invalid message, we mean a message that is not signed by the + // publisher, or a message that is not signed by the peer that sent it. + InvalidMessageDeliveriesWeight float64 `validate:"lt=0" mapstructure:"invalid-message-deliveries-weight"` + // InvalidMessageDeliveriesDecay decay factor used to decay the number of invalid message deliveries. + // The total number of invalid message deliveries is multiplied by this factor at each heartbeat interval to + // decay the number of invalid message deliveries, and prevent the peer from being disconnected if it stops + // sending invalid messages. + InvalidMessageDeliveriesDecay float64 `validate:"gt=0,lt=1" mapstructure:"invalid-message-deliveries-decay"` + // TimeInMeshQuantum is the time in mesh quantum for the GossipSub scoring system. It is used to gauge + // a discrete time interval for the time in mesh counter. + TimeInMeshQuantum time.Duration `validate:"gte=1h" mapstructure:"time-in-mesh-quantum"` + // Weight is the weight of a topic in the GossipSub scoring system. + // The overall score of a peer in a topic mesh is multiplied by the weight of the topic when calculating the overall score of the peer. + TopicWeight float64 `validate:"gt=0" mapstructure:"topic-weight"` + // MeshMessageDeliveriesDecay is applied to the number of actual message deliveries in a topic mesh + // at each decay interval (i.e., DecayInterval). + // It is used to decay the number of actual message deliveries, and prevents past message + // deliveries from affecting the current score of the peer. + MeshMessageDeliveriesDecay float64 `validate:"gt=0" mapstructure:"mesh-message-deliveries-decay"` + // MeshMessageDeliveriesCap is the maximum number of actual message deliveries in a topic + // mesh that is used to calculate the score of a peer in that topic mesh. + MeshMessageDeliveriesCap float64 `validate:"gt=0" mapstructure:"mesh-message-deliveries-cap"` + // MeshMessageDeliveryThreshold is the threshold for the number of actual message deliveries in a + // topic mesh that is used to calculate the score of a peer in that topic mesh. + // If the number of actual message deliveries in a topic mesh is less than this value, + // the peer will be penalized by square of the difference between the actual message deliveries and the threshold, + // i.e., -w * (actual - threshold)^2 where `actual` and `threshold` are the actual message deliveries and the + // threshold, respectively, and `w` is the weight (i.e., MeshMessageDeliveriesWeight). + MeshMessageDeliveryThreshold float64 `validate:"gt=0" mapstructure:"mesh-message-deliveries-threshold"` + // MeshDeliveriesWeight is the weight for applying penalty when a peer is under-performing in a topic mesh. + // Upon every decay interval, if the number of actual message deliveries is less than the topic mesh message deliveries threshold + // (i.e., MeshMessageDeliveriesThreshold), the peer will be penalized by square of the difference between the actual + // message deliveries and the threshold, multiplied by this weight, i.e., -w * (actual - threshold)^2 where w is the weight, and + // `actual` and `threshold` are the actual message deliveries and the threshold, respectively. + MeshDeliveriesWeight float64 `validate:"lt=0" mapstructure:"mesh-deliveries-weight"` + // MeshMessageDeliveriesWindow is the window size is time interval that we count a delivery of an already + // seen message towards the score of a peer in a topic mesh. The delivery is counted + // by GossipSub only if the previous sender of the message is different from the current sender. + MeshMessageDeliveriesWindow time.Duration `validate:"gte=1m" mapstructure:"mesh-message-deliveries-window"` + // MeshMessageDeliveryActivation is the time interval that we wait for a new peer that joins a topic mesh + // till start counting the number of actual message deliveries of that peer in that topic mesh. + MeshMessageDeliveryActivation time.Duration `validate:"gte=2m" mapstructure:"mesh-message-delivery-activation"` +} diff --git a/network/p2p/config/score_registry.go b/network/p2p/config/score_registry.go new file mode 100644 index 00000000000..3788451325a --- /dev/null +++ b/network/p2p/config/score_registry.go @@ -0,0 +1,103 @@ +package p2pconfig + +import "time" + +const ( + SpamRecordCacheKey = "spam-record-cache" + ScoringRegistryKey = "scoring-registry" + AppSpecificScoreRegistryKey = "app-specific-score" + StartupSilenceDurationKey = "startup-silence-duration" +) + +type ScoringRegistryParameters struct { + // StartupSilenceDuration defines the duration of time, after the node startup, + // during which the scoring registry remains inactive before penalizing nodes. + // Throughout this startup silence period, the application-specific penalty + // for all nodes will be set to 0, and any invalid control message notifications + // will be ignored. + // + // This configuration allows nodes to stabilize and initialize before + // applying penalties or responding processing invalid control message notifications. + StartupSilenceDuration time.Duration `validate:"gt=10m" mapstructure:"startup-silence-duration"` + AppSpecificScore AppSpecificScoreParameters `validate:"required" mapstructure:"app-specific-score"` + SpamRecordCache SpamRecordCacheParameters `validate:"required" mapstructure:"spam-record-cache"` + MisbehaviourPenalties MisbehaviourPenalties `validate:"required" mapstructure:"misbehaviour-penalties"` +} + +const ( + ScoreUpdateWorkerNumKey = "score-update-worker-num" + ScoreUpdateRequestQueueSizeKey = "score-update-request-queue-size" + ScoreTTLKey = "score-ttl" +) + +// AppSpecificScoreParameters is the parameters for the GossipSubAppSpecificScoreRegistry. +// Parameters are "numerical values" that are used to compute or build components that compute or maintain the application specific score of peers. +type AppSpecificScoreParameters struct { + // ScoreUpdateWorkerNum is the number of workers in the worker pool for handling the application specific score update of peers in a non-blocking way. + ScoreUpdateWorkerNum int `validate:"gt=0" mapstructure:"score-update-worker-num"` + + // ScoreUpdateRequestQueueSize is the size of the worker pool for handling the application specific score update of peers in a non-blocking way. + ScoreUpdateRequestQueueSize uint32 `validate:"gt=0" mapstructure:"score-update-request-queue-size"` + + // ScoreTTL is the time to live of the application specific score of a peer; the registry keeps a cached copy of the + // application specific score of a peer for this duration. When the duration expires, the application specific score + // of the peer is updated asynchronously. As long as the update is in progress, the cached copy of the application + // specific score of the peer is used even if it is expired. + ScoreTTL time.Duration `validate:"required" mapstructure:"score-ttl"` +} + +const ( + DecayKey = "decay" +) + +type SpamRecordCacheParameters struct { + // CacheSize is size of the cache used to store the spam records of peers. + // The spam records are used to penalize peers that send invalid messages. + CacheSize uint32 `validate:"gt=0" mapstructure:"cache-size"` + Decay SpamRecordCacheDecay `validate:"required" mapstructure:"decay"` +} + +const ( + PenaltyDecaySlowdownThresholdKey = "penalty-decay-slowdown-threshold" + DecayRateReductionFactorKey = "penalty-decay-rate-reduction-factor" + PenaltyDecayEvaluationPeriodKey = "penalty-decay-evaluation-period" + MinimumSpamPenaltyDecayFactorKey = "minimum-spam-penalty-decay-factor" + MaximumSpamPenaltyDecayFactorKey = "maximum-spam-penalty-decay-factor" + SkipDecayThresholdKey = "skip-decay-threshold" +) + +type SpamRecordCacheDecay struct { + // PenaltyDecaySlowdownThreshold defines the penalty level which the decay rate is reduced by `DecayRateReductionFactor` every time the penalty of a node falls below the threshold, thereby slowing down the decay process. + // This mechanism ensures that malicious nodes experience longer decay periods, while honest nodes benefit from quicker decay. + PenaltyDecaySlowdownThreshold float64 `validate:"lt=0" mapstructure:"penalty-decay-slowdown-threshold"` + + // DecayRateReductionFactor defines the value by which the decay rate is decreased every time the penalty is below the PenaltyDecaySlowdownThreshold. A reduced decay rate extends the time it takes for penalties to diminish. + DecayRateReductionFactor float64 `validate:"gt=0,lt=1" mapstructure:"penalty-decay-rate-reduction-factor"` + + // PenaltyDecayEvaluationPeriod defines the interval at which the decay for a spam record is okay to be adjusted. + PenaltyDecayEvaluationPeriod time.Duration `validate:"gt=0" mapstructure:"penalty-decay-evaluation-period"` + + SkipDecayThreshold float64 `validate:"gt=-1,lt=0" mapstructure:"skip-decay-threshold"` + + MinimumSpamPenaltyDecayFactor float64 `validate:"gt=0,lte=1" mapstructure:"minimum-spam-penalty-decay-factor"` + MaximumSpamPenaltyDecayFactor float64 `validate:"gt=0,lte=1" mapstructure:"maximum-spam-penalty-decay-factor"` +} + +const ( + MisbehaviourPenaltiesKey = "misbehaviour-penalties" + GraftMisbehaviourKey = "graft" + PruneMisbehaviourKey = "prune" + IHaveMisbehaviourKey = "ihave" + IWantMisbehaviourKey = "iwant" + PublishMisbehaviourKey = "publish" + ClusterPrefixedReductionFactorKey = "cluster-prefixed-reduction-factor" +) + +type MisbehaviourPenalties struct { + GraftMisbehaviour float64 `validate:"lt=0" mapstructure:"graft"` + PruneMisbehaviour float64 `validate:"lt=0" mapstructure:"prune"` + IHaveMisbehaviour float64 `validate:"lt=0" mapstructure:"ihave"` + IWantMisbehaviour float64 `validate:"lt=0" mapstructure:"iwant"` + PublishMisbehaviour float64 `validate:"lt=0" mapstructure:"publish"` + ClusterPrefixedReductionFactor float64 `validate:"gt=0,lte=1" mapstructure:"cluster-prefixed-reduction-factor"` +} diff --git a/network/p2p/inspector/control_message_metrics.go b/network/p2p/inspector/control_message_metrics.go deleted file mode 100644 index 9047d0f9484..00000000000 --- a/network/p2p/inspector/control_message_metrics.go +++ /dev/null @@ -1,101 +0,0 @@ -package inspector - -import ( - "fmt" - - pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine/common/worker" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/mempool/queue" - "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/inspector/internal" -) - -const ( - // DefaultControlMsgMetricsInspectorNumberOfWorkers default number of workers for the inspector component. - DefaultControlMsgMetricsInspectorNumberOfWorkers = 1 - // DefaultControlMsgMetricsInspectorQueueCacheSize is the default size of the message queue. - DefaultControlMsgMetricsInspectorQueueCacheSize = 100 - // rpcInspectorComponentName the rpc inspector component name. - rpcInspectorComponentName = "gossipsub_rpc_metrics_observer_inspector" -) - -// ObserveRPCMetricsRequest represents a request to capture metrics for the provided RPC -type ObserveRPCMetricsRequest struct { - // Nonce adds random value so that when msg req is stored on hero store a unique ID can be created from the struct fields. - Nonce []byte - // From the sender of the RPC. - From peer.ID - // rpc the rpc message. - rpc *pubsub.RPC -} - -// ControlMsgMetricsInspector a GossipSub RPC inspector that will observe incoming RPC's and collect metrics related to control messages. -type ControlMsgMetricsInspector struct { - component.Component - logger zerolog.Logger - // NumberOfWorkers number of component workers. - NumberOfWorkers int - // workerPool queue that stores *ObserveRPCMetricsRequest that will be processed by component workers. - workerPool *worker.Pool[*ObserveRPCMetricsRequest] - metrics p2p.GossipSubControlMetricsObserver -} - -var _ p2p.GossipSubRPCInspector = (*ControlMsgMetricsInspector)(nil) - -// Inspect submits a request to the worker pool to observe metrics for the rpc. -// All errors returned from this function can be considered benign. -func (c *ControlMsgMetricsInspector) Inspect(from peer.ID, rpc *pubsub.RPC) error { - nonce, err := internal.Nonce() - if err != nil { - return fmt.Errorf("failed to get observe rpc metrics request nonce: %w", err) - } - c.workerPool.Submit(&ObserveRPCMetricsRequest{Nonce: nonce, From: from, rpc: rpc}) - return nil -} - -// ObserveRPC collects metrics for the rpc. -// No error is ever returned from this func. -func (c *ControlMsgMetricsInspector) ObserveRPC(req *ObserveRPCMetricsRequest) error { - c.metrics.ObserveRPC(req.From, req.rpc) - return nil -} - -// Name returns the name of the rpc inspector. -func (c *ControlMsgMetricsInspector) Name() string { - return rpcInspectorComponentName -} - -// NewControlMsgMetricsInspector returns a new *ControlMsgMetricsInspector -func NewControlMsgMetricsInspector(logger zerolog.Logger, metricsObserver p2p.GossipSubControlMetricsObserver, numberOfWorkers int, heroStoreOpts ...queue.HeroStoreConfigOption) *ControlMsgMetricsInspector { - lg := logger.With().Str("component", "gossip_sub_rpc_metrics_observer_inspector").Logger() - c := &ControlMsgMetricsInspector{ - logger: lg, - NumberOfWorkers: numberOfWorkers, - metrics: metricsObserver, - } - - cfg := &queue.HeroStoreConfig{ - SizeLimit: DefaultControlMsgMetricsInspectorQueueCacheSize, - Collector: metrics.NewNoopCollector(), - } - - for _, opt := range heroStoreOpts { - opt(cfg) - } - store := queue.NewHeroStore(cfg.SizeLimit, logger, cfg.Collector) - pool := worker.NewWorkerPoolBuilder[*ObserveRPCMetricsRequest](c.logger, store, c.ObserveRPC).Build() - c.workerPool = pool - - builder := component.NewComponentManagerBuilder() - for i := 0; i < c.NumberOfWorkers; i++ { - builder.AddWorker(pool.WorkerLogic()) - } - c.Component = builder.Build() - - return c -} diff --git a/network/p2p/inspector/internal/cache/cache.go b/network/p2p/inspector/internal/cache/cache.go index 82d8f781a98..d64418d636f 100644 --- a/network/p2p/inspector/internal/cache/cache.go +++ b/network/p2p/inspector/internal/cache/cache.go @@ -51,11 +51,7 @@ type RecordCache struct { func NewRecordCache(config *RecordCacheConfig, recordEntityFactory recordEntityFactory) (*RecordCache, error) { backData := herocache.NewCache(config.sizeLimit, herocache.DefaultOversizeFactor, - // this cache is supposed to keep the cluster prefix control messages received record for the authorized (staked) nodes. Since the number of such nodes is - // expected to be small, we do not eject any records from the cache. The cache size must be large enough to hold all - // the records of the authorized nodes. Also, this cache is keeping at most one record per peer id, so the - // size of the cache must be at least the number of authorized nodes. - heropool.NoEjection, + heropool.LRUEjection, config.logger.With().Str("mempool", "gossipsub=cluster-prefix-control-messages-received-records").Logger(), config.collector) return &RecordCache{ @@ -65,19 +61,6 @@ func NewRecordCache(config *RecordCacheConfig, recordEntityFactory recordEntityF }, nil } -// Init initializes the record cache for the given peer id if it does not exist. -// Returns true if the record is initialized, false otherwise (i.e.: the record already exists). -// Args: -// - nodeID: the node ID of the sender of the control message. -// Returns: -// - true if the record is initialized, false otherwise (i.e.: the record already exists). -// Note that if Init is called multiple times for the same peer id, the record is initialized only once, and the -// subsequent calls return false and do not change the record (i.e.: the record is not re-initialized). -func (r *RecordCache) Init(nodeID flow.Identifier) bool { - entity := r.recordEntityFactory(nodeID) - return r.c.Add(entity) -} - // ReceivedClusterPrefixedMessage applies an adjustment that increments the number of cluster prefixed control messages received by a peer. // Returns number of cluster prefix control messages received after the adjustment. The record is initialized before // the adjustment func is applied that will increment the Gauge. @@ -88,35 +71,32 @@ func (r *RecordCache) Init(nodeID flow.Identifier) bool { // - exception only in cases of internal data inconsistency or bugs. No errors are expected. func (r *RecordCache) ReceivedClusterPrefixedMessage(nodeID flow.Identifier) (float64, error) { var err error - optimisticAdjustFunc := func() (flow.Entity, bool) { - return r.c.Adjust(nodeID, func(entity flow.Entity) flow.Entity { - entity, err = r.decayAdjustment(entity) // first decay the record - if err != nil { - return entity - } - return r.incrementAdjustment(entity) // then increment the record - }) + adjustFunc := func(entity flow.Entity) flow.Entity { + entity, err = r.decayAdjustment(entity) // first decay the record + if err != nil { + return entity + } + return r.incrementAdjustment(entity) // then increment the record } - // optimisticAdjustFunc is called assuming the record exists; if the record does not exist, - // it means the record was not initialized. In this case, initialize the record and call optimisticAdjustFunc again. - // If the record was initialized, optimisticAdjustFunc will be called only once. - adjustedEntity, adjusted := optimisticAdjustFunc() + adjustedEntity, adjusted := r.c.AdjustWithInit(nodeID, adjustFunc, func() flow.Entity { + return r.recordEntityFactory(nodeID) + }) + if err != nil { - return 0, fmt.Errorf("unexpected error while applying decay adjustment for node %s: %w", nodeID, err) + return 0, fmt.Errorf("unexpected error while applying decay and increment adjustments for node %s: %w", nodeID, err) } + if !adjusted { - r.Init(nodeID) - adjustedEntity, adjusted = optimisticAdjustFunc() - if !adjusted { - return 0, fmt.Errorf("unexpected record not found for node ID %s, even after an init attempt", nodeID) - } + return 0, fmt.Errorf("adjustment failed for node %s", nodeID) } - return adjustedEntity.(ClusterPrefixedMessagesReceivedRecord).Gauge, nil + record := mustBeClusterPrefixedMessageReceivedRecordEntity(adjustedEntity) + + return record.Gauge, nil } -// Get returns the current number of cluster prefixed control messages received from a peer. +// GetWithInit returns the current number of cluster prefixed control messages received from a peer. // The record is initialized before the count is returned. // Before the control messages received gauge value is returned it is decayed using the configured decay function. // Returns the record and true if the record exists, nil and false otherwise. @@ -125,30 +105,24 @@ func (r *RecordCache) ReceivedClusterPrefixedMessage(nodeID flow.Identifier) (fl // Returns: // - The cluster prefixed control messages received gauge value after the decay and true if the record exists, 0 and false otherwise. // No errors are expected during normal operation. -func (r *RecordCache) Get(nodeID flow.Identifier) (float64, bool, error) { - if r.Init(nodeID) { - return 0, true, nil - } - +func (r *RecordCache) GetWithInit(nodeID flow.Identifier) (float64, bool, error) { var err error - adjustedEntity, adjusted := r.c.Adjust(nodeID, func(entity flow.Entity) flow.Entity { + adjustLogic := func(entity flow.Entity) flow.Entity { // perform decay on gauge value entity, err = r.decayAdjustment(entity) return entity + } + adjustedEntity, adjusted := r.c.AdjustWithInit(nodeID, adjustLogic, func() flow.Entity { + return r.recordEntityFactory(nodeID) }) if err != nil { return 0, false, fmt.Errorf("unexpected error while applying decay adjustment for node %s: %w", nodeID, err) } if !adjusted { - return 0, false, fmt.Errorf("unexpected error record not found for node ID %s, even after an init attempt", nodeID) + return 0, false, fmt.Errorf("decay adjustment failed for node %s", nodeID) } - record, ok := adjustedEntity.(ClusterPrefixedMessagesReceivedRecord) - if !ok { - // sanity check - // This should never happen, because the cache only contains ClusterPrefixedMessagesReceivedRecord entities. - panic(fmt.Sprintf("invalid entity type, expected ClusterPrefixedMessagesReceivedRecord type, got: %T", adjustedEntity)) - } + record := mustBeClusterPrefixedMessageReceivedRecordEntity(adjustedEntity) return record.Gauge, true, nil } @@ -225,3 +199,19 @@ func defaultDecayFunction(decay float64) decayFunc { return recordEntity, nil } } + +// mustBeClusterPrefixedMessageReceivedRecordEntity is a helper function for type assertion of the flow.Entity to ClusterPrefixedMessagesReceivedRecord. +// It panics if the type assertion fails. +// Args: +// - entity: the flow.Entity to be type asserted. +// Returns: +// - the ClusterPrefixedMessagesReceivedRecord entity. +func mustBeClusterPrefixedMessageReceivedRecordEntity(entity flow.Entity) ClusterPrefixedMessagesReceivedRecord { + c, ok := entity.(ClusterPrefixedMessagesReceivedRecord) + if !ok { + // sanity check + // This should never happen, because the cache only contains ClusterPrefixedMessagesReceivedRecord entities. + panic(fmt.Sprintf("invalid entity type, expected ClusterPrefixedMessagesReceivedRecord type, got: %T", entity)) + } + return c +} diff --git a/network/p2p/inspector/internal/cache/cache_test.go b/network/p2p/inspector/internal/cache/cache_test.go index 2be9d4f2517..d6f5ffad908 100644 --- a/network/p2p/inspector/internal/cache/cache_test.go +++ b/network/p2p/inspector/internal/cache/cache_test.go @@ -8,7 +8,6 @@ import ( "github.com/rs/zerolog" "github.com/stretchr/testify/require" - "go.uber.org/atomic" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" @@ -28,18 +27,14 @@ func TestRecordCache_Init(t *testing.T) { nodeID2 := unittest.IdentifierFixture() // test initializing a record for an node ID that doesn't exist in the cache - initialized := cache.Init(nodeID1) - require.True(t, initialized, "expected record to be initialized") - gauge, ok, err := cache.Get(nodeID1) + gauge, ok, err := cache.GetWithInit(nodeID1) require.NoError(t, err) require.True(t, ok, "expected record to exist") require.Zerof(t, gauge, "expected gauge to be 0") require.Equal(t, uint(1), cache.Size(), "expected cache to have one additional record") // test initializing a record for an node ID that already exists in the cache - initialized = cache.Init(nodeID1) - require.False(t, initialized, "expected record not to be initialized") - gaugeAgain, ok, err := cache.Get(nodeID1) + gaugeAgain, ok, err := cache.GetWithInit(nodeID1) require.NoError(t, err) require.True(t, ok, "expected record to still exist") require.Zerof(t, gaugeAgain, "expected same gauge to be 0") @@ -47,9 +42,7 @@ func TestRecordCache_Init(t *testing.T) { require.Equal(t, uint(1), cache.Size(), "expected cache to still have one additional record") // test initializing a record for another node ID - initialized = cache.Init(nodeID2) - require.True(t, initialized, "expected record to be initialized") - gauge2, ok, err := cache.Get(nodeID2) + gauge2, ok, err := cache.GetWithInit(nodeID2) require.NoError(t, err) require.True(t, ok, "expected record to exist") require.Zerof(t, gauge2, "expected second gauge to be 0") @@ -71,26 +64,21 @@ func TestRecordCache_ConcurrentInit(t *testing.T) { for _, nodeID := range nodeIDs { go func(id flow.Identifier) { defer wg.Done() - cache.Init(id) + gauge, found, err := cache.GetWithInit(id) + require.NoError(t, err) + require.True(t, found) + require.Zerof(t, gauge, "expected all gauge values to be initialized to 0") }(nodeID) } unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") - - // ensure that all records are correctly initialized - for _, nodeID := range nodeIDs { - gauge, found, err := cache.Get(nodeID) - require.NoError(t, err) - require.True(t, found) - require.Zerof(t, gauge, "expected all gauge values to be initialized to 0") - } } // TestRecordCache_ConcurrentSameRecordInit tests the concurrent initialization of the same record. // The test covers the following scenarios: // 1. Multiple goroutines attempting to initialize the same record concurrently. // 2. Only one goroutine successfully initializes the record, and others receive false on initialization. -// 3. The record is correctly initialized in the cache and can be retrieved using the Get method. +// 3. The record is correctly initialized in the cache and can be retrieved using the GetWithInit method. func TestRecordCache_ConcurrentSameRecordInit(t *testing.T) { cache := cacheFixture(t, 100, defaultDecay, zerolog.Nop(), metrics.NewNoopCollector()) @@ -100,28 +88,20 @@ func TestRecordCache_ConcurrentSameRecordInit(t *testing.T) { var wg sync.WaitGroup wg.Add(concurrentAttempts) - successGauge := atomic.Int32{} - for i := 0; i < concurrentAttempts; i++ { go func() { defer wg.Done() - initSuccess := cache.Init(nodeID) - if initSuccess { - successGauge.Inc() - } + gauge, found, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, found) + require.Zero(t, gauge) }() } unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") // ensure that only one goroutine successfully initialized the record - require.Equal(t, int32(1), successGauge.Load()) - - // ensure that the record is correctly initialized in the cache - gauge, found, err := cache.Get(nodeID) - require.NoError(t, err) - require.True(t, found) - require.Zero(t, gauge) + require.Equal(t, uint(1), cache.Size()) } // TestRecordCache_ReceivedClusterPrefixedMessage tests the ReceivedClusterPrefixedMessage method of the RecordCache. @@ -135,22 +115,22 @@ func TestRecordCache_ReceivedClusterPrefixedMessage(t *testing.T) { nodeID1 := unittest.IdentifierFixture() nodeID2 := unittest.IdentifierFixture() - // initialize spam records for nodeID1 and nodeID2 - require.True(t, cache.Init(nodeID1)) - require.True(t, cache.Init(nodeID2)) - gauge, err := cache.ReceivedClusterPrefixedMessage(nodeID1) require.NoError(t, err) require.Equal(t, float64(1), gauge) // get will apply a slightl decay resulting // in a gauge value less than gauge which is 1 but greater than 0.9 - currentGauge, ok, err := cache.Get(nodeID1) + currentGauge, ok, err := cache.GetWithInit(nodeID1) require.NoError(t, err) require.True(t, ok) require.LessOrEqual(t, currentGauge, gauge) require.Greater(t, currentGauge, 0.9) + _, ok, err = cache.GetWithInit(nodeID2) + require.NoError(t, err) + require.True(t, ok) + // test adjusting the spam record for a non-existing node ID nodeID3 := unittest.IdentifierFixture() gauge3, err := cache.ReceivedClusterPrefixedMessage(nodeID3) @@ -172,11 +152,10 @@ func TestRecordCache_Decay(t *testing.T) { nodeID1 := unittest.IdentifierFixture() // initialize spam records for nodeID1 and nodeID2 - require.True(t, cache.Init(nodeID1)) gauge, err := cache.ReceivedClusterPrefixedMessage(nodeID1) require.Equal(t, float64(1), gauge) require.NoError(t, err) - gauge, ok, err := cache.Get(nodeID1) + gauge, ok, err := cache.GetWithInit(nodeID1) require.True(t, ok) require.NoError(t, err) // gauge should have been delayed slightly @@ -184,7 +163,7 @@ func TestRecordCache_Decay(t *testing.T) { time.Sleep(time.Second) - gauge, ok, err = cache.Get(nodeID1) + gauge, ok, err = cache.GetWithInit(nodeID1) require.True(t, ok) require.NoError(t, err) // gauge should have been delayed slightly, but closer to 0 @@ -203,9 +182,15 @@ func TestRecordCache_Identities(t *testing.T) { nodeID2 := unittest.IdentifierFixture() nodeID3 := unittest.IdentifierFixture() - require.True(t, cache.Init(nodeID1)) - require.True(t, cache.Init(nodeID2)) - require.True(t, cache.Init(nodeID3)) + _, ok, err := cache.GetWithInit(nodeID1) + require.NoError(t, err) + require.True(t, ok) + _, ok, err = cache.GetWithInit(nodeID2) + require.NoError(t, err) + require.True(t, ok) + _, ok, err = cache.GetWithInit(nodeID3) + require.NoError(t, err) + require.True(t, ok) // check if the NodeIDs method returns the correct set of node IDs identities := cache.NodeIDs() @@ -227,9 +212,15 @@ func TestRecordCache_Remove(t *testing.T) { nodeID2 := unittest.IdentifierFixture() nodeID3 := unittest.IdentifierFixture() - require.True(t, cache.Init(nodeID1)) - require.True(t, cache.Init(nodeID2)) - require.True(t, cache.Init(nodeID3)) + _, ok, err := cache.GetWithInit(nodeID1) + require.NoError(t, err) + require.True(t, ok) + _, ok, err = cache.GetWithInit(nodeID2) + require.NoError(t, err) + require.True(t, ok) + _, ok, err = cache.GetWithInit(nodeID3) + require.NoError(t, err) + require.True(t, ok) numOfIds := uint(3) require.Equal(t, numOfIds, cache.Size(), fmt.Sprintf("expected size of the cache to be %d", numOfIds)) @@ -238,10 +229,10 @@ func TestRecordCache_Remove(t *testing.T) { require.NotContains(t, nodeID1, cache.NodeIDs()) // check if the other node IDs are still in the cache - _, exists, err := cache.Get(nodeID2) + _, exists, err := cache.GetWithInit(nodeID2) require.NoError(t, err) require.True(t, exists) - _, exists, err = cache.Get(nodeID3) + _, exists, err = cache.GetWithInit(nodeID3) require.NoError(t, err) require.True(t, exists) @@ -259,7 +250,9 @@ func TestRecordCache_ConcurrentRemove(t *testing.T) { nodeIDs := unittest.IdentifierListFixture(10) for _, nodeID := range nodeIDs { - cache.Init(nodeID) + _, ok, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, ok) } var wg sync.WaitGroup @@ -289,7 +282,9 @@ func TestRecordCache_ConcurrentUpdatesAndReads(t *testing.T) { nodeIDs := unittest.IdentifierListFixture(10) for _, nodeID := range nodeIDs { - cache.Init(nodeID) + _, ok, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, ok) } var wg sync.WaitGroup @@ -306,7 +301,7 @@ func TestRecordCache_ConcurrentUpdatesAndReads(t *testing.T) { // get spam records concurrently go func(id flow.Identifier) { defer wg.Done() - _, found, err := cache.Get(id) + _, found, err := cache.GetWithInit(id) require.NoError(t, err) require.True(t, found) }(nodeID) @@ -316,7 +311,7 @@ func TestRecordCache_ConcurrentUpdatesAndReads(t *testing.T) { // ensure that the records are correctly updated in the cache for _, nodeID := range nodeIDs { - gauge, found, err := cache.Get(nodeID) + gauge, found, err := cache.GetWithInit(nodeID) require.NoError(t, err) require.True(t, found) // slight decay will result in 0.9 < gauge < 1 @@ -339,7 +334,9 @@ func TestRecordCache_ConcurrentInitAndRemove(t *testing.T) { nodeIDsToRemove := nodeIDs[10:] for _, nodeID := range nodeIDsToRemove { - cache.Init(nodeID) + _, ok, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, ok) } var wg sync.WaitGroup @@ -349,7 +346,9 @@ func TestRecordCache_ConcurrentInitAndRemove(t *testing.T) { for _, nodeID := range nodeIDsToAdd { go func(id flow.Identifier) { defer wg.Done() - cache.Init(id) + _, ok, err := cache.GetWithInit(id) + require.NoError(t, err) + require.True(t, ok) }(nodeID) } @@ -383,7 +382,9 @@ func TestRecordCache_ConcurrentInitRemoveUpdate(t *testing.T) { nodeIDsToAdjust := nodeIDs[20:] for _, nodeID := range nodeIDsToRemove { - cache.Init(nodeID) + _, ok, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, ok) } var wg sync.WaitGroup @@ -393,7 +394,9 @@ func TestRecordCache_ConcurrentInitRemoveUpdate(t *testing.T) { for _, nodeID := range nodeIDsToAdd { go func(id flow.Identifier) { defer wg.Done() - cache.Init(id) + _, ok, err := cache.GetWithInit(id) + require.NoError(t, err) + require.True(t, ok) }(nodeID) } @@ -431,7 +434,9 @@ func TestRecordCache_EdgeCasesAndInvalidInputs(t *testing.T) { nodeIDsToRemove := nodeIDs[10:20] for _, nodeID := range nodeIDsToRemove { - cache.Init(nodeID) + _, ok, err := cache.GetWithInit(nodeID) + require.NoError(t, err) + require.True(t, ok) } var wg sync.WaitGroup @@ -441,8 +446,7 @@ func TestRecordCache_EdgeCasesAndInvalidInputs(t *testing.T) { for _, nodeID := range nodeIDsToAdd { go func(id flow.Identifier) { defer wg.Done() - require.True(t, cache.Init(id)) - retrieved, ok, err := cache.Get(id) + retrieved, ok, err := cache.GetWithInit(id) require.NoError(t, err) require.True(t, ok) require.Zero(t, retrieved) diff --git a/network/p2p/inspector/internal/cache/cluster_prefixed_received_tracker.go b/network/p2p/inspector/internal/cache/cluster_prefixed_received_tracker.go index b112b7d7a7c..99e40884e1f 100644 --- a/network/p2p/inspector/internal/cache/cluster_prefixed_received_tracker.go +++ b/network/p2p/inspector/internal/cache/cluster_prefixed_received_tracker.go @@ -18,7 +18,8 @@ type ClusterPrefixedMessagesReceivedTracker struct { } // NewClusterPrefixedMessagesReceivedTracker returns a new *ClusterPrefixedMessagesReceivedTracker. -func NewClusterPrefixedMessagesReceivedTracker(logger zerolog.Logger, sizeLimit uint32, clusterPrefixedCacheCollector module.HeroCacheMetrics, decay float64) (*ClusterPrefixedMessagesReceivedTracker, error) { +func NewClusterPrefixedMessagesReceivedTracker(logger zerolog.Logger, sizeLimit uint32, clusterPrefixedCacheCollector module.HeroCacheMetrics, decay float64) (*ClusterPrefixedMessagesReceivedTracker, + error) { config := &RecordCacheConfig{ sizeLimit: sizeLimit, logger: logger, @@ -45,7 +46,7 @@ func (c *ClusterPrefixedMessagesReceivedTracker) Inc(nodeID flow.Identifier) (fl // Load loads the current number of cluster prefixed control messages received by a peer. // All errors returned from this func are unexpected and irrecoverable. func (c *ClusterPrefixedMessagesReceivedTracker) Load(nodeID flow.Identifier) (float64, error) { - count, _, err := c.cache.Get(nodeID) + count, _, err := c.cache.GetWithInit(nodeID) if err != nil { return 0, fmt.Errorf("failed to get cluster prefixed received tracker gauge value for peer %s: %w", nodeID, err) } diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index afcd0e7cce5..5556edf685d 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -191,8 +191,10 @@ func (c *ControlMsgValidationInspector) ActiveClustersChanged(clusterIDList flow // Returns: // - error: if a new inspect rpc request cannot be created, all errors returned are considered irrecoverable. func (c *ControlMsgValidationInspector) Inspect(from peer.ID, rpc *pubsub.RPC) error { + // first truncate the rpc to the configured max sample size; if needed c.truncateRPC(from, rpc) - // queue further async inspection + + // second, queue further async inspection req, err := NewInspectRPCRequest(from, rpc) if err != nil { c.logger.Error(). @@ -203,9 +205,39 @@ func (c *ControlMsgValidationInspector) Inspect(from peer.ID, rpc *pubsub.RPC) e return fmt.Errorf("failed to get inspect RPC request: %w", err) } c.workerPool.Submit(req) + return nil } +// updateMetrics updates the metrics for the received RPC. +// Args: +// - from: the sender. +// +// - rpc: the control message RPC. +func (c *ControlMsgValidationInspector) updateMetrics(from peer.ID, rpc *pubsub.RPC) { + includedMessages := len(rpc.GetPublish()) + iHaveCount, iWantCount, graftCount, pruneCount := 0, 0, 0, 0 + ctl := rpc.GetControl() + if ctl != nil { + iHaveCount = len(ctl.GetIhave()) + iWantCount = len(ctl.GetIwant()) + graftCount = len(ctl.GetGraft()) + pruneCount = len(ctl.GetPrune()) + } + c.metrics.OnIncomingRpcReceived(iHaveCount, iWantCount, graftCount, pruneCount, includedMessages) + if c.logger.GetLevel() > zerolog.TraceLevel { + return // skip logging if trace level is not enabled + } + c.logger.Trace(). + Str("peer_id", p2plogging.PeerId(from)). + Int("iHaveCount", iHaveCount). + Int("iWantCount", iWantCount). + Int("graftCount", graftCount). + Int("pruneCount", pruneCount). + Int("included_message_count", includedMessages). + Msg("received rpc with control messages") +} + // processInspectRPCReq func used by component workers to perform further inspection of RPC control messages that will validate ensure all control message // types are valid in the RPC. // Args: @@ -214,6 +246,7 @@ func (c *ControlMsgValidationInspector) Inspect(from peer.ID, rpc *pubsub.RPC) e // Returns: // - error: no error is expected to be returned from this func as they are logged and distributed in invalid control message notifications. func (c *ControlMsgValidationInspector) processInspectRPCReq(req *InspectRPCRequest) error { + c.updateMetrics(req.Peer, req.rpc) c.metrics.AsyncProcessingStarted() start := time.Now() defer func() { @@ -546,18 +579,18 @@ func (c *ControlMsgValidationInspector) truncateRPC(from peer.ID, rpc *pubsub.RP // - rpc: the rpc message to truncate. func (c *ControlMsgValidationInspector) truncateGraftMessages(rpc *pubsub.RPC) { grafts := rpc.GetControl().GetGraft() - totalGrafts := len(grafts) - if totalGrafts == 0 { - return + originalGraftSize := len(grafts) + if originalGraftSize <= c.config.GraftPruneMessageMaxSampleSize { + return // nothing to truncate } + + // truncate grafts and update metrics sampleSize := c.config.GraftPruneMessageMaxSampleSize - if sampleSize > totalGrafts { - sampleSize = totalGrafts - } - c.performSample(p2pmsg.CtrlMsgGraft, uint(totalGrafts), uint(sampleSize), func(i, j uint) { + c.performSample(p2pmsg.CtrlMsgGraft, uint(originalGraftSize), uint(sampleSize), func(i, j uint) { grafts[i], grafts[j] = grafts[j], grafts[i] }) rpc.Control.Graft = grafts[:sampleSize] + c.metrics.OnControlMessagesTruncated(p2pmsg.CtrlMsgGraft, originalGraftSize-len(rpc.Control.Graft)) } // truncatePruneMessages truncates the Prune control messages in the RPC. If the total number of Prunes in the RPC exceeds the configured @@ -566,18 +599,17 @@ func (c *ControlMsgValidationInspector) truncateGraftMessages(rpc *pubsub.RPC) { // - rpc: the rpc message to truncate. func (c *ControlMsgValidationInspector) truncatePruneMessages(rpc *pubsub.RPC) { prunes := rpc.GetControl().GetPrune() - totalPrunes := len(prunes) - if totalPrunes == 0 { - return + originalPruneSize := len(prunes) + if originalPruneSize <= c.config.GraftPruneMessageMaxSampleSize { + return // nothing to truncate } + sampleSize := c.config.GraftPruneMessageMaxSampleSize - if sampleSize > totalPrunes { - sampleSize = totalPrunes - } - c.performSample(p2pmsg.CtrlMsgPrune, uint(totalPrunes), uint(sampleSize), func(i, j uint) { + c.performSample(p2pmsg.CtrlMsgPrune, uint(originalPruneSize), uint(sampleSize), func(i, j uint) { prunes[i], prunes[j] = prunes[j], prunes[i] }) rpc.Control.Prune = prunes[:sampleSize] + c.metrics.OnControlMessagesTruncated(p2pmsg.CtrlMsgPrune, originalPruneSize-len(rpc.Control.Prune)) } // truncateIHaveMessages truncates the iHaves control messages in the RPC. If the total number of iHaves in the RPC exceeds the configured @@ -586,19 +618,23 @@ func (c *ControlMsgValidationInspector) truncatePruneMessages(rpc *pubsub.RPC) { // - rpc: the rpc message to truncate. func (c *ControlMsgValidationInspector) truncateIHaveMessages(rpc *pubsub.RPC) { ihaves := rpc.GetControl().GetIhave() - totalIHaves := len(ihaves) - if totalIHaves == 0 { + originalIHaveCount := len(ihaves) + if originalIHaveCount == 0 { return } - sampleSize := c.config.IHave.MaxSampleSize - if sampleSize > totalIHaves { - sampleSize = totalIHaves - } - c.performSample(p2pmsg.CtrlMsgIHave, uint(totalIHaves), uint(sampleSize), func(i, j uint) { - ihaves[i], ihaves[j] = ihaves[j], ihaves[i] - }) - rpc.Control.Ihave = ihaves[:sampleSize] + if originalIHaveCount > c.config.IHave.MaxSampleSize { + // truncate ihaves and update metrics + sampleSize := c.config.IHave.MaxSampleSize + if sampleSize > originalIHaveCount { + sampleSize = originalIHaveCount + } + c.performSample(p2pmsg.CtrlMsgIHave, uint(originalIHaveCount), uint(sampleSize), func(i, j uint) { + ihaves[i], ihaves[j] = ihaves[j], ihaves[i] + }) + rpc.Control.Ihave = ihaves[:sampleSize] + c.metrics.OnControlMessagesTruncated(p2pmsg.CtrlMsgIHave, originalIHaveCount-len(rpc.Control.Ihave)) + } c.truncateIHaveMessageIds(rpc) } @@ -609,18 +645,23 @@ func (c *ControlMsgValidationInspector) truncateIHaveMessages(rpc *pubsub.RPC) { func (c *ControlMsgValidationInspector) truncateIHaveMessageIds(rpc *pubsub.RPC) { for _, ihave := range rpc.GetControl().GetIhave() { messageIDs := ihave.GetMessageIDs() - totalMessageIDs := len(messageIDs) - if totalMessageIDs == 0 { - return + originalMessageIdCount := len(messageIDs) + if originalMessageIdCount == 0 { + continue // nothing to truncate; skip } - sampleSize := c.config.IHave.MaxMessageIDSampleSize - if sampleSize > totalMessageIDs { - sampleSize = totalMessageIDs + + if originalMessageIdCount > c.config.IHave.MaxMessageIDSampleSize { + sampleSize := c.config.IHave.MaxMessageIDSampleSize + if sampleSize > originalMessageIdCount { + sampleSize = originalMessageIdCount + } + c.performSample(p2pmsg.CtrlMsgIHave, uint(originalMessageIdCount), uint(sampleSize), func(i, j uint) { + messageIDs[i], messageIDs[j] = messageIDs[j], messageIDs[i] + }) + ihave.MessageIDs = messageIDs[:sampleSize] + c.metrics.OnIHaveControlMessageIdsTruncated(originalMessageIdCount - len(ihave.MessageIDs)) } - c.performSample(p2pmsg.CtrlMsgIHave, uint(totalMessageIDs), uint(sampleSize), func(i, j uint) { - messageIDs[i], messageIDs[j] = messageIDs[j], messageIDs[i] - }) - ihave.MessageIDs = messageIDs[:sampleSize] + c.metrics.OnIHaveMessageIDsReceived(ihave.GetTopicID(), len(ihave.MessageIDs)) } } @@ -630,18 +671,23 @@ func (c *ControlMsgValidationInspector) truncateIHaveMessageIds(rpc *pubsub.RPC) // - rpc: the rpc message to truncate. func (c *ControlMsgValidationInspector) truncateIWantMessages(from peer.ID, rpc *pubsub.RPC) { iWants := rpc.GetControl().GetIwant() - totalIWants := uint(len(iWants)) - if totalIWants == 0 { + originalIWantCount := uint(len(iWants)) + if originalIWantCount == 0 { return } - sampleSize := c.config.IWant.MaxSampleSize - if sampleSize > totalIWants { - sampleSize = totalIWants + + if originalIWantCount > c.config.IWant.MaxSampleSize { + // truncate iWants and update metrics + sampleSize := c.config.IWant.MaxSampleSize + if sampleSize > originalIWantCount { + sampleSize = originalIWantCount + } + c.performSample(p2pmsg.CtrlMsgIWant, originalIWantCount, sampleSize, func(i, j uint) { + iWants[i], iWants[j] = iWants[j], iWants[i] + }) + rpc.Control.Iwant = iWants[:sampleSize] + c.metrics.OnControlMessagesTruncated(p2pmsg.CtrlMsgIWant, int(originalIWantCount)-len(rpc.Control.Iwant)) } - c.performSample(p2pmsg.CtrlMsgIWant, totalIWants, sampleSize, func(i, j uint) { - iWants[i], iWants[j] = iWants[j], iWants[i] - }) - rpc.Control.Iwant = iWants[:sampleSize] c.truncateIWantMessageIds(from, rpc) } @@ -665,17 +711,19 @@ func (c *ControlMsgValidationInspector) truncateIWantMessageIds(from peer.ID, rp } for _, iWant := range rpc.GetControl().GetIwant() { messageIDs := iWant.GetMessageIDs() - totalMessageIDs := len(messageIDs) - if totalMessageIDs == 0 { - return + totalMessageIdCount := len(messageIDs) + if totalMessageIdCount == 0 { + continue // nothing to truncate; skip } - if sampleSize > totalMessageIDs { - sampleSize = totalMessageIDs + + if totalMessageIdCount > sampleSize { + c.performSample(p2pmsg.CtrlMsgIWant, uint(totalMessageIdCount), uint(sampleSize), func(i, j uint) { + messageIDs[i], messageIDs[j] = messageIDs[j], messageIDs[i] + }) + iWant.MessageIDs = messageIDs[:sampleSize] + c.metrics.OnIWantControlMessageIdsTruncated(totalMessageIdCount - len(iWant.MessageIDs)) } - c.performSample(p2pmsg.CtrlMsgIWant, uint(totalMessageIDs), uint(sampleSize), func(i, j uint) { - messageIDs[i], messageIDs[j] = messageIDs[j], messageIDs[i] - }) - iWant.MessageIDs = messageIDs[:sampleSize] + c.metrics.OnIWantMessageIDsReceived(len(iWant.MessageIDs)) } } diff --git a/network/p2p/mock/gossip_sub_application_specific_score_cache.go b/network/p2p/mock/gossip_sub_application_specific_score_cache.go index 68260079f1a..d4eb9c6cc32 100644 --- a/network/p2p/mock/gossip_sub_application_specific_score_cache.go +++ b/network/p2p/mock/gossip_sub_application_specific_score_cache.go @@ -15,8 +15,8 @@ type GossipSubApplicationSpecificScoreCache struct { mock.Mock } -// Add provides a mock function with given fields: peerID, score, _a2 -func (_m *GossipSubApplicationSpecificScoreCache) Add(peerID peer.ID, score float64, _a2 time.Time) error { +// AdjustWithInit provides a mock function with given fields: peerID, score, _a2 +func (_m *GossipSubApplicationSpecificScoreCache) AdjustWithInit(peerID peer.ID, score float64, _a2 time.Time) error { ret := _m.Called(peerID, score, _a2) var r0 error diff --git a/network/p2p/mock/gossip_sub_control_metrics_observer.go b/network/p2p/mock/gossip_sub_control_metrics_observer.go deleted file mode 100644 index 333bb990c6c..00000000000 --- a/network/p2p/mock/gossip_sub_control_metrics_observer.go +++ /dev/null @@ -1,36 +0,0 @@ -// Code generated by mockery v2.21.4. DO NOT EDIT. - -package mockp2p - -import ( - mock "github.com/stretchr/testify/mock" - - peer "github.com/libp2p/go-libp2p/core/peer" - - pubsub "github.com/libp2p/go-libp2p-pubsub" -) - -// GossipSubControlMetricsObserver is an autogenerated mock type for the GossipSubControlMetricsObserver type -type GossipSubControlMetricsObserver struct { - mock.Mock -} - -// ObserveRPC provides a mock function with given fields: _a0, _a1 -func (_m *GossipSubControlMetricsObserver) ObserveRPC(_a0 peer.ID, _a1 *pubsub.RPC) { - _m.Called(_a0, _a1) -} - -type mockConstructorTestingTNewGossipSubControlMetricsObserver interface { - mock.TestingT - Cleanup(func()) -} - -// NewGossipSubControlMetricsObserver creates a new instance of GossipSubControlMetricsObserver. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubControlMetricsObserver(t mockConstructorTestingTNewGossipSubControlMetricsObserver) *GossipSubControlMetricsObserver { - mock := &GossipSubControlMetricsObserver{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/gossip_sub_spam_record_cache.go b/network/p2p/mock/gossip_sub_spam_record_cache.go index 35e674fdffb..2fe7a692f89 100644 --- a/network/p2p/mock/gossip_sub_spam_record_cache.go +++ b/network/p2p/mock/gossip_sub_spam_record_cache.go @@ -14,18 +14,30 @@ type GossipSubSpamRecordCache struct { mock.Mock } -// Add provides a mock function with given fields: peerId, record -func (_m *GossipSubSpamRecordCache) Add(peerId peer.ID, record p2p.GossipSubSpamRecord) bool { - ret := _m.Called(peerId, record) +// Adjust provides a mock function with given fields: peerID, updateFunc +func (_m *GossipSubSpamRecordCache) Adjust(peerID peer.ID, updateFunc p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error) { + ret := _m.Called(peerID, updateFunc) - var r0 bool - if rf, ok := ret.Get(0).(func(peer.ID, p2p.GossipSubSpamRecord) bool); ok { - r0 = rf(peerId, record) + var r0 *p2p.GossipSubSpamRecord + var r1 error + if rf, ok := ret.Get(0).(func(peer.ID, p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error)); ok { + return rf(peerID, updateFunc) + } + if rf, ok := ret.Get(0).(func(peer.ID, p2p.UpdateFunction) *p2p.GossipSubSpamRecord); ok { + r0 = rf(peerID, updateFunc) } else { - r0 = ret.Get(0).(bool) + if ret.Get(0) != nil { + r0 = ret.Get(0).(*p2p.GossipSubSpamRecord) + } } - return r0 + if rf, ok := ret.Get(1).(func(peer.ID, p2p.UpdateFunction) error); ok { + r1 = rf(peerID, updateFunc) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // Get provides a mock function with given fields: peerID @@ -75,32 +87,6 @@ func (_m *GossipSubSpamRecordCache) Has(peerID peer.ID) bool { return r0 } -// Update provides a mock function with given fields: peerID, updateFunc -func (_m *GossipSubSpamRecordCache) Update(peerID peer.ID, updateFunc p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error) { - ret := _m.Called(peerID, updateFunc) - - var r0 *p2p.GossipSubSpamRecord - var r1 error - if rf, ok := ret.Get(0).(func(peer.ID, p2p.UpdateFunction) (*p2p.GossipSubSpamRecord, error)); ok { - return rf(peerID, updateFunc) - } - if rf, ok := ret.Get(0).(func(peer.ID, p2p.UpdateFunction) *p2p.GossipSubSpamRecord); ok { - r0 = rf(peerID, updateFunc) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*p2p.GossipSubSpamRecord) - } - } - - if rf, ok := ret.Get(1).(func(peer.ID, p2p.UpdateFunction) error); ok { - r1 = rf(peerID, updateFunc) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - type mockConstructorTestingTNewGossipSubSpamRecordCache interface { mock.TestingT Cleanup(func()) diff --git a/network/p2p/node/gossipsubMetrics.go b/network/p2p/node/gossipsubMetrics.go deleted file mode 100644 index 57dd6f065d1..00000000000 --- a/network/p2p/node/gossipsubMetrics.go +++ /dev/null @@ -1,58 +0,0 @@ -package p2pnode - -import ( - pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network/p2p" - p2plogging "github.com/onflow/flow-go/network/p2p/logging" -) - -// GossipSubControlMessageMetrics is a metrics and observability wrapper component for the incoming RPCs to a -// GossipSub router. It records metrics on the number of control messages received in each RPC. -type GossipSubControlMessageMetrics struct { - metrics module.GossipSubRpcInspectorMetrics - logger zerolog.Logger -} - -var _ p2p.GossipSubControlMetricsObserver = (*GossipSubControlMessageMetrics)(nil) - -func NewGossipSubControlMessageMetrics(metrics module.GossipSubRpcInspectorMetrics, logger zerolog.Logger) *GossipSubControlMessageMetrics { - return &GossipSubControlMessageMetrics{ - logger: logger.With().Str("module", "gossipsub-control-message-metrics").Logger(), - metrics: metrics, - } -} - -// ObserveRPC is invoked to record metrics on incoming RPC messages. -func (o *GossipSubControlMessageMetrics) ObserveRPC(from peer.ID, rpc *pubsub.RPC) { - lg := o.logger.With().Str("peer_id", p2plogging.PeerId(from)).Logger() - includedMessages := len(rpc.GetPublish()) - - ctl := rpc.GetControl() - if ctl == nil && includedMessages == 0 { - lg.Trace().Msg("received rpc with no control message and no publish messages") - return - } - - iHaveCount := len(ctl.GetIhave()) - iWantCount := len(ctl.GetIwant()) - graftCount := len(ctl.GetGraft()) - pruneCount := len(ctl.GetPrune()) - - lg.Trace(). - Int("iHaveCount", iHaveCount). - Int("iWantCount", iWantCount). - Int("graftCount", graftCount). - Int("pruneCount", pruneCount). - Int("included_message_count", includedMessages). - Msg("received rpc with control messages") - - o.metrics.OnIHaveReceived(iHaveCount) - o.metrics.OnIWantReceived(iWantCount) - o.metrics.OnGraftReceived(graftCount) - o.metrics.OnPruneReceived(pruneCount) - o.metrics.OnPublishedGossipMessagesReceived(includedMessages) -} diff --git a/network/p2p/node/internal/cache.go b/network/p2p/node/internal/cache.go index 6d8952e6628..e2a49ccca51 100644 --- a/network/p2p/node/internal/cache.go +++ b/network/p2p/node/internal/cache.go @@ -34,11 +34,7 @@ type DisallowListCache struct { func NewDisallowListCache(sizeLimit uint32, logger zerolog.Logger, collector module.HeroCacheMetrics) *DisallowListCache { backData := herocache.NewCache(sizeLimit, herocache.DefaultOversizeFactor, - // this cache is supposed to keep the disallow-list causes for the authorized (staked) nodes. Since the number of such nodes is - // expected to be small, we do not eject any records from the cache. The cache size must be large enough to hold all - // the spam records of the authorized nodes. Also, this cache is keeping at most one record per peer id, so the - // size of the cache must be at least the number of authorized nodes. - heropool.NoEjection, + heropool.LRUEjection, logger.With().Str("mempool", "disallow-list-records").Logger(), collector) @@ -72,20 +68,6 @@ func (d *DisallowListCache) IsDisallowListed(peerID peer.ID) ([]network.Disallow return causes, true } -// init initializes the disallow-list cache entity for the peerID. -// Args: -// - peerID: the peerID of the peer to be disallow-listed. -// Returns: -// - bool: true if the entity is successfully added to the cache. -// false if the entity already exists in the cache. -func (d *DisallowListCache) init(peerID peer.ID) bool { - return d.c.Add(&disallowListCacheEntity{ - peerID: peerID, - causes: make(map[network.DisallowListedCause]struct{}), - id: makeId(peerID), - }) -} - // DisallowFor disallow-lists a peer for a cause. // Args: // - peerID: the peerID of the peer to be disallow-listed. @@ -94,51 +76,26 @@ func (d *DisallowListCache) init(peerID peer.ID) bool { // - []network.DisallowListedCause: the list of causes for which the peer is disallow-listed. // - error: if the operation fails, error is irrecoverable. func (d *DisallowListCache) DisallowFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) { - // first, we try to optimistically add the peer to the disallow list. - causes, err := d.disallowListFor(peerID, cause) - switch { - case err == nil: - return causes, nil - case err == ErrDisallowCacheEntityNotFound: - // if the entity not exist, we initialize it and try again. - // Note: there is an edge case where the entity is initialized by another goroutine between the two calls. - // In this case, the init function is invoked twice, but it is not a problem because the underlying - // cache is thread-safe. Hence, we do not need to synchronize the two calls. In such cases, one of the - // two calls returns false, and the other call returns true. We do not care which call returns false, hence, - // we ignore the return value of the init function. - _ = d.init(peerID) - causes, err = d.disallowListFor(peerID, cause) - if err != nil { - // any error after the init is irrecoverable. - return nil, fmt.Errorf("failed to disallow list peer %s for cause %s: %w", peerID, cause, err) + entityId := makeId(peerID) + initLogic := func() flow.Entity { + return &disallowListCacheEntity{ + peerID: peerID, + causes: make(map[network.DisallowListedCause]struct{}), + entityId: entityId, } - return causes, nil - default: - return nil, fmt.Errorf("failed to disallow list peer %s for cause %s: %w", peerID, cause, err) } -} - -// disallowListFor is a helper function for disallowing a peer for a cause. -// It adds the cause to the disallow list cache entity for the peerID and returns the updated list of causes for the peer. -// Args: -// - peerID: the peerID of the peer to be disallow-listed. -// - cause: the cause for disallow-listing the peer. -// Returns: -// - the updated list of causes for the peer. -// - error if the entity for the peerID is not found in the cache it returns ErrDisallowCacheEntityNotFound, which is a benign error. -func (d *DisallowListCache) disallowListFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) { - adjustedEntity, adjusted := d.c.Adjust(makeId(peerID), func(entity flow.Entity) flow.Entity { + adjustLogic := func(entity flow.Entity) flow.Entity { dEntity := mustBeDisallowListEntity(entity) dEntity.causes[cause] = struct{}{} return dEntity - }) - + } + adjustedEntity, adjusted := d.c.AdjustWithInit(entityId, adjustLogic, initLogic) if !adjusted { - // if the entity is not found in the cache, we return a benign error. - return nil, ErrDisallowCacheEntityNotFound + return nil, fmt.Errorf("failed to disallow list peer %s for cause %s", peerID, cause) } dEntity := mustBeDisallowListEntity(adjustedEntity) + // returning a deep copy of causes (to avoid being mutated externally). updatedCauses := make([]network.DisallowListedCause, 0, len(dEntity.causes)) for c := range dEntity.causes { updatedCauses = append(updatedCauses, c) diff --git a/network/p2p/node/internal/cacheEntity.go b/network/p2p/node/internal/cacheEntity.go index e55b0d127b5..119a5b42c1d 100644 --- a/network/p2p/node/internal/cacheEntity.go +++ b/network/p2p/node/internal/cacheEntity.go @@ -16,7 +16,7 @@ type disallowListCacheEntity struct { causes map[network.DisallowListedCause]struct{} // id is the hash of the peerID which is used as the key for storing the entity in the cache. // we cache it internally to avoid hashing the peerID multiple times. - id flow.Identifier + entityId flow.Identifier } var _ flow.Entity = (*disallowListCacheEntity)(nil) @@ -25,7 +25,7 @@ var _ flow.Entity = (*disallowListCacheEntity)(nil) // Returns: // - the hash of the peerID as a flow.Identifier. func (d *disallowListCacheEntity) ID() flow.Identifier { - return d.id + return d.entityId } // Checksum returns the hash of the peerID, there is no use for this method in the cache. It is implemented to satisfy @@ -33,7 +33,7 @@ func (d *disallowListCacheEntity) ID() flow.Identifier { // Returns: // - the hash of the peerID as a flow.Identifier. func (d *disallowListCacheEntity) Checksum() flow.Identifier { - return d.id + return d.entityId } // makeId is a helper function for creating the id field of the disallowListCacheEntity by hashing the peerID. diff --git a/network/p2p/pubsub.go b/network/p2p/pubsub.go index 02b4546553d..7b4833736bb 100644 --- a/network/p2p/pubsub.go +++ b/network/p2p/pubsub.go @@ -86,11 +86,6 @@ type PubSubAdapterConfig interface { WithInspectorSuite(GossipSubInspectorSuite) } -// GossipSubControlMetricsObserver funcs used to observe gossipsub related metrics. -type GossipSubControlMetricsObserver interface { - ObserveRPC(peer.ID, *pubsub.RPC) -} - // GossipSubRPCInspector app specific RPC inspector used to inspect and validate incoming RPC messages before they are processed by libp2p. // Implementations must: // - be concurrency safe diff --git a/network/p2p/scoring/app_score_test.go b/network/p2p/scoring/app_score_test.go index d693b77207c..e373cd6ac0e 100644 --- a/network/p2p/scoring/app_score_test.go +++ b/network/p2p/scoring/app_score_test.go @@ -18,7 +18,7 @@ import ( "github.com/onflow/flow-go/network/internal/p2pfixtures" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/scoring" + p2pconfig "github.com/onflow/flow-go/network/p2p/config" p2ptest "github.com/onflow/flow-go/network/p2p/test" flowpubsub "github.com/onflow/flow-go/network/validator/pubsub" "github.com/onflow/flow-go/utils/unittest" @@ -142,7 +142,7 @@ func TestFullGossipSubConnectivityAmongHonestNodesWithMaliciousMajority(t *testi p2ptest.WithRole(flow.RoleAccess), // overrides the default peer scoring parameters to mute GossipSub traffic from/to honest nodes. p2ptest.EnablePeerScoringWithOverride(&p2p.PeerScoringConfigOverride{ - AppSpecificScoreParams: maliciousAppSpecificScore(flow.IdentityList{&con1Id, &con2Id}), + AppSpecificScoreParams: maliciousAppSpecificScore(flow.IdentityList{&con1Id, &con2Id}, defaultConfig.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol), }), ) @@ -223,14 +223,14 @@ func TestFullGossipSubConnectivityAmongHonestNodesWithMaliciousMajority(t *testi // maliciousAppSpecificScore returns a malicious app specific penalty function that rewards the malicious node and // punishes the honest nodes. -func maliciousAppSpecificScore(honestIds flow.IdentityList) func(peer.ID) float64 { +func maliciousAppSpecificScore(honestIds flow.IdentityList, optionCfg p2pconfig.ProtocolLevelGossipSubScoreParams) func(peer.ID) float64 { honestIdProvider := id.NewFixedIdentityProvider(honestIds) return func(p peer.ID) float64 { _, isHonest := honestIdProvider.ByPeerID(p) if isHonest { - return scoring.MaxAppSpecificPenalty + return optionCfg.AppSpecificScore.MaxAppSpecificPenalty } - return scoring.MaxAppSpecificReward + return optionCfg.AppSpecificScore.MaxAppSpecificReward } } diff --git a/network/p2p/scoring/decay_test.go b/network/p2p/scoring/decay_test.go index 2a15b60b754..643d22fba83 100644 --- a/network/p2p/scoring/decay_test.go +++ b/network/p2p/scoring/decay_test.go @@ -287,7 +287,7 @@ func TestDefaultDecayFunction(t *testing.T) { record: p2p.GossipSubSpamRecord{ Penalty: -100, Decay: 0.8, - LastDecayAdjustment: time.Now().Add(-flowConfig.NetworkConfig.GossipSub.ScoringParameters.SpamRecordCache.PenaltyDecayEvaluationPeriod), + LastDecayAdjustment: time.Now().Add(-flowConfig.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.PenaltyDecayEvaluationPeriod), }, lastUpdated: time.Now(), }, @@ -299,8 +299,9 @@ func TestDefaultDecayFunction(t *testing.T) { }, }, } - scoringRegistryConfig := flowConfig.NetworkConfig.GossipSub.ScoringParameters.SpamRecordCache - decayFunc := scoring.DefaultDecayFunction(scoringRegistryConfig.PenaltyDecaySlowdownThreshold, scoringRegistryConfig.DecayRateReductionFactor, scoringRegistryConfig.PenaltyDecayEvaluationPeriod) + scoringRegistryConfig := flowConfig.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters + decayFunc := scoring.DefaultDecayFunction(scoringRegistryConfig.SpamRecordCache.Decay) + for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { got, err := decayFunc(tt.args.record, tt.args.lastUpdated) diff --git a/network/p2p/scoring/internal/appSpecificScoreCache.go b/network/p2p/scoring/internal/appSpecificScoreCache.go index 10e2957e03e..a818046b06c 100644 --- a/network/p2p/scoring/internal/appSpecificScoreCache.go +++ b/network/p2p/scoring/internal/appSpecificScoreCache.go @@ -53,14 +53,14 @@ func NewAppSpecificScoreCache(sizeLimit uint32, logger zerolog.Logger, collector // - time.Time: the time at which the score was last updated. // - bool: true if the score was retrieved successfully, false otherwise. func (a *AppSpecificScoreCache) Get(peerID peer.ID) (float64, time.Time, bool) { - e, ok := a.c.ByID(flow.MakeID(peerID)) + e, ok := a.c.ByID(entityIdOf(peerID)) if !ok { return 0, time.Time{}, false } return e.(appSpecificScoreRecordEntity).Score, e.(appSpecificScoreRecordEntity).LastUpdated, true } -// Add adds the application specific score of a peer to the cache. +// AdjustWithInit adds the application specific score of a peer to the cache. // If the peer already has a score in the cache, the score is updated. // Args: // - peerID: the peer ID of the peer in the GossipSub protocol. @@ -68,28 +68,27 @@ func (a *AppSpecificScoreCache) Get(peerID peer.ID) (float64, time.Time, bool) { // - time: the time at which the score was last updated. // Returns: // - error on failure to add the score. The returned error is irrecoverable and indicates an exception. -func (a *AppSpecificScoreCache) Add(peerID peer.ID, score float64, time time.Time) error { - entityId := flow.MakeID(peerID) +func (a *AppSpecificScoreCache) AdjustWithInit(peerID peer.ID, score float64, time time.Time) error { + entityId := entityIdOf(peerID) - // first tries an optimistic add; if it fails, it tries an optimistic update - added := a.c.Add(appSpecificScoreRecordEntity{ - entityId: entityId, - PeerID: peerID, - Score: score, - LastUpdated: time, - }) - if !added { - _, ok := a.c.Adjust(entityId, func(entity flow.Entity) flow.Entity { - r := entity.(appSpecificScoreRecordEntity) - r.Score = score - r.LastUpdated = time - return r - }) - - if !ok { - return fmt.Errorf("failed to add app specific score record for peer %s", peerID) + initLogic := func() flow.Entity { + return appSpecificScoreRecordEntity{ + entityId: entityId, + PeerID: peerID, + Score: score, + LastUpdated: time, } } + adjustLogic := func(entity flow.Entity) flow.Entity { + r := entity.(appSpecificScoreRecordEntity) + r.Score = score + r.LastUpdated = time + return r + } + _, adjusted := a.c.AdjustWithInit(entityId, adjustLogic, initLogic) + if !adjusted { + return fmt.Errorf("failed to adjust app specific score for peer %s", peerID) + } return nil } diff --git a/network/p2p/scoring/internal/appSpecificScoreCache_test.go b/network/p2p/scoring/internal/appSpecificScoreCache_test.go index 415218f3c30..bea5f355833 100644 --- a/network/p2p/scoring/internal/appSpecificScoreCache_test.go +++ b/network/p2p/scoring/internal/appSpecificScoreCache_test.go @@ -23,7 +23,7 @@ func TestAppSpecificScoreCache(t *testing.T) { score := 5.0 updateTime := time.Now() - err := cache.Add(peerID, score, updateTime) + err := cache.AdjustWithInit(peerID, score, updateTime) require.Nil(t, err, "failed to add score to cache") // retrieve score from cache @@ -34,7 +34,7 @@ func TestAppSpecificScoreCache(t *testing.T) { // test cache update newScore := 10.0 - err = cache.Add(peerID, newScore, updateTime.Add(time.Minute)) + err = cache.AdjustWithInit(peerID, newScore, updateTime.Add(time.Minute)) require.Nil(t, err, "Failed to update score in cache") // retrieve updated score @@ -62,13 +62,13 @@ func TestAppSpecificScoreCache_Concurrent_Add_Get_Update(t *testing.T) { wg.Add(2) go func() { defer wg.Done() - err := cache.Add(peerId1, score1, lastUpdated1) + err := cache.AdjustWithInit(peerId1, score1, lastUpdated1) require.Nil(t, err, "failed to add score1 to cache") }() go func() { defer wg.Done() - err := cache.Add(peerId2, score2, lastUpdated2) + err := cache.AdjustWithInit(peerId2, score2, lastUpdated2) require.Nil(t, err, "failed to add score2 to cache") }() @@ -103,13 +103,13 @@ func TestAppSpecificScoreCache_Concurrent_Add_Get_Update(t *testing.T) { wg.Add(2) go func() { defer wg.Done() - err := cache.Add(peerId1, newScore1, lastUpdated1) + err := cache.AdjustWithInit(peerId1, newScore1, lastUpdated1) require.Nil(t, err, "failed to update score1 in cache") }() go func() { defer wg.Done() - err := cache.Add(peerId2, newScore2, lastUpdated2) + err := cache.AdjustWithInit(peerId2, newScore2, lastUpdated2) require.Nil(t, err, "failed to update score2 in cache") }() @@ -149,7 +149,7 @@ func TestAppSpecificScoreCache_Eviction(t *testing.T) { // add scores to cache for i := 0; i < len(peerIds); i++ { - err := cache.Add(peerIds[i], scores[i], time.Now()) + err := cache.AdjustWithInit(peerIds[i], scores[i], time.Now()) require.Nil(t, err, "failed to add score to cache") } diff --git a/network/p2p/scoring/internal/subscriptionCache.go b/network/p2p/scoring/internal/subscriptionCache.go index 95acafdd422..8aed3878753 100644 --- a/network/p2p/scoring/internal/subscriptionCache.go +++ b/network/p2p/scoring/internal/subscriptionCache.go @@ -1,7 +1,6 @@ package internal import ( - "errors" "fmt" "github.com/libp2p/go-libp2p/core/peer" @@ -15,8 +14,6 @@ import ( "github.com/onflow/flow-go/module/mempool/stdmap" ) -var ErrTopicRecordNotFound = fmt.Errorf("topic record not found") - // SubscriptionRecordCache manages the subscription records of peers in a network. // It uses a currentCycle counter to track the update cycles of the cache, ensuring the relevance of subscription data. type SubscriptionRecordCache struct { @@ -61,7 +58,7 @@ func NewSubscriptionRecordCache(sizeLimit uint32, // - []string: the list of topics the peer is subscribed to. // - bool: true if there is a record for the peer, false otherwise. func (s *SubscriptionRecordCache) GetSubscribedTopics(pid peer.ID) ([]string, bool) { - e, ok := s.c.ByID(flow.MakeID(pid)) + e, ok := s.c.ByID(entityIdOf(pid)) if !ok { return nil, false } @@ -87,7 +84,7 @@ func (s *SubscriptionRecordCache) MoveToNextUpdateCycle() uint64 { return s.currentCycle.Load() } -// AddTopicForPeer appends a topic to the list of topics a peer is subscribed to. If the peer is not subscribed to any +// AddWithInitTopicForPeer appends a topic to the list of topics a peer is subscribed to. If the peer is not subscribed to any // topics yet, a new record is created. // If the last update cycle is older than the current cycle, the list of topics for the peer is first cleared, and then // the topic is added to the list. This is to ensure that the list of topics for a peer is always up to date. @@ -98,40 +95,18 @@ func (s *SubscriptionRecordCache) MoveToNextUpdateCycle() uint64 { // - []string: the list of topics the peer is subscribed to after the update. // - error: an error if the update failed; any returned error is an irrecoverable error and indicates a bug or misconfiguration. // Implementation must be thread-safe. -func (s *SubscriptionRecordCache) AddTopicForPeer(pid peer.ID, topic string) ([]string, error) { - // first, we try to optimistically adjust the record assuming that the record already exists. - entityId := flow.MakeID(pid) - topics, err := s.addTopicForPeer(entityId, topic) - - switch { - case errors.Is(err, ErrTopicRecordNotFound): - // if the record does not exist, we initialize the record and try to adjust it again. - // Note: there is an edge case where the record is initialized by another goroutine between the two calls. - // In this case, the init function is invoked twice, but it is not a problem because the underlying - // cache is thread-safe. Hence, we do not need to synchronize the two calls. In such cases, one of the - // two calls returns false, and the other call returns true. We do not care which call returns false, hence, - // we ignore the return value of the init function. - _ = s.c.Add(SubscriptionRecordEntity{ +func (s *SubscriptionRecordCache) AddWithInitTopicForPeer(pid peer.ID, topic string) ([]string, error) { + entityId := entityIdOf(pid) + initLogic := func() flow.Entity { + return SubscriptionRecordEntity{ entityId: entityId, PeerID: pid, Topics: make([]string, 0), LastUpdatedCycle: s.currentCycle.Load(), - }) - // as the record is initialized, the adjust attempt should not return an error, and any returned error - // is an irrecoverable error and indicates a bug. - return s.addTopicForPeer(entityId, topic) - case err != nil: - // if the adjust function returns an unexpected error on the first attempt, we return the error directly. - return nil, err - default: - // if the adjust function returns no error, we return the updated list of topics. - return topics, nil + } } -} - -func (s *SubscriptionRecordCache) addTopicForPeer(entityId flow.Identifier, topic string) ([]string, error) { var rErr error - updatedEntity, adjusted := s.c.Adjust(entityId, func(entity flow.Entity) flow.Entity { + adjustLogic := func(entity flow.Entity) flow.Entity { record, ok := entity.(SubscriptionRecordEntity) if !ok { // sanity check @@ -162,15 +137,25 @@ func (s *SubscriptionRecordCache) addTopicForPeer(entityId flow.Identifier, topi // Return the adjusted record. return record - }) - + } + adjustedEntity, adjusted := s.c.AdjustWithInit(entityId, adjustLogic, initLogic) if rErr != nil { - return nil, fmt.Errorf("failed to adjust record: %w", rErr) + return nil, fmt.Errorf("failed to adjust record with error: %w", rErr) } - if !adjusted { - return nil, ErrTopicRecordNotFound + return nil, fmt.Errorf("failed to adjust record, entity not found") } - return updatedEntity.(SubscriptionRecordEntity).Topics, nil + return adjustedEntity.(SubscriptionRecordEntity).Topics, nil +} + +// entityIdOf converts a peer ID to a flow ID by taking the hash of the peer ID. +// This is used to convert the peer ID in a notion that is compatible with HeroCache. +// This is not a protocol-level conversion, and is only used internally by the cache, MUST NOT be exposed outside the cache. +// Args: +// - peerId: the peer ID of the peer in the GossipSub protocol. +// Returns: +// - flow.Identifier: the flow ID of the peer. +func entityIdOf(pid peer.ID) flow.Identifier { + return flow.MakeID(pid) } diff --git a/network/p2p/scoring/internal/subscriptionCache_test.go b/network/p2p/scoring/internal/subscriptionCache_test.go index 355bdab9523..54b88707702 100644 --- a/network/p2p/scoring/internal/subscriptionCache_test.go +++ b/network/p2p/scoring/internal/subscriptionCache_test.go @@ -39,10 +39,10 @@ func TestSubscriptionCache_GetSubscribedTopics(t *testing.T) { // case when the peer has a subscription topics := []string{"topic1", "topic2"} - updatedTopics, err := cache.AddTopicForPeer(peerID, topics[0]) + updatedTopics, err := cache.AddWithInitTopicForPeer(peerID, topics[0]) require.NoError(t, err, "adding topic 1 should not produce an error") require.Equal(t, topics[:1], updatedTopics, "updated topics should match the added topic") - updatedTopics, err = cache.AddTopicForPeer(peerID, topics[1]) + updatedTopics, err = cache.AddWithInitTopicForPeer(peerID, topics[1]) require.NoError(t, err, "adding topic 2 should not produce an error") require.Equal(t, topics, updatedTopics, "updated topics should match the added topic") @@ -89,11 +89,11 @@ func TestSubscriptionCache_TestAddTopicForPeer(t *testing.T) { secondTopic := "topic2" // add first topic to the existing peer - _, err := cache.AddTopicForPeer(existingPeerID, firstTopic) + _, err := cache.AddWithInitTopicForPeer(existingPeerID, firstTopic) require.NoError(t, err, "adding first topic to existing peer should not produce an error") // add second topic to the same peer - updatedTopics, err := cache.AddTopicForPeer(existingPeerID, secondTopic) + updatedTopics, err := cache.AddWithInitTopicForPeer(existingPeerID, secondTopic) require.NoError(t, err, "adding second topic to existing peer should not produce an error") require.ElementsMatch(t, []string{firstTopic, secondTopic}, updatedTopics, "updated topics should match the added topics") @@ -102,7 +102,7 @@ func TestSubscriptionCache_TestAddTopicForPeer(t *testing.T) { newTopic := "newTopic" // add a topic to the new peer - updatedTopics, err = cache.AddTopicForPeer(newPeerID, newTopic) + updatedTopics, err = cache.AddWithInitTopicForPeer(newPeerID, newTopic) require.NoError(t, err, "adding topic to new peer should not produce an error") require.Equal(t, []string{newTopic}, updatedTopics, "updated topics for new peer should match the added topic") @@ -124,18 +124,18 @@ func TestSubscriptionCache_DuplicateTopics(t *testing.T) { topic := "topic1" // add first topic to the existing peer - _, err := cache.AddTopicForPeer(peerID, topic) + _, err := cache.AddWithInitTopicForPeer(peerID, topic) require.NoError(t, err, "adding first topic to existing peer should not produce an error") // add second topic to the same peer - updatedTopics, err := cache.AddTopicForPeer(peerID, topic) + updatedTopics, err := cache.AddWithInitTopicForPeer(peerID, topic) require.NoError(t, err, "adding duplicate topic to existing peer should not produce an error") require.Equal(t, []string{topic}, updatedTopics, "duplicate topic should not be added") } -// TestSubscriptionCache_MoveUpdateCycle tests that (1) within one update cycle, "AddTopicForPeer" calls append the topics to the list of -// subscribed topics for peer, (2) as long as there is no "AddTopicForPeer" call, moving to the next update cycle -// does not change the subscribed topics for a peer, and (3) calling "AddTopicForPeer" after moving to the next update +// TestSubscriptionCache_MoveUpdateCycle tests that (1) within one update cycle, "AddWithInitTopicForPeer" calls append the topics to the list of +// subscribed topics for peer, (2) as long as there is no "AddWithInitTopicForPeer" call, moving to the next update cycle +// does not change the subscribed topics for a peer, and (3) calling "AddWithInitTopicForPeer" after moving to the next update // cycle clears the subscribed topics for a peer and adds the new topic. func TestSubscriptionCache_MoveUpdateCycle(t *testing.T) { sizeLimit := uint32(100) @@ -151,13 +151,13 @@ func TestSubscriptionCache_MoveUpdateCycle(t *testing.T) { topic4 := "topic4" // adds topic1, topic2, and topic3 to the peer - topics, err := cache.AddTopicForPeer(peerID, topic1) + topics, err := cache.AddWithInitTopicForPeer(peerID, topic1) require.NoError(t, err, "adding first topic to existing peer should not produce an error") require.Equal(t, []string{topic1}, topics, "updated topics should match the added topic") - topics, err = cache.AddTopicForPeer(peerID, topic2) + topics, err = cache.AddWithInitTopicForPeer(peerID, topic2) require.NoError(t, err, "adding second topic to existing peer should not produce an error") require.Equal(t, []string{topic1, topic2}, topics, "updated topics should match the added topics") - topics, err = cache.AddTopicForPeer(peerID, topic3) + topics, err = cache.AddWithInitTopicForPeer(peerID, topic3) require.NoError(t, err, "adding third topic to existing peer should not produce an error") require.Equal(t, []string{topic1, topic2, topic3}, topics, "updated topics should match the added topics") @@ -169,7 +169,7 @@ func TestSubscriptionCache_MoveUpdateCycle(t *testing.T) { // add topic4 to the peer; since we moved to the next update cycle, the topics for the peer should be cleared // and topic4 should be the only topic for the peer - topics, err = cache.AddTopicForPeer(peerID, topic4) + topics, err = cache.AddWithInitTopicForPeer(peerID, topic4) require.NoError(t, err, "adding fourth topic to existing peer should not produce an error") require.Equal(t, []string{topic4}, topics, "updated topics should match the added topic") @@ -197,12 +197,12 @@ func TestSubscriptionCache_MoveUpdateCycleWithDifferentPeers(t *testing.T) { topic2 := "topic2" // add topic1 to peer1 - topics, err := cache.AddTopicForPeer(peer1, topic1) + topics, err := cache.AddWithInitTopicForPeer(peer1, topic1) require.NoError(t, err, "adding first topic to peer1 should not produce an error") require.Equal(t, []string{topic1}, topics, "updated topics should match the added topic") // add topic2 to peer2 - topics, err = cache.AddTopicForPeer(peer2, topic2) + topics, err = cache.AddWithInitTopicForPeer(peer2, topic2) require.NoError(t, err, "adding first topic to peer2 should not produce an error") require.Equal(t, []string{topic2}, topics, "updated topics should match the added topic") @@ -219,7 +219,7 @@ func TestSubscriptionCache_MoveUpdateCycleWithDifferentPeers(t *testing.T) { require.ElementsMatch(t, []string{topic2}, topics, "retrieved topics should match the added topics") // now add topic2 to peer1; it should overwrite the previous topics for peer1, but not affect the topics for peer2 - topics, err = cache.AddTopicForPeer(peer1, topic2) + topics, err = cache.AddWithInitTopicForPeer(peer1, topic2) require.NoError(t, err, "adding second topic to peer1 should not produce an error") require.Equal(t, []string{topic2}, topics, "updated topics should match the added topic") @@ -230,7 +230,6 @@ func TestSubscriptionCache_MoveUpdateCycleWithDifferentPeers(t *testing.T) { // TestSubscriptionCache_ConcurrentUpdate tests subscription cache update in a concurrent environment. func TestSubscriptionCache_ConcurrentUpdate(t *testing.T) { - unittest.SkipUnless(t, unittest.TEST_TODO, "this test requires atomic AdjustOrGet method to be implemented for backend") sizeLimit := uint32(100) cache := internal.NewSubscriptionRecordCache( sizeLimit, @@ -248,7 +247,7 @@ func TestSubscriptionCache_ConcurrentUpdate(t *testing.T) { allUpdatesDone.Add(1) go func() { defer allUpdatesDone.Done() - _, err := cache.AddTopicForPeer(pid, topic) + _, err := cache.AddWithInitTopicForPeer(pid, topic) require.NoError(t, err, "adding topic to peer should not produce an error") }() } @@ -286,7 +285,7 @@ func TestSubscriptionCache_TestSizeLimit(t *testing.T) { // add topics to peers for _, pid := range peerIds { for _, topic := range topics { - _, err := cache.AddTopicForPeer(pid, topic) + _, err := cache.AddWithInitTopicForPeer(pid, topic) require.NoError(t, err, "adding topic to peer should not produce an error") } } @@ -300,7 +299,7 @@ func TestSubscriptionCache_TestSizeLimit(t *testing.T) { // add one more peer and verify that the first peer is evicted newPeerID := unittest.PeerIdFixture(t) - _, err := cache.AddTopicForPeer(newPeerID, topics[0]) + _, err := cache.AddWithInitTopicForPeer(newPeerID, topics[0]) require.NoError(t, err, "adding topic to peer should not produce an error") _, found := cache.GetSubscribedTopics(peerIds[0]) diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 750e1384915..4b56de3754e 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -8,6 +8,7 @@ import ( "github.com/go-playground/validator/v10" "github.com/libp2p/go-libp2p/core/peer" "github.com/rs/zerolog" + "go.uber.org/atomic" "github.com/onflow/flow-go/engine/common/worker" "github.com/onflow/flow-go/model/flow" @@ -26,76 +27,12 @@ import ( ) const ( - // MinimumSpamPenaltyDecayFactor is minimum speed at which the spam penalty value of a peer is decayed. - // Spam record will be initialized with a decay value between .5 , .7 and this value will then be decayed up to .99 on consecutive misbehavior's, - // The maximum decay value decays the penalty by 1% every second. The decay is applied geometrically, i.e., `newPenalty = oldPenalty * decay`, hence, the higher decay value - // indicates a lower decay speed, i.e., it takes more heartbeat intervals to decay a penalty back to zero when the decay value is high. - // assume: - // penalty = -100 (the maximum application specific penalty is -100) - // skipDecayThreshold = -0.1 - // it takes around 459 seconds for the penalty to decay to reach greater than -0.1 and turn into 0. - // x * 0.99 ^ n > -0.1 (assuming negative x). - // 0.99 ^ n > -0.1 / x - // Now we can take the logarithm of both sides (with any base, but let's use base 10 for simplicity). - // log( 0.99 ^ n ) < log( 0.1 / x ) - // Using the properties of logarithms, we can bring down the exponent: - // n * log( 0.99 ) < log( -0.1 / x ) - // And finally, we can solve for n: - // n > log( -0.1 / x ) / log( 0.99 ) - // We can plug in x = -100: - // n > log( -0.1 / -100 ) / log( 0.99 ) - // n > log( 0.001 ) / log( 0.99 ) - // n > -3 / log( 0.99 ) - // n > 458.22 - MinimumSpamPenaltyDecayFactor = 0.99 - // MaximumSpamPenaltyDecayFactor represents the maximum rate at which the spam penalty value of a peer decays. Decay speeds increase - // during sustained malicious activity, leading to a slower recovery of the app-specific score for the penalized node. Conversely, - // decay speeds decrease, allowing faster recoveries, when nodes exhibit fleeting misbehavior. - MaximumSpamPenaltyDecayFactor = 0.8 - // skipDecayThreshold is the threshold for which when the negative penalty is above this value, the decay function will not be called. - // instead, the penalty will be set to 0. This is to prevent the penalty from keeping a small negative value for a long time. - skipDecayThreshold = -0.1 - // graftMisbehaviourPenalty is the penalty applied to the application specific penalty when a peer conducts a graft misbehaviour. - graftMisbehaviourPenalty = -10 - // pruneMisbehaviourPenalty is the penalty applied to the application specific penalty when a peer conducts a prune misbehaviour. - pruneMisbehaviourPenalty = -10 - // iHaveMisbehaviourPenalty is the penalty applied to the application specific penalty when a peer conducts a iHave misbehaviour. - iHaveMisbehaviourPenalty = -10 - // iWantMisbehaviourPenalty is the penalty applied to the application specific penalty when a peer conducts a iWant misbehaviour. - iWantMisbehaviourPenalty = -10 - // clusterPrefixedPenaltyReductionFactor factor used to reduce the penalty for control message misbehaviours on cluster prefixed topics. This allows a more lenient punishment for nodes - // that fall behind and may need to request old data. - clusterPrefixedPenaltyReductionFactor = .5 - // rpcPublishMessageMisbehaviourPenalty is the penalty applied to the application specific penalty when a peer conducts a RpcPublishMessageMisbehaviourPenalty misbehaviour. - rpcPublishMessageMisbehaviourPenalty = -10 + // NotificationSilencedMsg log messages for silenced notifications + NotificationSilencedMsg = "ignoring invalid control message notification for peer during silence period" ) type SpamRecordInitFunc func() p2p.GossipSubSpamRecord -// GossipSubCtrlMsgPenaltyValue is the penalty value for each control message type. -type GossipSubCtrlMsgPenaltyValue struct { - Graft float64 // penalty value for an individual graft message misbehaviour. - Prune float64 // penalty value for an individual prune message misbehaviour. - IHave float64 // penalty value for an individual iHave message misbehaviour. - IWant float64 // penalty value for an individual iWant message misbehaviour. - // ClusterPrefixedPenaltyReductionFactor factor used to reduce the penalty for control message misbehaviours on cluster prefixed topics. This is allows a more lenient punishment for nodes - // that fall behind and may need to request old data. - ClusterPrefixedPenaltyReductionFactor float64 - RpcPublishMessage float64 // penalty value for an individual RpcPublishMessage message misbehaviour. -} - -// DefaultGossipSubCtrlMsgPenaltyValue returns the default penalty value for each control message type. -func DefaultGossipSubCtrlMsgPenaltyValue() GossipSubCtrlMsgPenaltyValue { - return GossipSubCtrlMsgPenaltyValue{ - Graft: graftMisbehaviourPenalty, - Prune: pruneMisbehaviourPenalty, - IHave: iHaveMisbehaviourPenalty, - IWant: iWantMisbehaviourPenalty, - ClusterPrefixedPenaltyReductionFactor: clusterPrefixedPenaltyReductionFactor, - RpcPublishMessage: rpcPublishMessageMisbehaviourPenalty, - } -} - // GossipSubAppSpecificScoreRegistry is the registry for the application specific score of peers in the GossipSub protocol. // The application specific score is part of the overall score of a peer, and is used to determine the peer's score based // on its behavior related to the application (Flow protocol). @@ -111,10 +48,7 @@ type GossipSubAppSpecificScoreRegistry struct { // spamScoreCache currently only holds the control message misbehaviour penalty (spam related penalty). spamScoreCache p2p.GossipSubSpamRecordCache - penalty GossipSubCtrlMsgPenaltyValue - - // initial application specific penalty record, used to initialize the penalty cache entry. - init func() p2p.GossipSubSpamRecord + penalty p2pconfig.MisbehaviourPenalties validator p2p.SubscriptionValidator @@ -129,6 +63,18 @@ type GossipSubAppSpecificScoreRegistry struct { // appScoreUpdateWorkerPool is the worker pool for handling the application specific score update of peers in a non-blocking way. appScoreUpdateWorkerPool *worker.Pool[peer.ID] + + // silencePeriodDuration duration that the startup silence period will last, during which nodes will not be penalized + silencePeriodDuration time.Duration + // silencePeriodStartTime time that the silence period begins, this is the time that the registry is started by the node. + silencePeriodStartTime time.Time + // silencePeriodElapsed atomic bool that stores a bool flag which indicates if the silence period is over or not. + silencePeriodElapsed *atomic.Bool + + unknownIdentityPenalty float64 + minAppSpecificPenalty float64 + stakedIdentityReward float64 + invalidSubscriptionPenalty float64 } // GossipSubAppSpecificScoreRegistryConfig is the configuration for the GossipSubAppSpecificScoreRegistry. @@ -143,16 +89,12 @@ type GossipSubAppSpecificScoreRegistryConfig struct { Validator p2p.SubscriptionValidator `validate:"required"` // Penalty encapsulates the penalty unit for each control message type misbehaviour. - Penalty GossipSubCtrlMsgPenaltyValue `validate:"required"` + Penalty p2pconfig.MisbehaviourPenalties `validate:"required"` // IdProvider is the identity provider used to translate peer ids at the networking layer to Flow identifiers (if // an authorized peer is found). IdProvider module.IdentityProvider `validate:"required"` - // Init is a factory function that returns a new GossipSubSpamRecord. It is used to initialize the spam record of - // a peer when the peer is first observed by the local peer. - Init func() p2p.GossipSubSpamRecord `validate:"required"` - // SpamRecordCacheFactory is a factory function that returns a new GossipSubSpamRecordCache. It is used to initialize the spamScoreCache. // The cache is used to store the application specific penalty of peers. SpamRecordCacheFactory func() p2p.GossipSubSpamRecordCache `validate:"required"` @@ -164,6 +106,12 @@ type GossipSubAppSpecificScoreRegistryConfig struct { HeroCacheMetricsFactory metrics.HeroCacheMetricsFactory `validate:"required"` NetworkingType network.NetworkingType `validate:"required"` + + // ScoringRegistryStartupSilenceDuration defines the duration of time, after the node startup, + // during which the scoring registry remains inactive before penalizing nodes. + ScoringRegistryStartupSilenceDuration time.Duration + + AppSpecificScoreParams p2pconfig.ApplicationSpecificScoreParameters `validate:"required"` } // NewGossipSubAppSpecificScoreRegistry returns a new GossipSubAppSpecificScoreRegistry. @@ -187,14 +135,19 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis metrics.GossipSubAppSpecificScoreUpdateQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) reg := &GossipSubAppSpecificScoreRegistry{ - logger: config.Logger.With().Str("module", "app_score_registry").Logger(), - spamScoreCache: config.SpamRecordCacheFactory(), - appScoreCache: config.AppScoreCacheFactory(), - penalty: config.Penalty, - init: config.Init, - validator: config.Validator, - idProvider: config.IdProvider, - scoreTTL: config.Parameters.ScoreTTL, + logger: config.Logger.With().Str("module", "app_score_registry").Logger(), + spamScoreCache: config.SpamRecordCacheFactory(), + appScoreCache: config.AppScoreCacheFactory(), + penalty: config.Penalty, + validator: config.Validator, + idProvider: config.IdProvider, + scoreTTL: config.Parameters.ScoreTTL, + silencePeriodDuration: config.ScoringRegistryStartupSilenceDuration, + silencePeriodElapsed: atomic.NewBool(false), + unknownIdentityPenalty: config.AppSpecificScoreParams.UnknownIdentityPenalty, + minAppSpecificPenalty: config.AppSpecificScoreParams.MinAppSpecificPenalty, + stakedIdentityReward: config.AppSpecificScoreParams.StakedIdentityReward, + invalidSubscriptionPenalty: config.AppSpecificScoreParams.InvalidSubscriptionPenalty, } reg.appScoreUpdateWorkerPool = worker.NewWorkerPoolBuilder[peer.ID](lg.With().Str("component", "app_specific_score_update_worker_pool").Logger(), @@ -213,11 +166,16 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis ready() reg.logger.Info().Msg("subscription validator is ready") } - <-ctx.Done() reg.logger.Info().Msg("stopping subscription validator") <-reg.validator.Done() reg.logger.Info().Msg("subscription validator stopped") + }).AddWorker(func(parent irrecoverable.SignalerContext, ready component.ReadyFunc) { + if !reg.silencePeriodStartTime.IsZero() { + parent.Throw(fmt.Errorf("gossipsub scoring registry started more than once")) + } + reg.silencePeriodStartTime = time.Now() + ready() }) for i := 0; i < config.Parameters.ScoreUpdateWorkerNum; i++ { @@ -242,6 +200,13 @@ var _ p2p.GossipSubInvCtrlMsgNotifConsumer = (*GossipSubAppSpecificScoreRegistry func (r *GossipSubAppSpecificScoreRegistry) AppSpecificScoreFunc() func(peer.ID) float64 { return func(pid peer.ID) float64 { lg := r.logger.With().Str("remote_peer_id", p2plogging.PeerId(pid)).Logger() + + // during startup silence period avoid penalizing nodes + if !r.afterSilencePeriod() { + lg.Trace().Msg("returning 0 app specific score penalty for node during silence period") + return 0 + } + appSpecificScore, lastUpdated, ok := r.appScoreCache.Get(pid) switch { case !ok: @@ -250,7 +215,6 @@ func (r *GossipSubAppSpecificScoreRegistry) AppSpecificScoreFunc() func(peer.ID) lg.Trace(). Bool("worker_submitted", submitted). Msg("application specific score not found in cache, submitting worker to update it") - return 0 // in the mean time, return 0, which is a neutral score. case time.Since(lastUpdated) > r.scoreTTL: // record found in the cache, but expired; submit a worker to update it. @@ -260,14 +224,12 @@ func (r *GossipSubAppSpecificScoreRegistry) AppSpecificScoreFunc() func(peer.ID) Float64("app_specific_score", appSpecificScore). Dur("score_ttl", r.scoreTTL). Msg("application specific score expired, submitting worker to update it") - return appSpecificScore // in the mean time, return the expired score. default: // record found in the cache. r.logger.Trace(). Float64("app_specific_score", appSpecificScore). Msg("application specific score found in cache") - return appSpecificScore } } @@ -342,7 +304,7 @@ func (r *GossipSubAppSpecificScoreRegistry) computeAppSpecificScore(pid peer.ID) // - error: an error if the update failed; any returned error is an irrecoverable error and indicates a bug or misconfiguration. func (r *GossipSubAppSpecificScoreRegistry) processAppSpecificScoreUpdateWork(p peer.ID) error { appSpecificScore := r.computeAppSpecificScore(p) - err := r.appScoreCache.Add(p, appSpecificScore, time.Now()) + err := r.appScoreCache.AdjustWithInit(p, appSpecificScore, time.Now()) if err != nil { // the error is considered fatal as it means the cache is not working properly. return fmt.Errorf("could not add application specific score %f for peer to cache: %w", appSpecificScore, err) @@ -364,7 +326,7 @@ func (r *GossipSubAppSpecificScoreRegistry) stakingScore(pid peer.ID) (float64, Err(err). Bool(logging.KeySuspicious, true). Msg("invalid peer identity, penalizing peer") - return DefaultUnknownIdentityPenalty, flow.Identifier{}, 0 + return r.unknownIdentityPenalty, flow.Identifier{}, 0 } lg = lg.With(). @@ -377,13 +339,13 @@ func (r *GossipSubAppSpecificScoreRegistry) stakingScore(pid peer.ID) (float64, if flowId.Role == flow.RoleAccess { lg.Trace(). Msg("pushing access node to edge by penalizing with minimum penalty value") - return MinAppSpecificPenalty, flowId.NodeID, flowId.Role + return r.minAppSpecificPenalty, flowId.NodeID, flowId.Role } lg.Trace(). Msg("rewarding well-behaved non-access node peer with maximum reward value") - return DefaultStakedIdentityReward, flowId.NodeID, flowId.Role + return r.stakedIdentityReward, flowId.NodeID, flowId.Role } func (r *GossipSubAppSpecificScoreRegistry) subscriptionPenalty(pid peer.ID, flowId flow.Identifier, role flow.Role) float64 { @@ -395,7 +357,7 @@ func (r *GossipSubAppSpecificScoreRegistry) subscriptionPenalty(pid peer.ID, flo Hex("flow_id", logging.ID(flowId)). Bool(logging.KeySuspicious, true). Msg("invalid subscription detected, penalizing peer") - return DefaultInvalidSubscriptionPenalty + return r.invalidSubscriptionPenalty } return 0 @@ -412,28 +374,25 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( Str("peer_id", p2plogging.PeerId(notification.PeerID)). Str("misbehavior_type", notification.MsgType.String()).Logger() - // try initializing the application specific penalty for the peer if it is not yet initialized. - // this is done to avoid the case where the peer is not yet cached and the application specific penalty is not yet initialized. - // initialization is successful only if the peer is not yet cached. If any error is occurred during initialization we log a fatal error - initRecord := r.init() - initialized := r.spamScoreCache.Add(notification.PeerID, initRecord) - if initialized { - lg.Trace().Str("peer_id", p2plogging.PeerId(notification.PeerID)).Msg("application specific penalty initialized for peer") + // during startup silence period avoid penalizing nodes, ignore all notifications + if !r.afterSilencePeriod() { + lg.Trace().Msg("ignoring invalid control message notification for peer during silence period") + return } - record, err := r.spamScoreCache.Update(notification.PeerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record, err := r.spamScoreCache.Adjust(notification.PeerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { penalty := 0.0 switch notification.MsgType { case p2pmsg.CtrlMsgGraft: - penalty += r.penalty.Graft + penalty += r.penalty.GraftMisbehaviour case p2pmsg.CtrlMsgPrune: - penalty += r.penalty.Prune + penalty += r.penalty.PruneMisbehaviour case p2pmsg.CtrlMsgIHave: - penalty += r.penalty.IHave + penalty += r.penalty.IHaveMisbehaviour case p2pmsg.CtrlMsgIWant: - penalty += r.penalty.IWant + penalty += r.penalty.IWantMisbehaviour case p2pmsg.RpcPublishMessage: - penalty += r.penalty.RpcPublishMessage + penalty += r.penalty.PublishMisbehaviour default: // the error is considered fatal as it means that we have an unsupported misbehaviour type, we should crash the node to prevent routing attack vulnerability. lg.Fatal().Str("misbehavior_type", notification.MsgType.String()).Msg("unknown misbehaviour type") @@ -441,7 +400,7 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( // reduce penalty for cluster prefixed topics allowing nodes that are potentially behind to catch up if notification.TopicType == p2p.CtrlMsgTopicTypeClusterPrefixed { - penalty *= r.penalty.ClusterPrefixedPenaltyReductionFactor + penalty *= r.penalty.ClusterPrefixedReductionFactor } record.Penalty += penalty @@ -458,10 +417,22 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( Msg("applied misbehaviour penalty and updated application specific penalty") } +// afterSilencePeriod returns true if registry silence period is over, false otherwise. +func (r *GossipSubAppSpecificScoreRegistry) afterSilencePeriod() bool { + if !r.silencePeriodElapsed.Load() { + if time.Since(r.silencePeriodStartTime) > r.silencePeriodDuration { + r.silencePeriodElapsed.Store(true) + return true + } + return false + } + return true +} + // DefaultDecayFunction is the default decay function that is used to decay the application specific penalty of a peer. // It is used if no decay function is provided in the configuration. // It decays the application specific penalty of a peer if it is negative. -func DefaultDecayFunction(slowerDecayPenaltyThreshold, decayRateDecrement float64, decayAdjustInterval time.Duration) netcache.PreprocessorFunc { +func DefaultDecayFunction(cfg p2pconfig.SpamRecordCacheDecay) netcache.PreprocessorFunc { return func(record p2p.GossipSubSpamRecord, lastUpdated time.Time) (p2p.GossipSubSpamRecord, error) { if record.Penalty >= 0 { // no need to decay the penalty if it is positive, the reason is currently the app specific penalty @@ -470,10 +441,10 @@ func DefaultDecayFunction(slowerDecayPenaltyThreshold, decayRateDecrement float6 return record, nil } - if record.Penalty > skipDecayThreshold { + if record.Penalty > cfg.SkipDecayThreshold { // penalty is negative but greater than the threshold, we set it to 0. record.Penalty = 0 - record.Decay = MaximumSpamPenaltyDecayFactor + record.Decay = cfg.MaximumSpamPenaltyDecayFactor record.LastDecayAdjustment = time.Time{} return record, nil } @@ -485,10 +456,10 @@ func DefaultDecayFunction(slowerDecayPenaltyThreshold, decayRateDecrement float6 } record.Penalty = penalty - if record.Penalty <= slowerDecayPenaltyThreshold { - if time.Since(record.LastDecayAdjustment) > decayAdjustInterval || record.LastDecayAdjustment.IsZero() { + if record.Penalty <= cfg.PenaltyDecaySlowdownThreshold { + if time.Since(record.LastDecayAdjustment) > cfg.PenaltyDecayEvaluationPeriod || record.LastDecayAdjustment.IsZero() { // reduces the decay speed flooring at MinimumSpamRecordDecaySpeed - record.Decay = math.Min(record.Decay+decayRateDecrement, MinimumSpamPenaltyDecayFactor) + record.Decay = math.Min(record.Decay+cfg.DecayRateReductionFactor, cfg.MinimumSpamPenaltyDecayFactor) record.LastDecayAdjustment = time.Now() } } @@ -496,13 +467,15 @@ func DefaultDecayFunction(slowerDecayPenaltyThreshold, decayRateDecrement float6 } } -// InitAppScoreRecordState initializes the gossipsub spam record state for a peer. +// InitAppScoreRecordStateFunc returns a callback that initializes the gossipsub spam record state for a peer. // Returns: -// - a gossipsub spam record with the default decay value and 0 penalty. -func InitAppScoreRecordState() p2p.GossipSubSpamRecord { - return p2p.GossipSubSpamRecord{ - Decay: MaximumSpamPenaltyDecayFactor, - Penalty: 0, - LastDecayAdjustment: time.Now(), +// - a func that returns a gossipsub spam record with the default decay value and 0 penalty. +func InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor float64) func() p2p.GossipSubSpamRecord { + return func() p2p.GossipSubSpamRecord { + return p2p.GossipSubSpamRecord{ + Decay: maximumSpamPenaltyDecayFactor, + Penalty: 0, + LastDecayAdjustment: time.Now(), + } } } diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index 1a454ee5312..ce5a522c17c 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -4,14 +4,17 @@ import ( "context" "fmt" "math" + "os" "sync" "testing" "time" "github.com/libp2p/go-libp2p/core/peer" + "github.com/rs/zerolog" "github.com/stretchr/testify/assert" testifymock "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + "go.uber.org/atomic" "github.com/onflow/flow-go/config" "github.com/onflow/flow-go/model/flow" @@ -39,16 +42,19 @@ func TestScoreRegistry_FreshStart(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), + reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), + withStakedIdentities(peerID), withValidSubscriptions(peerID)) - - // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) // initially, the spamRecords should not have the peer id, and there should be no app-specific score in the cache. require.False(t, spamRecords.Has(peerID)) @@ -57,13 +63,15 @@ func TestScoreRegistry_FreshStart(t *testing.T) { require.Equal(t, time.Time{}, updated) require.Equal(t, float64(0), score) + maxAppSpecificReward := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.MaxAppSpecificReward + queryTime := time.Now() require.Eventually(t, func() bool { // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. score := reg.AppSpecificScoreFunc()(peerID) // since the peer id does not have a spam record, the app specific score should be the max app specific reward, which // is the default reward for a staked peer that has valid subscriptions. - return score == scoring.MaxAppSpecificReward + return score == maxAppSpecificReward }, 5*time.Second, 100*time.Millisecond) // still the spamRecords should not have the peer id (as there is no spam record for the peer id). @@ -73,7 +81,7 @@ func TestScoreRegistry_FreshStart(t *testing.T) { score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - require.Equal(t, scoring.MaxAppSpecificReward, score) + require.Equal(t, maxAppSpecificReward, score) // stop the registry. cancel() @@ -88,19 +96,19 @@ func TestScoreRegistry_FreshStart(t *testing.T) { // asynchronous nature of app-specific score updates in GossipSub's cache. func TestScoreRegistry_PeerWithSpamRecord(t *testing.T) { t.Run("graft", func(t *testing.T) { - testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().Graft) + testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().GraftMisbehaviour) }) t.Run("prune", func(t *testing.T) { - testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().Prune) + testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().PruneMisbehaviour) }) t.Run("ihave", func(t *testing.T) { - testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHave) + testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHaveMisbehaviour) }) t.Run("iwant", func(t *testing.T) { - testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWant) + testScoreRegistryPeerWithSpamRecord(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWantMisbehaviour) }) t.Run("RpcPublishMessage", func(t *testing.T) { - testScoreRegistryPeerWithSpamRecord(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().RpcPublishMessage) + testScoreRegistryPeerWithSpamRecord(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().PublishMisbehaviour) }) } @@ -121,16 +129,19 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond - reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), + reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), + withStakedIdentities(peerID), withValidSubscriptions(peerID)) - - // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) // initially, the spamRecords should not have the peer id; also the app specific score record should not be in the cache. require.False(t, spamRecords.Has(peerID)) @@ -139,13 +150,15 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro require.Equal(t, time.Time{}, updated) require.Equal(t, float64(0), score) + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // eventually, the app specific score should be updated in the cache. require.Eventually(t, func() bool { // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. score := reg.AppSpecificScoreFunc()(peerID) // since the peer id does not have a spam record, the app specific score should be the max app specific reward, which // is the default reward for a staked peer that has valid subscriptions. - return scoring.MaxAppSpecificReward == score + return scoreOptParameters.MaxAppSpecificReward == score }, 5*time.Second, 100*time.Millisecond) // report a misbehavior for the peer id. @@ -158,8 +171,8 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. assert.True(t, ok) assert.NoError(t, err) - assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10. - assert.Equal(t, scoring.InitAppScoreRecordState().Decay, record.Decay) // decay should be initialized to the initial state. + assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10. + assert.Equal(t, scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. queryTime := time.Now() // eventually, the app specific score should be updated in the cache. @@ -189,19 +202,19 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro // penalty computation and updates to the score registry when a peer with an unknown identity sends these control messages. func TestScoreRegistry_SpamRecordWithUnknownIdentity(t *testing.T) { t.Run("graft", func(t *testing.T) { - testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().Graft) + testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().GraftMisbehaviour) }) t.Run("prune", func(t *testing.T) { - testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().Prune) + testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().PruneMisbehaviour) }) t.Run("ihave", func(t *testing.T) { - testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHave) + testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHaveMisbehaviour) }) t.Run("iwant", func(t *testing.T) { - testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWant) + testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWantMisbehaviour) }) t.Run("RpcPublishMessage", func(t *testing.T) { - testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().RpcPublishMessage) + testScoreRegistrySpamRecordWithUnknownIdentity(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().PublishMisbehaviour) }) } @@ -220,17 +233,19 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, + reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), withUnknownIdentity(peerID), withValidSubscriptions(peerID)) - - // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) // initially, the spamRecords should not have the peer id; also the app specific score record should not be in the cache. require.False(t, spamRecords.Has(peerID)) @@ -239,12 +254,14 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 require.Equal(t, time.Time{}, updated) require.Equal(t, float64(0), score) + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // eventually the app specific score should be updated in the cache to the penalty value for unknown identity. require.Eventually(t, func() bool { // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. score := reg.AppSpecificScoreFunc()(peerID) // peer does not have spam record, but has an unknown identity. Hence, the app specific score should be the staking penalty. - return scoring.DefaultUnknownIdentityPenalty == score + return scoreOptParameters.UnknownIdentityPenalty == score }, 5*time.Second, 100*time.Millisecond) // queryTime := time.Now() @@ -258,8 +275,8 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. require.True(t, ok) require.NoError(t, err) - require.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10, we account for decay. - require.Equal(t, scoring.InitAppScoreRecordState().Decay, record.Decay) // decay should be initialized to the initial state. + require.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10, we account for decay. + require.Equal(t, scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. queryTime := time.Now() // eventually, the app specific score should be updated in the cache. @@ -269,7 +286,7 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 // the peer has spam record as well as an unknown identity. Hence, the app specific score should be the spam penalty // and the staking penalty. // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 0.1% error. - return unittest.AreNumericallyClose(expectedPenalty+scoring.DefaultUnknownIdentityPenalty, score, 0.01) + return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, score, 0.01) }, 5*time.Second, 10*time.Millisecond) // the app specific score should now be updated in the cache. @@ -277,8 +294,8 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 require.True(t, exists) require.True(t, updated.After(queryTime)) - unittest.RequireNumericallyClose(t, expectedPenalty+scoring.DefaultUnknownIdentityPenalty, score, 0.01) - assert.Equal(t, scoring.InitAppScoreRecordState().Decay, record.Decay) // decay should be initialized to the initial state. + unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, score, 0.01) + assert.Equal(t, scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. // stop the registry. cancel() @@ -292,19 +309,19 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 // in spam activities, as indicated by these control messages. func TestScoreRegistry_SpamRecordWithSubscriptionPenalty(t *testing.T) { t.Run("graft", func(t *testing.T) { - testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().Graft) + testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgGraft, penaltyValueFixtures().GraftMisbehaviour) }) t.Run("prune", func(t *testing.T) { - testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().Prune) + testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgPrune, penaltyValueFixtures().PruneMisbehaviour) }) t.Run("ihave", func(t *testing.T) { - testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHave) + testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgIHave, penaltyValueFixtures().IHaveMisbehaviour) }) t.Run("iwant", func(t *testing.T) { - testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWant) + testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.CtrlMsgIWant, penaltyValueFixtures().IWantMisbehaviour) }) t.Run("RpcPublishMessage", func(t *testing.T) { - testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().RpcPublishMessage) + testScoreRegistrySpamRecordWithSubscriptionPenalty(t, p2pmsg.RpcPublishMessage, penaltyValueFixtures().PublishMisbehaviour) }) } @@ -323,16 +340,19 @@ func testScoreRegistrySpamRecordWithSubscriptionPenalty(t *testing.T, messageTyp cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), + reg, spamRecords, appScoreCache := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), + withStakedIdentities(peerID), withInvalidSubscriptions(peerID)) - - // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) // initially, the spamRecords should not have the peer id; also the app specific score record should not be in the cache. require.False(t, spamRecords.Has(peerID)) @@ -341,13 +361,15 @@ func testScoreRegistrySpamRecordWithSubscriptionPenalty(t *testing.T, messageTyp require.Equal(t, time.Time{}, updated) require.Equal(t, float64(0), score) + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // peer does not have spam record, but has invalid subscription. Hence, the app specific score should be subscription penalty. // eventually the app specific score should be updated in the cache to the penalty value for subscription penalty. require.Eventually(t, func() bool { // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. score := reg.AppSpecificScoreFunc()(peerID) // peer does not have spam record, but has an invalid subscription penalty. - return scoring.DefaultInvalidSubscriptionPenalty == score + return scoreOptParameters.InvalidSubscriptionPenalty == score }, 5*time.Second, 100*time.Millisecond) // report a misbehavior for the peer id. @@ -361,7 +383,7 @@ func testScoreRegistrySpamRecordWithSubscriptionPenalty(t *testing.T, messageTyp assert.True(t, ok) assert.NoError(t, err) assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) - assert.Equal(t, scoring.InitAppScoreRecordState().Decay, record.Decay) // decay should be initialized to the initial state. + assert.Equal(t, scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. queryTime := time.Now() // eventually, the app specific score should be updated in the cache. @@ -371,14 +393,14 @@ func testScoreRegistrySpamRecordWithSubscriptionPenalty(t *testing.T, messageTyp // the peer has spam record as well as an unknown identity. Hence, the app specific score should be the spam penalty // and the staking penalty. // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 0.1% error. - return unittest.AreNumericallyClose(expectedPenalty+scoring.DefaultInvalidSubscriptionPenalty, score, 0.01) + return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, score, 0.01) }, 5*time.Second, 10*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - unittest.RequireNumericallyClose(t, expectedPenalty+scoring.DefaultInvalidSubscriptionPenalty, score, 0.01) + unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, score, 0.01) // stop the registry. cancel() @@ -391,16 +413,19 @@ func TestScoreRegistry_SpamPenaltyDecaysInCache(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, _, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), + reg, _, _ := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), + withStakedIdentities(peerID), withValidSubscriptions(peerID)) - - // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) // report a misbehavior for the peer id. reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ @@ -439,14 +464,14 @@ func TestScoreRegistry_SpamPenaltyDecaysInCache(t *testing.T) { time.Sleep(1 * time.Second) // wait for the penalty to decay. // the upper bound is the sum of the penalties without decay. - scoreUpperBound := penaltyValueFixtures().Prune + - penaltyValueFixtures().Graft + - penaltyValueFixtures().IHave + - penaltyValueFixtures().IWant + - penaltyValueFixtures().RpcPublishMessage + scoreUpperBound := penaltyValueFixtures().PruneMisbehaviour + + penaltyValueFixtures().GraftMisbehaviour + + penaltyValueFixtures().IHaveMisbehaviour + + penaltyValueFixtures().IWantMisbehaviour + + penaltyValueFixtures().PublishMisbehaviour // the lower bound is the sum of the penalties with decay assuming the decay is applied 4 times to the sum of the penalties. // in reality, the decay is applied 4 times to the first penalty, then 3 times to the second penalty, and so on. - r := scoring.InitAppScoreRecordState() + r := scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)() scoreLowerBound := scoreUpperBound * math.Pow(r.Decay, 4) // eventually, the app specific score should be updated in the cache. @@ -471,22 +496,28 @@ func TestScoreRegistry_SpamPenaltyDecayToZero(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), - withValidSubscriptions(peerID), - withInitFunction(func() p2p.GossipSubSpamRecord { + reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + func() p2p.GossipSubSpamRecord { return p2p.GossipSubSpamRecord{ Decay: 0.02, // we choose a small decay value to speed up the test. Penalty: 0, } - })) + }, + withStakedIdentities(peerID), + withValidSubscriptions(peerID)) // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) + + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore // report a misbehavior for the peer id. reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ @@ -500,7 +531,7 @@ func TestScoreRegistry_SpamPenaltyDecayToZero(t *testing.T) { require.Eventually(t, func() bool { score := reg.AppSpecificScoreFunc()(peerID) // the penalty should be less than zero and greater than the penalty value (due to decay). - return score < 0 && score > penaltyValueFixtures().Graft + return score < 0 && score > penaltyValueFixtures().GraftMisbehaviour }, 5*time.Second, 100*time.Millisecond) require.Eventually(t, func() bool { @@ -511,7 +542,7 @@ func TestScoreRegistry_SpamPenaltyDecayToZero(t *testing.T) { require.Eventually(t, func() bool { // when the spam penalty is decayed to zero, the app specific penalty of the node should reset back to default staking reward. - return reg.AppSpecificScoreFunc()(peerID) == scoring.DefaultStakedIdentityReward + return reg.AppSpecificScoreFunc()(peerID) == scoreOptParameters.StakedIdentityReward }, 5*time.Second, 100*time.Millisecond) // the penalty should now be zero. @@ -533,27 +564,33 @@ func TestScoreRegistry_PersistingUnknownIdentityPenalty(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withUnknownIdentity(peerID), // the peer id has an unknown identity. - withValidSubscriptions(peerID), - withInitFunction(func() p2p.GossipSubSpamRecord { + reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + func() p2p.GossipSubSpamRecord { return p2p.GossipSubSpamRecord{ Decay: 0.02, // we choose a small decay value to speed up the test. Penalty: 0, } - })) + }, + withUnknownIdentity(peerID), // the peer id has an unknown identity. + withValidSubscriptions(peerID)) // starts the registry. ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) reg.Start(signalerCtx) - unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + defer stopRegistry(t, cancel, reg) + + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore // initially, the app specific score should be the default unknown identity penalty. require.Eventually(t, func() bool { score := reg.AppSpecificScoreFunc()(peerID) - return score == scoring.DefaultUnknownIdentityPenalty + return score == scoreOptParameters.UnknownIdentityPenalty }, 5*time.Second, 100*time.Millisecond) // report a misbehavior for the peer id. @@ -570,8 +607,8 @@ func TestScoreRegistry_PersistingUnknownIdentityPenalty(t *testing.T) { score := reg.AppSpecificScoreFunc()(peerID) // Ideally, the score should be the sum of the default invalid subscription penalty and the graft penalty, however, // due to exponential decay of the spam penalty and asynchronous update the app specific score; score should be in the range of [scoring. - // (scoring.DefaultUnknownIdentityPenalty+penaltyValueFixtures().Graft, scoring.DefaultUnknownIdentityPenalty). - return score < scoring.DefaultUnknownIdentityPenalty && score > scoring.DefaultUnknownIdentityPenalty+penaltyValueFixtures().Graft + // (scoring.DefaultUnknownIdentityPenalty+penaltyValueFixtures().GraftMisbehaviour, scoring.DefaultUnknownIdentityPenalty). + return score < scoreOptParameters.UnknownIdentityPenalty && score > scoreOptParameters.UnknownIdentityPenalty+penaltyValueFixtures().GraftMisbehaviour }, 5*time.Second, 100*time.Millisecond) require.Eventually(t, func() bool { @@ -582,7 +619,7 @@ func TestScoreRegistry_PersistingUnknownIdentityPenalty(t *testing.T) { require.Eventually(t, func() bool { // when the spam penalty is decayed to zero, the app specific penalty of the node should only contain the unknown identity penalty. - return reg.AppSpecificScoreFunc()(peerID) == scoring.DefaultUnknownIdentityPenalty + return reg.AppSpecificScoreFunc()(peerID) == scoreOptParameters.UnknownIdentityPenalty }, 5*time.Second, 100*time.Millisecond) // the spam penalty should now be zero in spamRecords. @@ -604,16 +641,18 @@ func TestScoreRegistry_PersistingInvalidSubscriptionPenalty(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond - reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, withStakedIdentities(peerID), - withInvalidSubscriptions(peerID), // the peer id has an invalid subscription. - withInitFunction(func() p2p.GossipSubSpamRecord { + reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + func() p2p.GossipSubSpamRecord { return p2p.GossipSubSpamRecord{ Decay: 0.02, // we choose a small decay value to speed up the test. Penalty: 0, } - })) + }, + withStakedIdentities(peerID), + withInvalidSubscriptions(peerID)) // the peer id has an invalid subscription // starts the registry. ctx, cancel := context.WithCancel(context.Background()) @@ -621,10 +660,12 @@ func TestScoreRegistry_PersistingInvalidSubscriptionPenalty(t *testing.T) { reg.Start(signalerCtx) unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + // initially, the app specific score should be the default invalid subscription penalty. require.Eventually(t, func() bool { score := reg.AppSpecificScoreFunc()(peerID) - return score == scoring.DefaultInvalidSubscriptionPenalty + return score == scoreOptParameters.InvalidSubscriptionPenalty }, 5*time.Second, 100*time.Millisecond) // report a misbehavior for the peer id. @@ -638,8 +679,8 @@ func TestScoreRegistry_PersistingInvalidSubscriptionPenalty(t *testing.T) { score := reg.AppSpecificScoreFunc()(peerID) // Ideally, the score should be the sum of the default invalid subscription penalty and the graft penalty, however, // due to exponential decay of the spam penalty and asynchronous update the app specific score; score should be in the range of [scoring. - // (DefaultInvalidSubscriptionPenalty+penaltyValueFixtures().Graft, scoring.DefaultInvalidSubscriptionPenalty). - return score < scoring.DefaultInvalidSubscriptionPenalty && score > scoring.DefaultInvalidSubscriptionPenalty+penaltyValueFixtures().Graft + // (DefaultInvalidSubscriptionPenalty+penaltyValueFixtures().GraftMisbehaviour, scoring.DefaultInvalidSubscriptionPenalty). + return score < scoreOptParameters.InvalidSubscriptionPenalty && score > scoreOptParameters.InvalidSubscriptionPenalty+penaltyValueFixtures().GraftMisbehaviour }, 5*time.Second, 100*time.Millisecond) require.Eventually(t, func() bool { @@ -650,7 +691,7 @@ func TestScoreRegistry_PersistingInvalidSubscriptionPenalty(t *testing.T) { require.Eventually(t, func() bool { // when the spam penalty is decayed to zero, the app specific penalty of the node should only contain the default invalid subscription penalty. - return reg.AppSpecificScoreFunc()(peerID) == scoring.DefaultUnknownIdentityPenalty + return reg.AppSpecificScoreFunc()(peerID) == scoreOptParameters.UnknownIdentityPenalty }, 5*time.Second, 100*time.Millisecond) // the spam penalty should now be zero in spamRecords. @@ -670,15 +711,16 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond // increase configured DecayRateReductionFactor so that the decay time is increased faster - cfg.NetworkConfig.GossipSub.ScoringParameters.SpamRecordCache.DecayRateReductionFactor = .1 - cfg.NetworkConfig.GossipSub.ScoringParameters.SpamRecordCache.PenaltyDecayEvaluationPeriod = time.Second + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.DecayRateReductionFactor = .1 + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.PenaltyDecayEvaluationPeriod = time.Second peer1 := unittest.PeerIdFixture(t) peer2 := unittest.PeerIdFixture(t) reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), withStakedIdentities(peer1, peer2), withValidSubscriptions(peer1, peer2)) @@ -690,16 +732,19 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { // initially, the spamRecords should not have the peer ids. assert.False(t, spamRecords.Has(peer1)) assert.False(t, spamRecords.Has(peer2)) + + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + scoringRegistryParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters // since the both peers do not have a spam record, their app specific score should be the max app specific reward, which // is the default reward for a staked peer that has valid subscriptions. require.Eventually(t, func() bool { // when the spam penalty is decayed to zero, the app specific penalty of the node should only contain the unknown identity penalty. - return scoring.MaxAppSpecificReward == reg.AppSpecificScoreFunc()(peer1) && scoring.MaxAppSpecificReward == reg.AppSpecificScoreFunc()(peer2) + return scoreOptParameters.MaxAppSpecificReward == reg.AppSpecificScoreFunc()(peer1) && scoreOptParameters.MaxAppSpecificReward == reg.AppSpecificScoreFunc()(peer2) }, 5*time.Second, 100*time.Millisecond) // simulate sustained malicious activity from peer1, eventually the decay speed // for a spam record should be reduced to the MinimumSpamPenaltyDecayFactor - prevDecay := scoring.MaximumSpamPenaltyDecayFactor + prevDecay := scoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor tolerance := 0.1 require.Eventually(t, func() bool { reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ @@ -711,7 +756,7 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { require.True(t, ok) assert.Less(t, math.Abs(prevDecay-record.Decay), tolerance) prevDecay = record.Decay - return record.Decay == scoring.MinimumSpamPenaltyDecayFactor + return record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor }, 5*time.Second, 500*time.Millisecond) // initialize a spam record for peer2 @@ -720,23 +765,23 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { MsgType: p2pmsg.CtrlMsgPrune, }) // reduce penalty and increase Decay to scoring.MinimumSpamPenaltyDecayFactor - record, err := spamRecords.Update(peer2, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { + record, err := spamRecords.Adjust(peer2, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { record.Penalty = -.1 - record.Decay = scoring.MinimumSpamPenaltyDecayFactor + record.Decay = scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor return record }) require.NoError(t, err) - require.True(t, record.Decay == scoring.MinimumSpamPenaltyDecayFactor) + require.True(t, record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor) require.True(t, record.Penalty == -.1) // simulate sustained good behavior from peer 2, each time the spam record is read from the cache // using Get method the record penalty will be decayed until it is eventually reset to // 0 at this point the decay speed for the record should be reset to MaximumSpamPenaltyDecayFactor - // eventually after penalty reaches the skipDecaThreshold the record decay will be reset to scoring.MaximumSpamPenaltyDecayFactor + // eventually after penalty reaches the skipDecaThreshold the record decay will be reset to scoringRegistryParameters.MaximumSpamPenaltyDecayFactor require.Eventually(t, func() bool { record, err, ok := spamRecords.Get(peer2) require.NoError(t, err) require.True(t, ok) - return record.Decay == scoring.MaximumSpamPenaltyDecayFactor && + return record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor && record.Penalty == 0 && record.LastDecayAdjustment.IsZero() }, 5*time.Second, time.Second) @@ -750,7 +795,7 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { record, err, ok := spamRecords.Get(peer1) require.NoError(t, err) require.True(t, ok) - return record.Decay == scoring.MinimumSpamPenaltyDecayFactor + return record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor }, 5*time.Second, 500*time.Millisecond) // stop the registry. @@ -769,10 +814,11 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) // refresh cached app-specific score every 100 milliseconds to speed up the test. - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), withStakedIdentities(peerIds...), withValidSubscriptions(peerIds...)) @@ -782,6 +828,8 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { reg.Start(signalerCtx) unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "failed to start GossipSubAppSpecificScoreRegistry") + scoreOptParameters := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore + for _, peerID := range peerIds { // initially, the spamRecords should not have the peer id. assert.False(t, spamRecords.Has(peerID)) @@ -792,7 +840,7 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { score := reg.AppSpecificScoreFunc()(peerID) // since the peer id does not have a spam record, the app specific score should be the max app specific reward, which // is the default reward for a staked peer that has valid subscriptions. - return score == scoring.MaxAppSpecificReward + return score == scoreOptParameters.MaxAppSpecificReward }, 5*time.Second, 100*time.Millisecond) } @@ -822,15 +870,15 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { }() unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") - // expected penalty should be penaltyValueFixtures().Graft * (1 + clusterReductionFactor) - expectedPenalty := penaltyValueFixture(ctlMsgType) * (1 + penaltyValueFixtures().ClusterPrefixedPenaltyReductionFactor) + // expected penalty should be penaltyValueFixtures().GraftMisbehaviour * (1 + clusterReductionFactor) + expectedPenalty := penaltyValueFixture(ctlMsgType) * (1 + penaltyValueFixtures().ClusterPrefixedReductionFactor) // the penalty should now be updated in the spamRecords record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. assert.True(t, ok) assert.NoError(t, err) assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) - assert.Equal(t, scoring.InitAppScoreRecordState().Decay, record.Decay) + assert.Equal(t, scoring.InitAppScoreRecordStateFunc(cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor)().Decay, record.Decay) // this peer has a spam record, with no subscription penalty. Hence, the app specific score should only be the spam penalty, // and the peer should be deprived of the default reward for its valid staked role. score := reg.AppSpecificScoreFunc()(peerID) @@ -847,6 +895,100 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { unittest.RequireCloseBefore(t, reg.Done(), 1*time.Second, "failed to stop GossipSubAppSpecificScoreRegistry") } +// TestScoringRegistrySilencePeriod ensures that the scoring registry does not penalize nodes during the silence period, and +// starts to penalize nodes only after the silence period is over. +func TestScoringRegistrySilencePeriod(t *testing.T) { + peerID := unittest.PeerIdFixture(t) + silenceDuration := 5 * time.Second + silencedNotificationLogs := atomic.NewInt32(0) + hook := zerolog.HookFunc(func(e *zerolog.Event, level zerolog.Level, message string) { + if level == zerolog.TraceLevel { + if message == scoring.NotificationSilencedMsg { + silencedNotificationLogs.Inc() + } + } + }) + logger := zerolog.New(os.Stdout).Level(zerolog.TraceLevel).Hook(hook) + + cfg, err := config.DefaultConfig() + require.NoError(t, err) + // refresh cached app-specific score every 100 milliseconds to speed up the test. + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 100 * time.Millisecond + maximumSpamPenaltyDecayFactor := cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor + reg, spamRecords, _ := newGossipSubAppSpecificScoreRegistry(t, + cfg.NetworkConfig.GossipSub.ScoringParameters, + scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor), + withUnknownIdentity(peerID), + withInvalidSubscriptions(peerID), + func(cfg *scoring.GossipSubAppSpecificScoreRegistryConfig) { + // we set the scoring registry silence duration 10 seconds + // the peer is not expected to be penalized for the first 5 seconds of the test + // after that an invalid control message notification is processed and the peer + // should be penalized + cfg.ScoringRegistryStartupSilenceDuration = silenceDuration + // hooked logger will capture the number of logs related to ignored notifications + cfg.Logger = logger + }) + + ctx, cancel := context.WithCancel(context.Background()) + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + defer stopRegistry(t, cancel, reg) + // capture approximate registry start time + reg.Start(signalerCtx) + unittest.RequireCloseBefore(t, reg.Ready(), 1*time.Second, "registry did not start in time") + + registryStartTime := time.Now() + expectedNumOfSilencedNotif := 0 + // while we are in the silence period all notifications should be ignored and the + // invalid subscription penalty should not be applied to the app specific score + // we ensure we stay within the silence duration by iterating only up until 1 second + // before silence period is over + for time.Since(registryStartTime) < (silenceDuration - time.Second) { + // report a misbehavior for the peer id. + reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ + PeerID: peerID, + MsgType: p2pmsg.CtrlMsgGraft, + }) + expectedNumOfSilencedNotif++ + // spam records should not be created during the silence period + _, err, ok := spamRecords.Get(peerID) + assert.False(t, ok) + assert.NoError(t, err) + // initially, the app specific score should be the default invalid subscription penalty. + require.Equal(t, float64(0), reg.AppSpecificScoreFunc()(peerID)) + } + + invalidSubscriptionPenalty := cfg.NetworkConfig.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.InvalidSubscriptionPenalty + + require.Eventually(t, func() bool { + // we expect to have logged a debug message for all notifications ignored. + require.Equal(t, int32(expectedNumOfSilencedNotif), silencedNotificationLogs.Load()) + // after silence period the invalid subscription penalty should be applied to the app specific score + return invalidSubscriptionPenalty == reg.AppSpecificScoreFunc()(peerID) + }, 2*time.Second, 200*time.Millisecond) + + // after silence period the peer has spam record as well as an unknown identity. Hence, the app specific score should be the spam penalty + // and the staking penalty. + reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ + PeerID: peerID, + MsgType: p2pmsg.CtrlMsgGraft, + }) + // the penalty should now be applied and spam records created. + record, err, ok := spamRecords.Get(peerID) + assert.True(t, ok) + assert.NoError(t, err) + expectedPenalty := penaltyValueFixtures().GraftMisbehaviour + assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) + assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + require.Eventually(t, func() bool { + // we expect to have logged a debug message for all notifications ignored. + require.Equal(t, int32(expectedNumOfSilencedNotif), silencedNotificationLogs.Load()) + // after silence period the invalid subscription penalty should be applied to the app specific score + return invalidSubscriptionPenalty+expectedPenalty-reg.AppSpecificScoreFunc()(peerID) < 0.1 + }, 2*time.Second, 200*time.Millisecond) +} + // withStakedIdentities returns a function that sets the identity provider to return staked identities for the given peer ids. // It is used for testing purposes, and causes the given peer id to benefit from the staked identity reward in GossipSub. func withStakedIdentities(peerIds ...peer.ID) func(cfg *scoring.GossipSubAppSpecificScoreRegistryConfig) { @@ -905,12 +1047,6 @@ func withInvalidSubscriptions(peer peer.ID) func(cfg *scoring.GossipSubAppSpecif } } -func withInitFunction(initFunction scoring.SpamRecordInitFunc) func(cfg *scoring.GossipSubAppSpecificScoreRegistryConfig) { - return func(cfg *scoring.GossipSubAppSpecificScoreRegistryConfig) { - cfg.Init = initFunction - } -} - // newGossipSubAppSpecificScoreRegistry creates a new instance of GossipSubAppSpecificScoreRegistry along with its associated // GossipSubSpamRecordCache and AppSpecificScoreCache. This function is primarily used in testing scenarios to set up a controlled // environment for evaluating the behavior of the GossipSub scoring mechanism. @@ -921,6 +1057,8 @@ func withInitFunction(initFunction scoring.SpamRecordInitFunc) func(cfg *scoring // // Parameters: // - t *testing.T: The test context, used for asserting the absence of errors during the setup. +// - params p2pconfig.ScoringParameters: The scoring parameters used to configure the registry. +// - initFunction scoring.SpamRecordInitFunc: The function used to initialize the spam records. // - opts ...func(*scoring.GossipSubAppSpecificScoreRegistryConfig): A variadic set of functions that modify the registry's configuration. // // Returns: @@ -932,13 +1070,17 @@ func withInitFunction(initFunction scoring.SpamRecordInitFunc) func(cfg *scoring // and an application-specific score cache with predefined sizes and functionalities. The function also configures the scoring parameters // with test-specific values, particularly modifying the ScoreTTL value for the purpose of the tests. The creation and configuration of // the GossipSubAppSpecificScoreRegistry are validated to ensure no errors occur during the process. -func newGossipSubAppSpecificScoreRegistry(t *testing.T, params p2pconfig.ScoringParameters, opts ...func(*scoring.GossipSubAppSpecificScoreRegistryConfig)) (*scoring.GossipSubAppSpecificScoreRegistry, +func newGossipSubAppSpecificScoreRegistry(t *testing.T, + params p2pconfig.ScoringParameters, + initFunction scoring.SpamRecordInitFunc, + opts ...func(*scoring.GossipSubAppSpecificScoreRegistryConfig)) (*scoring.GossipSubAppSpecificScoreRegistry, *netcache.GossipSubSpamRecordCache, *internal.AppSpecificScoreCache) { cache := netcache.NewGossipSubSpamRecordCache(100, unittest.Logger(), metrics.NewNoopCollector(), - scoring.DefaultDecayFunction(params.SpamRecordCache.PenaltyDecaySlowdownThreshold, params.SpamRecordCache.DecayRateReductionFactor, params.SpamRecordCache.PenaltyDecayEvaluationPeriod)) + initFunction, + scoring.DefaultDecayFunction(params.ScoringRegistryParameters.SpamRecordCache.Decay)) appSpecificScoreCache := internal.NewAppSpecificScoreCache(100, unittest.Logger(), metrics.NewNoopCollector()) validator := mockp2p.NewSubscriptionValidator(t) @@ -952,7 +1094,6 @@ func newGossipSubAppSpecificScoreRegistry(t *testing.T, params p2pconfig.Scoring validator.On("Done").Return(f()).Maybe() cfg := &scoring.GossipSubAppSpecificScoreRegistryConfig{ Logger: unittest.Logger(), - Init: scoring.InitAppScoreRecordState, Penalty: penaltyValueFixtures(), IdProvider: mock.NewIdentityProvider(t), Validator: validator, @@ -962,9 +1103,11 @@ func newGossipSubAppSpecificScoreRegistry(t *testing.T, params p2pconfig.Scoring SpamRecordCacheFactory: func() p2p.GossipSubSpamRecordCache { return cache }, - Parameters: params.AppSpecificScore, - HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), - NetworkingType: network.PrivateNetwork, + Parameters: params.ScoringRegistryParameters.AppSpecificScore, + HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), + NetworkingType: network.PrivateNetwork, + AppSpecificScoreParams: params.PeerScoring.Protocol.AppSpecificScore, + ScoringRegistryStartupSilenceDuration: 0, // turn off silence period by default } for _, opt := range opts { opt(cfg) @@ -979,14 +1122,14 @@ func newGossipSubAppSpecificScoreRegistry(t *testing.T, params p2pconfig.Scoring // penaltyValueFixtures returns a set of penalty values for testing purposes. // The values are not realistic. The important thing is that they are different from each other. This is to make sure // that the tests are not passing because of the default values. -func penaltyValueFixtures() scoring.GossipSubCtrlMsgPenaltyValue { - return scoring.GossipSubCtrlMsgPenaltyValue{ - Graft: -100, - Prune: -50, - IHave: -20, - IWant: -10, - ClusterPrefixedPenaltyReductionFactor: .5, - RpcPublishMessage: -10, +func penaltyValueFixtures() p2pconfig.MisbehaviourPenalties { + return p2pconfig.MisbehaviourPenalties{ + GraftMisbehaviour: -100, + PruneMisbehaviour: -50, + IHaveMisbehaviour: -20, + IWantMisbehaviour: -10, + ClusterPrefixedReductionFactor: .5, + PublishMisbehaviour: -10, } } @@ -995,16 +1138,21 @@ func penaltyValueFixture(msgType p2pmsg.ControlMessageType) float64 { penaltyValues := penaltyValueFixtures() switch msgType { case p2pmsg.CtrlMsgGraft: - return penaltyValues.Graft + return penaltyValues.GraftMisbehaviour case p2pmsg.CtrlMsgPrune: - return penaltyValues.Prune + return penaltyValues.PruneMisbehaviour case p2pmsg.CtrlMsgIHave: - return penaltyValues.IHave + return penaltyValues.IHaveMisbehaviour case p2pmsg.CtrlMsgIWant: - return penaltyValues.IWant + return penaltyValues.IWantMisbehaviour case p2pmsg.RpcPublishMessage: - return penaltyValues.RpcPublishMessage + return penaltyValues.PublishMisbehaviour default: - return penaltyValues.ClusterPrefixedPenaltyReductionFactor + return penaltyValues.ClusterPrefixedReductionFactor } } + +func stopRegistry(t *testing.T, cancel context.CancelFunc, registry *scoring.GossipSubAppSpecificScoreRegistry) { + cancel() + unittest.RequireCloseBefore(t, registry.Done(), 5*time.Second, "registry did not stop") +} diff --git a/network/p2p/scoring/score_option.go b/network/p2p/scoring/score_option.go index a9bc3c90f91..07b948d975e 100644 --- a/network/p2p/scoring/score_option.go +++ b/network/p2p/scoring/score_option.go @@ -22,287 +22,17 @@ import ( "github.com/onflow/flow-go/utils/logging" ) -const ( - // DefaultAppSpecificScoreWeight is the default weight for app-specific scores. It is used to scale the app-specific - // scores to the same range as the other scores. At the current version, we don't distinguish between the app-specific - // scores and the other scores, so we set it to 1. - DefaultAppSpecificScoreWeight = 1 - - // MaxAppSpecificReward is the default reward for well-behaving staked peers. If a peer does not have - // any misbehavior record, e.g., invalid subscription, invalid message, etc., it will be rewarded with this score. - MaxAppSpecificReward = float64(100) - - // MaxAppSpecificPenalty is the maximum penalty for sever offenses that we apply to a remote node score. The score - // mechanism of GossipSub in Flow is designed in a way that all other infractions are penalized with a fraction of - // this value. We have also set the other parameters such as DefaultGraylistThreshold, DefaultGossipThreshold and DefaultPublishThreshold to - // be a bit higher than this, i.e., MaxAppSpecificPenalty + 1. This ensures that a node with a score of MaxAppSpecificPenalty - // will be graylisted (i.e., all incoming and outgoing RPCs are rejected) and will not be able to publish or gossip any messages. - MaxAppSpecificPenalty = -1 * MaxAppSpecificReward - MinAppSpecificPenalty = -1 - - // DefaultStakedIdentityReward is the default reward for staking peers. It is applied to the peer's score when - // the peer does not have any misbehavior record, e.g., invalid subscription, invalid message, etc. - // The purpose is to reward the staking peers for their contribution to the network and prioritize them in neighbor selection. - DefaultStakedIdentityReward = MaxAppSpecificReward - - // DefaultUnknownIdentityPenalty is the default penalty for unknown identity. It is applied to the peer's score when - // the peer is not in the identity list. - DefaultUnknownIdentityPenalty = MaxAppSpecificPenalty - - // DefaultInvalidSubscriptionPenalty is the default penalty for invalid subscription. It is applied to the peer's score when - // the peer subscribes to a topic that it is not authorized to subscribe to. - DefaultInvalidSubscriptionPenalty = MaxAppSpecificPenalty - - // DefaultGossipThreshold when a peer's penalty drops below this threshold, - // no gossip is emitted towards that peer and gossip from that peer is ignored. - // - // Validation Constraint: GossipThreshold >= PublishThreshold && GossipThreshold < 0 - // - // How we use it: - // As current max penalty is -100, we set the threshold to -99 so that all gossips - // to and from peers with penalty -100 are ignored. - DefaultGossipThreshold = MaxAppSpecificPenalty + 1 - - // DefaultPublishThreshold when a peer's penalty drops below this threshold, - // self-published messages are not propagated towards this peer. - // - // Validation Constraint: - // PublishThreshold >= GraylistThreshold && PublishThreshold <= GossipThreshold && PublishThreshold < 0. - // - // How we use it: - // As current max penalty is -100, we set the threshold to -99 so that all penalized peers are deprived of - // receiving any published messages. - DefaultPublishThreshold = MaxAppSpecificPenalty + 1 - - // DefaultGraylistThreshold when a peer's penalty drops below this threshold, the peer is graylisted, i.e., - // incoming RPCs from the peer are ignored. - // - // Validation Constraint: - // GraylistThreshold =< PublishThreshold && GraylistThreshold =< GossipThreshold && GraylistThreshold < 0 - // - // How we use it: - // As current max penalty is -100, we set the threshold to -99 so that all penalized peers are graylisted. - DefaultGraylistThreshold = MaxAppSpecificPenalty + 1 - - // DefaultAcceptPXThreshold when a peer sends us PX information with a prune, we only accept it and connect to the supplied - // peers if the originating peer's penalty exceeds this threshold. - // - // Validation Constraint: must be non-negative. - // - // How we use it: - // As current max reward is 100, we set the threshold to 99 so that we only receive supplied peers from - // well-behaved peers. - DefaultAcceptPXThreshold = MaxAppSpecificReward - 1 - - // DefaultOpportunisticGraftThreshold when the median peer penalty in the mesh drops below this value, - // the peer may select more peers with penalty above the median to opportunistically graft on the mesh. - // - // Validation Constraint: must be non-negative. - // - // How we use it: - // We set it to the MaxAppSpecificReward + 1 so that we only opportunistically graft peers that are not access nodes (i.e., with MinAppSpecificPenalty), - // or penalized peers (i.e., with MaxAppSpecificPenalty). - DefaultOpportunisticGraftThreshold = MaxAppSpecificReward + 1 - - // MaxDebugLogs sets the max number of debug/trace log events per second. Logs emitted above - // this threshold are dropped. - MaxDebugLogs = 50 - - // defaultDecayInterval is the default decay interval for the overall score of a peer at the GossipSub scoring - // system. We set it to 1 minute so that it is not too short so that a malicious node can recover from a penalty - // and is not too long so that a well-behaved node can't recover from a penalty. - defaultDecayInterval = 1 * time.Minute - - // defaultDecayToZero is the default decay to zero for the overall score of a peer at the GossipSub scoring system. - // It defines the maximum value below which a peer scoring counter is reset to zero. - // This is to prevent the counter from decaying to a very small value. - // The default value is 0.01, which means that a counter will be reset to zero if it decays to 0.01. - // When a counter hits the DecayToZero threshold, it means that the peer did not exhibit the behavior - // for a long time, and we can reset the counter. - defaultDecayToZero = 0.01 - - // defaultTopicSkipAtomicValidation is the default value for the skip atomic validation flag for topics. - // We set it to true, which means gossipsub parameter validation will not fail if we leave some of the - // topic parameters at their default values, i.e., zero. This is because we are not setting all - // topic parameters at the current implementation. - defaultTopicSkipAtomicValidation = true - - // defaultTopicInvalidMessageDeliveriesWeight this value is applied to the square of the number of invalid message deliveries on a topic. - // It is used to penalize peers that send invalid messages. By an invalid message, we mean a message that is not signed by the - // publisher, or a message that is not signed by the peer that sent it. We set it to -1.0, which means that with around 14 invalid - // message deliveries within a gossipsub heartbeat interval, the peer will be disconnected. - // The supporting math is as follows: - // - each staked (i.e., authorized) peer is rewarded by the fixed reward of 100 (i.e., DefaultStakedIdentityReward). - // - x invalid message deliveries will result in a penalty of x^2 * DefaultTopicInvalidMessageDeliveriesWeight, i.e., -x^2. - // - the peer will be disconnected when its penalty reaches -100 (i.e., MaxAppSpecificPenalty). - // - so, the maximum number of invalid message deliveries that a peer can have before being disconnected is sqrt(200/DefaultTopicInvalidMessageDeliveriesWeight) ~ 14. - defaultTopicInvalidMessageDeliveriesWeight = -1.0 - - // defaultTopicInvalidMessageDeliveriesDecay decay factor used to decay the number of invalid message deliveries. - // The total number of invalid message deliveries is multiplied by this factor at each heartbeat interval to - // decay the number of invalid message deliveries, and prevent the peer from being disconnected if it stops - // sending invalid messages. We set it to 0.99, which means that the number of invalid message deliveries will - // decay by 1% at each heartbeat interval. - // The decay heartbeats are defined by the heartbeat interval of the gossipsub scoring system, which is 1 Minute (defaultDecayInterval). - defaultTopicInvalidMessageDeliveriesDecay = .99 - - // defaultTopicTimeInMeshQuantum is the default time in mesh quantum for the GossipSub scoring system. It is used to gauge - // a discrete time interval for the time in mesh counter. We set it to 1 hour, which means that every one complete hour a peer is - // in a topic mesh, the time in mesh counter will be incremented by 1 and is counted towards the availability score of the peer in that topic mesh. - // The reason of setting it to 1 hour is that we want to reward peers that are in a topic mesh for a long time, and we want to avoid rewarding peers that - // are churners, i.e., peers that join and leave a topic mesh frequently. - defaultTopicTimeInMesh = time.Hour - - // defaultTopicWeight is the default weight of a topic in the GossipSub scoring system. - // The overall score of a peer in a topic mesh is multiplied by the weight of the topic when calculating the overall score of the peer. - // We set it to 1.0, which means that the overall score of a peer in a topic mesh is not affected by the weight of the topic. - defaultTopicWeight = 1.0 - - // defaultTopicMeshMessageDeliveriesDecay is applied to the number of actual message deliveries in a topic mesh - // at each decay interval (i.e., defaultDecayInterval). - // It is used to decay the number of actual message deliveries, and prevents past message - // deliveries from affecting the current score of the peer. - // As the decay interval is 1 minute, we set it to 0.5, which means that the number of actual message - // deliveries will decay by 50% at each decay interval. - defaultTopicMeshMessageDeliveriesDecay = .5 - - // defaultTopicMeshMessageDeliveriesCap is the maximum number of actual message deliveries in a topic - // mesh that is used to calculate the score of a peer in that topic mesh. - // We set it to 1000, which means that the maximum number of actual message deliveries in a - // topic mesh that is used to calculate the score of a peer in that topic mesh is 1000. - // This is to prevent the score of a peer in a topic mesh from being affected by a large number of actual - // message deliveries and also affect the score of the peer in other topic meshes. - // When the total delivered messages in a topic mesh exceeds this value, the score of the peer in that topic - // mesh will not be affected by the actual message deliveries in that topic mesh. - // Moreover, this does not allow the peer to accumulate a large number of actual message deliveries in a topic mesh - // and then start under-performing in that topic mesh without being penalized. - defaultTopicMeshMessageDeliveriesCap = 1000 - - // defaultTopicMeshMessageDeliveriesThreshold is the threshold for the number of actual message deliveries in a - // topic mesh that is used to calculate the score of a peer in that topic mesh. - // If the number of actual message deliveries in a topic mesh is less than this value, - // the peer will be penalized by square of the difference between the actual message deliveries and the threshold, - // i.e., -w * (actual - threshold)^2 where `actual` and `threshold` are the actual message deliveries and the - // threshold, respectively, and `w` is the weight (i.e., defaultTopicMeshMessageDeliveriesWeight). - // We set it to 0.1 * defaultTopicMeshMessageDeliveriesCap, which means that if a peer delivers less tha 10% of the - // maximum number of actual message deliveries in a topic mesh, it will be considered as an under-performing peer - // in that topic mesh. - defaultTopicMeshMessageDeliveryThreshold = 0.1 * defaultTopicMeshMessageDeliveriesCap - - // defaultTopicMeshDeliveriesWeight is the weight for applying penalty when a peer is under-performing in a topic mesh. - // Upon every decay interval, if the number of actual message deliveries is less than the topic mesh message deliveries threshold - // (i.e., defaultTopicMeshMessageDeliveriesThreshold), the peer will be penalized by square of the difference between the actual - // message deliveries and the threshold, multiplied by this weight, i.e., -w * (actual - threshold)^2 where w is the weight, and - // `actual` and `threshold` are the actual message deliveries and the threshold, respectively. - // We set this value to be - 0.05 MaxAppSpecificReward / (defaultTopicMeshMessageDeliveriesThreshold^2). This guarantees that even if a peer - // is not delivering any message in a topic mesh, it will not be disconnected. - // Rather, looses part of the MaxAppSpecificReward that is awarded by our app-specific scoring function to all staked - // nodes by default will be withdrawn, and the peer will be slightly penalized. In other words, under-performing in a topic mesh - // will drop the overall score of a peer by 5% of the MaxAppSpecificReward that is awarded by our app-specific scoring function. - // It means that under-performing in a topic mesh will not cause a peer to be disconnected, but it will cause the peer to lose - // its MaxAppSpecificReward that is awarded by our app-specific scoring function. - // At this point, we do not want to disconnect a peer only because it is under-performing in a topic mesh as it might be - // causing a false positive network partition. - // TODO: we must increase the penalty for under-performing in a topic mesh in the future, and disconnect the peer if it is under-performing. - defaultTopicMeshMessageDeliveriesWeight = -0.05 * MaxAppSpecificReward / (defaultTopicMeshMessageDeliveryThreshold * defaultTopicMeshMessageDeliveryThreshold) - - // defaultMeshMessageDeliveriesWindow is the window size is time interval that we count a delivery of an already - // seen message towards the score of a peer in a topic mesh. The delivery is counted - // by GossipSub only if the previous sender of the message is different from the current sender. - // We set it to the decay interval of the GossipSub scoring system, which is 1 minute. - // It means that if a peer delivers a message that it has already seen less than one minute ago, - // the delivery will be counted towards the score of the peer in a topic mesh only if the previous sender of the message. - // This also prevents replay attacks of messages that are older than one minute. As replayed messages will not - // be counted towards the actual message deliveries of a peer in a topic mesh. - defaultMeshMessageDeliveriesWindow = defaultDecayInterval - - // defaultMeshMessageDeliveryActivation is the time interval that we wait for a new peer that joins a topic mesh - // till start counting the number of actual message deliveries of that peer in that topic mesh. - // We set it to 2 * defaultDecayInterval, which means that we wait for 2 decay intervals before start counting - // the number of actual message deliveries of a peer in a topic mesh. - // With a default decay interval of 1 minute, it means that we wait for 2 minutes before start counting the - // number of actual message deliveries of a peer in a topic mesh. This is to account for - // the time that it takes for a peer to start up and receive messages from other peers in the topic mesh. - defaultMeshMessageDeliveriesActivation = 2 * defaultDecayInterval - - // defaultBehaviorPenaltyThreshold is the threshold when the behavior of a peer is considered as bad by GossipSub. - // Currently, the misbehavior is defined as advertising an iHave without responding to the iWants (iHave broken promises), as well as attempting - // on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh - // for a while, and the remote peer keep attempting on GRAFT (aka GRAFT flood). - // When the misbehavior counter of a peer goes beyond this threshold, the peer is penalized by defaultBehaviorPenaltyWeight (see below) for the excess misbehavior. - // - // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. - // For iHave broken promises, the gossipsub scoring works as follows: - // It samples ONLY A SINGLE iHave out of the entire RPC. - // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. - // - // We set it to 10, meaning that we at most tolerate 10 of such RPCs containing iHave broken promises. After that, the peer is penalized for every - // excess RPC containing iHave broken promises. - // The counter is also decayed by (0.99) every decay interval (defaultDecayInterval) i.e., every minute. - // Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through - // the ALSP system). - defaultBehaviourPenaltyThreshold = 10 - - // defaultBehaviorPenaltyWeight is the weight for applying penalty when a peer misbehavior goes beyond the threshold. - // Misbehavior of a peer at gossipsub layer is defined as advertising an iHave without responding to the iWants (broken promises), as well as attempting - // on GRAFT when the peer is considered for a PRUNE backoff, i.e., the local peer does not allow the peer to join the local topic mesh - // This is detected by the GossipSub scoring system, and the peer is penalized by defaultBehaviorPenaltyWeight. - // - // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. - // For iHave broken promises, the gossipsub scoring works as follows: - // It samples ONLY A SINGLE iHave out of the entire RPC. - // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. - // - // The penalty is applied to the square of the difference between the misbehavior counter and the threshold, i.e., -|w| * (misbehavior counter - threshold)^2. - // We set it to 0.01 * MaxAppSpecificPenalty, which means that misbehaving 10 times more than the threshold (i.e., 10 + 10) will cause the peer to lose - // its entire AppSpecificReward that is awarded by our app-specific scoring function to all staked (i.e., authorized) nodes by default. - // Moreover, as the MaxAppSpecificPenalty is -MaxAppSpecificReward, misbehaving sqrt(2) * 10 times more than the threshold will cause the peer score - // to be dropped below the MaxAppSpecificPenalty, which is also below the GraylistThreshold, and the peer will be graylisted (i.e., disconnected). - // - // The math is as follows: -|w| * (misbehavior - threshold)^2 = 0.01 * MaxAppSpecificPenalty * (misbehavior - threshold)^2 < 2 * MaxAppSpecificPenalty - // if misbehavior > threshold + sqrt(2) * 10. - // As shown above, with this choice of defaultBehaviorPenaltyWeight, misbehaving sqrt(2) * 10 times more than the threshold will cause the peer score - // to be dropped below the MaxAppSpecificPenalty, which is also below the GraylistThreshold, and the peer will be graylisted (i.e., disconnected). This weight - // is chosen in a way that with almost a few misbehaviors more than the threshold, the peer will be graylisted. The rationale relies on the fact that - // the misbehavior counter is incremented by 1 for each RPC containing one or more broken promises. Hence, it is per RPC, and not per broken promise. - // Having sqrt(2) * 10 broken promises RPC is a blatant misbehavior, and the peer should be graylisted. With decay interval of 1 minute, and decay value of - // 0.99 we expect a graylisted node due to borken promises to get back in about 527 minutes, i.e., (0.99)^x * (sqrt(2) * 10)^2 * MaxAppSpecificPenalty > GraylistThreshold - // where x is the number of decay intervals that the peer is graylisted. As MaxAppSpecificPenalty and GraylistThresholds are close, we can simplify the inequality - // to (0.99)^x * (sqrt(2) * 10)^2 > 1 --> (0.99)^x * 200 > 1 --> (0.99)^x > 1/200 --> x > log(1/200) / log(0.99) --> x > 527.17 decay intervals, i.e., 527 minutes. - // Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through - // the ALSP system that are reported by the engines). - defaultBehaviourPenaltyWeight = 0.01 * MaxAppSpecificPenalty - - // defaultBehaviorPenaltyDecay is the decay interval for the misbehavior counter of a peer. The misbehavior counter is - // incremented by GossipSub for iHave broken promises or the GRAFT flooding attacks (i.e., each GRAFT received from a remote peer while that peer is on a PRUNE backoff). - // - // An iHave broken promise means that a peer advertises an iHave for a message, but does not respond to the iWant requests for that message. - // For iHave broken promises, the gossipsub scoring works as follows: - // It samples ONLY A SINGLE iHave out of the entire RPC. - // If that iHave is not followed by an actual message within the next 3 seconds, the peer misbehavior counter is incremented by 1. - // This means that regardless of how many iHave broken promises an RPC contains, the misbehavior counter is incremented by 1. - // That is why we decay the misbehavior counter very slow, as this counter indicates a severe misbehavior. - // - // The misbehavior counter is decayed per decay interval (i.e., defaultDecayInterval = 1 minute) by GossipSub. - // We set it to 0.99, which means that the misbehavior counter is decayed by 1% per decay interval. - // With the generous threshold that we set (i.e., defaultBehaviourPenaltyThreshold = 10), we take the peers going beyond the threshold as persistent misbehaviors, - // We expect honest peers never to go beyond the threshold, and if they do, we expect them to go back below the threshold quickly. - // - // Note that misbehaviors are counted by GossipSub across all topics (and is different from the Application Layer Misbehaviors that we count through - // the ALSP system that is based on the engines report). - defaultBehaviourPenaltyDecay = 0.99 -) - // ScoreOption is a functional option for configuring the peer scoring system. // TODO: rename it to ScoreManager. type ScoreOption struct { component.Component logger zerolog.Logger - peerScoreParams *pubsub.PeerScoreParams - peerThresholdParams *pubsub.PeerScoreThresholds - validator p2p.SubscriptionValidator - appScoreFunc func(peer.ID) float64 + peerScoreParams *pubsub.PeerScoreParams + peerThresholdParams *pubsub.PeerScoreThresholds + defaultTopicScoreParams *pubsub.TopicScoreParams + validator p2p.SubscriptionValidator + appScoreFunc func(peer.ID) float64 } type ScoreOptionConfig struct { @@ -366,7 +96,7 @@ func (c *ScoreOptionConfig) SetRegisterNotificationConsumerFunc(f func(p2p.Gossi // NewScoreOption creates a new penalty option with the given configuration. func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) (*ScoreOption, error) { - throttledSampler := logging.BurstSampler(MaxDebugLogs, time.Second) + throttledSampler := logging.BurstSampler(cfg.params.PeerScoring.Protocol.MaxDebugLogs, time.Second) logger := cfg.logger.With(). Str("module", "pubsub_score_option"). Logger(). @@ -374,38 +104,79 @@ func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) ( TraceSampler: throttledSampler, DebugSampler: throttledSampler, }) + validator := NewSubscriptionValidator(cfg.logger, provider) scoreRegistry, err := NewGossipSubAppSpecificScoreRegistry(&GossipSubAppSpecificScoreRegistryConfig{ Logger: logger, - Penalty: DefaultGossipSubCtrlMsgPenaltyValue(), + Penalty: cfg.params.ScoringRegistryParameters.MisbehaviourPenalties, Validator: validator, - Init: InitAppScoreRecordState, IdProvider: cfg.provider, HeroCacheMetricsFactory: cfg.heroCacheMetricsFactory, AppScoreCacheFactory: func() p2p.GossipSubApplicationSpecificScoreCache { collector := metrics.NewGossipSubApplicationSpecificScoreCacheMetrics(cfg.heroCacheMetricsFactory, cfg.networkingType) - return internal.NewAppSpecificScoreCache(cfg.params.SpamRecordCache.CacheSize, cfg.logger, collector) + return internal.NewAppSpecificScoreCache(cfg.params.ScoringRegistryParameters.SpamRecordCache.CacheSize, cfg.logger, collector) }, SpamRecordCacheFactory: func() p2p.GossipSubSpamRecordCache { collector := metrics.GossipSubSpamRecordCacheMetricsFactory(cfg.heroCacheMetricsFactory, cfg.networkingType) - return netcache.NewGossipSubSpamRecordCache(cfg.params.SpamRecordCache.CacheSize, cfg.logger, collector, - DefaultDecayFunction( - cfg.params.SpamRecordCache.PenaltyDecaySlowdownThreshold, - cfg.params.SpamRecordCache.DecayRateReductionFactor, - cfg.params.SpamRecordCache.PenaltyDecayEvaluationPeriod)) + return netcache.NewGossipSubSpamRecordCache(cfg.params.ScoringRegistryParameters.SpamRecordCache.CacheSize, cfg.logger, collector, + InitAppScoreRecordStateFunc(cfg.params.ScoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor), + DefaultDecayFunction(cfg.params.ScoringRegistryParameters.SpamRecordCache.Decay)) }, - Parameters: cfg.params.AppSpecificScore, - NetworkingType: cfg.networkingType, + Parameters: cfg.params.ScoringRegistryParameters.AppSpecificScore, + NetworkingType: cfg.networkingType, + AppSpecificScoreParams: cfg.params.PeerScoring.Protocol.AppSpecificScore, }) if err != nil { return nil, fmt.Errorf("failed to create gossipsub app specific score registry: %w", err) } s := &ScoreOption{ - logger: logger, - validator: validator, - peerScoreParams: defaultPeerScoreParams(), - appScoreFunc: scoreRegistry.AppSpecificScoreFunc(), + logger: logger, + validator: validator, + peerScoreParams: &pubsub.PeerScoreParams{ + Topics: make(map[string]*pubsub.TopicScoreParams), + // we don't set all the parameters, so we skip the atomic validation. + // atomic validation fails initialization if any parameter is not set. + SkipAtomicValidation: cfg.params.PeerScoring.Internal.TopicParameters.SkipAtomicValidation, + // DecayInterval is the interval over which we decay the effect of past behavior, so that + // a good or bad behavior will not have a permanent effect on the penalty. It is also the interval + // that GossipSub uses to refresh the scores of all peers. + DecayInterval: cfg.params.PeerScoring.Internal.DecayInterval, + // DecayToZero defines the maximum value below which a peer scoring counter is reset to zero. + // This is to prevent the counter from decaying to a very small value. + // When a counter hits the DecayToZero threshold, it means that the peer did not exhibit the behavior + // for a long time, and we can reset the counter. + DecayToZero: cfg.params.PeerScoring.Internal.DecayToZero, + // AppSpecificWeight is the weight of the application specific penalty. + AppSpecificWeight: cfg.params.PeerScoring.Internal.AppSpecificScoreWeight, + // PenaltyThreshold is the threshold above which a peer is penalized for GossipSub-level misbehaviors. + BehaviourPenaltyThreshold: cfg.params.PeerScoring.Internal.Behaviour.PenaltyThreshold, + // PenaltyWeight is the weight of the GossipSub-level penalty. + BehaviourPenaltyWeight: cfg.params.PeerScoring.Internal.Behaviour.PenaltyWeight, + // PenaltyDecay is the decay of the GossipSub-level penalty (applied every decay interval). + BehaviourPenaltyDecay: cfg.params.PeerScoring.Internal.Behaviour.PenaltyDecay, + }, + peerThresholdParams: &pubsub.PeerScoreThresholds{ + GossipThreshold: cfg.params.PeerScoring.Internal.Thresholds.Gossip, + PublishThreshold: cfg.params.PeerScoring.Internal.Thresholds.Publish, + GraylistThreshold: cfg.params.PeerScoring.Internal.Thresholds.Graylist, + AcceptPXThreshold: cfg.params.PeerScoring.Internal.Thresholds.AcceptPX, + OpportunisticGraftThreshold: cfg.params.PeerScoring.Internal.Thresholds.OpportunisticGraft, + }, + defaultTopicScoreParams: &pubsub.TopicScoreParams{ + TopicWeight: cfg.params.PeerScoring.Internal.TopicParameters.TopicWeight, + SkipAtomicValidation: cfg.params.PeerScoring.Internal.TopicParameters.SkipAtomicValidation, + InvalidMessageDeliveriesWeight: cfg.params.PeerScoring.Internal.TopicParameters.InvalidMessageDeliveriesWeight, + InvalidMessageDeliveriesDecay: cfg.params.PeerScoring.Internal.TopicParameters.InvalidMessageDeliveriesDecay, + TimeInMeshQuantum: cfg.params.PeerScoring.Internal.TopicParameters.TimeInMeshQuantum, + MeshMessageDeliveriesWeight: cfg.params.PeerScoring.Internal.TopicParameters.MeshDeliveriesWeight, + MeshMessageDeliveriesDecay: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesDecay, + MeshMessageDeliveriesCap: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesCap, + MeshMessageDeliveriesThreshold: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveryThreshold, + MeshMessageDeliveriesWindow: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesWindow, + MeshMessageDeliveriesActivation: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveryActivation, + }, + appScoreFunc: scoreRegistry.AppSpecificScoreFunc(), } // set the app specific penalty function for the penalty option @@ -418,13 +189,13 @@ func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) ( Msg("app specific score function is overridden, should never happen in production") } - if cfg.params.DecayInterval > 0 && cfg.params.DecayInterval != s.peerScoreParams.DecayInterval { + if cfg.params.PeerScoring.Internal.DecayInterval > 0 && cfg.params.PeerScoring.Internal.DecayInterval != s.peerScoreParams.DecayInterval { // overrides the default decay interval if the decay interval is set. - s.peerScoreParams.DecayInterval = cfg.params.DecayInterval + s.peerScoreParams.DecayInterval = cfg.params.PeerScoring.Internal.DecayInterval s.logger. Warn(). Str(logging.KeyNetworkingSecurity, "true"). - Dur("decay_interval_ms", cfg.params.DecayInterval). + Dur("decay_interval_ms", cfg.params.PeerScoring.Internal.DecayInterval). Msg("decay interval is overridden, should never happen in production") } @@ -462,8 +233,6 @@ func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) ( } func (s *ScoreOption) BuildFlowPubSubScoreOption() (*pubsub.PeerScoreParams, *pubsub.PeerScoreThresholds) { - s.preparePeerScoreThresholds() - s.logger.Info(). Float64("gossip_threshold", s.peerThresholdParams.GossipThreshold). Float64("publish_threshold", s.peerThresholdParams.PublishThreshold). @@ -481,16 +250,6 @@ func (s *ScoreOption) BuildFlowPubSubScoreOption() (*pubsub.PeerScoreParams, *pu return s.peerScoreParams, s.peerThresholdParams } -func (s *ScoreOption) preparePeerScoreThresholds() { - s.peerThresholdParams = &pubsub.PeerScoreThresholds{ - GossipThreshold: DefaultGossipThreshold, - PublishThreshold: DefaultPublishThreshold, - GraylistThreshold: DefaultGraylistThreshold, - AcceptPXThreshold: DefaultAcceptPXThreshold, - OpportunisticGraftThreshold: DefaultOpportunisticGraftThreshold, - } -} - // TopicScoreParams returns the topic score parameters for the given topic. If the topic // score parameters are not set, it returns the default topic score parameters. // The custom topic parameters are set at the initialization of the score option. @@ -502,59 +261,7 @@ func (s *ScoreOption) preparePeerScoreThresholds() { func (s *ScoreOption) TopicScoreParams(topic *pubsub.Topic) *pubsub.TopicScoreParams { params, exists := s.peerScoreParams.Topics[topic.String()] if !exists { - return DefaultTopicScoreParams() + return s.defaultTopicScoreParams } return params } - -func defaultPeerScoreParams() *pubsub.PeerScoreParams { - // DO NOT CHANGE THE DEFAULT VALUES, THEY ARE TUNED FOR THE BEST SECURITY PRACTICES. - return &pubsub.PeerScoreParams{ - Topics: make(map[string]*pubsub.TopicScoreParams), - // we don't set all the parameters, so we skip the atomic validation. - // atomic validation fails initialization if any parameter is not set. - SkipAtomicValidation: true, - // DecayInterval is the interval over which we decay the effect of past behavior, so that - // a good or bad behavior will not have a permanent effect on the penalty. It is also the interval - // that GossipSub uses to refresh the scores of all peers. - DecayInterval: defaultDecayInterval, - // DecayToZero defines the maximum value below which a peer scoring counter is reset to zero. - // This is to prevent the counter from decaying to a very small value. - // When a counter hits the DecayToZero threshold, it means that the peer did not exhibit the behavior - // for a long time, and we can reset the counter. - DecayToZero: defaultDecayToZero, - // AppSpecificWeight is the weight of the application specific penalty. - AppSpecificWeight: DefaultAppSpecificScoreWeight, - // BehaviourPenaltyThreshold is the threshold above which a peer is penalized for GossipSub-level misbehaviors. - BehaviourPenaltyThreshold: defaultBehaviourPenaltyThreshold, - // BehaviourPenaltyWeight is the weight of the GossipSub-level penalty. - BehaviourPenaltyWeight: defaultBehaviourPenaltyWeight, - // BehaviourPenaltyDecay is the decay of the GossipSub-level penalty (applied every decay interval). - BehaviourPenaltyDecay: defaultBehaviourPenaltyDecay, - } -} - -// DefaultTopicScoreParams returns the default score params for topics. -func DefaultTopicScoreParams() *pubsub.TopicScoreParams { - // DO NOT CHANGE THE DEFAULT VALUES, THEY ARE TUNED FOR THE BEST SECURITY PRACTICES. - p := &pubsub.TopicScoreParams{ - TopicWeight: defaultTopicWeight, - SkipAtomicValidation: defaultTopicSkipAtomicValidation, - InvalidMessageDeliveriesWeight: defaultTopicInvalidMessageDeliveriesWeight, - InvalidMessageDeliveriesDecay: defaultTopicInvalidMessageDeliveriesDecay, - TimeInMeshQuantum: defaultTopicTimeInMesh, - MeshMessageDeliveriesWeight: defaultTopicMeshMessageDeliveriesWeight, - MeshMessageDeliveriesDecay: defaultTopicMeshMessageDeliveriesDecay, - MeshMessageDeliveriesCap: defaultTopicMeshMessageDeliveriesCap, - MeshMessageDeliveriesThreshold: defaultTopicMeshMessageDeliveryThreshold, - MeshMessageDeliveriesWindow: defaultMeshMessageDeliveriesWindow, - MeshMessageDeliveriesActivation: defaultMeshMessageDeliveriesActivation, - } - - if p.MeshMessageDeliveriesWeight >= 0 { - // GossipSub also does a validation, but we want to panic as early as possible. - panic(fmt.Sprintf("invalid mesh message deliveries weight %f", p.MeshMessageDeliveriesWeight)) - } - - return p -} diff --git a/network/p2p/scoring/scoring_test.go b/network/p2p/scoring/scoring_test.go index b71c662a300..cee819d3c85 100644 --- a/network/p2p/scoring/scoring_test.go +++ b/network/p2p/scoring/scoring_test.go @@ -104,7 +104,7 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { cfg, err := config.DefaultConfig() require.NoError(t, err) - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond // speed up the test + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond // speed up the test node1, id1 := p2ptest.NodeFixture( t, diff --git a/network/p2p/scoring/subscriptionCache.go b/network/p2p/scoring/subscriptionCache.go index 8eae60bd385..a58ab79db5c 100644 --- a/network/p2p/scoring/subscriptionCache.go +++ b/network/p2p/scoring/subscriptionCache.go @@ -31,5 +31,5 @@ type SubscriptionCache interface { // - []string: the list of topics the peer is subscribed to after the update. // - error: an error if the update failed; any returned error is an irrecoverable error and indicates a bug or misconfiguration. // Implementation must be thread-safe. - AddTopicForPeer(pid peer.ID, topic string) ([]string, error) + AddWithInitTopicForPeer(pid peer.ID, topic string) ([]string, error) } diff --git a/network/p2p/scoring/subscription_provider.go b/network/p2p/scoring/subscription_provider.go index 6c6b5a50b43..2bfd43bb870 100644 --- a/network/p2p/scoring/subscription_provider.go +++ b/network/p2p/scoring/subscription_provider.go @@ -134,7 +134,7 @@ func (s *SubscriptionProvider) updateTopics() error { continue } - updatedTopics, err := s.cache.AddTopicForPeer(p, topic) + updatedTopics, err := s.cache.AddWithInitTopicForPeer(p, topic) if err != nil { // this is an irrecoverable error; hence, we crash the node. return fmt.Errorf("failed to update topics for peer %s: %w", p, err) diff --git a/network/p2p/scoring/subscription_validator_test.go b/network/p2p/scoring/subscription_validator_test.go index b4edad1f04c..1a6a4b6bfcb 100644 --- a/network/p2p/scoring/subscription_validator_test.go +++ b/network/p2p/scoring/subscription_validator_test.go @@ -164,6 +164,7 @@ func TestSubscriptionValidator_InvalidSubscriptions(t *testing.T) { // 4. Verification node also publishes a chunk request on the RequestChunks channel. // 5. Test checks that consensus node does not receive the chunk request while the other verification node does. func TestSubscriptionValidator_Integration(t *testing.T) { + unittest.SkipUnless(t, unittest.TEST_FLAKY, "flaky test") ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) @@ -171,7 +172,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { require.NoError(t, err) // set a low update interval to speed up the test cfg.NetworkConfig.GossipSub.SubscriptionProvider.UpdateInterval = 10 * time.Millisecond - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond sporkId := unittest.IdentifierFixture() diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 8b8f33067d4..50fb4cb80ca 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -825,6 +825,22 @@ func MockInspectorNotificationDistributorReadyDoneAware(d *mockp2p.GossipSubInsp }()).Maybe() } +// MockScoringRegistrySubscriptionValidatorReadyDoneAware mocks the Ready and Done methods of the subscription validator to return a channel that is already closed, +// so that the distributor is considered ready and done when the test needs. +func MockScoringRegistrySubscriptionValidatorReadyDoneAware(s *mockp2p.SubscriptionValidator) { + s.On("Start", mockery.Anything).Return().Maybe() + s.On("Ready").Return(func() <-chan struct{} { + ch := make(chan struct{}) + close(ch) + return ch + }()).Maybe() + s.On("Done").Return(func() <-chan struct{} { + ch := make(chan struct{}) + close(ch) + return ch + }()).Maybe() +} + // GossipSubRpcFixtures returns a slice of random message IDs for testing. // Args: // - t: *testing.T instance diff --git a/network/p2p/tracer/gossipSubScoreTracer_test.go b/network/p2p/tracer/gossipSubScoreTracer_test.go index d12de3e634a..dab069024c9 100644 --- a/network/p2p/tracer/gossipSubScoreTracer_test.go +++ b/network/p2p/tracer/gossipSubScoreTracer_test.go @@ -80,7 +80,7 @@ func TestGossipSubScoreTracer(t *testing.T) { cfg.NetworkConfig.GossipSub.RpcTracer.ScoreTracerInterval = 1 * time.Second // the libp2p node updates the subscription list as well as the app-specific score every 10 milliseconds (for testing purposes) cfg.NetworkConfig.GossipSub.SubscriptionProvider.UpdateInterval = 10 * time.Millisecond - cfg.NetworkConfig.GossipSub.ScoringParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond + cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond tracerNode, tracerId := p2ptest.NodeFixture( t, sporkId, diff --git a/network/p2p/unicast/cache/unicastConfigCache.go b/network/p2p/unicast/cache/unicastConfigCache.go index 13c000110fe..52845d7f33a 100644 --- a/network/p2p/unicast/cache/unicastConfigCache.go +++ b/network/p2p/unicast/cache/unicastConfigCache.go @@ -2,7 +2,6 @@ package unicastcache import ( "fmt" - "sync" "github.com/libp2p/go-libp2p/core/peer" "github.com/rs/zerolog" @@ -15,13 +14,7 @@ import ( "github.com/onflow/flow-go/network/p2p/unicast" ) -// ErrUnicastConfigNotFound is a benign error that indicates that the unicast config does not exist in the cache. It is not a fatal error. -var ErrUnicastConfigNotFound = fmt.Errorf("unicast config not found") - type UnicastConfigCache struct { - // mutex is temporarily protect the edge case in HeroCache that optimistic adjustment causes the cache to be full. - // TODO: remove this mutex after the HeroCache is fixed. - mutex sync.RWMutex peerCache *stdmap.Backend cfgFactory func() unicast.Config // factory function that creates a new unicast config. } @@ -56,7 +49,7 @@ func NewUnicastConfigCache( } } -// Adjust applies the given adjust function to the unicast config of the given peer ID, and stores the adjusted config in the cache. +// AdjustWithInit applies the given adjust function to the unicast config of the given peer ID, and stores the adjusted config in the cache. // It returns an error if the adjustFunc returns an error. // Note that if the Adjust is called when the config does not exist, the config is initialized and the // adjust function is applied to the initialized config again. In this case, the adjust function should not return an error. @@ -65,51 +58,11 @@ func NewUnicastConfigCache( // - adjustFunc: the function that adjusts the unicast config. // Returns: // - error any returned error should be considered as an irrecoverable error and indicates a bug. -func (d *UnicastConfigCache) Adjust(peerID peer.ID, adjustFunc unicast.UnicastConfigAdjustFunc) (*unicast.Config, error) { - d.mutex.Lock() // making optimistic adjustment atomic. - defer d.mutex.Unlock() - - // first we translate the peer id to a flow id (taking - peerIdHash := PeerIdToFlowId(peerID) - adjustedUnicastCfg, err := d.adjust(peerIdHash, adjustFunc) - if err != nil { - if err == ErrUnicastConfigNotFound { - // if the config does not exist, we initialize the config and try to adjust it again. - // Note: there is an edge case where the config is initialized by another goroutine between the two calls. - // In this case, the init function is invoked twice, but it is not a problem because the underlying - // cache is thread-safe. Hence, we do not need to synchronize the two calls. In such cases, one of the - // two calls returns false, and the other call returns true. We do not care which call returns false, hence, - // we ignore the return value of the init function. - e := UnicastConfigEntity{ - PeerId: peerID, - Config: d.cfgFactory(), - } - - _ = d.peerCache.Add(e) - - // as the config is initialized, the adjust function should not return an error, and any returned error - // is an irrecoverable error and indicates a bug. - return d.adjust(peerIdHash, adjustFunc) - } - // if the adjust function returns an unexpected error on the first attempt, we return the error directly. - // any returned error should be considered as an irrecoverable error and indicates a bug. - return nil, fmt.Errorf("failed to adjust unicast config: %w", err) - } - // if the adjust function returns no error on the first attempt, we return the adjusted config. - return adjustedUnicastCfg, nil -} - -// adjust applies the given adjust function to the unicast config of the given origin id. -// It returns an error if the adjustFunc returns an error or if the config does not exist. -// Args: -// - peerIDHash: the hash value of the peer id of the unicast config (i.e., the ID of the unicast config entity). -// - adjustFunc: the function that adjusts the unicast config. -// Returns: -// - error if the adjustFunc returns an error or if the config does not exist (ErrUnicastConfigNotFound). Except the ErrUnicastConfigNotFound, -// any other error should be treated as an irrecoverable error and indicates a bug. -func (d *UnicastConfigCache) adjust(peerIdHash flow.Identifier, adjustFunc unicast.UnicastConfigAdjustFunc) (*unicast.Config, error) { +func (d *UnicastConfigCache) AdjustWithInit(peerID peer.ID, adjustFunc unicast.UnicastConfigAdjustFunc) (*unicast.Config, error) { + entityId := entityIdOf(peerID) var rErr error - adjustedEntity, adjusted := d.peerCache.Adjust(peerIdHash, func(entity flow.Entity) flow.Entity { + // wraps external adjust function to adjust the unicast config. + wrapAdjustFunc := func(entity flow.Entity) flow.Entity { cfgEntity, ok := entity.(UnicastConfigEntity) if !ok { // sanity check @@ -127,14 +80,23 @@ func (d *UnicastConfigCache) adjust(peerIdHash flow.Identifier, adjustFunc unica // Return the adjusted config. cfgEntity.Config = adjustedCfg return cfgEntity - }) + } + initFunc := func() flow.Entity { + return UnicastConfigEntity{ + PeerId: peerID, + Config: d.cfgFactory(), + EntityId: entityId, + } + } + + adjustedEntity, adjusted := d.peerCache.AdjustWithInit(entityId, wrapAdjustFunc, initFunc) if rErr != nil { - return nil, fmt.Errorf("failed to adjust config: %w", rErr) + return nil, fmt.Errorf("adjust operation aborted with an error: %w", rErr) } if !adjusted { - return nil, ErrUnicastConfigNotFound + return nil, fmt.Errorf("adjust operation aborted, entity not found") } return &unicast.Config{ @@ -143,37 +105,27 @@ func (d *UnicastConfigCache) adjust(peerIdHash flow.Identifier, adjustFunc unica }, nil } -// GetOrInit returns the unicast config for the given peer id. If the config does not exist, it creates a new config +// GetWithInit returns the unicast config for the given peer id. If the config does not exist, it creates a new config // using the factory function and stores it in the cache. // Args: // - peerID: the peer id of the unicast config. // Returns: // - *Config, the unicast config for the given peer id. // - error if the factory function returns an error. Any error should be treated as an irrecoverable error and indicates a bug. -func (d *UnicastConfigCache) GetOrInit(peerID peer.ID) (*unicast.Config, error) { - // first we translate the peer id to a flow id (taking - flowPeerId := PeerIdToFlowId(peerID) - cfg, ok := d.get(flowPeerId) - if !ok { - _ = d.peerCache.Add(UnicastConfigEntity{ - PeerId: peerID, - Config: d.cfgFactory(), - }) - cfg, ok = d.get(flowPeerId) - if !ok { - return nil, fmt.Errorf("failed to initialize unicast config for peer %s", peerID) +func (d *UnicastConfigCache) GetWithInit(peerID peer.ID) (*unicast.Config, error) { + // ensuring that the init-and-get operation is atomic. + entityId := entityIdOf(peerID) + initFunc := func() flow.Entity { + return UnicastConfigEntity{ + PeerId: peerID, + Config: d.cfgFactory(), + EntityId: entityId, } } - return cfg, nil -} - -// Get returns the unicast config of the given peer ID. -func (d *UnicastConfigCache) get(peerIDHash flow.Identifier) (*unicast.Config, bool) { - entity, ok := d.peerCache.ByID(peerIDHash) + entity, ok := d.peerCache.GetWithInit(entityId, initFunc) if !ok { - return nil, false + return nil, fmt.Errorf("get or init for unicast config for peer %s failed", peerID) } - cfg, ok := entity.(UnicastConfigEntity) if !ok { // sanity check @@ -185,7 +137,7 @@ func (d *UnicastConfigCache) get(peerIDHash flow.Identifier) (*unicast.Config, b return &unicast.Config{ StreamCreationRetryAttemptBudget: cfg.StreamCreationRetryAttemptBudget, ConsecutiveSuccessfulStream: cfg.ConsecutiveSuccessfulStream, - }, true + }, nil } // Size returns the number of unicast configs in the cache. diff --git a/network/p2p/unicast/cache/unicastConfigCache_test.go b/network/p2p/unicast/cache/unicastConfigCache_test.go index 4d07c9980d2..23d83f1a354 100644 --- a/network/p2p/unicast/cache/unicastConfigCache_test.go +++ b/network/p2p/unicast/cache/unicastConfigCache_test.go @@ -60,9 +60,9 @@ func TestUnicastConfigCache_Adjust_Init(t *testing.T) { peerID1 := unittest.PeerIdFixture(t) peerID2 := unittest.PeerIdFixture(t) - // Initializing the unicast config for peerID1 through GetOrInit. - // unicast config for peerID1 does not exist in the cache, so it must be initialized when using GetOrInit. - cfg, err := cache.GetOrInit(peerID1) + // Initializing the unicast config for peerID1 through GetWithInit. + // unicast config for peerID1 does not exist in the cache, so it must be initialized when using GetWithInit. + cfg, err := cache.GetWithInit(peerID1) require.NoError(t, err) require.NotNil(t, cfg, "unicast config must not be nil") require.Equal(t, unicastConfigFixture(), *cfg, "unicast config must be initialized with the default values") @@ -70,15 +70,15 @@ func TestUnicastConfigCache_Adjust_Init(t *testing.T) { // Initializing and adjusting the unicast config for peerID2 through Adjust. // unicast config for peerID2 does not exist in the cache, so it must be initialized when using Adjust. - cfg, err = cache.Adjust(peerID2, adjustFuncIncrement) + cfg, err = cache.AdjustWithInit(peerID2, adjustFuncIncrement) require.NoError(t, err) // adjusting a non-existing unicast config must not initialize the config. require.Equal(t, uint(2), cache.Size(), "cache size must be 2") require.Equal(t, cfg.StreamCreationRetryAttemptBudget, unicastConfigFixture().StreamCreationRetryAttemptBudget+1, "stream backoff must be 2") - // Retrieving the unicast config of peerID2 through GetOrInit. + // Retrieving the unicast config of peerID2 through GetWithInit. // retrieve the unicast config for peerID2 and assert than it is initialized with the default values; and the adjust function is applied. - cfg, err = cache.GetOrInit(peerID2) + cfg, err = cache.GetWithInit(peerID2) require.NoError(t, err, "unicast config must exist in the cache") require.NotNil(t, cfg, "unicast config must not be nil") // retrieving an existing unicast config must not change the cache size. @@ -88,7 +88,7 @@ func TestUnicastConfigCache_Adjust_Init(t *testing.T) { // Adjusting the unicast config of peerID1 through Adjust. // unicast config for peerID1 already exists in the cache, so it must be adjusted when using Adjust. - cfg, err = cache.Adjust(peerID1, adjustFuncIncrement) + cfg, err = cache.AdjustWithInit(peerID1, adjustFuncIncrement) require.NoError(t, err) // adjusting an existing unicast config must not change the cache size. require.Equal(t, uint(2), cache.Size(), "cache size must be 2") @@ -96,7 +96,7 @@ func TestUnicastConfigCache_Adjust_Init(t *testing.T) { // Recurring adjustment of the unicast config of peerID1 through Adjust. // unicast config for peerID1 already exists in the cache, so it must be adjusted when using Adjust. - cfg, err = cache.Adjust(peerID1, adjustFuncIncrement) + cfg, err = cache.AdjustWithInit(peerID1, adjustFuncIncrement) require.NoError(t, err) // adjusting an existing unicast config must not change the cache size. require.Equal(t, uint(2), cache.Size(), "cache size must be 2") @@ -130,7 +130,7 @@ func TestUnicastConfigCache_Concurrent_Adjust(t *testing.T) { wg.Add(1) go func(peerId peer.ID) { defer wg.Done() - _, err := cache.Adjust(peerId, func(cfg unicast.Config) (unicast.Config, error) { + _, err := cache.AdjustWithInit(peerId, func(cfg unicast.Config) (unicast.Config, error) { cfg.StreamCreationRetryAttemptBudget++ return cfg, nil }) @@ -151,7 +151,7 @@ func TestUnicastConfigCache_Concurrent_Adjust(t *testing.T) { wg.Done() peerID := peerIds[j] - cfg, err := cache.GetOrInit(peerID) + cfg, err := cache.GetWithInit(peerID) require.NoError(t, err) require.Equal(t, uint64(j+1), @@ -182,7 +182,7 @@ func TestConcurrent_Adjust_And_Get_Is_Safe(t *testing.T) { go func() { defer wg.Done() peerId := unittest.PeerIdFixture(t) - updatedConfig, err := cache.Adjust(peerId, func(cfg unicast.Config) (unicast.Config, error) { + updatedConfig, err := cache.AdjustWithInit(peerId, func(cfg unicast.Config) (unicast.Config, error) { cfg.StreamCreationRetryAttemptBudget = 2 // some random adjustment cfg.ConsecutiveSuccessfulStream = 3 // some random adjustment return cfg, nil @@ -199,7 +199,7 @@ func TestConcurrent_Adjust_And_Get_Is_Safe(t *testing.T) { go func() { wg.Done() peerId := unittest.PeerIdFixture(t) - cfg, err := cache.GetOrInit(peerId) + cfg, err := cache.GetWithInit(peerId) require.NoError(t, err) // concurrent retrieval must not fail. require.Equal(t, unicastConfigFixture().StreamCreationRetryAttemptBudget, cfg.StreamCreationRetryAttemptBudget) require.Equal(t, uint64(0), cfg.ConsecutiveSuccessfulStream) @@ -229,7 +229,7 @@ func TestUnicastConfigCache_LRU_Eviction(t *testing.T) { peerIds[i] = peerId } for i := 0; i < int(sizeLimit+1); i++ { - updatedConfig, err := cache.Adjust(peerIds[i], func(cfg unicast.Config) (unicast.Config, error) { + updatedConfig, err := cache.AdjustWithInit(peerIds[i], func(cfg unicast.Config) (unicast.Config, error) { cfg.StreamCreationRetryAttemptBudget = 2 // some random adjustment cfg.ConsecutiveSuccessfulStream = 3 // some random adjustment return cfg, nil @@ -241,7 +241,7 @@ func TestUnicastConfigCache_LRU_Eviction(t *testing.T) { // except the first peer id, all other peer ids should stay intact in the cache. for i := 1; i < int(sizeLimit+1); i++ { - cfg, err := cache.GetOrInit(peerIds[i]) + cfg, err := cache.GetWithInit(peerIds[i]) require.NoError(t, err) require.Equal(t, uint64(2), cfg.StreamCreationRetryAttemptBudget) require.Equal(t, uint64(3), cfg.ConsecutiveSuccessfulStream) @@ -251,7 +251,7 @@ func TestUnicastConfigCache_LRU_Eviction(t *testing.T) { // querying the first peer id should return a fresh unicast config, // since it should be evicted due to LRU eviction, and the initiated with the default values. - cfg, err := cache.GetOrInit(peerIds[0]) + cfg, err := cache.GetWithInit(peerIds[0]) require.NoError(t, err) require.Equal(t, unicastConfigFixture().StreamCreationRetryAttemptBudget, cfg.StreamCreationRetryAttemptBudget) require.Equal(t, uint64(0), cfg.ConsecutiveSuccessfulStream) diff --git a/network/p2p/unicast/cache/unicastConfigEntity.go b/network/p2p/unicast/cache/unicastConfigEntity.go index c1db31523fe..7e84b89deab 100644 --- a/network/p2p/unicast/cache/unicastConfigEntity.go +++ b/network/p2p/unicast/cache/unicastConfigEntity.go @@ -11,18 +11,15 @@ import ( // It implements the flow.Entity interface. type UnicastConfigEntity struct { unicast.Config - PeerId peer.ID // remote peer id; used as the "key" in the unicast config cache. - id flow.Identifier // cache the id for fast lookup (HeroCache). + PeerId peer.ID // remote peer id; used as the "key" in the unicast config cache. + EntityId flow.Identifier // cache the id for fast lookup (HeroCache). } var _ flow.Entity = (*UnicastConfigEntity)(nil) // ID returns the ID of the unicast config entity; it is hash value of the peer id. func (d UnicastConfigEntity) ID() flow.Identifier { - if d.id == flow.ZeroID { - d.id = PeerIdToFlowId(d.PeerId) - } - return d.id + return d.EntityId } // Checksum acts the same as ID. @@ -30,7 +27,13 @@ func (d UnicastConfigEntity) Checksum() flow.Identifier { return d.ID() } -// PeerIdToFlowId converts a peer id to a flow id (hash value of the peer id). -func PeerIdToFlowId(pid peer.ID) flow.Identifier { - return flow.MakeIDFromFingerPrint([]byte(pid)) +// entityIdOf converts a peer ID to a flow ID by taking the hash of the peer ID. +// This is used to convert the peer ID in a notion that is compatible with HeroCache. +// This is not a protocol-level conversion, and is only used internally by the cache, MUST NOT be exposed outside the cache. +// Args: +// - peerId: the peer ID of the peer in the GossipSub protocol. +// Returns: +// - flow.Identifier: the flow ID of the peer. +func entityIdOf(pid peer.ID) flow.Identifier { + return flow.MakeID(pid) } diff --git a/network/p2p/unicast/cache/unicastConfigEntity_test.go b/network/p2p/unicast/cache/unicastConfigEntity_test.go index d7bad635c04..a8994b4375d 100644 --- a/network/p2p/unicast/cache/unicastConfigEntity_test.go +++ b/network/p2p/unicast/cache/unicastConfigEntity_test.go @@ -5,6 +5,7 @@ import ( "github.com/stretchr/testify/require" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/p2p/unicast" unicastcache "github.com/onflow/flow-go/network/p2p/unicast/cache" "github.com/onflow/flow-go/utils/unittest" @@ -20,12 +21,13 @@ func TestUnicastConfigEntity(t *testing.T) { StreamCreationRetryAttemptBudget: 20, ConsecutiveSuccessfulStream: 30, }, + EntityId: flow.MakeID(peerID), } t.Run( "Test ID and Checksum", func(t *testing.T) { // id and checksum methods must return the same value as expected. - expectedID := unicastcache.PeerIdToFlowId(peerID) + expectedID := flow.MakeID(peerID) require.Equal(t, expectedID, d.ID()) require.Equal(t, expectedID, d.Checksum()) @@ -36,9 +38,11 @@ func TestUnicastConfigEntity(t *testing.T) { ) t.Run("ID is only calculated from peer.ID", func(t *testing.T) { + peerId := unittest.PeerIdFixture(t) d2 := &unicastcache.UnicastConfigEntity{ - PeerId: unittest.PeerIdFixture(t), - Config: d.Config, + PeerId: peerId, + Config: d.Config, + EntityId: flow.MakeID(peerId), } require.NotEqual(t, d.ID(), d2.ID()) // different peer id, different id. @@ -47,6 +51,7 @@ func TestUnicastConfigEntity(t *testing.T) { Config: unicast.Config{ StreamCreationRetryAttemptBudget: 200, }, + EntityId: d.EntityId, } require.Equal(t, d.ID(), d3.ID()) // same peer id, same id, even though the unicast config is different. }) diff --git a/network/p2p/unicast/dialConfigCache.go b/network/p2p/unicast/dialConfigCache.go index fc4c3199b5b..9696e3dcc14 100644 --- a/network/p2p/unicast/dialConfigCache.go +++ b/network/p2p/unicast/dialConfigCache.go @@ -7,14 +7,14 @@ import ( // ConfigCache is a thread-safe cache for dial configs. It is used by the unicast service to store // the dial configs for peers. type ConfigCache interface { - // GetOrInit returns the dial config for the given peer id. If the config does not exist, it creates a new config + // GetWithInit returns the dial config for the given peer id. If the config does not exist, it creates a new config // using the factory function and stores it in the cache. // Args: // - peerID: the peer id of the dial config. // Returns: // - *Config, the dial config for the given peer id. // - error if the factory function returns an error. Any error should be treated as an irrecoverable error and indicates a bug. - GetOrInit(peerID peer.ID) (*Config, error) + GetWithInit(peerID peer.ID) (*Config, error) // Adjust adjusts the dial config for the given peer id using the given adjustFunc. // It returns an error if the adjustFunc returns an error. @@ -23,7 +23,7 @@ type ConfigCache interface { // - adjustFunc: the function that adjusts the dial config. // Returns: // - error if the adjustFunc returns an error. Any error should be treated as an irrecoverable error and indicates a bug. - Adjust(peerID peer.ID, adjustFunc UnicastConfigAdjustFunc) (*Config, error) + AdjustWithInit(peerID peer.ID, adjustFunc UnicastConfigAdjustFunc) (*Config, error) // Size returns the number of dial configs in the cache. Size() uint diff --git a/network/p2p/unicast/manager.go b/network/p2p/unicast/manager.go index f7c6a9742fb..e51c3d0fc42 100644 --- a/network/p2p/unicast/manager.go +++ b/network/p2p/unicast/manager.go @@ -234,7 +234,7 @@ func (m *Manager) createStream(ctx context.Context, peerID peer.ID, protocol pro return nil, fmt.Errorf("failed to upgrade raw stream: %w", err) } - updatedConfig, err := m.dialConfigCache.Adjust(peerID, func(config Config) (Config, error) { + updatedConfig, err := m.dialConfigCache.AdjustWithInit(peerID, func(config Config) (Config, error) { config.ConsecutiveSuccessfulStream++ // increase consecutive successful stream count. return config, nil }) @@ -353,7 +353,7 @@ func retryFailedError(dialAttempts, maxAttempts uint64, err error) error { // - dial config for the given peer id. // - error if the dial config cannot be retrieved or adjusted; any error is irrecoverable and indicates a fatal error. func (m *Manager) getDialConfig(peerID peer.ID) (*Config, error) { - dialCfg, err := m.dialConfigCache.GetOrInit(peerID) + dialCfg, err := m.dialConfigCache.GetWithInit(peerID) if err != nil { return nil, fmt.Errorf("failed to get or init dial config for peer id: %w", err) } @@ -361,7 +361,7 @@ func (m *Manager) getDialConfig(peerID peer.ID) (*Config, error) { if dialCfg.StreamCreationRetryAttemptBudget == uint64(0) && dialCfg.ConsecutiveSuccessfulStream >= m.streamZeroBackoffResetThreshold { // reset the stream creation backoff budget to the default value if the number of consecutive successful streams reaches the threshold, // as the stream creation is reliable enough to be trusted again. - dialCfg, err = m.dialConfigCache.Adjust(peerID, func(config Config) (Config, error) { + dialCfg, err = m.dialConfigCache.AdjustWithInit(peerID, func(config Config) (Config, error) { config.StreamCreationRetryAttemptBudget = m.maxStreamCreationAttemptTimes m.metrics.OnStreamCreationRetryBudgetUpdated(config.StreamCreationRetryAttemptBudget) m.metrics.OnStreamCreationRetryBudgetResetToDefault() @@ -385,7 +385,7 @@ func (m *Manager) getDialConfig(peerID peer.ID) (*Config, error) { // - connected indicates whether there is a connection to the peer. // - error if the dial config cannot be adjusted; any error is irrecoverable and indicates a fatal error. func (m *Manager) adjustUnsuccessfulStreamAttempt(peerID peer.ID) (*Config, error) { - updatedCfg, err := m.dialConfigCache.Adjust(peerID, func(config Config) (Config, error) { + updatedCfg, err := m.dialConfigCache.AdjustWithInit(peerID, func(config Config) (Config, error) { // consecutive successful stream count is reset to 0 if we fail to create a stream or connection to the peer. config.ConsecutiveSuccessfulStream = 0 diff --git a/network/p2p/unicast/manager_test.go b/network/p2p/unicast/manager_test.go index 70165c4cbb1..1ab85e16cd8 100644 --- a/network/p2p/unicast/manager_test.go +++ b/network/p2p/unicast/manager_test.go @@ -142,7 +142,7 @@ func TestUnicastManager_SuccessfulStream(t *testing.T) { require.NotNil(t, s) // The unicast config must be updated with the backoff budget decremented. - unicastCfg, err := configCache.GetOrInit(peerID) + unicastCfg, err := configCache.GetWithInit(peerID) require.NoError(t, err) require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes, unicastCfg.StreamCreationRetryAttemptBudget) // stream backoff budget must remain intact. require.Equal(t, uint64(1), unicastCfg.ConsecutiveSuccessfulStream) // consecutive successful stream must incremented. @@ -172,7 +172,7 @@ func TestUnicastManager_StreamBackoff(t *testing.T) { require.Nil(t, s) // The unicast config must be updated with the backoff budget decremented. - unicastCfg, err := configCache.GetOrInit(peerID) + unicastCfg, err := configCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be decremented by 1 since all budget is used up. require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes-1, unicastCfg.StreamCreationRetryAttemptBudget) @@ -202,7 +202,7 @@ func TestUnicastManager_StreamFactory_StreamBackoff(t *testing.T) { require.Nil(t, s) // The unicast config must be updated with the stream backoff budget decremented. - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be decremented by 1. require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes-1, unicastCfg.StreamCreationRetryAttemptBudget) @@ -234,7 +234,7 @@ func TestUnicastManager_Stream_ConsecutiveStreamCreation_Increment(t *testing.T) require.NotNil(t, s) // The unicast config must be updated with the stream backoff budget decremented. - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be intact (all stream creation attempts are successful). require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes, unicastCfg.StreamCreationRetryAttemptBudget) @@ -254,7 +254,7 @@ func TestUnicastManager_Stream_ConsecutiveStreamCreation_Reset(t *testing.T) { Return(nil, fmt.Errorf("some error")). Once() - adjustedUnicastConfig, err := unicastConfigCache.Adjust(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { + adjustedUnicastConfig, err := unicastConfigCache.AdjustWithInit(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { // sets the consecutive successful stream to 5 meaning that the last 5 stream creation attempts were successful. unicastConfig.ConsecutiveSuccessfulStream = 5 // sets the stream back budget to 0 meaning that the stream backoff budget is exhausted. @@ -273,7 +273,7 @@ func TestUnicastManager_Stream_ConsecutiveStreamCreation_Reset(t *testing.T) { require.Nil(t, s) // The unicast config must be updated with the stream backoff budget decremented. - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be intact (we can't decrement it below 0). @@ -320,7 +320,7 @@ func TestUnicastManager_StreamFactory_ErrNoAddresses(t *testing.T) { require.Error(t, err) require.Nil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be reduced by 1 due to failed stream creation. @@ -349,7 +349,7 @@ func TestUnicastManager_Stream_ErrSecurityProtocolNegotiationFailed(t *testing.T require.Error(t, err) require.Nil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream retry budget must be decremented by 1 (since we didn't have a successful stream creation, the budget is decremented). require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes-1, unicastCfg.StreamCreationRetryAttemptBudget) @@ -376,7 +376,7 @@ func TestUnicastManager_StreamFactory_ErrGaterDisallowedConnection(t *testing.T) require.Error(t, err) require.Nil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must be reduced by 1 due to failed stream creation. require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes-1, unicastCfg.StreamCreationRetryAttemptBudget) @@ -413,7 +413,7 @@ func TestUnicastManager_Stream_BackoffBudgetDecremented(t *testing.T) { require.Error(t, err) require.Nil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) if i == int(maxStreamRetryBudget)-1 { @@ -423,7 +423,7 @@ func TestUnicastManager_Stream_BackoffBudgetDecremented(t *testing.T) { } } // At this time the backoff budget for connection must be 0. - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) require.Equal(t, uint64(0), unicastCfg.StreamCreationRetryAttemptBudget) @@ -432,7 +432,7 @@ func TestUnicastManager_Stream_BackoffBudgetDecremented(t *testing.T) { require.Error(t, err) require.Nil(t, s) - unicastCfg, err = unicastConfigCache.GetOrInit(peerID) + unicastCfg, err = unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) require.Equal(t, uint64(0), unicastCfg.StreamCreationRetryAttemptBudget) } @@ -450,7 +450,7 @@ func TestUnicastManager_Stream_BackoffBudgetResetToDefault(t *testing.T) { streamFactory.On("NewStream", mock.Anything, peerID, mock.Anything).Return(&p2ptest.MockStream{}, nil).Once() // update the unicast config of the peer to have a zero stream backoff budget but a consecutive successful stream counter above the reset threshold. - adjustedCfg, err := unicastConfigCache.Adjust(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { + adjustedCfg, err := unicastConfigCache.AdjustWithInit(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { unicastConfig.StreamCreationRetryAttemptBudget = 0 unicastConfig.ConsecutiveSuccessfulStream = cfg.NetworkConfig.Unicast.UnicastManager.StreamZeroRetryResetThreshold + 1 return unicastConfig, nil @@ -466,7 +466,7 @@ func TestUnicastManager_Stream_BackoffBudgetResetToDefault(t *testing.T) { require.NoError(t, err) require.NotNil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) // stream backoff budget must reset to default. require.Equal(t, cfg.NetworkConfig.Unicast.UnicastManager.MaxStreamCreationRetryAttemptTimes, unicastCfg.StreamCreationRetryAttemptBudget) @@ -483,7 +483,7 @@ func TestUnicastManager_Stream_NoBackoff_When_Budget_Is_Zero(t *testing.T) { // mocks that it attempts to create a stream once and fails, and does not retry. streamFactory.On("NewStream", mock.Anything, peerID, mock.Anything).Return(nil, fmt.Errorf("some error")).Once() - adjustedCfg, err := unicastConfigCache.Adjust(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { + adjustedCfg, err := unicastConfigCache.AdjustWithInit(peerID, func(unicastConfig unicast.Config) (unicast.Config, error) { unicastConfig.ConsecutiveSuccessfulStream = 2 // set the consecutive successful stream to 2, which is below the reset threshold. unicastConfig.StreamCreationRetryAttemptBudget = 0 // set the stream backoff budget to 0, meaning that the stream backoff budget is exhausted. return unicastConfig, nil @@ -499,7 +499,7 @@ func TestUnicastManager_Stream_NoBackoff_When_Budget_Is_Zero(t *testing.T) { require.Error(t, err) require.Nil(t, s) - unicastCfg, err := unicastConfigCache.GetOrInit(peerID) + unicastCfg, err := unicastConfigCache.GetWithInit(peerID) require.NoError(t, err) require.Equal(t, uint64(0), unicastCfg.StreamCreationRetryAttemptBudget) // stream backoff budget must remain zero. require.Equal(t, uint64(0), unicastCfg.ConsecutiveSuccessfulStream) // consecutive successful stream must be set to zero. diff --git a/state/protocol/badger/mutator_test.go b/state/protocol/badger/mutator_test.go index 53ecd1a6e79..8a63f20aa29 100644 --- a/state/protocol/badger/mutator_test.go +++ b/state/protocol/badger/mutator_test.go @@ -20,7 +20,6 @@ import ( "github.com/onflow/flow-go/engine" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/module/metrics" mockmodule "github.com/onflow/flow-go/module/mock" "github.com/onflow/flow-go/module/signature" @@ -868,7 +867,7 @@ func TestExtendEpochTransitionValid(t *testing.T) { // add a participant for the next epoch epoch2NewParticipant := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - epoch2Participants := append(participants, epoch2NewParticipant).Sort(order.Canonical) + epoch2Participants := append(participants, epoch2NewParticipant).Sort(flow.Canonical) // create the epoch setup event for the second epoch epoch2Setup := unittest.EpochSetupFixture( @@ -1319,7 +1318,7 @@ func TestExtendEpochSetupInvalid(t *testing.T) { // add a participant for the next epoch epoch2NewParticipant := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - epoch2Participants := append(participants, epoch2NewParticipant).Sort(order.Canonical) + epoch2Participants := append(participants, epoch2NewParticipant).Sort(flow.Canonical) // this function will return a VALID setup event and seal, we will modify // in different ways in each test case @@ -1437,7 +1436,7 @@ func TestExtendEpochCommitInvalid(t *testing.T) { epoch2Participants := append( participants.Filter(filter.Not(filter.HasRole(flow.RoleConsensus))), epoch2NewParticipant, - ).Sort(order.Canonical) + ).Sort(flow.Canonical) // factory method to create a valid EpochSetup method w.r.t. the generated state createSetup := func(block *flow.Block) (*flow.EpochSetup, *flow.ExecutionReceipt, *flow.Seal) { @@ -1620,7 +1619,7 @@ func TestExtendEpochTransitionWithoutCommit(t *testing.T) { // add a participant for the next epoch epoch2NewParticipant := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - epoch2Participants := append(participants, epoch2NewParticipant).Sort(order.Canonical) + epoch2Participants := append(participants, epoch2NewParticipant).Sort(flow.Canonical) // create the epoch setup event for the second epoch epoch2Setup := unittest.EpochSetupFixture( @@ -1761,7 +1760,7 @@ func TestEmergencyEpochFallback(t *testing.T) { // add a participant for the next epoch epoch2NewParticipant := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - epoch2Participants := append(participants, epoch2NewParticipant).Sort(order.Canonical) + epoch2Participants := append(participants, epoch2NewParticipant).Sort(flow.Canonical) // create the epoch setup event for the second epoch epoch2Setup := unittest.EpochSetupFixture( @@ -1851,7 +1850,7 @@ func TestEmergencyEpochFallback(t *testing.T) { // add a participant for the next epoch epoch2NewParticipant := unittest.IdentityFixture(unittest.WithRole(flow.RoleVerification)) - epoch2Participants := append(participants, epoch2NewParticipant).Sort(order.Canonical) + epoch2Participants := append(participants, epoch2NewParticipant).Sort(flow.Canonical) // create the epoch setup event for the second epoch // this event is invalid because it used a non-contiguous first view diff --git a/state/protocol/badger/snapshot.go b/state/protocol/badger/snapshot.go index 1a121e81748..90cdebc6db9 100644 --- a/state/protocol/badger/snapshot.go +++ b/state/protocol/badger/snapshot.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/model/flow/mapfunc" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/state/fork" "github.com/onflow/flow-go/state/protocol" "github.com/onflow/flow-go/state/protocol/inmem" @@ -111,7 +110,7 @@ func (s *Snapshot) Identities(selector flow.IdentityFilter) (flow.IdentityList, } // sort the identities so the 'IsCached' binary search works - identities := setup.Participants.Sort(order.Canonical) + identities := setup.Participants.Sort(flow.Canonical) // get identities that are in either last/next epoch but NOT in the current epoch var otherEpochIdentities flow.IdentityList @@ -173,7 +172,7 @@ func (s *Snapshot) Identities(selector flow.IdentityFilter) (flow.IdentityList, identities = identities.Filter(selector) // apply a deterministic sort to the participants - identities = identities.Sort(order.Canonical) + identities = identities.Sort(flow.Canonical) return identities, nil } diff --git a/state/protocol/badger/validity.go b/state/protocol/badger/validity.go index 264831512ec..acece515f64 100644 --- a/state/protocol/badger/validity.go +++ b/state/protocol/badger/validity.go @@ -10,7 +10,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/factory" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/state/protocol" ) @@ -87,7 +86,7 @@ func verifyEpochSetup(setup *flow.EpochSetup, verifyNetworkAddress bool) error { } // the participants must be listed in canonical order - if !setup.Participants.Sorted(order.Canonical) { + if !flow.IsIdentityListCanonical(setup.Participants) { return fmt.Errorf("participants are not canonically ordered") } @@ -235,7 +234,7 @@ func IsValidRootSnapshot(snap protocol.Snapshot, verifyResultID bool) error { if err != nil { return fmt.Errorf("could not get identities for root snapshot: %w", err) } - if !identities.Sorted(order.Canonical) { + if !flow.IsIdentityListCanonical(identities) { return fmt.Errorf("identities are not canonically ordered") } diff --git a/state/protocol/snapshot.go b/state/protocol/snapshot.go index f557bc59fbc..ca781b6e6cb 100644 --- a/state/protocol/snapshot.go +++ b/state/protocol/snapshot.go @@ -50,7 +50,7 @@ type Snapshot interface { // epoch. At the end of an epoch, this includes identities scheduled to join // in the next epoch but are not active yet. // - // Identities are guaranteed to be returned in canonical order (order.Canonical). + // Identities are guaranteed to be returned in canonical order (flow.Canonical). // // It allows us to provide optional upfront filters which can be used by the // implementation to speed up database lookups. diff --git a/storage/badger/blocks.go b/storage/badger/blocks.go index cc0a35e3acd..9d3b64a1ffc 100644 --- a/storage/badger/blocks.go +++ b/storage/badger/blocks.go @@ -109,6 +109,7 @@ func (b *Blocks) IndexBlockForCollections(blockID flow.Identifier, collIDs []flo } // InsertLastFullBlockHeightIfNotExists inserts the last full block height +// Calling this function multiple times is a no-op and returns no expected errors. func (b *Blocks) InsertLastFullBlockHeightIfNotExists(height uint64) error { return operation.RetryOnConflict(b.db.Update, func(tx *badger.Txn) error { err := operation.InsertLastCompleteBlockHeightIfNotExists(height)(tx) diff --git a/storage/badger/operation/events_test.go b/storage/badger/operation/events_test.go index ed2395e69d7..9896c02fd69 100644 --- a/storage/badger/operation/events_test.go +++ b/storage/badger/operation/events_test.go @@ -2,9 +2,10 @@ package operation import ( "bytes" - "sort" "testing" + "golang.org/x/exp/slices" + "github.com/dgraph-io/badger/v2" "github.com/stretchr/testify/require" @@ -114,24 +115,14 @@ func TestRetrieveEventByBlockIDTxID(t *testing.T) { }) } -// Event retrieval does not guarantee any order, hence a sort function to help compare expected and actual events +// Event retrieval does not guarantee any order, +// Hence, we a sort the events for comparing the expected and actual events. func sortEvent(events []flow.Event) { - sort.Slice(events, func(i, j int) bool { - - tComp := bytes.Compare(events[i].TransactionID[:], events[j].TransactionID[:]) - if tComp < 0 { - return true + slices.SortFunc(events, func(i, j flow.Event) int { + tComp := bytes.Compare(i.TransactionID[:], j.TransactionID[:]) + if tComp != 0 { + return tComp } - if tComp > 0 { - return false - } - - txIndex := events[i].TransactionIndex == events[j].TransactionIndex - if !txIndex { - return events[i].TransactionIndex < events[j].TransactionIndex - } - - return events[i].EventIndex < events[j].EventIndex - + return int(i.EventIndex) - int(j.EventIndex) }) } diff --git a/storage/pebble/bootstrap.go b/storage/pebble/bootstrap.go index 54a3ab182b3..f00d4991cda 100644 --- a/storage/pebble/bootstrap.go +++ b/storage/pebble/bootstrap.go @@ -12,6 +12,7 @@ import ( "go.uber.org/atomic" "golang.org/x/sync/errgroup" + "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/convert" "github.com/onflow/flow-go/ledger/complete/wal" ) @@ -26,6 +27,7 @@ type RegisterBootstrap struct { checkpointFileName string leafNodeChan chan *wal.LeafNode rootHeight uint64 + rootHash ledger.RootHash registerCount *atomic.Uint64 } @@ -36,6 +38,7 @@ func NewRegisterBootstrap( db *pebble.DB, checkpointFile string, rootHeight uint64, + rootHash ledger.RootHash, log zerolog.Logger, ) (*RegisterBootstrap, error) { // check for pre-populated heights, fail if it is populated @@ -57,6 +60,7 @@ func NewRegisterBootstrap( checkpointFileName: checkpointFileName, leafNodeChan: make(chan *wal.LeafNode, checkpointLeafNodeBufSize), rootHeight: rootHeight, + rootHash: rootHash, registerCount: atomic.NewUint64(0), }, nil } @@ -131,6 +135,12 @@ func (b *RegisterBootstrap) IndexCheckpointFile(ctx context.Context, workerCount cct, cancel := context.WithCancel(ctx) defer cancel() + // validate the checkpoint has correct root hash + err := wal.CheckpointHasRootHash(b.log, b.checkpointDir, b.checkpointFileName, b.rootHash) + if err != nil { + return fmt.Errorf("the root checkpoint to have the trie root hash %v does not match with the root state commitment: %w", b.rootHash, err) + } + g, gCtx := errgroup.WithContext(cct) start := time.Now() @@ -141,7 +151,7 @@ func (b *RegisterBootstrap) IndexCheckpointFile(ctx context.Context, workerCount }) } - err := wal.OpenAndReadLeafNodesFromCheckpointV6(b.leafNodeChan, b.checkpointDir, b.checkpointFileName, b.log) + err = wal.OpenAndReadLeafNodesFromCheckpointV6(b.leafNodeChan, b.checkpointDir, b.checkpointFileName, b.log) if err != nil { return fmt.Errorf("error reading leaf node: %w", err) } diff --git a/storage/pebble/bootstrap_test.go b/storage/pebble/bootstrap_test.go index 5e35ac156a2..bf35ec915b3 100644 --- a/storage/pebble/bootstrap_test.go +++ b/storage/pebble/bootstrap_test.go @@ -28,13 +28,14 @@ func TestRegisterBootstrap_NewBootstrap(t *testing.T) { t.Parallel() unittest.RunWithTempDir(t, func(dir string) { rootHeight := uint64(1) + rootHash := ledger.RootHash(unittest.StateCommitmentFixture()) log := zerolog.New(io.Discard) p, err := OpenRegisterPebbleDB(dir) require.NoError(t, err) // set heights require.NoError(t, initHeights(p, rootHeight)) // errors if FirstHeight or LastHeight are populated - _, err = NewRegisterBootstrap(p, dir, rootHeight, log) + _, err = NewRegisterBootstrap(p, dir, rootHeight, rootHash, log) require.ErrorIs(t, err, ErrAlreadyBootstrapped) }) } @@ -45,12 +46,13 @@ func TestRegisterBootstrap_IndexCheckpointFile_Happy(t *testing.T) { rootHeight := uint64(10000) unittest.RunWithTempDir(t, func(dir string) { tries, registerIDs := simpleTrieWithValidRegisterIDs(t) + rootHash := tries[0].RootHash() fileName := "simple-checkpoint" require.NoErrorf(t, wal.StoreCheckpointV6Concurrently(tries, dir, fileName, log), "fail to store checkpoint") checkpointFile := path.Join(dir, fileName) pb, dbDir := createPebbleForTest(t) - bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, log) + bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, rootHash, log) require.NoError(t, err) err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) require.NoError(t, err) @@ -79,12 +81,13 @@ func TestRegisterBootstrap_IndexCheckpointFile_Empty(t *testing.T) { rootHeight := uint64(10000) unittest.RunWithTempDir(t, func(dir string) { tries := []*trie.MTrie{trie.NewEmptyMTrie()} + rootHash := tries[0].RootHash() fileName := "empty-checkpoint" require.NoErrorf(t, wal.StoreCheckpointV6Concurrently(tries, dir, fileName, log), "fail to store checkpoint") checkpointFile := path.Join(dir, fileName) pb, dbDir := createPebbleForTest(t) - bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, log) + bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, rootHash, log) require.NoError(t, err) err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) require.NoError(t, err) @@ -113,6 +116,7 @@ func TestRegisterBootstrap_IndexCheckpointFile_FormatIssue(t *testing.T) { emptyTrie := trie.NewEmptyMTrie() trieWithInvalidEntry, _, err := trie.NewTrieWithUpdatedRegisters(emptyTrie, paths, payloads, true) require.NoError(t, err) + rootHash := trieWithInvalidEntry.RootHash() log := zerolog.New(io.Discard) unittest.RunWithTempDir(t, func(dir string) { @@ -122,7 +126,7 @@ func TestRegisterBootstrap_IndexCheckpointFile_FormatIssue(t *testing.T) { checkpointFile := path.Join(dir, fileName) pb, dbDir := createPebbleForTest(t) - bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, log) + bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, rootHash, log) require.NoError(t, err) err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) require.ErrorContains(t, err, "unexpected ledger key format") @@ -138,6 +142,7 @@ func TestRegisterBootstrap_IndexCheckpointFile_CorruptedCheckpointFile(t *testin log := zerolog.New(io.Discard) unittest.RunWithTempDir(t, func(dir string) { tries, _ := largeTrieWithValidRegisterIDs(t) + rootHash := tries[0].RootHash() checkpointFileName := "large-checkpoint-incomplete" require.NoErrorf(t, wal.StoreCheckpointV6Concurrently(tries, dir, checkpointFileName, log), "fail to store checkpoint") // delete 2nd part of the file (2nd subtrie) @@ -145,7 +150,7 @@ func TestRegisterBootstrap_IndexCheckpointFile_CorruptedCheckpointFile(t *testin err := os.RemoveAll(fileToDelete) require.NoError(t, err) pb, dbDir := createPebbleForTest(t) - bootstrap, err := NewRegisterBootstrap(pb, checkpointFileName, rootHeight, log) + bootstrap, err := NewRegisterBootstrap(pb, checkpointFileName, rootHeight, rootHash, log) require.NoError(t, err) err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) require.ErrorIs(t, err, os.ErrNotExist) @@ -159,11 +164,12 @@ func TestRegisterBootstrap_IndexCheckpointFile_MultipleBatch(t *testing.T) { rootHeight := uint64(10000) unittest.RunWithTempDir(t, func(dir string) { tries, registerIDs := largeTrieWithValidRegisterIDs(t) + rootHash := tries[0].RootHash() fileName := "large-checkpoint" require.NoErrorf(t, wal.StoreCheckpointV6Concurrently(tries, dir, fileName, log), "fail to store checkpoint") checkpointFile := path.Join(dir, fileName) pb, dbDir := createPebbleForTest(t) - bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, log) + bootstrap, err := NewRegisterBootstrap(pb, checkpointFile, rootHeight, rootHash, log) require.NoError(t, err) err = bootstrap.IndexCheckpointFile(context.Background(), workerCount) require.NoError(t, err) diff --git a/storage/pebble/registers.go b/storage/pebble/registers.go index a1f3bc20bed..40a4657d4af 100644 --- a/storage/pebble/registers.go +++ b/storage/pebble/registers.go @@ -59,9 +59,12 @@ func (s *Registers) Get( fmt.Sprintf("height %d not indexed, indexed range is [%d-%d]", height, s.firstHeight, latestHeight), ) } - iter := s.db.NewIter(&pebble.IterOptions{ + iter, err := s.db.NewIter(&pebble.IterOptions{ UseL6Filters: true, }) + if err != nil { + return nil, err + } defer iter.Close() diff --git a/utils/unittest/cluster.go b/utils/unittest/cluster.go index 80d8627342c..a88f4c6c13a 100644 --- a/utils/unittest/cluster.go +++ b/utils/unittest/cluster.go @@ -2,12 +2,12 @@ package unittest import ( "fmt" - "sort" + + "golang.org/x/exp/slices" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/factory" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" ) // TransactionForCluster generates a transaction that will be assigned to the @@ -51,9 +51,7 @@ func ClusterAssignment(n uint, nodes flow.IdentityList) flow.AssignmentList { collectors := nodes.Filter(filter.HasRole(flow.RoleCollection)) // order, so the same list results in the same - sort.Slice(collectors, func(i, j int) bool { - return order.Canonical(collectors[i], collectors[j]) - }) + slices.SortFunc(collectors, flow.Canonical) assignments := make(flow.AssignmentList, n) for i, collector := range collectors { diff --git a/utils/unittest/execution_state.go b/utils/unittest/execution_state.go index 13815c5c001..5edbe68de04 100644 --- a/utils/unittest/execution_state.go +++ b/utils/unittest/execution_state.go @@ -24,7 +24,7 @@ const ServiceAccountPrivateKeySignAlgo = crypto.ECDSAP256 const ServiceAccountPrivateKeyHashAlgo = hash.SHA2_256 // Pre-calculated state commitment with root account with the above private key -const GenesisStateCommitmentHex = "d95624ee3fb5e59dbe0bb66fe0953b00a1381e4855c46ddeff091acdfac6dc7c" +const GenesisStateCommitmentHex = "35e276ebe06f7eef128eec4c82e12472e500e45251d14db90f38170bbb00e3c5" var GenesisStateCommitment flow.StateCommitment @@ -88,10 +88,10 @@ func genesisCommitHexByChainID(chainID flow.ChainID) string { return GenesisStateCommitmentHex } if chainID == flow.Testnet { - return "432922e741cf9bebd1395792731adab32896cc4fb9a9e64dbe458a2d3b55beae" + return "68afd3b0377941d4fb1c043d80444bfc0a0e39eb27e685f07ed2438ebdcbdabd" } if chainID == flow.Sandboxnet { return "e1c08b17f9e5896f03fe28dd37ca396c19b26628161506924fbf785834646ea1" } - return "84e93540ba0b4edeac629327fd8fe6b33e9ef43800645eb74557faefa96cf8b7" + return "eb70898c2bec1094e902cb97018ab1ccf222b35f980bcae027cc47dcc87b87e8" } diff --git a/utils/unittest/fixtures.go b/utils/unittest/fixtures.go index ce8390fbb8a..dfb706be6d4 100644 --- a/utils/unittest/fixtures.go +++ b/utils/unittest/fixtures.go @@ -33,7 +33,6 @@ import ( "github.com/onflow/flow-go/model/encoding" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" - "github.com/onflow/flow-go/model/flow/order" "github.com/onflow/flow-go/model/messages" "github.com/onflow/flow-go/model/verification" "github.com/onflow/flow-go/module" @@ -1714,6 +1713,11 @@ func EventsFixture( return events } +func EventTypeFixture(chainID flow.ChainID) flow.EventType { + eventType := fmt.Sprintf("A.%s.TestContract.TestEvent1", RandomAddressFixtureForChain(chainID)) + return flow.EventType(eventType) +} + // EventFixture returns an event func EventFixture( eType flow.EventType, @@ -1971,7 +1975,7 @@ func VoteWithBeaconSig() func(*hotstuff.Vote) { func WithParticipants(participants flow.IdentityList) func(*flow.EpochSetup) { return func(setup *flow.EpochSetup) { - setup.Participants = participants.Sort(order.Canonical) + setup.Participants = participants.Sort(flow.Canonical) setup.Assignments = ClusterAssignment(1, participants) } } @@ -2003,7 +2007,7 @@ func EpochSetupFixture(opts ...func(setup *flow.EpochSetup)) *flow.EpochSetup { Counter: uint64(rand.Uint32()), FirstView: uint64(0), FinalView: uint64(rand.Uint32() + 1000), - Participants: participants.Sort(order.Canonical), + Participants: participants.Sort(flow.Canonical), RandomSource: SeedFixture(flow.EpochSetupRandomSourceLength), DKGPhase1FinalView: 100, DKGPhase2FinalView: 200, @@ -2181,7 +2185,7 @@ func RootSnapshotFixtureWithChainID( chainID flow.ChainID, opts ...func(*flow.Block), ) *inmem.Snapshot { - block, result, seal := BootstrapFixtureWithChainID(participants.Sort(order.Canonical), chainID, opts...) + block, result, seal := BootstrapFixtureWithChainID(participants.Sort(flow.Canonical), chainID, opts...) qc := QuorumCertificateFixture(QCWithRootBlockID(block.ID())) root, err := inmem.SnapshotFromBootstrapState(block, result, seal, qc) if err != nil { diff --git a/utils/unittest/generator/events.go b/utils/unittest/generator/events.go index b57b27b5a7b..9400dc99287 100644 --- a/utils/unittest/generator/events.go +++ b/utils/unittest/generator/events.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow/protobuf/go/flow/entities" "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/utils/unittest" ) type EventGeneratorOption func(*Events) @@ -41,8 +42,13 @@ func EventGenerator(opts ...EventGeneratorOption) *Events { } func (g *Events) New() flow.Event { - location := common.StringLocation("test") - identifier := fmt.Sprintf("FooEvent%d", g.count) + address, err := common.BytesToAddress(unittest.RandomAddressFixture().Bytes()) + if err != nil { + panic(fmt.Sprintf("unexpected error while creating random address: %s", err)) + } + + location := common.NewAddressLocation(nil, address, "TestContract") + identifier := fmt.Sprintf("TestContract.FooEvent%d", g.count) typeID := location.TypeID(nil, identifier) testEventType := &cadence.EventType{ diff --git a/utils/unittest/mockEntity.go b/utils/unittest/mockEntity.go index 60dfca14ac7..4b1b66300ad 100644 --- a/utils/unittest/mockEntity.go +++ b/utils/unittest/mockEntity.go @@ -7,6 +7,7 @@ import ( // MockEntity implements a bare minimum entity for sake of test. type MockEntity struct { Identifier flow.Identifier + Nonce uint64 } func (m MockEntity) ID() flow.Identifier { diff --git a/utils/unittest/mocks/mock_getters.go b/utils/unittest/mocks/mock_getters.go new file mode 100644 index 00000000000..c36d53c75f0 --- /dev/null +++ b/utils/unittest/mocks/mock_getters.go @@ -0,0 +1,55 @@ +package mocks + +import "github.com/onflow/flow-go/storage" + +// StorageMapGetter implements a simple generic getter function for mock storage methods. +// This is useful to avoid duplicating boilerplate code for mock storage methods. +// +// Example: +// Instead of the following code: +// +// results.On("ByID", mock.AnythingOfType("flow.Identifier")).Return( +// func(resultID flow.Identifier) (*flow.ExecutionResult, error) { +// if result, ok := s.resultMap[resultID]; ok { +// return result, nil +// } +// return nil, storage.ErrNotFound +// }, +// ) +// +// Use this: +// +// results.On("ByID", mock.AnythingOfType("flow.Identifier")).Return( +// mocks.StorageMapGetter(s.resultMap), +// ) +func StorageMapGetter[K comparable, V any](m map[K]V) func(key K) (V, error) { + return func(key K) (V, error) { + if val, ok := m[key]; ok { + return val, nil + } + return *new(V), storage.ErrNotFound + } +} + +// ConvertStorageOutput maps the output type from a getter function to a different type. +// This is useful to avoid maintaining multiple maps for the same data. +// +// Example usage: +// +// blockMap := map[uint64]*flow.Block{} +// +// headers.On("BlockIDByHeight", mock.AnythingOfType("uint64")).Return( +// mocks.ConvertStorageOutput( +// mocks.StorageMapGetter(s.blockMap), +// func(block *flow.Block) flow.Identifier { return block.ID() }, +// ), +// ) +func ConvertStorageOutput[K comparable, V any, R any](fn func(key K) (V, error), mapper func(V) R) func(key K) (R, error) { + return func(key K) (R, error) { + v, err := fn(key) + if err != nil { + return *new(R), err + } + return mapper(v), err + } +} diff --git a/utils/unittest/service_events_fixtures.go b/utils/unittest/service_events_fixtures.go index 38b46e77292..5f62b194bc3 100644 --- a/utils/unittest/service_events_fixtures.go +++ b/utils/unittest/service_events_fixtures.go @@ -1,6 +1,9 @@ package unittest import ( + "crypto/rand" + "encoding/hex" + "github.com/onflow/cadence" "github.com/onflow/cadence/encoding/ccf" "github.com/onflow/cadence/runtime/common" @@ -12,19 +15,23 @@ import ( // This file contains service event fixtures for testing purposes. +func EpochSetupRandomSourceFixture() []byte { + source := make([]byte, flow.EpochSetupRandomSourceLength) + _, err := rand.Read(source) + if err != nil { + panic(err) + } + return source +} + // EpochSetupFixtureByChainID returns an EpochSetup service event as a Cadence event // representation and as a protocol model representation. func EpochSetupFixtureByChainID(chain flow.ChainID) (flow.Event, *flow.EpochSetup) { events := systemcontracts.ServiceEventsForChain(chain) event := EventFixture(events.EpochSetup.EventType(), 1, 1, IdentifierFixture(), 0) - event.Payload = EpochSetupFixtureCCF - - // randomSource is [0,0,...,1,2,3,4] - randomSource := make([]uint8, flow.EpochSetupRandomSourceLength) - for i := 0; i < 4; i++ { - randomSource[flow.EpochSetupRandomSourceLength-1-i] = uint8(4 - i) - } + randomSource := EpochSetupRandomSourceFixture() + event.Payload = EpochSetupFixtureCCF(randomSource) expected := &flow.EpochSetup{ Counter: 1, @@ -165,7 +172,8 @@ func VersionBeaconFixtureByChainID(chain flow.ChainID) (flow.Event, *flow.Versio return event, expected } -func createEpochSetupEvent() cadence.Event { +func createEpochSetupEvent(randomSource []byte) cadence.Event { + randomSourceHex := hex.EncodeToString(randomSource) return cadence.NewEvent([]cadence.Value{ // counter @@ -184,7 +192,7 @@ func createEpochSetupEvent() cadence.Event { createEpochCollectors(), // randomSource - cadence.String("01020304"), + cadence.String(randomSourceHex), // DKGPhase1FinalView cadence.UInt64(150), @@ -1019,8 +1027,8 @@ func ufix64FromString(s string) cadence.UFix64 { return f } -var EpochSetupFixtureCCF = func() []byte { - b, err := ccf.Encode(createEpochSetupEvent()) +func EpochSetupFixtureCCF(randomSource []byte) []byte { + b, err := ccf.Encode(createEpochSetupEvent(randomSource)) if err != nil { panic(err) } @@ -1029,7 +1037,54 @@ var EpochSetupFixtureCCF = func() []byte { panic(err) } return b -}() +} + +func EpochSetupCCFWithNonHexRandomSource() []byte { + // randomSource of correct length but made of non hex characters + randomSource := "ZZ" + for len(randomSource) != 2*flow.EpochSetupRandomSourceLength { + randomSource = randomSource + "aa" + } + + event := cadence.NewEvent([]cadence.Value{ + // counter + cadence.NewUInt64(1), + + // nodeInfo + createEpochNodes(), + + // firstView + cadence.NewUInt64(100), + + // finalView + cadence.NewUInt64(200), + + // collectorClusters + createEpochCollectors(), + + // randomSource + cadence.String(randomSource), + + // DKGPhase1FinalView + cadence.UInt64(150), + + // DKGPhase2FinalView + cadence.UInt64(160), + + // DKGPhase3FinalView + cadence.UInt64(170), + }).WithType(newFlowEpochEpochSetupEventType()) + + b, err := ccf.Encode(event) + if err != nil { + panic(err) + } + _, err = ccf.Decode(nil, b) + if err != nil { + panic(err) + } + return b +} var EpochCommitFixtureCCF = func() []byte { b, err := ccf.Encode(createEpochCommittedEvent())