diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 1258b0a45c8..74591646f17 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -120,36 +120,37 @@ import ( // For a node running as a standalone process, the config fields will be populated from the command line params, // while for a node running as a library, the config fields are expected to be initialized by the caller. type AccessNodeConfig struct { - supportsObserver bool // True if this is an Access node that supports observers and consensus follower engines - collectionGRPCPort uint - executionGRPCPort uint - pingEnabled bool - nodeInfoFile string - apiRatelimits map[string]int - apiBurstlimits map[string]int - rpcConf rpc.Config - stateStreamConf statestreambackend.Config - stateStreamFilterConf map[string]int - ExecutionNodeAddress string // deprecated - HistoricalAccessRPCs []access.AccessAPIClient - logTxTimeToFinalized bool - logTxTimeToExecuted bool - logTxTimeToFinalizedExecuted bool - retryEnabled bool - rpcMetricsEnabled bool - executionDataSyncEnabled bool - executionDataDir string - executionDataStartHeight uint64 - executionDataConfig edrequester.ExecutionDataConfig - PublicNetworkConfig PublicNetworkConfig - TxResultCacheSize uint - TxErrorMessagesCacheSize uint - executionDataIndexingEnabled bool - registersDBPath string - checkpointFile string - scriptExecutorConfig query.QueryConfig - scriptExecMinBlock uint64 - scriptExecMaxBlock uint64 + supportsObserver bool // True if this is an Access node that supports observers and consensus follower engines + collectionGRPCPort uint + executionGRPCPort uint + pingEnabled bool + nodeInfoFile string + apiRatelimits map[string]int + apiBurstlimits map[string]int + rpcConf rpc.Config + stateStreamConf statestreambackend.Config + stateStreamFilterConf map[string]int + ExecutionNodeAddress string // deprecated + HistoricalAccessRPCs []access.AccessAPIClient + logTxTimeToFinalized bool + logTxTimeToExecuted bool + logTxTimeToFinalizedExecuted bool + retryEnabled bool + rpcMetricsEnabled bool + executionDataSyncEnabled bool + publicNetworkExecutionDataEnabled bool + executionDataDir string + executionDataStartHeight uint64 + executionDataConfig edrequester.ExecutionDataConfig + PublicNetworkConfig PublicNetworkConfig + TxResultCacheSize uint + TxErrorMessagesCacheSize uint + executionDataIndexingEnabled bool + registersDBPath string + checkpointFile string + scriptExecutorConfig query.QueryConfig + scriptExecMinBlock uint64 + scriptExecMaxBlock uint64 } type PublicNetworkConfig struct { @@ -225,9 +226,10 @@ func DefaultAccessNodeConfig() *AccessNodeConfig { BindAddress: cmd.NotSet, Metrics: metrics.NewNoopCollector(), }, - executionDataSyncEnabled: true, - executionDataDir: filepath.Join(homedir, ".flow", "execution_data"), - executionDataStartHeight: 0, + executionDataSyncEnabled: true, + publicNetworkExecutionDataEnabled: false, + executionDataDir: filepath.Join(homedir, ".flow", "execution_data"), + executionDataStartHeight: 0, executionDataConfig: edrequester.ExecutionDataConfig{ InitialBlockHeight: 0, MaxSearchAhead: edrequester.DefaultMaxSearchAhead, @@ -272,6 +274,7 @@ type FlowAccessNodeBuilder struct { FollowerCore module.HotStuffFollower Validator hotstuff.Validator ExecutionDataDownloader execution_data.Downloader + PublicBlobService network.BlobService ExecutionDataRequester state_synchronization.ExecutionDataRequester ExecutionDataStore execution_data.ExecutionDataStore ExecutionDataCache *execdatacache.ExecutionDataCache @@ -664,6 +667,28 @@ func (builder *FlowAccessNodeBuilder) BuildExecutionSyncComponents() *FlowAccess return builder.ExecutionDataRequester, nil }) + if builder.publicNetworkExecutionDataEnabled { + builder.Component("public network execution data service", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + opts := []network.BlobServiceOption{ + blob.WithBitswapOptions( + bitswap.WithTracer( + blob.NewTracer(node.Logger.With().Str("public_blob_service", channels.PublicExecutionDataService.String()).Logger()), + ), + ), + } + + net := builder.AccessNodeConfig.PublicNetworkConfig.Network + + var err error + builder.PublicBlobService, err = net.RegisterBlobService(channels.PublicExecutionDataService, ds, opts...) + if err != nil { + return nil, fmt.Errorf("could not register blob service: %w", err) + } + + return builder.PublicBlobService, nil + }) + } + if builder.executionDataIndexingEnabled { var indexedBlockHeight storage.ConsumerProgress @@ -1029,6 +1054,10 @@ func (builder *FlowAccessNodeBuilder) extraFlags() { "execution-data-sync-enabled", defaultConfig.executionDataSyncEnabled, "whether to enable the execution data sync protocol") + flags.BoolVar(&builder.publicNetworkExecutionDataEnabled, + "public-network-execution-data-sync-enabled", + defaultConfig.publicNetworkExecutionDataEnabled, + "[experimental] whether to enable the execution data sync protocol on public network") flags.StringVar(&builder.executionDataDir, "execution-data-dir", defaultConfig.executionDataDir, "directory to use for Execution Data database") flags.Uint64Var(&builder.executionDataStartHeight, "execution-data-start-height", diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index d8078580be9..91d2b50e4f1 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -6,18 +6,24 @@ import ( "encoding/json" "errors" "fmt" + "os" + "path/filepath" "strings" "time" + badger "github.com/ipfs/go-ds-badger2" dht "github.com/libp2p/go-libp2p-kad-dht" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/routing" "github.com/onflow/crypto" + "github.com/onflow/go-bitswap" "github.com/rs/zerolog" "github.com/spf13/pflag" "google.golang.org/grpc/credentials" + "github.com/onflow/flow-go/admin/commands" + stateSyncCommands "github.com/onflow/flow-go/admin/commands/state_synchronization" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/consensus" "github.com/onflow/flow-go/consensus/hotstuff" @@ -35,6 +41,7 @@ import ( "github.com/onflow/flow-go/engine/access/rpc" "github.com/onflow/flow-go/engine/access/rpc/backend" rpcConnection "github.com/onflow/flow-go/engine/access/rpc/connection" + "github.com/onflow/flow-go/engine/access/state_stream" statestreambackend "github.com/onflow/flow-go/engine/access/state_stream/backend" "github.com/onflow/flow-go/engine/common/follower" synceng "github.com/onflow/flow-go/engine/common/synchronization" @@ -43,12 +50,18 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/blobs" "github.com/onflow/flow-go/module/chainsync" + "github.com/onflow/flow-go/module/executiondatasync/execution_data" + execdatacache "github.com/onflow/flow-go/module/executiondatasync/execution_data/cache" finalizer "github.com/onflow/flow-go/module/finalizer/consensus" "github.com/onflow/flow-go/module/grpcserver" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/local" + "github.com/onflow/flow-go/module/mempool/herocache" "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/module/state_synchronization" + edrequester "github.com/onflow/flow-go/module/state_synchronization/requester" consensus_follower "github.com/onflow/flow-go/module/upstream" "github.com/onflow/flow-go/network" alspmgr "github.com/onflow/flow-go/network/alsp/manager" @@ -56,6 +69,7 @@ import ( "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/converter" "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/p2p/blob" p2pbuilder "github.com/onflow/flow-go/network/p2p/builder" p2pbuilderconfig "github.com/onflow/flow-go/network/p2p/builder/config" "github.com/onflow/flow-go/network/p2p/cache" @@ -74,6 +88,8 @@ import ( badgerState "github.com/onflow/flow-go/state/protocol/badger" "github.com/onflow/flow-go/state/protocol/blocktimer" "github.com/onflow/flow-go/state/protocol/events/gadgets" + "github.com/onflow/flow-go/storage" + bstorage "github.com/onflow/flow-go/storage/badger" "github.com/onflow/flow-go/utils/grpcutils" "github.com/onflow/flow-go/utils/io" ) @@ -109,11 +125,17 @@ type ObserverServiceConfig struct { apiTimeout time.Duration upstreamNodeAddresses []string upstreamNodePublicKeys []string + executionDataSyncEnabled bool + executionDataDir string + executionDataStartHeight uint64 + executionDataConfig edrequester.ExecutionDataConfig + executionDataCacheSize uint32 // TODO: remove it when state stream is added upstreamIdentities flow.IdentitySkeletonList // the identity list of upstream peers the node uses to forward API requests to } // DefaultObserverServiceConfig defines all the default values for the ObserverServiceConfig func DefaultObserverServiceConfig() *ObserverServiceConfig { + homedir, _ := os.UserHomeDir() return &ObserverServiceConfig{ rpcConf: rpc.Config{ UnsecureGRPCListenAddr: "0.0.0.0:9000", @@ -147,6 +169,18 @@ func DefaultObserverServiceConfig() *ObserverServiceConfig { apiTimeout: 3 * time.Second, upstreamNodeAddresses: []string{}, upstreamNodePublicKeys: []string{}, + executionDataSyncEnabled: false, + executionDataDir: filepath.Join(homedir, ".flow", "execution_data"), + executionDataStartHeight: 0, + executionDataConfig: edrequester.ExecutionDataConfig{ + InitialBlockHeight: 0, + MaxSearchAhead: edrequester.DefaultMaxSearchAhead, + FetchTimeout: edrequester.DefaultFetchTimeout, + MaxFetchTimeout: edrequester.DefaultMaxFetchTimeout, + RetryDelay: edrequester.DefaultRetryDelay, + MaxRetryDelay: edrequester.DefaultMaxRetryDelay, + }, + executionDataCacheSize: state_stream.DefaultCacheSize, } } @@ -183,6 +217,10 @@ type ObserverServiceBuilder struct { // grpc servers secureGrpcServer *grpcserver.GrpcServer unsecureGrpcServer *grpcserver.GrpcServer + + ExecutionDataDownloader execution_data.Downloader + ExecutionDataRequester state_synchronization.ExecutionDataRequester + ExecutionDataStore execution_data.ExecutionDataStore } // deriveBootstrapPeerIdentities derives the Flow Identity of the bootstrap peers from the parameters. @@ -533,6 +571,64 @@ func (builder *ObserverServiceBuilder) extraFlags() { defaultConfig.upstreamNodePublicKeys, "the networking public key of the upstream access node (in the same order as the upstream node addresses) e.g. \"d57a5e9c5.....\",\"44ded42d....\"") flags.BoolVar(&builder.rpcMetricsEnabled, "rpc-metrics-enabled", defaultConfig.rpcMetricsEnabled, "whether to enable the rpc metrics") + + // ExecutionDataRequester config + flags.BoolVar(&builder.executionDataSyncEnabled, + "execution-data-sync-enabled", + defaultConfig.executionDataSyncEnabled, + "whether to enable the execution data sync protocol") + flags.StringVar(&builder.executionDataDir, + "execution-data-dir", + defaultConfig.executionDataDir, + "directory to use for Execution Data database") + flags.Uint64Var(&builder.executionDataStartHeight, + "execution-data-start-height", + defaultConfig.executionDataStartHeight, + "height of first block to sync execution data from when starting with an empty Execution Data database") + flags.Uint64Var(&builder.executionDataConfig.MaxSearchAhead, + "execution-data-max-search-ahead", + defaultConfig.executionDataConfig.MaxSearchAhead, + "max number of heights to search ahead of the lowest outstanding execution data height") + flags.DurationVar(&builder.executionDataConfig.FetchTimeout, + "execution-data-fetch-timeout", + defaultConfig.executionDataConfig.FetchTimeout, + "initial timeout to use when fetching execution data from the network. timeout increases using an incremental backoff until execution-data-max-fetch-timeout. e.g. 30s") + flags.DurationVar(&builder.executionDataConfig.MaxFetchTimeout, + "execution-data-max-fetch-timeout", + defaultConfig.executionDataConfig.MaxFetchTimeout, + "maximum timeout to use when fetching execution data from the network e.g. 300s") + flags.DurationVar(&builder.executionDataConfig.RetryDelay, + "execution-data-retry-delay", + defaultConfig.executionDataConfig.RetryDelay, + "initial delay for exponential backoff when fetching execution data fails e.g. 10s") + flags.DurationVar(&builder.executionDataConfig.MaxRetryDelay, + "execution-data-max-retry-delay", + defaultConfig.executionDataConfig.MaxRetryDelay, + "maximum delay for exponential backoff when fetching execution data fails e.g. 5m") + flags.Uint32Var(&builder.executionDataCacheSize, + "execution-data-cache-size", + defaultConfig.executionDataCacheSize, + "block execution data cache size") + }).ValidateFlags(func() error { + if builder.executionDataSyncEnabled { + if builder.executionDataConfig.FetchTimeout <= 0 { + return errors.New("execution-data-fetch-timeout must be greater than 0") + } + if builder.executionDataConfig.MaxFetchTimeout < builder.executionDataConfig.FetchTimeout { + return errors.New("execution-data-max-fetch-timeout must be greater than execution-data-fetch-timeout") + } + if builder.executionDataConfig.RetryDelay <= 0 { + return errors.New("execution-data-retry-delay must be greater than 0") + } + if builder.executionDataConfig.MaxRetryDelay < builder.executionDataConfig.RetryDelay { + return errors.New("execution-data-max-retry-delay must be greater than or equal to execution-data-retry-delay") + } + if builder.executionDataConfig.MaxSearchAhead == 0 { + return errors.New("execution-data-max-search-ahead must be greater than 0") + } + } + + return nil }) } @@ -678,6 +774,9 @@ func (builder *ObserverServiceBuilder) Initialize() error { builder.enqueueConnectWithStakedAN() + if builder.executionDataSyncEnabled { + builder.BuildExecutionSyncComponents() + } builder.enqueueRPCServer() if builder.BaseConfig.MetricsEnabled { @@ -816,6 +915,172 @@ func (builder *ObserverServiceBuilder) Build() (cmd.Node, error) { return builder.FlowNodeBuilder.Build() } +func (builder *ObserverServiceBuilder) BuildExecutionSyncComponents() *ObserverServiceBuilder { + var ds *badger.Datastore + var bs network.BlobService + var processedBlockHeight storage.ConsumerProgress + var processedNotifications storage.ConsumerProgress + var publicBsDependable *module.ProxiedReadyDoneAware + var execDataDistributor *edrequester.ExecutionDataDistributor + var execDataCacheBackend *herocache.BlockExecutionData + + // setup dependency chain to ensure indexer starts after the requester + requesterDependable := module.NewProxiedReadyDoneAware() + + builder. + AdminCommand("read-execution-data", func(config *cmd.NodeConfig) commands.AdminCommand { + return stateSyncCommands.NewReadExecutionDataCommand(builder.ExecutionDataStore) + }). + Module("execution data datastore and blobstore", func(node *cmd.NodeConfig) error { + datastoreDir := filepath.Join(builder.executionDataDir, "blobstore") + err := os.MkdirAll(datastoreDir, 0700) + if err != nil { + return err + } + + ds, err = badger.NewDatastore(datastoreDir, &badger.DefaultOptions) + if err != nil { + return err + } + + builder.ShutdownFunc(func() error { + if err := ds.Close(); err != nil { + return fmt.Errorf("could not close execution data datastore: %w", err) + } + return nil + }) + + return nil + }). + Module("processed block height consumer progress", func(node *cmd.NodeConfig) error { + // Note: progress is stored in the datastore's DB since that is where the jobqueue + // writes execution data to. + processedBlockHeight = bstorage.NewConsumerProgress(ds.DB, module.ConsumeProgressExecutionDataRequesterBlockHeight) + return nil + }). + Module("processed notifications consumer progress", func(node *cmd.NodeConfig) error { + // Note: progress is stored in the datastore's DB since that is where the jobqueue + // writes execution data to. + processedNotifications = bstorage.NewConsumerProgress(ds.DB, module.ConsumeProgressExecutionDataRequesterNotification) + return nil + }). + Module("blobservice peer manager dependencies", func(node *cmd.NodeConfig) error { + publicBsDependable = module.NewProxiedReadyDoneAware() + builder.PeerManagerDependencies.Add(publicBsDependable) + return nil + }). + Module("execution datastore", func(node *cmd.NodeConfig) error { + blobstore := blobs.NewBlobstore(ds) + builder.ExecutionDataStore = execution_data.NewExecutionDataStore(blobstore, execution_data.DefaultSerializer) + return nil + }). + Module("execution data cache", func(node *cmd.NodeConfig) error { + var heroCacheCollector module.HeroCacheMetrics = metrics.NewNoopCollector() + if builder.HeroCacheMetricsEnable { + heroCacheCollector = metrics.AccessNodeExecutionDataCacheMetrics(builder.MetricsRegisterer) + } + + execDataCacheBackend = herocache.NewBlockExecutionData(builder.executionDataCacheSize, builder.Logger, heroCacheCollector) + + return nil + }). + Component("public execution data service", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + opts := []network.BlobServiceOption{ + blob.WithBitswapOptions( + bitswap.WithTracer( + blob.NewTracer(node.Logger.With().Str("public_blob_service", channels.PublicExecutionDataService.String()).Logger()), + ), + ), + } + + var err error + bs, err = node.EngineRegistry.RegisterBlobService(channels.PublicExecutionDataService, ds, opts...) + if err != nil { + return nil, fmt.Errorf("could not register blob service: %w", err) + } + + // add blobservice into ReadyDoneAware dependency passed to peer manager + // this starts the blob service and configures peer manager to wait for the blobservice + // to be ready before starting + publicBsDependable.Init(bs) + + builder.ExecutionDataDownloader = execution_data.NewDownloader(bs) + + return builder.ExecutionDataDownloader, nil + }). + Component("execution data requester", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { + // Validation of the start block height needs to be done after loading state + if builder.executionDataStartHeight > 0 { + if builder.executionDataStartHeight <= builder.FinalizedRootBlock.Header.Height { + return nil, fmt.Errorf( + "execution data start block height (%d) must be greater than the root block height (%d)", + builder.executionDataStartHeight, builder.FinalizedRootBlock.Header.Height) + } + + latestSeal, err := builder.State.Sealed().Head() + if err != nil { + return nil, fmt.Errorf("failed to get latest sealed height") + } + + // Note: since the root block of a spork is also sealed in the root protocol state, the + // latest sealed height is always equal to the root block height. That means that at the + // very beginning of a spork, this check will always fail. Operators should not specify + // an InitialBlockHeight when starting from the beginning of a spork. + if builder.executionDataStartHeight > latestSeal.Height { + return nil, fmt.Errorf( + "execution data start block height (%d) must be less than or equal to the latest sealed block height (%d)", + builder.executionDataStartHeight, latestSeal.Height) + } + + // executionDataStartHeight is provided as the first block to sync, but the + // requester expects the initial last processed height, which is the first height - 1 + builder.executionDataConfig.InitialBlockHeight = builder.executionDataStartHeight - 1 + } else { + builder.executionDataConfig.InitialBlockHeight = builder.FinalizedRootBlock.Header.Height + } + + execDataDistributor = edrequester.NewExecutionDataDistributor() + + // Execution Data cache with a downloader as the backend. This is used by the requester + // to download and cache execution data for each block. It shares a cache backend instance + // with the datastore implementation. + executionDataCache := execdatacache.NewExecutionDataCache( + builder.ExecutionDataDownloader, + builder.Storage.Headers, + builder.Storage.Seals, + builder.Storage.Results, + execDataCacheBackend, + ) + + r, err := edrequester.New( + builder.Logger, + metrics.NewExecutionDataRequesterCollector(), + builder.ExecutionDataDownloader, + executionDataCache, + processedBlockHeight, + processedNotifications, + builder.State, + builder.Storage.Headers, + builder.executionDataConfig, + execDataDistributor, + ) + if err != nil { + return nil, fmt.Errorf("failed to create execution data requester: %w", err) + } + builder.ExecutionDataRequester = r + + builder.FollowerDistributor.AddOnBlockFinalizedConsumer(builder.ExecutionDataRequester.OnBlockFinalized) + + // add requester into ReadyDoneAware dependency passed to indexer. This allows the indexer + // to wait for the requester to be ready before starting. + requesterDependable.Init(builder.ExecutionDataRequester) + + return builder.ExecutionDataRequester, nil + }) + + return builder +} + // enqueuePublicNetworkInit enqueues the observer network component initialized for the observer func (builder *ObserverServiceBuilder) enqueuePublicNetworkInit() { var publicLibp2pNode p2p.LibP2PNode diff --git a/fvm/environment/env.go b/fvm/environment/env.go index 59dc4f83416..031eb460dc4 100644 --- a/fvm/environment/env.go +++ b/fvm/environment/env.go @@ -41,6 +41,8 @@ type Environment interface { ConvertedServiceEvents() flow.ServiceEventList // SystemContracts + ContractFunctionInvoker + AccountsStorageCapacity( addresses []flow.Address, payer flow.Address, diff --git a/fvm/environment/invoker.go b/fvm/environment/invoker.go new file mode 100644 index 00000000000..8041acdb363 --- /dev/null +++ b/fvm/environment/invoker.go @@ -0,0 +1,28 @@ +package environment + +import ( + "github.com/onflow/cadence" + "github.com/onflow/cadence/runtime/sema" + + "github.com/onflow/flow-go/model/flow" +) + +// ContractFunctionSpec specify all the information, except the function's +// address and arguments, needed to invoke the contract function. +type ContractFunctionSpec struct { + AddressFromChain func(flow.Chain) flow.Address + LocationName string + FunctionName string + ArgumentTypes []sema.Type +} + +// ContractFunctionInvoker invokes a contract function +type ContractFunctionInvoker interface { + Invoke( + spec ContractFunctionSpec, + arguments []cadence.Value, + ) ( + cadence.Value, + error, + ) +} diff --git a/fvm/environment/mock/contract_function_invoker.go b/fvm/environment/mock/contract_function_invoker.go new file mode 100644 index 00000000000..a01c2ac4f4c --- /dev/null +++ b/fvm/environment/mock/contract_function_invoker.go @@ -0,0 +1,55 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mock + +import ( + cadence "github.com/onflow/cadence" + environment "github.com/onflow/flow-go/fvm/environment" + mock "github.com/stretchr/testify/mock" +) + +// ContractFunctionInvoker is an autogenerated mock type for the ContractFunctionInvoker type +type ContractFunctionInvoker struct { + mock.Mock +} + +// Invoke provides a mock function with given fields: spec, arguments +func (_m *ContractFunctionInvoker) Invoke(spec environment.ContractFunctionSpec, arguments []cadence.Value) (cadence.Value, error) { + ret := _m.Called(spec, arguments) + + var r0 cadence.Value + var r1 error + if rf, ok := ret.Get(0).(func(environment.ContractFunctionSpec, []cadence.Value) (cadence.Value, error)); ok { + return rf(spec, arguments) + } + if rf, ok := ret.Get(0).(func(environment.ContractFunctionSpec, []cadence.Value) cadence.Value); ok { + r0 = rf(spec, arguments) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(cadence.Value) + } + } + + if rf, ok := ret.Get(1).(func(environment.ContractFunctionSpec, []cadence.Value) error); ok { + r1 = rf(spec, arguments) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +type mockConstructorTestingTNewContractFunctionInvoker interface { + mock.TestingT + Cleanup(func()) +} + +// NewContractFunctionInvoker creates a new instance of ContractFunctionInvoker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewContractFunctionInvoker(t mockConstructorTestingTNewContractFunctionInvoker) *ContractFunctionInvoker { + mock := &ContractFunctionInvoker{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/fvm/environment/mock/environment.go b/fvm/environment/mock/environment.go index 39ec38fff64..07dec698cb1 100644 --- a/fvm/environment/mock/environment.go +++ b/fvm/environment/mock/environment.go @@ -953,6 +953,32 @@ func (_m *Environment) InteractionUsed() (uint64, error) { return r0, r1 } +// Invoke provides a mock function with given fields: spec, arguments +func (_m *Environment) Invoke(spec environment.ContractFunctionSpec, arguments []cadence.Value) (cadence.Value, error) { + ret := _m.Called(spec, arguments) + + var r0 cadence.Value + var r1 error + if rf, ok := ret.Get(0).(func(environment.ContractFunctionSpec, []cadence.Value) (cadence.Value, error)); ok { + return rf(spec, arguments) + } + if rf, ok := ret.Get(0).(func(environment.ContractFunctionSpec, []cadence.Value) cadence.Value); ok { + r0 = rf(spec, arguments) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(cadence.Value) + } + } + + if rf, ok := ret.Get(1).(func(environment.ContractFunctionSpec, []cadence.Value) error); ok { + r1 = rf(spec, arguments) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // IsServiceAccountAuthorizer provides a mock function with given fields: func (_m *Environment) IsServiceAccountAuthorizer() bool { ret := _m.Called() diff --git a/fvm/environment/system_contracts.go b/fvm/environment/system_contracts.go index 2b59445f2a4..cd7c9db9339 100644 --- a/fvm/environment/system_contracts.go +++ b/fvm/environment/system_contracts.go @@ -12,15 +12,6 @@ import ( "github.com/onflow/flow-go/module/trace" ) -// ContractFunctionSpec specify all the information, except the function's -// address and arguments, needed to invoke the contract function. -type ContractFunctionSpec struct { - AddressFromChain func(flow.Chain) flow.Address - LocationName string - FunctionName string - ArgumentTypes []sema.Type -} - // SystemContracts provides methods for invoking system contract functions as // service account. type SystemContracts struct { diff --git a/fvm/evm/testutils/backend.go b/fvm/evm/testutils/backend.go index c25a3626b0c..d93dd3e20fa 100644 --- a/fvm/evm/testutils/backend.go +++ b/fvm/evm/testutils/backend.go @@ -33,11 +33,12 @@ func RunWithTestFlowEVMRootAddress(t testing.TB, backend atree.Ledger, f func(fl func RunWithTestBackend(t testing.TB, f func(*TestBackend)) { tb := &TestBackend{ - TestValueStore: GetSimpleValueStore(), - testEventEmitter: getSimpleEventEmitter(), - testMeter: getSimpleMeter(), - TestBlockInfo: &TestBlockInfo{}, - TestRandomGenerator: getSimpleRandomGenerator(), + TestValueStore: GetSimpleValueStore(), + testEventEmitter: getSimpleEventEmitter(), + testMeter: getSimpleMeter(), + TestBlockInfo: &TestBlockInfo{}, + TestRandomGenerator: getSimpleRandomGenerator(), + TestContractFunctionInvoker: &TestContractFunctionInvoker{}, } f(tb) } @@ -160,6 +161,7 @@ type TestBackend struct { *testEventEmitter *TestBlockInfo *TestRandomGenerator + *TestContractFunctionInvoker } var _ types.Backend = &TestBackend{} @@ -430,3 +432,28 @@ func getSimpleRandomGenerator() *TestRandomGenerator { }, } } + +type TestContractFunctionInvoker struct { + InvokeFunc func( + spec environment.ContractFunctionSpec, + arguments []cadence.Value, + ) ( + cadence.Value, + error, + ) +} + +var _ environment.ContractFunctionInvoker = &TestContractFunctionInvoker{} + +func (t *TestContractFunctionInvoker) Invoke( + spec environment.ContractFunctionSpec, + arguments []cadence.Value, +) ( + cadence.Value, + error, +) { + if t.InvokeFunc == nil { + panic("InvokeFunc method is not set") + } + return t.InvokeFunc(spec, arguments) +} diff --git a/fvm/evm/types/handler.go b/fvm/evm/types/handler.go index 764ab95c788..43850a4df6a 100644 --- a/fvm/evm/types/handler.go +++ b/fvm/evm/types/handler.go @@ -50,6 +50,7 @@ type Backend interface { environment.EventEmitter environment.BlockInfo environment.RandomGenerator + environment.ContractFunctionInvoker } // AddressAllocator allocates addresses, used by the handler diff --git a/integration/go.mod b/integration/go.mod index 1cdea718d4e..d34d93663ec 100644 --- a/integration/go.mod +++ b/integration/go.mod @@ -23,8 +23,8 @@ require ( github.com/onflow/crypto v0.25.0 github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.2-0.20240206003101-928bf99024d7 github.com/onflow/flow-core-contracts/lib/go/templates v0.15.2-0.20240206003101-928bf99024d7 - github.com/onflow/flow-emulator v0.59.1-0.20240125162502-f1971b5d4630 - github.com/onflow/flow-go v0.33.2-0.20240125033624-f3bd7a6004cf + github.com/onflow/flow-emulator v1.0.0-M2.0.20240213001509-046672c2367a + github.com/onflow/flow-go v0.33.2-0.20240213000303-9fb4c62e8291 github.com/onflow/flow-go-sdk v1.0.0-M3 github.com/onflow/flow-go/insecure v0.0.0-00010101000000-000000000000 github.com/onflow/flow/protobuf/go/flow v0.3.7 diff --git a/integration/go.sum b/integration/go.sum index 5ce63017778..7f5d67e809f 100644 --- a/integration/go.sum +++ b/integration/go.sum @@ -2546,8 +2546,8 @@ github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.2-0.20240206003101- github.com/onflow/flow-core-contracts/lib/go/contracts v0.15.2-0.20240206003101-928bf99024d7/go.mod h1:GK+Ik1K3L3v8xmHmRQv5yxJz81lYhdYSNm0PQ63Xrws= github.com/onflow/flow-core-contracts/lib/go/templates v0.15.2-0.20240206003101-928bf99024d7 h1:WAx8ftVz1BeXiKvQ9gLKEf1J3NBWK26Pbczd0iH4C6I= github.com/onflow/flow-core-contracts/lib/go/templates v0.15.2-0.20240206003101-928bf99024d7/go.mod h1:MZ2j5YVTQiSE0B99zuaYhxvGG5GcvimWpQK1Fw/1QBg= -github.com/onflow/flow-emulator v0.59.1-0.20240125162502-f1971b5d4630 h1:2ipUBOI0N5HfPWgFgZll9LRee0zrZMFif5RAxKGv8sQ= -github.com/onflow/flow-emulator v0.59.1-0.20240125162502-f1971b5d4630/go.mod h1:ZU4Rxte3YVURNoQAhhtPAm+g6DgjUc6VP2FWViQ6Q+A= +github.com/onflow/flow-emulator v1.0.0-M2.0.20240213001509-046672c2367a h1:Si1w6KjM+tPq7LjVwt7QX3Lh4nbUZ4quNWhdtZwsPLQ= +github.com/onflow/flow-emulator v1.0.0-M2.0.20240213001509-046672c2367a/go.mod h1:G+0/cyxeqsZ3SnJPrl0dnbtBooz05v077pWqHBgpHBU= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20240205224107-320aa3cf09e0 h1:u6/YcUvO8jU0f3Evb/6agzXqeOo+VbL2a3mmj/5ifRs= github.com/onflow/flow-ft/lib/go/contracts v0.7.1-0.20240205224107-320aa3cf09e0/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-go-sdk v1.0.0-M1/go.mod h1:TDW0MNuCs4SvqYRUzkbRnRmHQL1h4X8wURsCw9P9beo= diff --git a/integration/localnet/builder/bootstrap.go b/integration/localnet/builder/bootstrap.go index 5dce8572a98..56ad1a947a9 100644 --- a/integration/localnet/builder/bootstrap.go +++ b/integration/localnet/builder/bootstrap.go @@ -429,11 +429,11 @@ func prepareAccessService(container testnet.ContainerConfig, i int, n int) Servi "--log-tx-time-to-finalized-executed", "--execution-data-sync-enabled=true", "--execution-data-dir=/data/execution-data", + "--public-network-execution-data-sync-enabled=true", "--execution-data-indexing-enabled=true", "--execution-state-dir=/data/execution-state", "--script-execution-mode=execution-nodes-only", "--event-query-mode=execution-nodes-only", - fmt.Sprintf("--state-stream-addr=%s:%s", container.ContainerName, testnet.ExecutionStatePort), ) service.AddExposedPorts( @@ -464,6 +464,8 @@ func prepareObserverService(i int, observerName string, agPublicKey string) Serv fmt.Sprintf("--secure-rpc-addr=%s:%s", observerName, testnet.GRPCSecurePort), fmt.Sprintf("--http-addr=%s:%s", observerName, testnet.GRPCWebPort), fmt.Sprintf("--rest-addr=%s:%s", observerName, testnet.RESTPort), + "--execution-data-dir=/data/execution-data", + "--execution-data-sync-enabled=true", ) service.AddExposedPorts( diff --git a/network/channels/channels.go b/network/channels/channels.go index 817e11c54db..8de000a799d 100644 --- a/network/channels/channels.go +++ b/network/channels/channels.go @@ -102,6 +102,7 @@ func PublicChannels() ChannelList { return ChannelList{ PublicSyncCommittee, PublicReceiveBlocks, + PublicExecutionDataService, } } @@ -154,9 +155,10 @@ const ( ProvideApprovalsByChunk = RequestApprovalsByChunk // Public network channels - PublicPushBlocks = Channel("public-push-blocks") - PublicReceiveBlocks = PublicPushBlocks - PublicSyncCommittee = Channel("public-sync-committee") + PublicPushBlocks = Channel("public-push-blocks") + PublicReceiveBlocks = PublicPushBlocks + PublicSyncCommittee = Channel("public-sync-committee") + PublicExecutionDataService = Channel("public-execution-data-service") // Execution data service ExecutionDataService = Channel("execution-data-service")