From ade23d5bca0cfcc8784c703519990400e634551b Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Wed, 27 Nov 2024 20:25:42 +0100 Subject: [PATCH 01/12] Use flow-go Components for composing --- api/profiler.go | 80 +- api/server.go | 113 ++- api/stream.go | 14 +- bootstrap/bootstrap.go | 1085 +++++++++++++------------- bootstrap/builder.go | 609 +++++++++++++++ cmd/run/cmd.go | 55 +- go.mod | 102 ++- go.sum | 369 ++++++++- models/engine.go | 55 -- models/stream.go | 98 ++- models/stream_test.go | 63 +- services/ingestion/engine.go | 44 +- services/ingestion/engine_test.go | 1204 +++++++++++++++-------------- services/signer/signer.go | 92 +++ 14 files changed, 2556 insertions(+), 1427 deletions(-) create mode 100644 bootstrap/builder.go delete mode 100644 models/engine.go create mode 100644 services/signer/signer.go diff --git a/api/profiler.go b/api/profiler.go index cda1b8b89..973ba349b 100644 --- a/api/profiler.go +++ b/api/profiler.go @@ -7,16 +7,24 @@ import ( "net/http" _ "net/http/pprof" "strconv" + "time" + + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" "github.com/rs/zerolog" ) type ProfileServer struct { - logger zerolog.Logger + log zerolog.Logger server *http.Server endpoint string + + startupCompleted chan struct{} } +var _ component.Component = (*ProfileServer)(nil) + func NewProfileServer( logger zerolog.Logger, host string, @@ -24,37 +32,69 @@ func NewProfileServer( ) *ProfileServer { endpoint := net.JoinHostPort(host, strconv.Itoa(port)) return &ProfileServer{ - logger: logger, - server: &http.Server{Addr: endpoint}, - endpoint: endpoint, + log: logger, + server: &http.Server{Addr: endpoint}, + endpoint: endpoint, + startupCompleted: make(chan struct{}), } } -func (s *ProfileServer) ListenAddr() string { - return s.endpoint -} +func (s *ProfileServer) Start(ctx irrecoverable.SignalerContext) { + defer close(s.startupCompleted) + + s.server.BaseContext = func(_ net.Listener) context.Context { + return ctx + } -func (s *ProfileServer) Start() { go func() { - err := s.server.ListenAndServe() - if err != nil { + s.log.Info().Msgf("Profiler server started: %s", s.endpoint) + + if err := s.server.ListenAndServe(); err != nil { + // http.ErrServerClosed is returned when Close or Shutdown is called + // we don't consider this an error, so print this with debug level instead if errors.Is(err, http.ErrServerClosed) { - s.logger.Warn().Msg("Profiler server shutdown") - return + s.log.Debug().Err(err).Msg("Profiler server shutdown") + } else { + s.log.Err(err).Msg("error running profiler server") } - s.logger.Err(err).Msg("failed to start Profiler server") - panic(err) } }() } -func (s *ProfileServer) Stop() error { - ctx, cancel := context.WithTimeout(context.Background(), shutdownTimeout) - defer cancel() +func (s *ProfileServer) Ready() <-chan struct{} { + ready := make(chan struct{}) + + go func() { + <-s.startupCompleted + close(ready) + }() - return s.server.Shutdown(ctx) + return ready } -func (s *ProfileServer) Close() error { - return s.server.Close() +func (s *ProfileServer) Done() <-chan struct{} { + done := make(chan struct{}) + go func() { + <-s.startupCompleted + defer close(done) + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + err := s.server.Shutdown(ctx) + if err == nil { + s.log.Info().Msg("Profiler server graceful shutdown completed") + } + + if errors.Is(err, ctx.Err()) { + s.log.Warn().Msg("Profiler server graceful shutdown timed out") + err := s.server.Close() + if err != nil { + s.log.Err(err).Msg("error closing profiler server") + } + } else { + s.log.Err(err).Msg("error shutting down profiler server") + } + }() + return done } diff --git a/api/server.go b/api/server.go index 0883edb3c..f964da028 100644 --- a/api/server.go +++ b/api/server.go @@ -18,7 +18,9 @@ import ( "strings" "time" - "github.com/onflow/go-ethereum/core" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + gethVM "github.com/onflow/go-ethereum/core/vm" gethLog "github.com/onflow/go-ethereum/log" "github.com/onflow/go-ethereum/rpc" @@ -57,8 +59,12 @@ type Server struct { config config.Config collector metrics.Collector + + startupCompleted chan struct{} } +var _ component.Component = (*Server)(nil) + const ( shutdownTimeout = 5 * time.Second batchRequestLimit = 50 @@ -79,10 +85,11 @@ func NewServer( gethLog.SetDefault(gethLog.NewLogger(zeroSlog)) return &Server{ - logger: logger, - timeouts: rpc.DefaultHTTPTimeouts, - config: cfg, - collector: collector, + logger: logger, + timeouts: rpc.DefaultHTTPTimeouts, + config: cfg, + collector: collector, + startupCompleted: make(chan struct{}), } } @@ -179,9 +186,10 @@ func (h *Server) disableWS() bool { } // Start starts the HTTP server if it is enabled and not already running. -func (h *Server) Start() error { +func (h *Server) Start(ctx irrecoverable.SignalerContext) { + defer close(h.startupCompleted) if h.endpoint == "" || h.listener != nil { - return nil // already running or not configured + return // already running or not configured } // Initialize the server. @@ -192,16 +200,21 @@ func (h *Server) Start() error { h.server.ReadHeaderTimeout = h.timeouts.ReadHeaderTimeout h.server.WriteTimeout = h.timeouts.WriteTimeout h.server.IdleTimeout = h.timeouts.IdleTimeout + h.server.BaseContext = func(_ net.Listener) context.Context { + return ctx + } } + listenConfig := net.ListenConfig{} // Start the server. - listener, err := net.Listen("tcp", h.endpoint) + listener, err := listenConfig.Listen(ctx, "tcp", h.endpoint) if err != nil { // If the server fails to start, we need to clear out the RPC and WS // configurations so they can be configured another time. h.disableRPC() h.disableWS() - return err + ctx.Throw(err) + return } h.listener = listener @@ -213,7 +226,7 @@ func (h *Server) Start() error { return } h.logger.Err(err).Msg("failed to start API server") - panic(err) + ctx.Throw(err) } }() @@ -225,8 +238,17 @@ func (h *Server) Start() error { url := fmt.Sprintf("ws://%v", listener.Addr()) h.logger.Info().Msgf("JSON-RPC over WebSocket enabled: %s", url) } +} - return nil +func (h *Server) Ready() <-chan struct{} { + ready := make(chan struct{}) + + go func() { + <-h.startupCompleted + close(ready) + }() + + return ready } // disableRPC stops the JSON-RPC over HTTP handler. @@ -296,41 +318,50 @@ func (h *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { w.WriteHeader(http.StatusNotFound) } -// Stop shuts down the HTTP server. -func (h *Server) Stop() { - if h.listener == nil { - return // not running - } +// Done shuts down the HTTP server. +func (h *Server) Done() <-chan struct{} { + done := make(chan struct{}) - // Shut down the server. - httpHandler := h.httpHandler - if httpHandler != nil { - httpHandler.server.Stop() - h.httpHandler = nil - } + go func() { + defer close(done) - wsHandler := h.wsHandler - if wsHandler != nil { - wsHandler.server.Stop() - h.wsHandler = nil - } + if h.listener == nil { + return // not running + } - ctx, cancel := context.WithTimeout(context.Background(), shutdownTimeout) - defer cancel() - err := h.server.Shutdown(ctx) - if err != nil && err == ctx.Err() { - h.logger.Warn().Msg("HTTP server graceful shutdown timed out") - h.server.Close() - } + // Shut down the server. + httpHandler := h.httpHandler + if httpHandler != nil { + httpHandler.server.Stop() + h.httpHandler = nil + } + + wsHandler := h.wsHandler + if wsHandler != nil { + wsHandler.server.Stop() + h.wsHandler = nil + } + + ctx, cancel := context.WithTimeout(context.Background(), shutdownTimeout) + defer cancel() + err := h.server.Shutdown(ctx) + if err != nil && err == ctx.Err() { + h.logger.Warn().Msg("HTTP server graceful shutdown timed out") + h.server.Close() + } - h.listener.Close() - h.logger.Info().Msgf( - "HTTP server stopped, endpoint: %s", h.listener.Addr(), - ) + h.listener.Close() + h.logger.Info().Msgf( + "HTTP server stopped, endpoint: %s", h.listener.Addr(), + ) + + // Clear out everything to allow re-configuring it later. + h.host, h.port, h.endpoint = "", 0, "" + h.server, h.listener = nil, nil + + }() - // Clear out everything to allow re-configuring it later. - h.host, h.port, h.endpoint = "", 0, "" - h.server, h.listener = nil, nil + return done } // CheckTimeouts ensures that timeout values are meaningful diff --git a/api/stream.go b/api/stream.go index 98a8f4fba..a68b6d4ce 100644 --- a/api/stream.go +++ b/api/stream.go @@ -178,7 +178,7 @@ func newSubscription[T any]( rpcSub := notifier.CreateSubscription() - subs := models.NewSubscription(logger, callback(notifier, rpcSub)) + subs := models.NewSubscription(callback(notifier, rpcSub)) l := logger.With(). Str("gateway-subscription-id", fmt.Sprintf("%p", subs)). @@ -190,16 +190,8 @@ func newSubscription[T any]( go func() { defer publisher.Unsubscribe(subs) - for { - select { - case err := <-subs.Error(): - l.Debug().Err(err).Msg("subscription returned error") - return - case err := <-rpcSub.Err(): - l.Debug().Err(err).Msg("client unsubscribed") - return - } - } + err := <-rpcSub.Err() + l.Debug().Err(err).Msg("client unsubscribed") }() l.Info().Msg("new heads subscription created") diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 0499bad8b..aa529df91 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -1,36 +1,20 @@ package bootstrap import ( - "context" - "errors" "fmt" - "math" - "time" - pebbleDB "github.com/cockroachdb/pebble" - "github.com/onflow/flow-evm-gateway/metrics" + "github.com/onflow/flow-evm-gateway/models" + "github.com/onflow/flow-evm-gateway/services/requester" + "github.com/onflow/flow-evm-gateway/storage" + "github.com/onflow/flow-evm-gateway/storage/pebble" "github.com/onflow/flow-go-sdk/access" "github.com/onflow/flow-go-sdk/access/grpc" - "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/fvm/evm" - flowGo "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/component" - flowMetrics "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/module/util" gethTypes "github.com/onflow/go-ethereum/core/types" + "github.com/rs/zerolog" - "github.com/sethvargo/go-limiter/memorystore" grpcOpts "google.golang.org/grpc" - "github.com/onflow/flow-evm-gateway/api" "github.com/onflow/flow-evm-gateway/config" - "github.com/onflow/flow-evm-gateway/models" - errs "github.com/onflow/flow-evm-gateway/models/errors" - "github.com/onflow/flow-evm-gateway/services/ingestion" - "github.com/onflow/flow-evm-gateway/services/replayer" - "github.com/onflow/flow-evm-gateway/services/requester" - "github.com/onflow/flow-evm-gateway/storage" - "github.com/onflow/flow-evm-gateway/storage/pebble" ) type Storages struct { @@ -48,405 +32,389 @@ type Publishers struct { Logs *models.Publisher[[]*gethTypes.Log] } -type Bootstrap struct { - logger zerolog.Logger - config config.Config - client *requester.CrossSporkClient - storages *Storages - publishers *Publishers - collector metrics.Collector - server *api.Server - metrics *flowMetrics.Server - events *ingestion.Engine - profiler *api.ProfileServer - db *pebbleDB.DB - keystore *requester.KeyStore -} - -func New(config config.Config) (*Bootstrap, error) { - logger := zerolog.New(config.LogWriter). - With().Timestamp().Str("version", api.Version). - Logger().Level(config.LogLevel) - - client, err := setupCrossSporkClient(config, logger) - if err != nil { - return nil, err - } - - db, storages, err := setupStorage(config, client, logger) - if err != nil { - return nil, err - } - - return &Bootstrap{ - publishers: &Publishers{ - Block: models.NewPublisher[*models.Block](), - Transaction: models.NewPublisher[*gethTypes.Transaction](), - Logs: models.NewPublisher[[]*gethTypes.Log](), - }, - db: db, - storages: storages, - logger: logger, - config: config, - client: client, - collector: metrics.NewCollector(logger), - }, nil -} - -func (b *Bootstrap) StartEventIngestion(ctx context.Context) error { - l := b.logger.With().Str("component", "bootstrap-ingestion").Logger() - l.Info().Msg("bootstrap starting event ingestion") - - // get latest cadence block from the network and the database - latestCadenceBlock, err := b.client.GetLatestBlock(context.Background(), true) - if err != nil { - return fmt.Errorf("failed to get latest cadence block: %w", err) - } - - latestCadenceHeight, err := b.storages.Blocks.LatestCadenceHeight() - if err != nil { - return err - } - - // make sure the provided block to start the indexing can be loaded - _, err = b.client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) - if err != nil { - return fmt.Errorf( - "failed to get provided cadence height %d: %w", - latestCadenceHeight, - err, - ) - } - - l.Info(). - Uint64("start-cadence-height", latestCadenceHeight). - Uint64("latest-cadence-height", latestCadenceBlock.Height). - Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). - Msg("indexing cadence height information") - - chainID := b.config.FlowNetworkID - - // create event subscriber - subscriber := ingestion.NewRPCEventSubscriber( - b.logger, - b.client, - chainID, - b.keystore, - latestCadenceHeight, - ) - - callTracerCollector, err := replayer.NewCallTracerCollector(b.logger) - if err != nil { - return err - } - blocksProvider := replayer.NewBlocksProvider( - b.storages.Blocks, - chainID, - callTracerCollector.TxTracer(), - ) - replayerConfig := replayer.Config{ - ChainID: chainID, - RootAddr: evm.StorageAccountAddress(chainID), - CallTracerCollector: callTracerCollector, - ValidateResults: true, - } - - // initialize event ingestion engine - b.events = ingestion.NewEventIngestionEngine( - subscriber, - blocksProvider, - b.storages.Storage, - b.storages.Registers, - b.storages.Blocks, - b.storages.Receipts, - b.storages.Transactions, - b.storages.Traces, - b.publishers.Block, - b.publishers.Logs, - b.logger, - b.collector, - replayerConfig, - ) - - StartEngine(ctx, b.events, l) - return nil -} - -func (b *Bootstrap) StopEventIngestion() { - if b.events == nil { - return - } - b.logger.Warn().Msg("stopping event ingestion engine") - b.events.Stop() -} - -func (b *Bootstrap) StartAPIServer(ctx context.Context) error { - b.logger.Info().Msg("bootstrap starting metrics server") - - b.server = api.NewServer(b.logger, b.collector, b.config) - - // create transaction pool - txPool := requester.NewTxPool( - b.client, - b.publishers.Transaction, - b.logger, - b.config, - ) - - blocksProvider := replayer.NewBlocksProvider( - b.storages.Blocks, - b.config.FlowNetworkID, - nil, - ) - - accountKeys := make([]*requester.AccountKey, 0) - if !b.config.IndexOnly { - account, err := b.client.GetAccount(ctx, b.config.COAAddress) - if err != nil { - return fmt.Errorf( - "failed to get signer info account for address: %s, with: %w", - b.config.COAAddress, - err, - ) - } - signer, err := createSigner(ctx, b.config, b.logger) - if err != nil { - return err - } - for _, key := range account.Keys { - // Skip account keys that do not use the same Publick Key as the - // configured crypto.Signer object. - if !key.PublicKey.Equals(signer.PublicKey()) { - continue - } - accountKeys = append(accountKeys, &requester.AccountKey{ - AccountKey: *key, - Address: b.config.COAAddress, - Signer: signer, - }) - } - } - - b.keystore = requester.NewKeyStore(accountKeys) - - evm, err := requester.NewEVM( - b.storages.Registers, - blocksProvider, - b.client, - b.config, - b.logger, - b.storages.Blocks, - txPool, - b.collector, - b.keystore, - ) - if err != nil { - return fmt.Errorf("failed to create EVM requester: %w", err) - } - - // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval - // if no limit is defined we specify max value, effectively disabling rate-limiting - rateLimit := b.config.RateLimit - if rateLimit == 0 { - b.logger.Warn().Msg("no rate-limiting is set") - rateLimit = math.MaxInt - } - ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) - if err != nil { - return fmt.Errorf("failed to create rate limiter: %w", err) - } - - // get the height from which the indexing resumed since the last restart, - // this is needed for the `eth_syncing` endpoint. - indexingResumedHeight, err := b.storages.Blocks.LatestEVMHeight() - if err != nil { - return fmt.Errorf("failed to retrieve the indexing resumed height: %w", err) - } - - blockchainAPI := api.NewBlockChainAPI( - b.logger, - b.config, - evm, - b.storages.Blocks, - b.storages.Transactions, - b.storages.Receipts, - ratelimiter, - b.collector, - indexingResumedHeight, - ) - - streamAPI := api.NewStreamAPI( - b.logger, - b.config, - b.storages.Blocks, - b.storages.Transactions, - b.storages.Receipts, - b.publishers.Block, - b.publishers.Transaction, - b.publishers.Logs, - ) - - pullAPI := api.NewPullAPI( - b.logger, - b.config, - b.storages.Blocks, - b.storages.Transactions, - b.storages.Receipts, - ratelimiter, - ) - - debugAPI := api.NewDebugAPI( - b.storages.Registers, - b.storages.Traces, - b.storages.Blocks, - b.storages.Transactions, - b.storages.Receipts, - b.client, - b.config, - b.logger, - b.collector, - ratelimiter, - ) - - var walletAPI *api.WalletAPI - if b.config.WalletEnabled { - walletAPI = api.NewWalletAPI(b.config, blockchainAPI) - } - - supportedAPIs := api.SupportedAPIs( - blockchainAPI, - streamAPI, - pullAPI, - debugAPI, - walletAPI, - b.config, - ) - - if err := b.server.EnableRPC(supportedAPIs); err != nil { - return err - } - - if b.config.WSEnabled { - if err := b.server.EnableWS(supportedAPIs); err != nil { - return err - } - } - - if err := b.server.SetListenAddr(b.config.RPCHost, b.config.RPCPort); err != nil { - return err - } - - if err := b.server.Start(); err != nil { - return err - } - - b.logger.Info().Msgf("API server started: %s", b.server.ListenAddr()) - return nil -} - -func (b *Bootstrap) StopAPIServer() { - if b.server == nil { - return - } - b.logger.Warn().Msg("shutting down API server") - b.server.Stop() -} - -func (b *Bootstrap) StartMetricsServer(ctx context.Context) error { - b.logger.Info().Msg("bootstrap starting metrics server") - - b.metrics = flowMetrics.NewServer(b.logger, uint(b.config.MetricsPort)) - err := util.WaitClosed(ctx, b.metrics.Ready()) - if err != nil { - return fmt.Errorf("failed to start metrics server: %w", err) - } - - return nil -} - -func (b *Bootstrap) StopMetricsServer() { - if b.metrics == nil { - return - } - b.logger.Warn().Msg("shutting down metrics server") - <-b.metrics.Done() -} - -func (b *Bootstrap) StartProfilerServer(_ context.Context) error { - if !b.config.ProfilerEnabled { - return nil - } - b.logger.Info().Msg("bootstrap starting profiler server") - - b.profiler = api.NewProfileServer(b.logger, b.config.ProfilerHost, b.config.ProfilerPort) - - b.profiler.Start() - b.logger.Info().Msgf("Profiler server started: %s", b.profiler.ListenAddr()) - - return nil -} - -func (b *Bootstrap) StopProfilerServer() { - if b.profiler == nil { - return - } - - b.logger.Warn().Msg("shutting down profiler server") - - err := b.profiler.Stop() - if err != nil { - if errors.Is(err, context.DeadlineExceeded) { - b.logger.Warn().Msg("Profiler server graceful shutdown timed out") - b.profiler.Close() - } else { - b.logger.Err(err).Msg("Profiler server graceful shutdown failed") - } - } -} - -func (b *Bootstrap) StopDB() { - if b.db == nil { - return - } - err := b.db.Close() - if err != nil { - b.logger.Err(err).Msg("PebbleDB graceful shutdown failed") - } -} - -func (b *Bootstrap) StopClient() { - if b.client == nil { - return - } - err := b.client.Close() - if err != nil { - b.logger.Err(err).Msg("CrossSporkClient graceful shutdown failed") - } -} - -// StartEngine starts provided engine and panics if there are startup errors. -func StartEngine( - ctx context.Context, - engine models.Engine, - logger zerolog.Logger, -) { - l := logger.With().Type("engine", engine).Logger() - - l.Info().Msg("starting engine") - start := time.Now() - go func() { - err := engine.Run(ctx) - if err != nil { - l.Fatal().Err(err).Msg("engine failed to run") - } - }() - - <-engine.Ready() - l.Info(). - Dur("duration", time.Since(start)). - Msg("engine started successfully") -} - +// type Bootstrap struct { +// logger zerolog.Logger +// config config.Config +// client *requester.CrossSporkClient +// storages *Storages +// publishers *Publishers +// collector metrics.Collector +// server *api.Server +// metrics *flowMetrics.Server +// events *ingestion.Engine +// profiler *api.ProfileServer +// db *pebbleDB.DB +// } +// +// func New(config config.Config) (*Bootstrap, error) { +// logger := zerolog.New(config.LogWriter). +// With().Timestamp().Str("version", api.Version). +// Logger().Level(config.LogLevel) +// +// client, err := setupCrossSporkClient(config, logger) +// if err != nil { +// return nil, err +// } +// +// db, storages, err := setupStorage(config, client, logger) +// if err != nil { +// return nil, err +// } +// +// return &Bootstrap{ +// publishers: &Publishers{ +// Block: models.NewPublisher[*models.Block](), +// Transaction: models.NewPublisher[*gethTypes.Transaction](), +// Logs: models.NewPublisher[[]*gethTypes.Log](), +// }, +// db: db, +// storages: storages, +// logger: logger, +// config: config, +// client: client, +// collector: metrics.NewCollector(logger), +// }, nil +// } +// +// func (b *Bootstrap) StartEventIngestion(ctx context.Context) error { +// l := b.logger.With().Str("component", "bootstrap-ingestion").Logger() +// l.Info().Msg("bootstrap starting event ingestion") +// +// // get latest cadence block from the network and the database +// latestCadenceBlock, err := b.client.GetLatestBlock(context.Background(), true) +// if err != nil { +// return fmt.Errorf("failed to get latest cadence block: %w", err) +// } +// +// latestCadenceHeight, err := b.storages.Blocks.LatestCadenceHeight() +// if err != nil { +// return err +// } +// +// // make sure the provided block to start the indexing can be loaded +// _, err = b.client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) +// if err != nil { +// return fmt.Errorf( +// "failed to get provided cadence height %d: %w", +// latestCadenceHeight, +// err, +// ) +// } +// +// l.Info(). +// Uint64("start-cadence-height", latestCadenceHeight). +// Uint64("latest-cadence-height", latestCadenceBlock.Height). +// Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). +// Msg("indexing cadence height information") +// +// chainID := b.config.FlowNetworkID +// +// // create event subscriber +// subscriber := ingestion.NewRPCEventSubscriber( +// b.logger, +// b.client, +// chainID, +// latestCadenceHeight, +// ) +// +// callTracerCollector, err := replayer.NewCallTracerCollector(b.logger) +// if err != nil { +// return err +// } +// blocksProvider := replayer.NewBlocksProvider( +// b.storages.Blocks, +// chainID, +// callTracerCollector.TxTracer(), +// ) +// replayerConfig := replayer.Config{ +// ChainID: chainID, +// RootAddr: evm.StorageAccountAddress(chainID), +// CallTracerCollector: callTracerCollector, +// ValidateResults: true, +// } +// +// // initialize event ingestion engine +// b.events = ingestion.NewEventIngestionEngine( +// subscriber, +// blocksProvider, +// b.storages.Storage, +// b.storages.Registers, +// b.storages.Blocks, +// b.storages.Receipts, +// b.storages.Transactions, +// b.storages.Traces, +// b.publishers.Block, +// b.publishers.Logs, +// b.logger, +// b.collector, +// replayerConfig, +// ) +// +// StartEngine(ctx, b.events, l) +// return nil +// } +// +// func (b *Bootstrap) StopEventIngestion() { +// if b.events == nil { +// return +// } +// b.logger.Warn().Msg("stopping event ingestion engine") +// b.events.Stop() +// } +// +// func (b *Bootstrap) StartAPIServer(ctx context.Context) error { +// b.logger.Info().Msg("bootstrap starting Metrics server") +// +// b.server = api.NewServer(b.logger, b.collector, b.config) +// +// // create the signer based on either a single coa key being provided and using a simple in-memory +// // signer, or multiple keys being provided and using signer with key-rotation mechanism.bootstrap/bootstrap.go +// var signer crypto.Signer +// var err error +// switch { +// case b.config.COAKey != nil: +// signer, err = crypto.NewInMemorySigner(b.config.COAKey, crypto.SHA3_256) +// case b.config.COAKeys != nil: +// signer, err = requester.NewKeyRotationSigner(b.config.COAKeys, crypto.SHA3_256) +// case len(b.config.COACloudKMSKeys) > 0: +// signer, err = requester.NewKMSKeyRotationSigner( +// ctx, +// b.config.COACloudKMSKeys, +// b.logger, +// ) +// default: +// return fmt.Errorf("must provide either single COA / keylist of COA keys / COA cloud KMS keys") +// } +// if err != nil { +// return fmt.Errorf("failed to create a COA signer: %w", err) +// } +// +// // create transaction pool +// txPool := requester.NewTxPool( +// b.client, +// b.publishers.Transaction, +// b.logger, +// ) +// +// blocksProvider := replayer.NewBlocksProvider( +// b.storages.Blocks, +// b.config.FlowNetworkID, +// nil, +// ) +// +// evm, err := requester.NewEVM( +// b.storages.Registers, +// blocksProvider, +// b.client, +// b.config, +// signer, +// b.logger, +// b.storages.Blocks, +// txPool, +// b.collector, +// ) +// if err != nil { +// return fmt.Errorf("failed to create EVM requester: %w", err) +// } +// +// // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval +// // if no limit is defined we specify max value, effectively disabling rate-limiting +// rateLimit := b.config.RateLimit +// if rateLimit == 0 { +// b.logger.Warn().Msg("no rate-limiting is set") +// rateLimit = math.MaxInt +// } +// ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) +// if err != nil { +// return fmt.Errorf("failed to create rate limiter: %w", err) +// } +// +// blockchainAPI, err := api.NewBlockChainAPI( +// b.logger, +// b.config, +// evm, +// b.storages.Blocks, +// b.storages.Transactions, +// b.storages.Receipts, +// ratelimiter, +// b.collector, +// ) +// if err != nil { +// return err +// } +// +// streamAPI := api.NewStreamAPI( +// b.logger, +// b.config, +// b.storages.Blocks, +// b.storages.Transactions, +// b.storages.Receipts, +// b.publishers.Block, +// b.publishers.Transaction, +// b.publishers.Logs, +// ) +// +// pullAPI := api.NewPullAPI( +// b.logger, +// b.config, +// b.storages.Blocks, +// b.storages.Transactions, +// b.storages.Receipts, +// ratelimiter, +// ) +// +// debugAPI := api.NewDebugAPI( +// b.storages.Registers, +// b.storages.Traces, +// b.storages.Blocks, +// b.storages.Transactions, +// b.storages.Receipts, +// b.client, +// b.config, +// b.logger, +// b.collector, +// ) +// +// var walletAPI *api.WalletAPI +// if b.config.WalletEnabled { +// walletAPI = api.NewWalletAPI(b.config, blockchainAPI) +// } +// +// supportedAPIs := api.SupportedAPIs( +// blockchainAPI, +// streamAPI, +// pullAPI, +// debugAPI, +// walletAPI, +// b.config, +// ) +// +// if err := b.server.EnableRPC(supportedAPIs); err != nil { +// return err +// } +// +// if b.config.WSEnabled { +// if err := b.server.EnableWS(supportedAPIs); err != nil { +// return err +// } +// } +// +// if err := b.server.SetListenAddr(b.config.RPCHost, b.config.RPCPort); err != nil { +// return err +// } +// +// if err := b.server.Start(); err != nil { +// return err +// } +// +// b.logger.Info().Msgf("API server started: %s", b.server.ListenAddr()) +// return nil +// } +// +// func (b *Bootstrap) StopAPIServer() { +// if b.server == nil { +// return +// } +// b.logger.Warn().Msg("shutting down API server") +// b.server.Stop() +// } +// +// func (b *Bootstrap) StartMetricsServer(ctx context.Context) error { +// b.logger.Info().Msg("bootstrap starting Metrics server") +// +// b.metrics = flowMetrics.NewServer(b.logger, uint(b.config.MetricsPort)) +// err := util.WaitClosed(ctx, b.metrics.Ready()) +// if err != nil { +// return fmt.Errorf("failed to start Metrics server: %w", err) +// } +// +// return nil +// } +// +// func (b *Bootstrap) StopMetricsServer() { +// if b.metrics == nil { +// return +// } +// b.logger.Warn().Msg("shutting down Metrics server") +// <-b.metrics.Done() +// } +// +// func (b *Bootstrap) StartProfilerServer(_ context.Context) error { +// if !b.config.ProfilerEnabled { +// return nil +// } +// b.logger.Info().Msg("bootstrap starting profiler server") +// +// b.profiler = api.NewProfileServer(b.logger, b.config.ProfilerHost, b.config.ProfilerPort) +// +// b.profiler.Start() +// +// return nil +// } +// +// func (b *Bootstrap) StopProfilerServer() { +// if b.profiler == nil { +// return +// } +// +// b.logger.Warn().Msg("shutting down profiler server") +// +// err := b.profiler.Stop() +// if err != nil { +// if errors.Is(err, context.DeadlineExceeded) { +// b.logger.Warn().Msg("Profiler server graceful shutdown timed out") +// b.profiler.Close() +// } else { +// b.logger.Err(err).Msg("Profiler server graceful shutdown failed") +// } +// } +// } +// +// func (b *Bootstrap) StopDB() { +// if b.db == nil { +// return +// } +// err := b.db.Close() +// if err != nil { +// b.logger.Err(err).Msg("PebbleDB graceful shutdown failed") +// } +// } +// +// func (b *Bootstrap) StopClient() { +// if b.client == nil { +// return +// } +// err := b.client.Close() +// if err != nil { +// b.logger.Err(err).Msg("CrossSporkClient graceful shutdown failed") +// } +// } +// +// // StartEngine starts provided engine and panics if there are startup errors. +// func StartEngine( +// +// ctx context.Context, +// engine models.Engine, +// logger zerolog.Logger, +// +// ) { +// l := logger.With().Type("engine", engine).Logger() +// +// l.Info().Msg("starting engine") +// start := time.Now() +// go func() { +// err := engine.Run(ctx) +// if err != nil { +// l.Fatal().Err(err).Msg("engine failed to run") +// } +// }() +// +// <-engine.Ready() +// l.Info(). +// Dur("duration", time.Since(start)). +// Msg("engine started successfully") +// } +// // setupCrossSporkClient sets up a cross-spork AN client. func setupCrossSporkClient(config config.Config, logger zerolog.Logger) (*requester.CrossSporkClient, error) { // create access client with cross-spork capabilities @@ -487,142 +455,141 @@ func setupCrossSporkClient(config config.Config, logger zerolog.Logger) (*reques return client, nil } -// setupStorage creates storage and initializes it with configured starting cadence height -// in case such a height doesn't already exist in the database. -func setupStorage( - config config.Config, - client *requester.CrossSporkClient, - logger zerolog.Logger, -) (*pebbleDB.DB, *Storages, error) { - // create pebble storage from the provided database root directory - db, err := pebble.OpenDB(config.DatabaseDir) - if err != nil { - return nil, nil, err - } - store := pebble.New(db, logger) - - blocks := pebble.NewBlocks(store, config.FlowNetworkID) - storageAddress := evm.StorageAccountAddress(config.FlowNetworkID) - registerStore := pebble.NewRegisterStorage(store, storageAddress) - - // hard set the start cadence height, this is used when force reindexing - if config.ForceStartCadenceHeight != 0 { - logger.Warn().Uint64("height", config.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") - if err := blocks.SetLatestCadenceHeight(config.ForceStartCadenceHeight, nil); err != nil { - return nil, nil, err - } - } - - // if database is not initialized require init height - if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { - batch := store.NewBatch() - defer func(batch *pebbleDB.Batch) { - err := batch.Close() - if err != nil { - // we don't know what went wrong, so this is fatal - logger.Fatal().Err(err).Msg("failed to close batch") - } - }(batch) - - cadenceHeight := config.InitCadenceHeight - evmBlokcHeight := uint64(0) - cadenceBlock, err := client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) - if err != nil { - return nil, nil, fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) - } - - snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) - if err != nil { - return nil, nil, fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) - } - - delta := storage.NewRegisterDelta(snapshot) - accountStatus := environment.NewAccountStatus() - err = delta.SetValue( - storageAddress[:], - []byte(flowGo.AccountStatusKey), - accountStatus.ToBytes(), - ) - if err != nil { - return nil, nil, fmt.Errorf("could not set account status: %w", err) - } - - err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) - if err != nil { - return nil, nil, fmt.Errorf("could not store register updates: %w", err) - } - - if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { - return nil, nil, fmt.Errorf( - "failed to init the database for block height: %d and ID: %s, with : %w", - cadenceHeight, - cadenceBlock.ID, - err, - ) - } - - err = batch.Commit(pebbleDB.Sync) - if err != nil { - return nil, nil, fmt.Errorf("could not commit register updates: %w", err) - } - - logger.Info(). - Stringer("fvm_address_for_evm_storage_account", storageAddress). - Msgf("database initialized with cadence height: %d", cadenceHeight) - } - //else { - // // TODO(JanezP): verify storage account owner is correct - //} - - return db, &Storages{ - Storage: store, - Blocks: blocks, - Registers: registerStore, - Transactions: pebble.NewTransactions(store), - Receipts: pebble.NewReceipts(store), - Traces: pebble.NewTraces(store), - }, nil -} - -// Run will run complete bootstrap of the EVM gateway with all the engines. -// Run is a blocking call, but it does signal readiness of the service -// through a channel provided as an argument. -func Run(ctx context.Context, cfg config.Config, ready component.ReadyFunc) error { - boot, err := New(cfg) - if err != nil { - return err - } - - // Start the API Server first, to avoid any races with incoming - // EVM events, that might affect the starting state. - if err := boot.StartAPIServer(ctx); err != nil { - return fmt.Errorf("failed to start API server: %w", err) - } - - if err := boot.StartEventIngestion(ctx); err != nil { - return fmt.Errorf("failed to start event ingestion engine: %w", err) - } - - if err := boot.StartMetricsServer(ctx); err != nil { - return fmt.Errorf("failed to start metrics server: %w", err) - } - - if err := boot.StartProfilerServer(ctx); err != nil { - return fmt.Errorf("failed to start profiler server: %w", err) - } - - // mark ready - ready() - - // if context is canceled start shutdown - <-ctx.Done() - boot.logger.Warn().Msg("bootstrap received context cancellation, stopping services") - - boot.StopEventIngestion() - boot.StopMetricsServer() - boot.StopAPIServer() - boot.StopClient() - boot.StopDB() - - return nil -} +// +//// setupStorage creates storage and initializes it with configured starting cadence height +//// in case such a height doesn't already exist in the database. +//func setupStorage( +// config config.Config, +// client *requester.CrossSporkClient, +// logger zerolog.Logger, +//) (*pebbleDB.DB, *Storages, error) { +// // create pebble storage from the provided database root directory +// db, err := pebble.OpenDB(config.DatabaseDir) +// if err != nil { +// return nil, nil, err +// } +// store := pebble.New(db, logger) +// +// blocks := pebble.NewBlocks(store, config.FlowNetworkID) +// storageAddress := evm.StorageAccountAddress(config.FlowNetworkID) +// registerStore := pebble.NewRegisterStorage(store, storageAddress) +// +// // hard set the start cadence height, this is used when force reindexing +// if config.ForceStartCadenceHeight != 0 { +// logger.Warn().Uint64("height", config.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") +// if err := blocks.SetLatestCadenceHeight(config.ForceStartCadenceHeight, nil); err != nil { +// return nil, nil, err +// } +// } +// +// // if database is not initialized require init height +// if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { +// batch := store.NewBatch() +// defer func(batch *pebbleDB.Batch) { +// err := batch.Close() +// if err != nil { +// // we don't know what went wrong, so this is fatal +// logger.Fatal().Err(err).Msg("failed to close batch") +// } +// }(batch) +// +// cadenceHeight := config.InitCadenceHeight +// evmBlokcHeight := uint64(0) +// cadenceBlock, err := client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) +// if err != nil { +// return nil, nil, fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) +// } +// +// snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) +// if err != nil { +// return nil, nil, fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) +// } +// +// delta := storage.NewRegisterDelta(snapshot) +// accountStatus := environment.NewAccountStatus() +// err = delta.SetValue( +// storageAddress[:], +// []byte(flowGo.AccountStatusKey), +// accountStatus.ToBytes(), +// ) +// if err != nil { +// return nil, nil, fmt.Errorf("could not set account status: %w", err) +// } +// +// err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) +// if err != nil { +// return nil, nil, fmt.Errorf("could not store register updates: %w", err) +// } +// +// if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { +// return nil, nil, fmt.Errorf( +// "failed to init the database for block height: %d and ID: %s, with : %w", +// cadenceHeight, +// cadenceBlock.ID, +// err, +// ) +// } +// +// err = batch.Commit(pebbleDB.Sync) +// if err != nil { +// return nil, nil, fmt.Errorf("could not commit register updates: %w", err) +// } +// +// logger.Info(). +// Stringer("fvm_address_for_evm_storage_account", storageAddress). +// Msgf("database initialized with cadence height: %d", cadenceHeight) +// } +// //else { +// // // TODO(JanezP): verify storage account owner is correct +// //} +// +// return db, &Storages{ +// Storage: store, +// Blocks: blocks, +// Registers: registerStore, +// Transactions: pebble.NewTransactions(store), +// Receipts: pebble.NewReceipts(store), +// Traces: pebble.NewTraces(store), +// }, nil +//} +// +//// Run will run complete bootstrap of the EVM gateway with all the engines. +//// Run is a blocking call, but it does signal readiness of the service +//// through a channel provided as an argument. +//func Run(ctx context.Context, cfg config.Config, ready component.ReadyFunc) error { +// boot, err := New(cfg) +// if err != nil { +// return err +// } +// +// if err := boot.StartEventIngestion(ctx); err != nil { +// return fmt.Errorf("failed to start event ingestion engine: %w", err) +// } +// +// if err := boot.StartAPIServer(ctx); err != nil { +// return fmt.Errorf("failed to start API server: %w", err) +// } +// +// if err := boot.StartMetricsServer(ctx); err != nil { +// return fmt.Errorf("failed to start Metrics server: %w", err) +// } +// +// if err := boot.StartProfilerServer(ctx); err != nil { +// return fmt.Errorf("failed to start profiler server: %w", err) +// } +// +// // mark ready +// ready() +// +// // if context is canceled start shutdown +// <-ctx.Done() +// boot.logger.Warn().Msg("bootstrap received context cancellation, stopping services") +// +// boot.StopEventIngestion() +// boot.StopMetricsServer() +// boot.StopAPIServer() +// boot.StopClient() +// boot.StopDB() +// +// return nil +//} diff --git a/bootstrap/builder.go b/bootstrap/builder.go new file mode 100644 index 000000000..58d75bc7d --- /dev/null +++ b/bootstrap/builder.go @@ -0,0 +1,609 @@ +package bootstrap + +import ( + "context" + "errors" + "fmt" + "math" + "time" + + "github.com/cockroachdb/pebble" + "github.com/hashicorp/go-multierror" + "github.com/onflow/flow-evm-gateway/api" + "github.com/onflow/flow-evm-gateway/config" + metrics2 "github.com/onflow/flow-evm-gateway/metrics" + "github.com/onflow/flow-evm-gateway/models" + errs "github.com/onflow/flow-evm-gateway/models/errors" + "github.com/onflow/flow-evm-gateway/services/ingestion" + "github.com/onflow/flow-evm-gateway/services/replayer" + "github.com/onflow/flow-evm-gateway/services/requester" + "github.com/onflow/flow-evm-gateway/services/signer" + "github.com/onflow/flow-evm-gateway/storage" + pebble2 "github.com/onflow/flow-evm-gateway/storage/pebble" + "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm" + flowGo "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/metrics" + gethTypes "github.com/onflow/go-ethereum/core/types" + "github.com/rs/zerolog" + "github.com/sethvargo/go-limiter/memorystore" +) + +type EVMGatewayNodeImp struct { + cmd.NodeImp + config.Config +} + +// NewNode returns a new node instance +func NewNode( + component component.Component, + cfg config.Config, + logger zerolog.Logger, + cleanup func() error, + handleFatal func(error), +) *EVMGatewayNodeImp { + return &EVMGatewayNodeImp{ + Config: cfg, + NodeImp: cmd.NewBaseNode( + component, + logger.With(). + Str("node_role", "EVM Gateway"). + Logger(), + cleanup, + handleFatal, + ), + } +} + +type EVMGatewayNodeBuilder struct { + config.Config + Logger zerolog.Logger + componentBuilder component.ComponentManagerBuilder + components []cmd.NamedComponentFunc[config.Config] + postShutdownFns []func() error + modules []namedModuleFunc + + Metrics metrics2.Collector + DB *pebble.DB + Client *requester.CrossSporkClient + Storages *Storages + // Signer is used for signing flow transactions + Signer crypto.Signer + Publishers *Publishers +} + +func (fnb *EVMGatewayNodeBuilder) Build() (cmd.Node, error) { + // Run the prestart initialization. This includes anything that should be done before + // starting the components. + if err := fnb.onStart(); err != nil { + return nil, err + } + + return NewNode( + fnb.componentBuilder.Build(), + fnb.Config, + fnb.Logger, + fnb.postShutdown, + fnb.handleFatal, + ), nil +} + +func (fnb *EVMGatewayNodeBuilder) onStart() error { + + if err := fnb.initDB(); err != nil { + return err + } + + if err := fnb.initMetrics(); err != nil { + return err + } + + if err := fnb.initClient(); err != nil { + return err + } + + if err := fnb.initStorage(); err != nil { + return err + } + + // run all modules + if err := fnb.handleModules(); err != nil { + return fmt.Errorf("could not handle modules: %w", err) + } + + // run all components + return fnb.handleComponents() +} + +func (fnb *EVMGatewayNodeBuilder) initDB() error { + pebbleDB, err := pebble2.OpenDB(fnb.DatabaseDir) + if err != nil { + return fmt.Errorf("failed to open db for dir: %s, with: %w", fnb.DatabaseDir, err) + } + + fnb.DB = pebbleDB + + fnb.ShutdownFunc(func() error { + if err := fnb.DB.Close(); err != nil { + return fmt.Errorf("error closing pebble database: %w", err) + } + return nil + }) + + return err +} + +func (fnb *EVMGatewayNodeBuilder) Component(name string, f cmd.ReadyDoneFactory[config.Config]) *EVMGatewayNodeBuilder { + fnb.components = append(fnb.components, cmd.NamedComponentFunc[config.Config]{ + FN: f, + Name: name, + }) + return fnb +} + +// postShutdown is called by the node before exiting +// put any cleanup code here that should be run after all components have stopped +func (fnb *EVMGatewayNodeBuilder) postShutdown() error { + var errs *multierror.Error + + for _, fn := range fnb.postShutdownFns { + err := fn() + if err != nil { + errs = multierror.Append(errs, err) + } + } + fnb.Logger.Info().Msg("database has been closed") + return errs.ErrorOrNil() +} + +// handleFatal handles irrecoverable errors by logging them and exiting the process. +func (fnb *EVMGatewayNodeBuilder) handleFatal(err error) { + fnb.Logger.Fatal().Err(err).Msg("unhandled irrecoverable error") +} + +func NewEVMGatewayNodeBuilder( + config config.Config, +) *EVMGatewayNodeBuilder { + + logger := zerolog.New(config.LogWriter). + With().Timestamp().Str("version", api.Version). + Logger().Level(config.LogLevel) + + return &EVMGatewayNodeBuilder{ + Logger: logger, + Config: config, + componentBuilder: component.NewComponentManagerBuilder(), + } +} + +func (fnb *EVMGatewayNodeBuilder) Initialize() error { + fnb.PrintBuildDetails() + + fnb.EnqueueMetricsServerInit() + + return nil +} + +func (fnb *EVMGatewayNodeBuilder) LoadComponentsAndModules() { + fnb.initPublishers() + + fnb.Component("Transaction Signer", fnb.initSigner) + fnb.Component("API Server", fnb.apiServerComponent) + fnb.Component("Event Ingestion Engine", fnb.eventIngestionEngineComponent) + fnb.Component("Metrics Server", fnb.metricsServerComponent) + fnb.Component("Profiler Server", fnb.profilerServerComponent) +} + +func (fnb *EVMGatewayNodeBuilder) metricsServerComponent(config config.Config) (module.ReadyDoneAware, error) { + server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) profilerServerComponent(config config.Config) (module.ReadyDoneAware, error) { + server := api.NewProfileServer(fnb.Logger, config.ProfilerHost, config.ProfilerPort) + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module.ReadyDoneAware, error) { + log := fnb.Logger + + log.Info().Msg("bootstrap starting Metrics server") + + server := api.NewServer(log, fnb.Metrics, cfg) + + // create transaction pool + txPool := requester.NewTxPool( + fnb.Client, + fnb.Publishers.Transaction, + log, + ) + + blocksProvider := replayer.NewBlocksProvider( + fnb.Storages.Blocks, + cfg.FlowNetworkID, + nil, + ) + + evm, err := requester.NewEVM( + fnb.Storages.Registers, + blocksProvider, + fnb.Client, + cfg, + fnb.Signer, + log, + fnb.Storages.Blocks, + txPool, + fnb.Metrics, + ) + if err != nil { + return nil, fmt.Errorf("failed to create EVM requester: %w", err) + } + + // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval + // if no limit is defined we specify max value, effectively disabling rate-limiting + rateLimit := cfg.RateLimit + if rateLimit == 0 { + log.Warn().Msg("no rate-limiting is set") + rateLimit = math.MaxInt + } + ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) + if err != nil { + return nil, fmt.Errorf("failed to create rate limiter: %w", err) + } + + blockchainAPI, err := api.NewBlockChainAPI( + log, + cfg, + evm, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + ratelimiter, + fnb.Metrics, + ) + if err != nil { + return nil, fmt.Errorf("failed to create blockchain API: %w", err) + } + + streamAPI := api.NewStreamAPI( + log, + cfg, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + fnb.Publishers.Block, + fnb.Publishers.Transaction, + fnb.Publishers.Logs, + ) + + pullAPI := api.NewPullAPI( + log, + cfg, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + ratelimiter, + ) + + debugAPI := api.NewDebugAPI( + fnb.Storages.Registers, + fnb.Storages.Traces, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + fnb.Client, + cfg, + log, + fnb.Metrics, + ) + + var walletAPI *api.WalletAPI + if cfg.WalletEnabled { + walletAPI = api.NewWalletAPI(cfg, blockchainAPI) + } + + supportedAPIs := api.SupportedAPIs( + blockchainAPI, + streamAPI, + pullAPI, + debugAPI, + walletAPI, + cfg, + ) + + if err := server.EnableRPC(supportedAPIs); err != nil { + return nil, err + } + + if cfg.WSEnabled { + if err := server.EnableWS(supportedAPIs); err != nil { + return nil, err + } + } + + if err := server.SetListenAddr(cfg.RPCHost, cfg.RPCPort); err != nil { + return nil, err + } + + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Config) (module.ReadyDoneAware, error) { + l := fnb.Logger.With().Str("component", "bootstrap-ingestion").Logger() + l.Info().Msg("bootstrap starting event ingestion") + + // get latest cadence block from the network and the database + latestCadenceBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) + if err != nil { + return nil, fmt.Errorf("failed to get latest cadence block: %w", err) + } + + latestCadenceHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() + if err != nil { + return nil, err + } + + // make sure the provided block to start the indexing can be loaded + _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) + if err != nil { + return nil, fmt.Errorf( + "failed to get provided cadence height %d: %w", + latestCadenceHeight, + err, + ) + } + + l.Info(). + Uint64("start-cadence-height", latestCadenceHeight). + Uint64("latest-cadence-height", latestCadenceBlock.Height). + Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). + Msg("indexing cadence height information") + + chainID := cfg.FlowNetworkID + + // create event subscriber + subscriber := ingestion.NewRPCEventSubscriber( + fnb.Logger, + fnb.Client, + chainID, + latestCadenceHeight, + ) + + callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) + if err != nil { + return nil, err + } + blocksProvider := replayer.NewBlocksProvider( + fnb.Storages.Blocks, + chainID, + callTracerCollector.TxTracer(), + ) + replayerConfig := replayer.Config{ + ChainID: chainID, + RootAddr: evm.StorageAccountAddress(chainID), + CallTracerCollector: callTracerCollector, + ValidateResults: true, + } + + // initialize event ingestion engine + events := ingestion.NewEventIngestionEngine( + subscriber, + blocksProvider, + fnb.Storages.Storage, + fnb.Storages.Registers, + fnb.Storages.Blocks, + fnb.Storages.Receipts, + fnb.Storages.Transactions, + fnb.Storages.Traces, + fnb.Publishers.Block, + fnb.Publishers.Logs, + fnb.Logger, + fnb.Metrics, + replayerConfig, + ) + + return events, nil +} + +func (fnb *EVMGatewayNodeBuilder) PrintBuildDetails() { + fnb.Logger.Info().Str("version", api.Version).Msg("build details") +} + +// ShutdownFunc adds a callback function that is called after all components have exited. +func (fnb *EVMGatewayNodeBuilder) ShutdownFunc(fn func() error) *EVMGatewayNodeBuilder { + fnb.postShutdownFns = append(fnb.postShutdownFns, fn) + return fnb +} + +func (fnb *EVMGatewayNodeBuilder) EnqueueMetricsServerInit() { + fnb.Component("Metrics server", func(config config.Config) (module.ReadyDoneAware, error) { + server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) + return server, nil + }) +} + +func (fnb *EVMGatewayNodeBuilder) initMetrics() error { + fnb.Metrics = metrics2.NewCollector(fnb.Logger) + return nil +} + +func (fnb *EVMGatewayNodeBuilder) initStorage() error { + logger := fnb.Logger + cfg := fnb.Config + + store := pebble2.New(fnb.DB, fnb.Logger) + + blocks := pebble2.NewBlocks(store, cfg.FlowNetworkID) + storageAddress := evm.StorageAccountAddress(cfg.FlowNetworkID) + registerStore := pebble2.NewRegisterStorage(store, storageAddress) + + // hard set the start cadence height, this is used when force reindexing + if cfg.ForceStartCadenceHeight != 0 { + logger.Warn().Uint64("height", cfg.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") + if err := blocks.SetLatestCadenceHeight(cfg.ForceStartCadenceHeight, nil); err != nil { + return err + } + } + + // if database is not initialized require init height + if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { + // TODO(JanezP): move this to a separate function + err = func() (innerErr error) { + batch := store.NewBatch() + defer func(batch *pebble.Batch) { + innerErr = batch.Close() + }(batch) + + cadenceHeight := cfg.InitCadenceHeight + evmBlokcHeight := uint64(0) + cadenceBlock, err := fnb.Client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) + if err != nil { + return fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) + } + + snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) + if err != nil { + return fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) + } + + delta := storage.NewRegisterDelta(snapshot) + accountStatus := environment.NewAccountStatus() + err = delta.SetValue( + storageAddress[:], + []byte(flowGo.AccountStatusKey), + accountStatus.ToBytes(), + ) + if err != nil { + return fmt.Errorf("could not set account status: %w", err) + } + + err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) + if err != nil { + return fmt.Errorf("could not store register updates: %w", err) + } + + if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { + return fmt.Errorf( + "failed to init the database for block height: %d and ID: %s, with : %w", + cadenceHeight, + cadenceBlock.ID, + err, + ) + } + + err = batch.Commit(pebble.Sync) + if err != nil { + return fmt.Errorf("could not commit register updates: %w", err) + } + + logger.Info(). + Stringer("fvm_address_for_evm_storage_account", storageAddress). + Msgf("database initialized with cadence height: %d", cadenceHeight) + + return nil + }() + + if err != nil { + return fmt.Errorf("failed to init the database: %w", err) + } + } + //else { + // // TODO(JanezP): verify storage account owner is correct + //} + + fnb.Storages = &Storages{ + Storage: store, + Blocks: blocks, + Registers: registerStore, + Transactions: pebble2.NewTransactions(store), + Receipts: pebble2.NewReceipts(store), + Traces: pebble2.NewTraces(store), + } + + return nil +} +func (fnb *EVMGatewayNodeBuilder) initSigner(config config.Config) (module.ReadyDoneAware, error) { + sig := signer.NewSigner(fnb.Logger, config) + fnb.Signer = sig + return sig, nil +} + +func (fnb *EVMGatewayNodeBuilder) initPublishers() { + fnb.Publishers = &Publishers{} + + fnb.Component("Block Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[*models.Block](fnb.Logger) + fnb.Publishers.Block = p + return p, nil + }) + fnb.Component("Transaction Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[*gethTypes.Transaction](fnb.Logger) + fnb.Publishers.Transaction = p + return p, nil + }) + fnb.Component("Logs Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[[]*gethTypes.Log](fnb.Logger) + fnb.Publishers.Logs = p + return p, nil + }) +} + +func (fnb *EVMGatewayNodeBuilder) initClient() error { + logger := fnb.Logger + cfg := fnb.Config + + client, err := setupCrossSporkClient(cfg, logger) + if err != nil { + return fmt.Errorf("failed to create cross-spork client: %w", err) + } + + fnb.Client = client + + fnb.ShutdownFunc(func() error { + if err := fnb.Client.Close(); err != nil { + return fmt.Errorf("error closing cross-spork client: %w", err) + } + return nil + }) + + return nil +} + +type BuilderFunc func(nodeConfig config.Config) error + +type namedModuleFunc struct { + fn BuilderFunc + name string +} + +// handleModules initializes the given module. +func (fnb *EVMGatewayNodeBuilder) handleModule(v namedModuleFunc) error { + fnb.Logger.Info().Str("module", v.name).Msg("module initialization started") + err := v.fn(fnb.Config) + if err != nil { + return fmt.Errorf("module %s initialization failed: %w", v.name, err) + } + + fnb.Logger.Info().Str("module", v.name).Msg("module initialization complete") + return nil +} + +// handleModules initializes all modules that have been enqueued on this node builder. +func (fnb *EVMGatewayNodeBuilder) handleModules() error { + for _, f := range fnb.modules { + if err := fnb.handleModule(f); err != nil { + return err + } + } + + return nil +} + +func (fnb *EVMGatewayNodeBuilder) handleComponents() error { + cmd.AddWorkersFromComponents(fnb.Logger, fnb.Config, fnb.componentBuilder, fnb.components) + return nil +} diff --git a/cmd/run/cmd.go b/cmd/run/cmd.go index b88bfa6cb..a25f18ead 100644 --- a/cmd/run/cmd.go +++ b/cmd/run/cmd.go @@ -1,15 +1,11 @@ package run import ( - "context" - "errors" + "encoding/json" "fmt" "math/big" "os" - "os/signal" "strings" - "sync" - "syscall" "time" "github.com/onflow/flow-evm-gateway/bootstrap" @@ -30,10 +26,6 @@ var Cmd = &cobra.Command{ Use: "run", Short: "Runs the EVM Gateway Node", RunE: func(command *cobra.Command, _ []string) error { - - ctx, cancel := context.WithCancel(command.Context()) - defer cancel() - // create multi-key account // TODO(JanezP): move to separate command if _, exists := os.LookupEnv("MULTIKEY_MODE"); exists { @@ -45,46 +37,19 @@ var Cmd = &cobra.Command{ return fmt.Errorf("failed to parse flags: %w", err) } - done := make(chan struct{}) - ready := make(chan struct{}) - once := sync.Once{} - closeReady := func() { - once.Do(func() { - close(ready) - }) - } - go func() { - defer close(done) - // In case an error happens before ready is called we need to close the ready channel - defer closeReady() + builder := bootstrap.NewEVMGatewayNodeBuilder(cfg) - err := bootstrap.Run( - ctx, - cfg, - closeReady, - ) - if err != nil && !errors.Is(err, context.Canceled) { - log.Err(err).Msg("Gateway runtime error") - } - }() - - <-ready - - osSig := make(chan os.Signal, 1) - signal.Notify(osSig, syscall.SIGINT, syscall.SIGTERM) - - // wait for gateway to exit or for a shutdown signal - select { - case <-osSig: - log.Info().Msg("OS Signal to shutdown received, shutting down") - cancel() - case <-done: - log.Info().Msg("done, shutting down") + if err := builder.Initialize(); err != nil { + builder.Logger.Fatal().Err(err).Send() } - // Wait for the gateway to completely stop - <-done + builder.LoadComponentsAndModules() + node, err := builder.Build() + if err != nil { + builder.Logger.Fatal().Err(err).Send() + } + node.Run() return nil }, } diff --git a/go.mod b/go.mod index af84aab9f..ca78e9c8c 100644 --- a/go.mod +++ b/go.mod @@ -22,7 +22,6 @@ require ( go.uber.org/ratelimit v0.3.1 golang.org/x/exp v0.0.0-20240119083558-1b970713d09a golang.org/x/sync v0.8.0 - google.golang.org/grpc v1.63.2 ) require ( @@ -37,6 +36,22 @@ require ( 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.2 // indirect + github.com/aws/aws-sdk-go-v2 v1.27.0 // indirect + github.com/aws/aws-sdk-go-v2/config v1.27.15 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.17.15 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.3 // 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.3.7 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.9 // 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.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.28.9 // indirect + github.com/aws/smithy-go v1.20.2 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.10.0 // indirect @@ -51,45 +66,66 @@ require ( 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-ipa v0.0.0-20240223125850-b1e8a79f509c // indirect github.com/crate-crypto/go-kzg-4844 v1.0.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 github.com/deckarep/golang-set/v2 v2.6.0 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/dgraph-io/badger/v2 v2.2007.4 // indirect github.com/dgraph-io/ristretto v0.1.0 // indirect github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 // indirect github.com/dlclark/regexp2 v1.7.0 // indirect + github.com/docker/go-units v0.5.0 // indirect github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 // indirect 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/c-kzg-4844 v1.0.0 // indirect github.com/ethereum/go-ethereum v1.13.10 // indirect github.com/ethereum/go-verkle v0.1.1-0.20240306133620-7d920df305f0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/frankban/quicktest v1.14.4 // indirect + github.com/flynn/noise v1.0.1 // indirect + github.com/francoispqt/gojay v1.2.13 // indirect github.com/fsnotify/fsnotify v1.6.0 // indirect github.com/fxamacker/cbor/v2 v2.4.1-0.20230228173756-c0c9f774e40c // indirect github.com/fxamacker/circlehash v0.3.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.3 // indirect + github.com/gammazero/deque v0.1.0 // indirect + github.com/gammazero/workerpool v1.1.2 // indirect github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff // indirect github.com/getsentry/sentry-go v0.27.0 // indirect + github.com/go-kit/kit v0.12.0 // indirect + github.com/go-kit/log v0.2.1 // indirect + github.com/go-logfmt/logfmt v0.5.1 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.1 // indirect github.com/go-sourcemap/sourcemap v2.1.3+incompatible // indirect + 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/gogo/protobuf v1.3.2 // indirect github.com/golang/glog v1.2.0 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect + github.com/google/gopacket v1.1.19 // indirect github.com/google/pprof v0.0.0-20231229205709-960ae82b1e42 // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/google/uuid v1.6.0 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect github.com/gorilla/websocket v1.5.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 // indirect + github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/golang-lru v1.0.2 // indirect @@ -97,45 +133,80 @@ require ( github.com/hashicorp/hcl v1.0.0 // indirect github.com/holiman/bloomfilter/v2 v2.0.3 // indirect github.com/holiman/uint256 v1.3.0 // indirect + github.com/huandu/go-clone v1.6.0 // indirect + github.com/huandu/go-clone/generic v1.7.2 // 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.17.1-0.20240131173518-89bceff34bf1 // indirect github.com/ipfs/go-block-format v0.2.0 // indirect github.com/ipfs/go-cid v0.4.1 // indirect + github.com/ipfs/go-cidutil v0.1.0 // indirect github.com/ipfs/go-datastore v0.6.0 // indirect + github.com/ipfs/go-ds-badger2 v0.1.3 // indirect + github.com/ipfs/go-ipfs-delay v0.0.1 // indirect + github.com/ipfs/go-ipfs-pq v0.0.3 // indirect github.com/ipfs/go-ipfs-util v0.0.3 // indirect github.com/ipfs/go-ipld-format v0.6.0 // indirect + github.com/ipfs/go-log v1.0.5 // indirect github.com/ipfs/go-log/v2 v2.5.1 // indirect github.com/ipfs/go-metrics-interface v0.0.1 // indirect + github.com/ipfs/go-peertaskqueue v0.8.1 // indirect + github.com/ipld/go-ipld-prime v0.21.0 // indirect github.com/jackpal/go-nat-pmp v1.0.2 // indirect + github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect github.com/jbenet/goprocess v0.1.4 // indirect + github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/k0kubun/pp v3.0.1+incompatible // indirect github.com/kevinburke/go-bindata v3.24.0+incompatible // indirect github.com/klauspost/compress v1.17.4 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect + github.com/koron/go-ssdp v0.0.4 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/libp2p/go-buffer-pool v0.1.0 // indirect + github.com/libp2p/go-cidranger v1.1.0 // indirect + github.com/libp2p/go-flow-metrics v0.1.0 // indirect github.com/libp2p/go-libp2p v0.32.2 // indirect + github.com/libp2p/go-libp2p-asn-util v0.4.1 // indirect + github.com/libp2p/go-libp2p-kad-dht v0.25.2 // indirect + github.com/libp2p/go-libp2p-kbucket v0.6.3 // indirect + github.com/libp2p/go-libp2p-pubsub v0.10.0 // indirect + github.com/libp2p/go-libp2p-record v0.2.0 // indirect + github.com/libp2p/go-libp2p-routing-helpers v0.7.3 // indirect + github.com/libp2p/go-msgio v0.3.0 // indirect + github.com/libp2p/go-nat v0.2.0 // indirect + github.com/libp2p/go-netroute v0.2.1 // indirect + github.com/libp2p/go-reuseport v0.4.0 // indirect + github.com/libp2p/go-yamux/v4 v4.0.1 // indirect github.com/logrusorgru/aurora/v4 v4.0.0 // indirect + github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/magiconair/properties v1.8.7 // indirect + github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.15 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect + github.com/miekg/dns v1.1.57 // indirect + github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b // indirect + 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/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 github.com/multiformats/go-multiaddr v0.12.2 // indirect + github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect + github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect github.com/multiformats/go-multibase v0.2.0 // indirect github.com/multiformats/go-multicodec v0.9.0 // indirect github.com/multiformats/go-multihash v0.2.3 // indirect github.com/multiformats/go-multistream v0.5.0 // 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/crypto v0.25.2 // indirect github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0 // indirect @@ -146,21 +217,34 @@ require ( github.com/onflow/flow-nft/lib/go/templates v1.2.1 // indirect github.com/onflow/flow/protobuf/go/flow v0.4.7 // indirect github.com/onflow/sdks v0.6.0-preview.1 // indirect - github.com/onsi/gomega v1.18.1 // indirect + github.com/onflow/wal v1.0.2 // indirect + github.com/onsi/ginkgo/v2 v2.13.2 // indirect + github.com/opencontainers/runtime-spec v1.1.0 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 // indirect github.com/pelletier/go-toml/v2 v2.0.6 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/polydawn/refmt v0.89.0 // indirect + github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/prometheus/client_model v0.5.0 // indirect github.com/prometheus/common v0.45.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect github.com/psiemens/sconfig v0.1.0 // indirect + github.com/quic-go/qpack v0.4.0 // indirect + github.com/quic-go/qtls-go1-20 v0.4.1 // indirect + github.com/quic-go/quic-go v0.40.1 // indirect + github.com/quic-go/webtransport-go v0.6.0 // indirect + github.com/raulk/go-watchdog v1.3.0 // indirect github.com/rivo/uniseg v0.4.4 // indirect github.com/rogpeppe/go-internal v1.10.0 // indirect github.com/samber/lo v1.39.0 // indirect + github.com/schollz/progressbar/v3 v3.13.1 // indirect github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect + github.com/shirou/gopsutil/v3 v3.22.2 // indirect github.com/slok/go-http-metrics v0.10.0 // indirect + github.com/sony/gobreaker v0.5.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/spf13/afero v1.10.0 // indirect github.com/spf13/cast v1.5.0 // indirect @@ -180,7 +264,9 @@ require ( github.com/vmihailenco/msgpack v4.0.4+incompatible // indirect github.com/vmihailenco/msgpack/v4 v4.3.11 // 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 go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 // indirect @@ -193,14 +279,20 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.0.0 // indirect go.uber.org/atomic v1.11.0 // indirect + go.uber.org/dig v1.17.1 // indirect + go.uber.org/fx v1.20.1 // indirect + go.uber.org/mock v0.4.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/crypto v0.28.0 // indirect + golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.25.0 // indirect golang.org/x/oauth2 v0.17.0 // indirect golang.org/x/sys v0.26.0 // indirect + golang.org/x/term v0.25.0 // indirect golang.org/x/text v0.19.0 // indirect golang.org/x/time v0.5.0 // indirect + golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect gonum.org/v1/gonum v0.14.0 // indirect google.golang.org/api v0.162.0 // indirect @@ -208,9 +300,13 @@ require ( google.golang.org/genproto v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de // indirect + google.golang.org/grpc v1.63.2 // indirect + google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 // indirect google.golang.org/protobuf v1.33.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect lukechampine.com/blake3 v1.3.0 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) + +replace github.com/onflow/flow-go => ../flow-go-2 diff --git a/go.sum b/go.sum index f4226b2e0..a9131103f 100644 --- a/go.sum +++ b/go.sum @@ -1,5 +1,7 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= @@ -35,6 +37,8 @@ cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= cloud.google.com/go/iam v1.1.6/go.mod h1:O0zxdPeGBoFdWW3HWmBxJsk0pfvNM/p/qa82rWOGTwI= cloud.google.com/go/kms v1.15.7 h1:7caV9K3yIxvlQPAcaFffhlT7d1qpxjB1wHBtjWa13SM= cloud.google.com/go/kms v1.15.7/go.mod h1:ub54lbsa6tDkUwnu4W7Yt1aAIFLnspgh0kPGToDukeI= +cloud.google.com/go/profiler v0.3.0 h1:R6y/xAeifaUXxd2x6w+jIwKxoKl8Cv5HJvcvASTPWJo= +cloud.google.com/go/profiler v0.3.0/go.mod h1:9wYk9eY4iZHsev8TQb61kh3wiOiSyz/xOYixWPzweCU= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -47,9 +51,15 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9 cloud.google.com/go/storage v1.14.0/go.mod h1:GrKmX003DSIwi9o29oFT7YDnHYwZoctc3fOKtUw0Xmo= cloud.google.com/go/storage v1.36.0 h1:P0mOkAcaJxhCTvAkMhxMfrTKiNcub4YmmPBtlhAyTr8= cloud.google.com/go/storage v1.36.0/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= +dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU= +dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= +dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= +git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/DataDog/zstd v1.5.2 h1:vUG4lAyuPCXO0TLbXvPv7EB7cNK1QV/luu55UHLrrn8= github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= @@ -67,8 +77,52 @@ github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= +github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +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.27.0 h1:7bZWKoXhzI+mMR/HjdMx8ZCC5+6fY0lS5tr0bbgiLlo= +github.com/aws/aws-sdk-go-v2 v1.27.0/go.mod h1:ffIFB97e2yNsv4aTSGkqtHnppsIJzw7G7BReUZ3jCXM= +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.27.15 h1:uNnGLZ+DutuNEkuPh6fwqK7LpEiPmzb7MIMA1mNWEUc= +github.com/aws/aws-sdk-go-v2/config v1.27.15/go.mod h1:7j7Kxx9/7kTmL7z4LlhwQe63MYEE5vkVV6nWg4ZAI8M= +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.17.15 h1:YDexlvDRCA8ems2T5IP1xkMtOZ1uLJOCJdTr0igs5zo= +github.com/aws/aws-sdk-go-v2/credentials v1.17.15/go.mod h1:vxHggqW6hFNaeNC0WyXS3VdyjcV0a4KMUY4dKJ96buU= +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.16.3 h1:dQLK4TjtnlRGb0czOht2CevZ5l6RSyRWAnKeGd7VAFE= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.3/go.mod h1:TL79f2P6+8Q7dTsILpiVST+AL9lkF6PPGI167Ny0Cjw= +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.3.7 h1:lf/8VTF2cM+N4SLzaYJERKEWAXq8MOMpZfU6wEPWsPk= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.7/go.mod h1:4SjkU7QiqK2M9oozyMzfZ/23LmUY+h3oFqhdeP5OMiI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7 h1:4OYVp0705xu8yjdyoWix0r9wPIRXnIzzOoUpQVHIJ/g= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7/go.mod h1:vd7ESTEvI76T2Na050gODNmNU7+OyKrIKroYTu4ABiI= +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.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= +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/accept-encoding v1.11.2 h1:Ji0DY1xUsUr3I8cHps0G+XM3WWU16lP6yG8qu1GAZAs= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2/go.mod h1:5CsjAbs3NlGQyZNFACh+zztPDI7fU6eW9QsxjfnuBKg= +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.11.9 h1:Wx0rlZoEJR7JwlSZcHnEa7CNjrSIyVxMFWGAaXy4fJY= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.9/go.mod h1:aVMHdE0aHO3v+f/iw01fmXV/5DbfQ3Bi9nN7nd9bE9Y= +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/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.20.8 h1:Kv1hwNG6jHC/sxMTe5saMjH6t6ZLkgfvVxyEjfWL1ks= +github.com/aws/aws-sdk-go-v2/service/sso v1.20.8/go.mod h1:c1qtZUWtygI6ZdvKppzCSXsDOq5I4luJPZ0Ud3juFCA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2 h1:nWBZ1xHCF+A7vv9sDzJOq4NWIdzFYm0kH7Pr4OjHYsQ= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2/go.mod h1:9lmoVDVLz/yUZwLaQ676TK02fhCu4+PgRSmMaKR1ozk= +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.28.9 h1:Qp6Boy0cGDloOE3zI6XhNLNZgjNS8YmiFQFHe71SaW0= +github.com/aws/aws-sdk-go-v2/service/sts v1.28.9/go.mod h1:0Aqn1MnEuitqfsCNyKsdKLhDUOr4txD/g19EfiUqgws= +github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= +github.com/aws/smithy-go v1.20.2 h1:tbp628ireGtzcHDDmLT/6ADHidqnwgF57XOXZe6tp4Q= +github.com/aws/smithy-go v1.20.2/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= 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= github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o= github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -77,10 +131,12 @@ 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/bits-and-blooms/bitset v1.10.0 h1:ePXTeiPEazB5+opbv5fr8umg2R/1NlzgDsyepwsSr88= github.com/bits-and-blooms/bitset v1.10.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= github.com/btcsuite/btcd/btcec/v2 v2.3.4 h1:3EJjcN70HCu/mwqlUsGK8GcNVyLVxFDlWurTXGPFfiQ= github.com/btcsuite/btcd/btcec/v2 v2.3.4/go.mod h1:zYzJ8etWJQIv1Ogk7OzpWjowwOdXY1W/17j2MW85J04= github.com/btcsuite/btcd/chaincfg/chainhash v1.0.2 h1:KdUfX2zKommPRa+PD0sWZUyXe9w277ABlgELO7H04IM= github.com/btcsuite/btcd/chaincfg/chainhash v1.0.2/go.mod h1:7SFka0XMvUgj3hfZtydOrQY2mwhPclbT2snogU7SQQc= +github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -98,6 +154,7 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/readline v1.5.0/go.mod h1:x22KAscuvRqlLoK9CsoYsmxoXZMMFVyOl86cAH8qUic= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/chzyer/test v0.0.0-20210722231415-061457976a23/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= @@ -122,17 +179,25 @@ github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/Yj 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= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd/v22 v22.1.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= +github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= 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.4 h1:wfIWP927BUkWJb2NmU/kNDYIBTh/ziUX91+lVfRxZq4= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/crate-crypto/go-ipa v0.0.0-20240223125850-b1e8a79f509c h1:uQYC5Z1mdLRPrZhHjHxufI8+2UG/i25QG92j0Er9p6I= @@ -140,15 +205,20 @@ github.com/crate-crypto/go-ipa v0.0.0-20240223125850-b1e8a79f509c/go.mod h1:geZJ github.com/crate-crypto/go-kzg-4844 v1.0.0 h1:TsSgHwrkTKecKJ4kadtHi4b3xHW5dCFUDFnUp1TsawI= github.com/crate-crypto/go-kzg-4844 v1.0.0/go.mod h1:1kMhvPgI0Ky3yIa+9lFySEBUBXkYxeOi8ZF1sYioxhc= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/cskr/pubsub v1.0.2 h1:vlOzMhl6PFn60gRlTQQsIfVwaPB/B/8MziK8FhEPt/0= +github.com/cskr/pubsub v1.0.2/go.mod h1:/8MzYXk/NJAz782G8RPkFzXTZVu63VotefPnR9TIRis= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR6AkioZ1ySsx5yxlDQZ8stG2b88gTPxgJU= +github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= github.com/deckarep/golang-set/v2 v2.6.0 h1:XfcQbWM1LlMB8BsJ8N9vW5ehnnPVIw0je80NsVHagjM= github.com/deckarep/golang-set/v2 v2.6.0/go.mod h1:VAky9rY/yGXJOLEDv3OMci+7wtDpOF4IN+y82NBOac4= github.com/decred/dcrd/crypto/blake256 v1.0.1 h1:7PltbUIQB7u/FfZ39+DGa/ShuMyJ5ilcvdfma9wOH6Y= github.com/decred/dcrd/crypto/blake256 v1.0.1/go.mod h1:2OfgNZ5wDpcsFmHmCK5gZTPcCXqlm2ArzUIkw9czNJo= github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 h1:8UrgZ3GkP4i/CLijOJx79Yu+etlyjdBU4sfcs2WYQMs= github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0= +github.com/dgraph-io/badger/v2 v2.2007.3/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDmM2p1e+NhhnBCAE= 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.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= @@ -161,6 +231,7 @@ github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8 github.com/dlclark/regexp2 v1.4.1-0.20201116162257-a2a8dda75c91/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= github.com/dlclark/regexp2 v1.7.0 h1:7lJfhqlPssTb1WQx4yvTHN0uElPEv52sbaECrAQxjAo= github.com/dlclark/regexp2 v1.7.0/go.mod h1:DHkYz0B9wPfa6wondMfaivmHpzrQ3v9q8cnmRbL6yW8= +github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dop251/goja v0.0.0-20211022113120-dc8c55024d06/go.mod h1:R9ET47fwRVRPZnOGvHxxhuZcbrMCuiqOz3Rlrh4KSnk= @@ -173,6 +244,9 @@ github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkp github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/ef-ds/deque v1.0.4 h1:iFAZNmveMT9WERAkqLJ+oaABF9AcVQ5AjXem/hroniI= github.com/ef-ds/deque v1.0.4/go.mod h1:gXDnTC3yqvBcHbq2lcExjtAcVrOnJCbMcZXmuj8Z4tg= +github.com/elastic/gosigar v0.12.0/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= +github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4= +github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -191,8 +265,13 @@ github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2 github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fjl/memsize v0.0.2 h1:27txuSD9or+NZlnOWdKUxeBzTAUkWCVh+4Gf2dWFOzA= github.com/fjl/memsize v0.0.2/go.mod h1:VvhXpOYNQvB+uIk2RvXzuaQtkQJzzIx6lSBe1xv7hi0= -github.com/frankban/quicktest v1.14.4 h1:g2rn0vABPOOXmZUj+vbmUp0lPoXEMuhTpIluN0XL9UY= -github.com/frankban/quicktest v1.14.4/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= +github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= +github.com/flynn/noise v1.0.1 h1:vPp/jdQLXC6ppsXSj/pM3W1BIJ5FEHE2TulSJBpb43Y= +github.com/flynn/noise v1.0.1/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= +github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= +github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= +github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHkI4W8= +github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= @@ -201,6 +280,12 @@ github.com/fxamacker/cbor/v2 v2.4.1-0.20230228173756-c0c9f774e40c h1:5tm/Wbs9d9r github.com/fxamacker/cbor/v2 v2.4.1-0.20230228173756-c0c9f774e40c/go.mod h1:TA1xS00nchWmaBnEIxPSE5oHLuJBAVvqrtAnWBwBCVo= github.com/fxamacker/circlehash v0.3.0 h1:XKdvTtIJV9t7DDUtsf0RIpC1OcxZtPbmgIH7ekx28WA= github.com/fxamacker/circlehash v0.3.0/go.mod h1:3aq3OfVvsWtkWMb6A1owjOQFA+TLsD5FgJflnaQwtMM= +github.com/gabriel-vasile/mimetype v1.4.3 h1:in2uUcidCuFcDKtdcBxlR0rJ1+fsokWf+uqxgUFjbI0= +github.com/gabriel-vasile/mimetype v1.4.3/go.mod h1:d8uq/6HKRL6CGdk+aubisF/M5GcPfT7nKyLpA0lbSSk= +github.com/gammazero/deque v0.1.0 h1:f9LnNmq66VDeuAlSAapemq/U7hJ2jpIWa4c09q8Dlik= +github.com/gammazero/deque v0.1.0/go.mod h1:KQw7vFau1hHuM8xmI9RbgKFbAsQFWmBpqQ2KenFLk6M= +github.com/gammazero/workerpool v1.1.2 h1:vuioDQbgrz4HoaCi2q1HLlOXdpbap5AET7xu5/qj87g= +github.com/gammazero/workerpool v1.1.2/go.mod h1:UelbXcO0zCIGFcufcirHhq2/xtLXJdQ29qZNlXG9OjQ= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff h1:tY80oXqGNY4FhTFhk+o9oFHGINQ/+vhlm8HFzi6znCI= github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww= github.com/getsentry/sentry-go v0.27.0 h1:Pv98CIbtB3LkMWmXi4Joa5OOcwbmnX88sF5qbK3r3Ps= @@ -208,6 +293,8 @@ github.com/getsentry/sentry-go v0.27.0/go.mod h1:lc76E2QywIyW8WuBnwl8Lc4bkmQH4+w github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= +github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= @@ -228,21 +315,36 @@ github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ4 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-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= +github.com/go-playground/validator/v10 v10.14.1 h1:9c50NUPC30zyuKprjL3vNZ0m5oG+jU0zvx4AqHGnv4k= +github.com/go-playground/validator/v10 v10.14.1/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-sourcemap/sourcemap v2.1.3+incompatible h1:W1iEw64niKVGogNgBN3ePyLFfuisuzeidWPMPWmECqU= github.com/go-sourcemap/sourcemap v2.1.3+incompatible/go.mod h1:F8jJfvm2KbVjc5NqelyYJmf/v5J0dwNLS2mL4sNA1Jg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= +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-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0= 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= +github.com/godbus/dbus/v5 v5.1.0/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw= github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= @@ -256,6 +358,7 @@ github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -283,6 +386,7 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk= @@ -300,12 +404,17 @@ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/gopacket v1.1.19 h1:ves8RnFZPGiFnTS0uPQStjwru6uO6h+nlr9j6fL7kF8= +github.com/google/gopacket v1.1.19/go.mod h1:iJ8V8n6KS+z2U1A8pUwu8bW5SyEMkXJB8Yo/Vo+TKTo= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= @@ -322,7 +431,6 @@ github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hf github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20201218002935-b9804c9f04c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20230207041349-798e818bf904/go.mod h1:uglQLonpP8qtYCYyzA+8c/9qtqgA3qsXGYqCPKARAFg= github.com/google/pprof v0.0.0-20231229205709-960ae82b1e42 h1:dHLYa5D8/Ta0aLR2XcPsrkpAgGeFs6thhMcQK0oQ0n8= github.com/google/pprof v0.0.0-20231229205709-960ae82b1e42/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik= @@ -330,21 +438,34 @@ github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm4 github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= +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.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= +github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= +github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c h1:7lF+Vz0LqiRidnzC1Oq86fpX1q/iEv2KJdrCtttYjT4= +github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 h1:pRhl55Yx1eC7BZ1N+BBWwnKaMyD8uC+34TLdndZMAKk= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0/go.mod h1:XKMd7iuf/RGPSMJ/U4HP0zS2Z9Fh8Ps9a+6X26m/tmI= +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= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 h1:Wqo399gCIufwto+VfwCSvsnfGpF/w5E9CNxSwbpD6No= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0/go.mod h1:qmOFXW2epJhM0qSnUUYpldc7gVz2KMQwJ/QYCDIa7XU= @@ -368,6 +489,8 @@ github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iU github.com/holiman/uint256 v1.3.0 h1:4wdcm/tnd0xXdu7iS3ruNvxkWwrb4aeBQv19ayYn8F4= github.com/holiman/uint256 v1.3.0/go.mod h1:EOMSn4q6Nyt9P6efbI3bueV4e1b3dGlUCXeiRV4ng7E= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/huandu/go-assert v1.1.5 h1:fjemmA7sSfYHJD7CUqs9qTwwfdNAx7/j2/ZlHXzNB3c= +github.com/huandu/go-assert v1.1.5/go.mod h1:yOLvuqZwmcHIC5rIzrBhT7D3Q9c3GFnd0JrPVhn/06U= github.com/huandu/go-clone v1.6.0 h1:HMo5uvg4wgfiy5FoGOqlFLQED/VGRm2D9Pi8g1FXPGc= github.com/huandu/go-clone v1.6.0/go.mod h1:ReGivhG6op3GYr+UY3lS6mxjKp7MIGTknuU5TbTVaXE= github.com/huandu/go-clone/generic v1.7.2 h1:47pQphxs1Xc9cVADjOHN+Bm5D0hNagwH9UXErbxgVKA= @@ -388,37 +511,67 @@ github.com/ipfs/go-block-format v0.2.0 h1:ZqrkxBA2ICbDRbK8KJs/u0O3dlp6gmAuuXUJNi github.com/ipfs/go-block-format v0.2.0/go.mod h1:+jpL11nFx5A/SPpsoBn6Bzkra/zaArfSmsknbPMYgzM= github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= +github.com/ipfs/go-cidutil v0.1.0 h1:RW5hO7Vcf16dplUU60Hs0AKDkQAVPVplr7lk97CFL+Q= +github.com/ipfs/go-cidutil v0.1.0/go.mod h1:e7OEVBMIv9JaOxt9zaGEmAoSlXW9jdFZ5lP/0PwcfpA= +github.com/ipfs/go-datastore v0.5.1/go.mod h1:9zhEApYMTl17C8YDp7JmU7sQZi2/wqiYh73hakZ90Bk= github.com/ipfs/go-datastore v0.6.0 h1:JKyz+Gvz1QEZw0LsX1IBn+JFCJQH4SJVFtM4uWU0Myk= github.com/ipfs/go-datastore v0.6.0/go.mod h1:rt5M3nNbSO/8q1t4LNkLyUwRs8HupMeN/8O4Vn9YAT8= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= +github.com/ipfs/go-ds-badger2 v0.1.3 h1:Zo9JicXJ1DmXTN4KOw7oPXkspZ0AWHcAFCP1tQKnegg= +github.com/ipfs/go-ds-badger2 v0.1.3/go.mod h1:TPhhljfrgewjbtuL/tczP8dNrBYwwk+SdPYbms/NO9w= +github.com/ipfs/go-ipfs-blocksutil v0.0.1 h1:Eh/H4pc1hsvhzsQoMEP3Bke/aW5P5rVM1IWFJMcGIPQ= +github.com/ipfs/go-ipfs-blocksutil v0.0.1/go.mod h1:Yq4M86uIOmxmGPUHv/uI7uKqZNtLb449gwKqXjIsnRk= +github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= +github.com/ipfs/go-ipfs-delay v0.0.1 h1:r/UXYyRcddO6thwOnhiznIAiSvxMECGgtv35Xs1IeRQ= +github.com/ipfs/go-ipfs-delay v0.0.1/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= +github.com/ipfs/go-ipfs-pq v0.0.3 h1:YpoHVJB+jzK15mr/xsWC574tyDLkezVrDNeaalQBsTE= +github.com/ipfs/go-ipfs-pq v0.0.3/go.mod h1:btNw5hsHBpRcSSgZtiNm/SLj5gYIZ18AKtv3kERkRb4= github.com/ipfs/go-ipfs-util v0.0.3 h1:2RFdGez6bu2ZlZdI+rWfIdbQb1KudQp3VGwPtdNCmE0= github.com/ipfs/go-ipfs-util v0.0.3/go.mod h1:LHzG1a0Ig4G+iZ26UUOMjHd+lfM84LZCrn17xAKWBvs= github.com/ipfs/go-ipld-format v0.6.0 h1:VEJlA2kQ3LqFSIm5Vu6eIlSxD/Ze90xtc4Meten1F5U= github.com/ipfs/go-ipld-format v0.6.0/go.mod h1:g4QVMTn3marU3qXchwjpKPKgJv+zF+OlaKMyhJ4LHPg= github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= +github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= +github.com/ipfs/go-log/v2 v2.5.0/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= github.com/ipfs/go-log/v2 v2.5.1 h1:1XdUzF7048prq4aBjDQQ4SL5RxftpRGdXhNRwKSAlcY= github.com/ipfs/go-log/v2 v2.5.1/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= +github.com/ipfs/go-peertaskqueue v0.8.1 h1:YhxAs1+wxb5jk7RvS0LHdyiILpNmRIRnZVztekOF0pg= +github.com/ipfs/go-peertaskqueue v0.8.1/go.mod h1:Oxxd3eaK279FxeydSPPVGHzbwVeHjatZ2GA8XD+KbPU= +github.com/ipld/go-ipld-prime v0.21.0 h1:n4JmcpOlPDIxBcY037SVfpd1G+Sj1nKZah0m6QH9C2E= +github.com/ipld/go-ipld-prime v0.21.0/go.mod h1:3RLqy//ERg/y5oShXXdx5YIp50cFGOanyMctpPjsvxQ= github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= +github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABoLk/+KKHggpk= +github.com/jbenet/go-temp-err-catcher v0.1.0/go.mod h1:0kJRvmDZXNMIiJirNPEYfhpPwbGVtZVWC34vc5WLsDk= github.com/jbenet/goprocess v0.1.4 h1:DRGOFReOMqqDNXwW70QkacFW0YN9QnwLV0Vqk+3oU0o= github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= +github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= +github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= +github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= +github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= +github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 h1:uC1QfSlInpQF+M0ao65imhwqKnz3Q2z/d8PWZRMQvDM= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= +github.com/k0kubun/go-ansi v0.0.0-20180517002512-3bf9e2903213/go.mod h1:vNUNkEQ1e29fT/6vq2aBdFsgNPmy8qMdSay1npru+Sw= github.com/k0kubun/pp v3.0.1+incompatible h1:3tqvf7QgUnZ5tXO6pNAZlrvHgl6DvifjDrd9g2S9Z40= github.com/k0kubun/pp v3.0.1+incompatible/go.mod h1:GWse8YhT0p8pT4ir3ZgBbfZild3tgzSScAn6HmfYukg= github.com/kevinburke/go-bindata v3.24.0+incompatible h1:qajFA3D0pH94OTLU4zcCCKCDgR+Zr2cZK/RPJHDdFoY= github.com/kevinburke/go-bindata v3.24.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.12.3/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= @@ -428,14 +581,18 @@ github.com/klauspost/cpuid/v2 v2.0.12/go.mod h1:g2LTdtYhdyuGPqyWyv7qRAmj1WBqxuOb github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/koron/go-ssdp v0.0.4 h1:1IDwrghSKYM7yLf7XCzbByg2sJ/JcNOZRXS2jczTwz0= +github.com/koron/go-ssdp v0.0.4/go.mod h1:oDXq+E5IL5q0U8uSBcoAXzTzInwy5lEgC91HoKtbmZk= github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/pty v1.1.3/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= @@ -444,41 +601,81 @@ github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+ github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= +github.com/libp2p/go-addr-util v0.1.0 h1:acKsntI33w2bTU7tC9a0SaPimJGfSI0bFKC18ChxeVI= +github.com/libp2p/go-addr-util v0.1.0/go.mod h1:6I3ZYuFr2O/9D+SoyM0zEw0EF3YkldtTX406BpdQMqw= github.com/libp2p/go-buffer-pool v0.1.0 h1:oK4mSFcQz7cTQIfqbe4MIj9gLW+mnanjyFtc6cdF0Y8= github.com/libp2p/go-buffer-pool v0.1.0/go.mod h1:N+vh8gMqimBzdKkSMVuydVDq+UV5QTWy5HSiZacSbPg= github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38yPW7c= github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic= +github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFGkx3Q3WM= +github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro= github.com/libp2p/go-libp2p v0.32.2 h1:s8GYN4YJzgUoyeYNPdW7JZeZ5Ee31iNaIBfGYMAY4FQ= github.com/libp2p/go-libp2p v0.32.2/go.mod h1:E0LKe+diV/ZVJVnOJby8VC5xzHF0660osg71skcxJvk= github.com/libp2p/go-libp2p-asn-util v0.4.1 h1:xqL7++IKD9TBFMgnLPZR6/6iYhawHKHl950SO9L6n94= github.com/libp2p/go-libp2p-asn-util v0.4.1/go.mod h1:d/NI6XZ9qxw67b4e+NgpQexCIiFYJjErASrYW4PFDN8= +github.com/libp2p/go-libp2p-kad-dht v0.25.2 h1:FOIk9gHoe4YRWXTu8SY9Z1d0RILol0TrtApsMDPjAVQ= +github.com/libp2p/go-libp2p-kad-dht v0.25.2/go.mod h1:6za56ncRHYXX4Nc2vn8z7CZK0P4QiMcrn77acKLM2Oo= github.com/libp2p/go-libp2p-kbucket v0.6.3 h1:p507271wWzpy2f1XxPzCQG9NiN6R6lHL9GiSErbQQo0= github.com/libp2p/go-libp2p-kbucket v0.6.3/go.mod h1:RCseT7AH6eJWxxk2ol03xtP9pEHetYSPXOaJnOiD8i0= github.com/libp2p/go-libp2p-pubsub v0.10.0 h1:wS0S5FlISavMaAbxyQn3dxMOe2eegMfswM471RuHJwA= github.com/libp2p/go-libp2p-pubsub v0.10.0/go.mod h1:1OxbaT/pFRO5h+Dpze8hdHQ63R0ke55XTs6b6NwLLkw= +github.com/libp2p/go-libp2p-record v0.2.0 h1:oiNUOCWno2BFuxt3my4i1frNrt7PerzB3queqa1NkQ0= +github.com/libp2p/go-libp2p-record v0.2.0/go.mod h1:I+3zMkvvg5m2OcSdoL0KPljyJyvNDFGKX7QdlpYUcwk= +github.com/libp2p/go-libp2p-routing-helpers v0.7.3 h1:u1LGzAMVRK9Nqq5aYDVOiq/HaB93U9WWczBzGyAC5ZY= +github.com/libp2p/go-libp2p-routing-helpers v0.7.3/go.mod h1:cN4mJAD/7zfPKXBcs9ze31JGYAZgzdABEm+q/hkswb8= +github.com/libp2p/go-libp2p-testing v0.12.0 h1:EPvBb4kKMWO29qP4mZGyhVzUyR25dvfUIK5WDu6iPUA= +github.com/libp2p/go-libp2p-testing v0.12.0/go.mod h1:KcGDRXyN7sQCllucn1cOOS+Dmm7ujhfEyXQL5lvkcPg= github.com/libp2p/go-msgio v0.3.0 h1:mf3Z8B1xcFN314sWX+2vOTShIE0Mmn2TXn3YCUQGNj0= github.com/libp2p/go-msgio v0.3.0/go.mod h1:nyRM819GmVaF9LX3l03RMh10QdOroF++NBbxAb0mmDM= +github.com/libp2p/go-nat v0.2.0 h1:Tyz+bUFAYqGyJ/ppPPymMGbIgNRH+WqC5QrT5fKrrGk= +github.com/libp2p/go-nat v0.2.0/go.mod h1:3MJr+GRpRkyT65EpVPBstXLvOlAPzUVlG6Pwg9ohLJk= +github.com/libp2p/go-netroute v0.2.1 h1:V8kVrpD8GK0Riv15/7VN6RbUQ3URNZVosw7H2v9tksU= +github.com/libp2p/go-netroute v0.2.1/go.mod h1:hraioZr0fhBjG0ZRXJJ6Zj2IVEVNx6tDTFQfSmcq7mQ= +github.com/libp2p/go-reuseport v0.4.0 h1:nR5KU7hD0WxXCJbmw7r2rhRYruNRl2koHw8fQscQm2s= +github.com/libp2p/go-reuseport v0.4.0/go.mod h1:ZtI03j/wO5hZVDFo2jKywN6bYKWLOy8Se6DrI2E1cLU= +github.com/libp2p/go-yamux/v4 v4.0.1 h1:FfDR4S1wj6Bw2Pqbc8Uz7pCxeRBPbwsBbEdfwiCypkQ= +github.com/libp2p/go-yamux/v4 v4.0.1/go.mod h1:NWjl8ZTLOGlozrXSOZ/HlfG++39iKNnM5wwmtQP1YB4= github.com/logrusorgru/aurora/v4 v4.0.0 h1:sRjfPpun/63iADiSvGGjgA1cAYegEWMPCJdUpJYn9JA= github.com/logrusorgru/aurora/v4 v4.0.0/go.mod h1:lP0iIa2nrnT/qoFXcOZSrZQpJ1o6n2CUf/hyHi2Q4ZQ= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= +github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= +github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk= +github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= +github.com/mattn/go-runewidth v0.0.14/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= +github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= +github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= github.com/miekg/dns v1.1.57 h1:Jzi7ApEIzwEPLHWRcafCN9LZSBbqQpxjt/wpgvg7wcM= github.com/miekg/dns v1.1.57/go.mod h1:uqRjCRUuEAA6qsOiJvDd+CFo/vW+y5WR6SNmHE55hZk= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= +github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc h1:PTfri+PuQmWDqERdnNMiD9ZejrlswWrCpBEZgWOiTrc= +github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= +github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= +github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db h1:62I3jR2EmQ4l5rM/4FEfDWcRD+abF5XlKShorW5LRoQ= @@ -493,28 +690,39 @@ github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iP 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/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mr-tron/base58 v1.1.2/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/multiformats/go-base32 v0.1.0 h1:pVx9xoSPqEIQG8o+UbAe7DNi51oej1NtK+aGkbLYxPE= github.com/multiformats/go-base32 v0.1.0/go.mod h1:Kj3tFY6zNr+ABYMqeUNeGvkIC/UYgtWibDcT0rExnbI= github.com/multiformats/go-base36 v0.2.0 h1:lFsAbNOGeKtuKozrtBsAkSVhv1p9D0/qedU9rQyccr0= github.com/multiformats/go-base36 v0.2.0/go.mod h1:qvnKE++v+2MWCfePClUEjE78Z7P2a1UV0xHgWc0hkp4= +github.com/multiformats/go-multiaddr v0.1.1/go.mod h1:aMKBKNEYmzmDmxfX88/vz+J5IU55txyt0p4aiWVohjo= +github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y95ncPj0vFwVF6k6wJ4= github.com/multiformats/go-multiaddr v0.12.2 h1:9G9sTY/wCYajKa9lyfWPmpZAwe6oV+Wb1zcmMS1HG24= github.com/multiformats/go-multiaddr v0.12.2/go.mod h1:GKyaTYjZRdcUhyOetrxTk9z0cW+jA/YrnqTOvKgi44M= github.com/multiformats/go-multiaddr-dns v0.3.1 h1:QgQgR+LQVt3NPTjbrLLpsaT2ufAA2y0Mkk+QRVJbW3A= github.com/multiformats/go-multiaddr-dns v0.3.1/go.mod h1:G/245BRQ6FJGmryJCrOuTdB37AMA5AMOVuO6NY3JwTk= +github.com/multiformats/go-multiaddr-fmt v0.1.0 h1:WLEFClPycPkp4fnIzoFoV9FVd49/eQsuaL3/CWe167E= +github.com/multiformats/go-multiaddr-fmt v0.1.0/go.mod h1:hGtDIW4PU4BqJ50gW2quDuPVjyWNZxToGUh/HwTZYJo= github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivncJHmHnnd87g= github.com/multiformats/go-multibase v0.2.0/go.mod h1:bFBZX4lKCA/2lyOFSAoKH5SS6oPyjtnzK/XTFDPkNuk= github.com/multiformats/go-multicodec v0.9.0 h1:pb/dlPnzee/Sxv/j4PmkDRxCOi3hXTz3IbPKOXWJkmg= github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= +github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U= github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM= github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE= github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA= +github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8= github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +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/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= @@ -554,17 +762,20 @@ github.com/onflow/wal v1.0.2/go.mod h1:iMC8gkLqu4nkbkAla5HkSBb+FGyQOZiWz3DYm2wSX github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= -github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= -github.com/onsi/ginkgo/v2 v2.0.0/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= +github.com/onsi/ginkgo/v2 v2.13.2 h1:Bi2gGVkfn6gQcjNjZJVO8Gf0FHzMPf2phUei9tejVMs= +github.com/onsi/ginkgo/v2 v2.13.2/go.mod h1:XStQ8QcGwLyF4HdfcZB8SFOS/MWCgDuXMSBe6zrvLgM= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= -github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= -github.com/onsi/gomega v1.18.1/go.mod h1:0q+aL8jAiMXy9hbwj2mr5GziHiwhAIQpFmmtT5hitRs= +github.com/onsi/gomega v1.29.0 h1:KIA/t2t5UBzoirT4H9tsML45GEbo3ouUnBHsCfD2tVg= +github.com/onsi/gomega v1.29.0/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= +github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.1.0 h1:HHUyrt9mwHUjtasSbXSMvs4cyFxh+Bll4AjJ9odEGpg= +github.com/opencontainers/runtime-spec v1.1.0/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= +github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= @@ -582,6 +793,11 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= 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.89.0 h1:ADJTApkvkeBZsN0tBTx8QjpD9JkmxbKp0cxfr9qszm4= +github.com/polydawn/refmt v0.89.0/go.mod h1:/zvteZs/GwLtCgZ4BL6CBsk9IKIlexP43ObX9AxTqTw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= +github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= @@ -591,10 +807,12 @@ github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1: github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= +github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= +github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= @@ -602,6 +820,16 @@ github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3c github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/psiemens/sconfig v0.1.0 h1:xfWqW+TRpih7mXZIqKYTmpRhlZLQ1kbxV8EjllPv76s= github.com/psiemens/sconfig v0.1.0/go.mod h1:+MLKqdledP/8G3rOBpknbLh0IclCf4WneJUtS26JB2U= +github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo= +github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A= +github.com/quic-go/qtls-go1-20 v0.4.1 h1:D33340mCNDAIKBqXuAvexTNMUByrYmFYVfKfDN5nfFs= +github.com/quic-go/qtls-go1-20 v0.4.1/go.mod h1:X9Nh97ZL80Z+bX/gUXMbipO6OxdiDi58b/fMC9mAL+k= +github.com/quic-go/quic-go v0.40.1 h1:X3AGzUNFs0jVuO3esAGnTfvdgvL4fq655WaOi1snv1Q= +github.com/quic-go/quic-go v0.40.1/go.mod h1:PeN7kuVJ4xZbxSv/4OX6S1USOX8MJvydwpTx31vx60c= +github.com/quic-go/webtransport-go v0.6.0 h1:CvNsKqc4W2HljHJnoT+rMmbRJybShZ0YPFDD3NxaZLY= +github.com/quic-go/webtransport-go v0.6.0/go.mod h1:9KjU4AEBqEQidGHNDkZrb8CAa1abRaosM2yGOyiikEc= +github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk= +github.com/raulk/go-watchdog v1.3.0/go.mod h1:fIvOnLbF0b0ZwkB9YU4mOW9Did//4vPZtDqv66NfsMU= github.com/rivo/uniseg v0.2.0/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= @@ -618,6 +846,7 @@ github.com/rs/zerolog v1.33.0 h1:1cU2KZkvPxNyfgEmhHAz/1A9Bz+llsdYzklWFzgp0r8= github.com/rs/zerolog v1.33.0/go.mod h1:/7mN4D5sKwJLZQ2b/znpjC3/GQWY/xaDXUM0kKWRHss= github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/samber/lo v1.39.0 h1:4gTz1wUhNYLhFSKl6O+8peW0v2F4BCY034GRpU9WnuA= @@ -626,18 +855,53 @@ github.com/samber/slog-zerolog v1.0.0 h1:YpRy0xux1uJr0Ng3wrEjv9nyvb4RAoNqkS611Uj github.com/samber/slog-zerolog v1.0.0/go.mod h1:N2/g/mNGRY1zqsydIYE0uKipSSFsPDjytoVkRnZ0Jp0= 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/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sethvargo/go-limiter v1.0.0 h1:JqW13eWEMn0VFv86OKn8wiYJY/m250WoXdrjRV0kLe4= github.com/sethvargo/go-limiter v1.0.0/go.mod h1:01b6tW25Ap+MeLYBuD4aHunMrJoNO5PVUFdS9rac3II= github.com/sethvargo/go-retry v0.2.3 h1:oYlgvIvsju3jNbottWABtbnoLC+GDtLdBHxKWxQm/iU= github.com/sethvargo/go-retry v0.2.3/go.mod h1:1afjQuvh7s4gflMObvjLPaWgluLLyhA1wmVZ6KLpICw= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil/v3 v3.22.2 h1:wCrArWFkHYIdDxx/FSfF5RB4dpJYW6t7rcp3+zL8uks= +github.com/shirou/gopsutil/v3 v3.22.2/go.mod h1:WapW1AOOPlHyXr+yOyw3uYx36enocrtSoSBy0L5vUHY= +github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4/go.mod h1:XhFIlyj5a1fBNx5aJTbKoIq0mNaPvOagO+HjB3EtxrY= +github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48/go.mod h1:5u70Mqkb5O5cxEA8nxTsgrgLehJeAw6Oc4Ab1c/P1HM= +github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470/go.mod h1:2dOwnU2uBioM+SGy2aZoq1f/Sd1l9OkAeAUvjSyvgU0= +github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= +github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= +github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d/go.mod h1:05UtEgK5zq39gLST6uB0cf3NEHjETfB4Fgr3Gx5R9Vw= +github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c/go.mod h1:8d3azKNyqcHP1GaQE/c6dDgjkgSx2BZ4IoEi4F1reUI= +github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b/go.mod h1:ZpfEhSmds4ytuByIcDnOLkTHGUI6KNqRNPDLHDk+mUU= +github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20/go.mod h1:UDKB5a1T23gOMUJrI+uSuH0VRDStOiUVSjBTRDVBVag= +github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9/go.mod h1:+rgNQw2P9ARFAs37qieuu7ohDNQ3gds9msbT2yn85sg= +github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50/go.mod h1:zPn1wHpTIePGnXSHpsVPWEktKXHr6+SS6x/IKRb7cpw= +github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc/go.mod h1:aYMfkZ6DWSJPJ6c4Wwz3QtW22G7mf/PEgaB9k/ik5+Y= +github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= +github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191/go.mod h1:e2qWDig5bLteJ4fwvDAc2NHzqFEthkqn7aOZAOpj+PQ= +github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241/go.mod h1:NPpHK2TI7iSaM0buivtFUc9offApnI0Alt/K8hcHy0I= +github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122/go.mod h1:b5uSkrEVM1jQUspwbixRBhaIjIzL2xazXp6kntxYle0= +github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2/go.mod h1:eWdoE5JD4R5UVWDucdOPg1g2fqQRq78IQa9zlOV1vpQ= +github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82/go.mod h1:TCR1lToEk4d2s07G3XGfz2QrgHXg4RJBvjrOozvoWfk= +github.com/shurcooL/sanitized_anchor_name v0.0.0-20170918181015-86672fcb3f95/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= +github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/slok/go-http-metrics v0.10.0 h1:rh0LaYEKza5eaYRGDXujKrOln57nHBi4TtVhmNEpbgM= github.com/slok/go-http-metrics v0.10.0/go.mod h1:lFqdaS4kWMfUKCSukjC47PdCeTk+hXDUVm8kLHRqJ38= +github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= +github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= +github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg= +github.com/smartystreets/goconvey v1.7.2/go.mod h1:Vw0tHAZW6lzCRk3xgdin6fKYcG+G3Pg9vgXWeJpQFMM= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/sony/gobreaker v0.5.0 h1:dRCvqm0P490vZPmy7ppEk2qCnCieBooFJ+YoXGYB+yg= +github.com/sony/gobreaker v0.5.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE= +github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -672,10 +936,12 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/subosito/gotenv v1.4.2 h1:X1TuBLAMDFbaTAChgCBLu3DU3UPyELpnF2jjJ2cz/S8= @@ -684,10 +950,13 @@ github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbe github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc= +github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= 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/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= 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-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= @@ -699,14 +968,25 @@ github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGr github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= +github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli v1.22.10 h1:p8Fspmz3iTctJstry1PYS3HVdllxnEzTEsgIgtxTrCk= +github.com/urfave/cli v1.22.10/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/cli/v2 v2.25.7 h1:VAzn5oq403l5pHjc4OhD54+XGO9cdKVL/7lDjF+iKUs= github.com/urfave/cli/v2 v2.25.7/go.mod h1:8qnjx1vcq5s2/wpsqoZFndg2CE5tNFyrTvS6SinrnYQ= +github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU= +github.com/viant/toolbox v0.24.0/go.mod h1:OxMCG57V0PXuIP2HNQrtJf2CjqdmbrOx5EkMILuUhzM= github.com/vmihailenco/msgpack v4.0.4+incompatible h1:dSLoQfGFAo3F6OoNhwUmLwVgaUXK79GlxNBwueZn0xI= github.com/vmihailenco/msgpack v4.0.4+incompatible/go.mod h1:fy3FlTQTDXWkZ7Bh6AcGMlsjHatGryHQYUTf1ShIgkk= github.com/vmihailenco/msgpack/v4 v4.3.11 h1:Q47CePddpNGNhk4GCnAx9DDtASi2rasatE0cd26cZoE= github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/tagparser v0.1.1 h1:quXMXlA39OCbd2wAdTsGDlK9RkOk6Wuw+x37wVyIuWY= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= +github.com/warpfork/go-testmark v0.12.1 h1:rMgCpJfwy1sJ50x0M0NgyphxYYPMOODIJHhsXyEHU0s= +github.com/warpfork/go-testmark v0.12.1/go.mod h1:kHwy7wfvGSPh1rQJYKayD4AbtNaeyZdcGi9tNJTaa5Y= +github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0 h1:GDDkbFiaK8jsSDJfjId/PEGEShv6ugrt4kYsC5UIDaQ= +github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= +github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= +github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= @@ -719,6 +999,8 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/zeebo/assert v1.1.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= @@ -727,6 +1009,7 @@ github.com/zeebo/blake3 v0.2.3/go.mod h1:mjJjZpnsyIVtVgTOSpJ9vmRE4wgDeyt2HU3qXvv github.com/zeebo/pcg v1.0.1 h1:lyqfGeWiv4ahac6ttHs+I5hwtH/+1mrhlCtVNQM2kHo= github.com/zeebo/pcg v1.0.1/go.mod h1:09F0S9iiKrwn9rlI5yjLkmrug154/YRW6KnnXVDM/l4= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= +go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -754,29 +1037,46 @@ go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/dig v1.17.1 h1:Tga8Lz8PcYNsWsyHMZ1Vm0OQOUaJNDyvPImgbAu9YSc= +go.uber.org/dig v1.17.1/go.mod h1:Us0rSJiThwCv2GteUN0Q7OKvU7n5J4dxZ9JKUXozFdE= +go.uber.org/fx v1.20.1 h1:zVwVQGS8zYvhh9Xxcu4w1M6ESyeMzebzj2NbSayZ4Mk= +go.uber.org/fx v1.20.1/go.mod h1:iSYNbHf2y55acNCwCXKx7LbWb5WG1Bnue5RDXz1OREg= go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/mock v0.4.0 h1:VcM4ZOtdbR4f6VXfiOpwpVJDL6lCReaZ6mw31wqh7KU= +go.uber.org/mock v0.4.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= +go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/ratelimit v0.3.1 h1:K4qVE+byfv/B3tC+4nYWP7v/6SimcO7HzHekoMNBma0= go.uber.org/ratelimit v0.3.1/go.mod h1:6euWsTB6U/Nb3X++xEUXA8ciPJvr19Q/0h1+oDcJhRk= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= +golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= @@ -796,6 +1096,7 @@ golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRj golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= 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= +golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -824,11 +1125,14 @@ golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= 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= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181029044818-c44066c5c816/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -858,14 +1162,16 @@ golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= 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= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -876,6 +1182,7 @@ golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.17.0 h1:6m3ZPmLEFdVxKKWnKq4VqZ60gutO35zm+zrAHVmHyDQ= golang.org/x/oauth2 v0.17.0/go.mod h1:OzPDGQiuQMguemayvdylqddI7qcD9lnSDb+1FiwQ5HA= +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= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -890,14 +1197,17 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.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-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181029174526-d69651ed3497/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190316082340-a2f829d7f35f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -910,11 +1220,13 @@ golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -926,23 +1238,27 @@ golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210225134936-a50acf3fe073/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423185535-09eb48e85fd7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220111092808-5a964db01320/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -959,6 +1275,7 @@ golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -972,18 +1289,23 @@ golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +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.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030000716-a0a13e073c7b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= @@ -993,6 +1315,8 @@ golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgw golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1023,7 +1347,6 @@ golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82u golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -1040,6 +1363,9 @@ golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSm golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.14.0 h1:2NiG67LD1tEH0D7kM+ps2V+fXmsAnpUeec7n8tcr4S0= gonum.org/v1/gonum v0.14.0/go.mod h1:AoWeoz0becf9QMWtE8iWXNXc27fK4fNeHNf/oMejGfU= +google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= @@ -1062,6 +1388,8 @@ google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjR google.golang.org/api v0.162.0 h1:Vhs54HkaEpkMBdgGdOT2P6F0csGG/vxDS0hWHJzmmps= google.golang.org/api v0.162.0/go.mod h1:6SulDkfoBIg4NFmCuZ39XeeAgSHCPecfSUuDyYlAHs0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= @@ -1071,6 +1399,10 @@ google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCID google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181029155118-b69ba1387ce2/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181202183823-bd91e49a0898/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= +google.golang.org/genproto v0.0.0-20190306203927-b5d61aea6440/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1112,6 +1444,9 @@ google.golang.org/genproto/googleapis/api v0.0.0-20240227224415-6ceb2ff114de h1: google.golang.org/genproto/googleapis/api v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:5iCWqnniDlqZHrd3neWVTOwvh/v6s3232omMecelax8= google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de h1:cZGRis4/ot9uVm639a+rHCUaG0JJHEsdyzSQTMX+suY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:H4O17MA/PE9BsGx3w+a+W2VOLLD1Qf7oJneAoU6WktY= +google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= +google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -1131,6 +1466,8 @@ google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA5 google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +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= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -1143,6 +1480,7 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= @@ -1155,6 +1493,7 @@ gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= 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/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= @@ -1176,6 +1515,8 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= +grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= +honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= @@ -1192,3 +1533,5 @@ 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= +sourcegraph.com/sourcegraph/go-diff v0.5.0/go.mod h1:kuch7UrkMzY0X+p9CRK03kfuPQ2zzQcaEFbx8wA8rck= +sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4/go.mod h1:ketZ/q3QxT9HOBeFhu6RdvsftgpsbFHBF5Cas6cDKZ0= diff --git a/models/engine.go b/models/engine.go deleted file mode 100644 index edb328689..000000000 --- a/models/engine.go +++ /dev/null @@ -1,55 +0,0 @@ -package models - -import ( - "context" -) - -// Engine defines a processing unit -type Engine interface { - // Run the engine with context, errors are not expected. - Run(ctx context.Context) error - // Stop the engine. - Stop() - // Done signals the engine was stopped. - Done() <-chan struct{} - // Ready signals the engine was started. - Ready() <-chan struct{} -} - -type EngineStatus struct { - done chan struct{} - ready chan struct{} - stop chan struct{} -} - -func NewEngineStatus() *EngineStatus { - return &EngineStatus{ - done: make(chan struct{}), - ready: make(chan struct{}), - stop: make(chan struct{}), - } -} - -func (e *EngineStatus) Ready() <-chan struct{} { - return e.ready -} - -func (e *EngineStatus) Stopped() <-chan struct{} { - return e.stop -} - -func (e *EngineStatus) Done() <-chan struct{} { - return e.done -} - -func (e *EngineStatus) MarkReady() { - close(e.ready) -} - -func (e *EngineStatus) MarkDone() { - close(e.done) -} - -func (e *EngineStatus) MarkStopped() { - close(e.stop) -} diff --git a/models/stream.go b/models/stream.go index 90fd59104..d8d33d8de 100644 --- a/models/stream.go +++ b/models/stream.go @@ -3,27 +3,50 @@ package models import ( "sync" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" "github.com/rs/zerolog" ) type Publisher[T any] struct { + component.Component + cm *component.ComponentManager + + log zerolog.Logger + mux sync.RWMutex subscribers map[Subscriber[T]]struct{} + + publishChan chan T + + publisherExited chan struct{} } -func NewPublisher[T any]() *Publisher[T] { - return &Publisher[T]{ - mux: sync.RWMutex{}, - subscribers: make(map[Subscriber[T]]struct{}), +func NewPublisher[T any](log zerolog.Logger) *Publisher[T] { + p := &Publisher[T]{ + mux: sync.RWMutex{}, + log: log, + subscribers: make(map[Subscriber[T]]struct{}), + publishChan: make(chan T), + publisherExited: make(chan struct{}), } + + builder := component.NewComponentManagerBuilder() + + builder.AddWorker(p.publishWorker) + + p.cm = builder.Build() + p.Component = p.cm + + return p } func (p *Publisher[T]) Publish(data T) { - p.mux.RLock() - defer p.mux.RUnlock() - - for s := range p.subscribers { - s.Notify(data) + select { + case <-p.publisherExited: + return + default: + p.publishChan <- data } } @@ -41,36 +64,53 @@ func (p *Publisher[T]) Unsubscribe(s Subscriber[T]) { delete(p.subscribers, s) } +func (p *Publisher[T]) publishWorker(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + defer func() { + close(p.publisherExited) + close(p.publishChan) + }() + ready() + + for { + select { + case <-ctx.Done(): + return + case data := <-p.publishChan: + stop := func() bool { + p.mux.RLock() + defer p.mux.RUnlock() + + for s := range p.subscribers { + err := s.Notify(data) + if err != nil { + p.log.Error().Err(err).Msg("failed to notify subscriber") + ctx.Throw(err) + return true + } + } + return false + }() + if stop { + return + } + } + } +} + type Subscriber[T any] interface { - Notify(data T) - Error() <-chan error + Notify(data T) error } type Subscription[T any] struct { - logger zerolog.Logger - err chan error callback func(data T) error } -func NewSubscription[T any](logger zerolog.Logger, callback func(T) error) *Subscription[T] { +func NewSubscription[T any](callback func(T) error) *Subscription[T] { return &Subscription[T]{ - logger: logger, callback: callback, - err: make(chan error, 1), - } -} - -func (b *Subscription[T]) Notify(data T) { - err := b.callback(data) - if err != nil { - select { - case b.err <- err: - default: - b.logger.Debug().Err(err).Msg("failed to send error to subscription") - } } } -func (b *Subscription[T]) Error() <-chan error { - return b.err +func (b *Subscription[T]) Notify(data T) error { + return b.callback(data) } diff --git a/models/stream_test.go b/models/stream_test.go index ddc2b0fda..ded635dff 100644 --- a/models/stream_test.go +++ b/models/stream_test.go @@ -1,7 +1,6 @@ package models_test import ( - "fmt" "sync" "sync/atomic" "testing" @@ -110,35 +109,35 @@ func Test_Stream(t *testing.T) { waitAllUnsubscribed.Wait() close(stopPublishing) }) - - t.Run("error handling", func(t *testing.T) { - p := newMockPublisher() - s := &mockSubscription{} - errContent := fmt.Errorf("failed to process data") - - s.Subscription = models.NewSubscription[mockData](zerolog.Nop(), func(data mockData) error { - s.callCount.Add(1) - return errContent - }) - - p.Subscribe(s) - - shouldReceiveError := make(chan struct{}) - ready := make(chan struct{}) - go func() { - close(ready) - select { - case err := <-s.Error(): - require.ErrorIs(t, err, errContent) - case <-shouldReceiveError: - require.Fail(t, "should have received error") - } - }() - <-ready - - p.Publish(mockData{}) - close(shouldReceiveError) - }) + // + //t.Run("error handling", func(t *testing.T) { + // p := newMockPublisher() + // s := &mockSubscription{} + // errContent := fmt.Errorf("failed to process data") + // + // s.Subscription = models.NewSubscription[mockData](func(data mockData) error { + // s.callCount.Add(1) + // return errContent + // }) + // + // p.Subscribe(s) + // + // shouldReceiveError := make(chan struct{}) + // ready := make(chan struct{}) + // go func() { + // close(ready) + // select { + // case err := <-s.Error(): + // require.ErrorIs(t, err, errContent) + // case <-shouldReceiveError: + // require.Fail(t, "should have received error") + // } + // }() + // <-ready + // + // p.Publish(mockData{}) + // close(shouldReceiveError) + //}) } type mockData struct{} @@ -150,7 +149,7 @@ type mockSubscription struct { func newMockSubscription() *mockSubscription { s := &mockSubscription{} - s.Subscription = models.NewSubscription[mockData](zerolog.Nop(), func(data mockData) error { + s.Subscription = models.NewSubscription[mockData](func(data mockData) error { s.callCount.Add(1) return nil }) @@ -162,5 +161,5 @@ func (s *mockSubscription) CallCount() uint64 { } func newMockPublisher() *models.Publisher[mockData] { - return models.NewPublisher[mockData]() + return models.NewPublisher[mockData](zerolog.Nop()) } diff --git a/services/ingestion/engine.go b/services/ingestion/engine.go index 2145e6497..8f1d4c6ec 100644 --- a/services/ingestion/engine.go +++ b/services/ingestion/engine.go @@ -1,9 +1,11 @@ package ingestion import ( - "context" "fmt" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + flowGo "github.com/onflow/flow-go/model/flow" pebbleDB "github.com/cockroachdb/pebble" @@ -20,8 +22,6 @@ import ( "github.com/onflow/flow-go/fvm/evm/offchain/sync" ) -var _ models.Engine = &Engine{} - // Engine is an implementation of the event ingestion engine. // // This engine monitors the Flow network for two types of EVM events: @@ -37,7 +37,8 @@ var _ models.Engine = &Engine{} // it will just overwrite the current indexed data. Idempotency is an important // requirement of the implementation of this engine. type Engine struct { - *models.EngineStatus + component.Component + cm *component.ComponentManager subscriber EventSubscriber blocksProvider *replayer.BlocksProvider @@ -72,8 +73,7 @@ func NewEventIngestionEngine( ) *Engine { log = log.With().Str("component", "ingestion").Logger() - return &Engine{ - EngineStatus: models.NewEngineStatus(), + e := &Engine{ subscriber: subscriber, blocksProvider: blocksProvider, @@ -89,12 +89,13 @@ func NewEventIngestionEngine( collector: collector, replayerConfig: replayerConfig, } -} -// Stop the engine. -func (e *Engine) Stop() { - e.MarkDone() - <-e.Stopped() + builder := component.NewComponentManagerBuilder() + builder.AddWorker(e.run) + e.cm = builder.Build() + e.Component = e.cm + + return e } // Run the Cadence event ingestion engine. @@ -112,34 +113,29 @@ func (e *Engine) Stop() { // handled by restarting the engine. This can happen if the client connection to the event subscription // drops. // All other errors are unexpected. -func (e *Engine) Run(ctx context.Context) error { +func (e *Engine) run(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { e.log.Info().Msg("starting ingestion") - e.MarkReady() - defer e.MarkStopped() - events := e.subscriber.Subscribe(ctx) + ready() + for { select { - case <-e.Done(): - // stop the engine - return nil + case <-ctx.Done(): + return case events, ok := <-events: if !ok { - return nil + return } if events.Err != nil { - return fmt.Errorf( - "failure in event subscription with: %w", - events.Err, - ) + return } err := e.processEvents(events.Events) if err != nil { e.log.Error().Err(err).Msg("failed to process EVM events") - return err + return } } } diff --git a/services/ingestion/engine_test.go b/services/ingestion/engine_test.go index 384f0da8b..da77d6633 100644 --- a/services/ingestion/engine_test.go +++ b/services/ingestion/engine_test.go @@ -1,573 +1,587 @@ package ingestion import ( - "context" "encoding/hex" - "encoding/json" "math/big" - "testing" - - "github.com/onflow/flow-evm-gateway/storage" - - pebbleDB "github.com/cockroachdb/pebble" - "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/fvm/evm" - "github.com/onflow/flow-go/fvm/evm/events" - flowGo "github.com/onflow/flow-go/model/flow" - - "github.com/onflow/flow-evm-gateway/metrics" - "github.com/onflow/flow-evm-gateway/services/ingestion/mocks" - "github.com/onflow/flow-evm-gateway/services/replayer" - "github.com/onflow/flow-evm-gateway/storage/pebble" "github.com/onflow/cadence" - "github.com/onflow/flow-evm-gateway/models" - - "github.com/onflow/flow-go-sdk" + "github.com/onflow/flow-go/fvm/evm/events" "github.com/onflow/flow-go/fvm/evm/types" + flowGo "github.com/onflow/flow-go/model/flow" gethCommon "github.com/onflow/go-ethereum/common" gethTypes "github.com/onflow/go-ethereum/core/types" - "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - - storageMock "github.com/onflow/flow-evm-gateway/storage/mocks" ) -func TestSerialBlockIngestion(t *testing.T) { - - t.Run("successfully ingest serial blocks", func(t *testing.T) { - receipts := &storageMock.ReceiptIndexer{} - transactions := &storageMock.TransactionIndexer{} - latestHeight := uint64(10) - - store, registerStore := setupStore(t) - - blocks := &storageMock.BlockIndexer{} - blocks. - On("LatestCadenceHeight"). - Return(func() (uint64, error) { - return latestHeight, nil - }). - Once() // make sure this isn't called multiple times - - traces := &storageMock.TraceIndexer{} - - eventsChan := make(chan models.BlockEvents) - - subscriber := &mocks.EventSubscriber{} - subscriber. - On("Subscribe", mock.Anything). - Return(func(ctx context.Context) <-chan models.BlockEvents { - return eventsChan - }) - - engine := NewEventIngestionEngine( - subscriber, - replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - store, - registerStore, - blocks, - receipts, - transactions, - traces, - models.NewPublisher[*models.Block](), - models.NewPublisher[[]*gethTypes.Log](), - zerolog.Nop(), - metrics.NopCollector, - defaultReplayerConfig(), - ) - - done := make(chan struct{}) - go func() { - err := engine.Run(context.Background()) - assert.NoError(t, err) - close(done) - }() - - storedCounter := 0 - runs := uint64(20) - for i := latestHeight + 1; i < latestHeight+runs; i++ { - cadenceHeight := i + 10 - blockCdc, block, blockEvent, err := newBlock(i, nil) - require.NoError(t, err) - - blocks. - On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - assert.Equal(t, block, storeBlock) - assert.Equal(t, cadenceHeight, h) - storedCounter++ - return nil - }). - Once() - - eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - Events: []flow.Event{{ - Type: string(blockEvent.Etype), - Value: blockCdc, - }}, - Height: cadenceHeight, - }) - } - - close(eventsChan) - <-done - assert.Equal(t, runs-1, uint64(storedCounter)) - }) - - t.Run("fail with events out of sequence", func(t *testing.T) { - receipts := &storageMock.ReceiptIndexer{} - transactions := &storageMock.TransactionIndexer{} - latestHeight := uint64(10) - - store, registerStore := setupStore(t) - - blocks := &storageMock.BlockIndexer{} - blocks. - On("LatestCadenceHeight"). - Return(func() (uint64, error) { - return latestHeight, nil - }). - Once() // make sure this isn't called multiple times - - traces := &storageMock.TraceIndexer{} - - eventsChan := make(chan models.BlockEvents) - subscriber := &mocks.EventSubscriber{} - subscriber. - On("Subscribe", mock.Anything). - Return(func(ctx context.Context) <-chan models.BlockEvents { - return eventsChan - }) - - engine := NewEventIngestionEngine( - subscriber, - replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - store, - registerStore, - blocks, - receipts, - transactions, - traces, - models.NewPublisher[*models.Block](), - models.NewPublisher[[]*gethTypes.Log](), - zerolog.Nop(), - metrics.NopCollector, - defaultReplayerConfig(), - ) - - waitErr := make(chan struct{}) - // catch eventual error due to out of sequence block height - go func() { - err := engine.Run(context.Background()) - assert.ErrorIs(t, err, models.ErrInvalidHeight) - assert.ErrorContains(t, err, "invalid height: received new block: 20, non-sequential of latest block: 11") - close(waitErr) - }() - - // first create one successful block event - blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) - cadenceHeight := latestHeight + 10 - require.NoError(t, err) - - blocks. - On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - assert.Equal(t, block, storeBlock) - assert.Equal(t, cadenceHeight, h) - return nil - }). - Once() // this should only be called for first valid block - - cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ - Events: []flow.Event{{ - Type: string(blockEvent.Etype), - Value: blockCdc, - }}, - Height: cadenceHeight, - }) - require.NoError(t, err) - - eventsChan <- models.BlockEvents{ - Events: cadenceEvents, - } - - // fail with next block height being incorrect - blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height - require.NoError(t, err) - - cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ - Events: []flow.Event{{ - Type: string(blockEvent.Etype), - Value: blockCdc, - }}, - Height: cadenceHeight + 1, - }) - require.NoError(t, err) - - eventsChan <- models.BlockEvents{ - Events: cadenceEvents, - } - - close(eventsChan) - <-waitErr - }) -} - -func TestBlockAndTransactionIngestion(t *testing.T) { - - t.Run("successfully ingest transaction and block", func(t *testing.T) { - receipts := &storageMock.ReceiptIndexer{} - transactions := &storageMock.TransactionIndexer{} - latestHeight := uint64(10) - nextHeight := latestHeight + 1 - blockID := flow.Identifier{0x01} - - store, registerStore := setupStore(t) - - blocks := &storageMock.BlockIndexer{} - blocks. - On("LatestCadenceHeight"). - Return(func() (uint64, error) { - return latestHeight, nil - }). - Once() // make sure this isn't called multiple times - - blocks. - On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). - Return(func(h uint64, _ *pebbleDB.Batch) error { - assert.Equal(t, nextHeight, h) - return nil - }) - - eventsChan := make(chan models.BlockEvents) - subscriber := &mocks.EventSubscriber{} - subscriber. - On("Subscribe", mock.Anything). - Return(func(ctx context.Context) <-chan models.BlockEvents { - return eventsChan - }) - - txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) - require.NoError(t, err) - blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) - require.NoError(t, err) - - traces := &storageMock.TraceIndexer{} - traces. - On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - assert.Equal(t, transaction.Hash(), txID) - return nil - }) - - engine := NewEventIngestionEngine( - subscriber, - replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - store, - registerStore, - blocks, - receipts, - transactions, - traces, - models.NewPublisher[*models.Block](), - models.NewPublisher[[]*gethTypes.Log](), - zerolog.Nop(), - metrics.NopCollector, - defaultReplayerConfig(), - ) - - done := make(chan struct{}) - go func() { - err := engine.Run(context.Background()) - assert.NoError(t, err) - close(done) - }() - - blocks. - On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - assert.Equal(t, block, storeBlock) - assert.Equal(t, blockID, id) - assert.Equal(t, nextHeight, h) - return nil - }). - Once() - - transactions. - On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal - return nil - }). - Once() - - receipts. - On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { - assert.Len(t, receipts, 1) - rcp := receipts[0] - - assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) - assert.Len(t, rcp.Logs, len(result.Logs)) - assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) - return nil - }). - Once() - - eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - Events: []flow.Event{{ - Type: string(blockEvent.Etype), - Value: blockCdc, - }, { - Type: string(txEvent.Etype), - Value: txCdc, - }}, - Height: nextHeight, - BlockID: blockID, - }) - - close(eventsChan) - <-done - }) - - t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { - receipts := &storageMock.ReceiptIndexer{} - transactions := &storageMock.TransactionIndexer{} - latestHeight := uint64(10) - nextHeight := latestHeight + 1 - - store, registerStore := setupStore(t) - - blocks := &storageMock.BlockIndexer{} - blocks. - On("LatestCadenceHeight"). - Return(func() (uint64, error) { - return latestHeight, nil - }). - On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). - Return(func(h uint64) error { return nil }) - - eventsChan := make(chan models.BlockEvents) - subscriber := &mocks.EventSubscriber{} - subscriber. - On("Subscribe", mock.Anything). - Return(func(ctx context.Context) <-chan models.BlockEvents { - return eventsChan - }) - - txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) - require.NoError(t, err) - blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) - require.NoError(t, err) - - traces := &storageMock.TraceIndexer{} - traces. - On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - assert.Equal(t, transaction.Hash(), txID) - return nil - }) - - engine := NewEventIngestionEngine( - subscriber, - replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - store, - registerStore, - blocks, - receipts, - transactions, - traces, - models.NewPublisher[*models.Block](), - models.NewPublisher[[]*gethTypes.Log](), - zerolog.Nop(), - metrics.NopCollector, - defaultReplayerConfig(), - ) - - done := make(chan struct{}) - go func() { - err := engine.Run(context.Background()) - assert.NoError(t, err) - close(done) - }() - - blocksFirst := false // flag indicating we stored block first - blocks. - On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - blocksFirst = true - return nil - }). - Once() - - transactions. - On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - require.True(t, blocksFirst) - return nil - }). - Once() - - receipts. - On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { - require.True(t, blocksFirst) - return nil - }). - Once() - - eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - Events: []flow.Event{ - // first transaction - { - Type: string(txEvent.Etype), - Value: txCdc, - }, - // and then block (out-of-order) - { - Type: string(blockEvent.Etype), - Value: blockCdc, - }, - }, - Height: nextHeight, - }) - - close(eventsChan) - <-done - }) - - t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { - receipts := &storageMock.ReceiptIndexer{} - transactions := &storageMock.TransactionIndexer{} - latestCadenceHeight := uint64(0) - - store, registerStore := setupStore(t) - - blocks := &storageMock.BlockIndexer{} - blocks. - On("LatestCadenceHeight"). - Return(func() (uint64, error) { - return latestCadenceHeight, nil - }). - Once() // make sure this isn't called multiple times - - traces := &storageMock.TraceIndexer{} - - eventsChan := make(chan models.BlockEvents) - subscriber := &mocks.EventSubscriber{} - subscriber. - On("Subscribe", mock.Anything). - Return(func(ctx context.Context) <-chan models.BlockEvents { - return eventsChan - }). - Once() - - engine := NewEventIngestionEngine( - subscriber, - replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - store, - registerStore, - blocks, - receipts, - transactions, - traces, - models.NewPublisher[*models.Block](), - models.NewPublisher[[]*gethTypes.Log](), - zerolog.Nop(), - metrics.NopCollector, - defaultReplayerConfig(), - ) - - done := make(chan struct{}) - go func() { - err := engine.Run(context.Background()) - assert.NoError(t, err) - close(done) - }() - - evmHeight := uint64(0) - events := make([]flow.Event, 0) - blockIndexedFirst := false - txsStored := 0 - eventCount := 5 - txHashes := make([]gethCommon.Hash, eventCount) - - for i := 0; i < eventCount; i++ { - txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) - txHashes[i] = res.TxHash - require.NoError(t, err) - - // add a single transaction for each block - transactions. - On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal - require.True(t, blockIndexedFirst) - txsStored++ - return nil - }). - Once() - - receipts. - On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). - Once() - - traces. - On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - assert.Equal(t, transaction.Hash(), txID) - return nil - }) - - events = append(events, flow.Event{ - Type: string(txEvent.Etype), - Value: txCdc, - }) - } - - blocksStored := 0 - blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) - require.NoError(t, err) - - blocks. - On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - assert.Equal(t, block, storeBlock) - assert.Equal(t, evmHeight, block.Height) - assert.Equal(t, latestCadenceHeight+1, h) - blockIndexedFirst = true - blocksStored++ - return nil - }). - Once() - - events = append(events, flow.Event{ - Type: string(blockEvent.Etype), - Value: blockCdc, - }) - - // this messes up order of events to test if we still process events in-order - // it will make transaction event first and then block event - events[0], events[1] = events[1], events[0] - // and it will make the first block be swapped with second block out-of-order - events[1], events[2] = events[2], events[1] - - eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - Events: events, - Height: latestCadenceHeight + 1, - }) - - close(eventsChan) - <-done - assert.Equal(t, eventCount, txsStored) - assert.Equal(t, 1, blocksStored) - }) -} +// +//import ( +// "context" +// "encoding/hex" +// "encoding/json" +// "math/big" +// "testing" +// +// "github.com/onflow/flow-evm-gateway/storage" +// +// pebbleDB "github.com/cockroachdb/pebble" +// "github.com/onflow/flow-go/fvm/environment" +// "github.com/onflow/flow-go/fvm/evm" +// "github.com/onflow/flow-go/fvm/evm/events" +// flowGo "github.com/onflow/flow-go/model/flow" +// +// "github.com/onflow/flow-evm-gateway/metrics" +// "github.com/onflow/flow-evm-gateway/services/ingestion/mocks" +// "github.com/onflow/flow-evm-gateway/services/replayer" +// "github.com/onflow/flow-evm-gateway/storage/pebble" +// +// "github.com/onflow/cadence" +// +// "github.com/onflow/flow-evm-gateway/models" +// +// "github.com/onflow/flow-go-sdk" +// "github.com/onflow/flow-go/fvm/evm/types" +// gethCommon "github.com/onflow/go-ethereum/common" +// gethTypes "github.com/onflow/go-ethereum/core/types" +// "github.com/rs/zerolog" +// "github.com/stretchr/testify/assert" +// "github.com/stretchr/testify/mock" +// "github.com/stretchr/testify/require" +// +// storageMock "github.com/onflow/flow-evm-gateway/storage/mocks" +//) +// +//func TestSerialBlockIngestion(t *testing.T) { +// +// t.Run("successfully ingest serial blocks", func(t *testing.T) { +// receipts := &storageMock.ReceiptIndexer{} +// transactions := &storageMock.TransactionIndexer{} +// latestHeight := uint64(10) +// +// store, registerStore := setupStore(t) +// +// blocks := &storageMock.BlockIndexer{} +// blocks. +// On("LatestCadenceHeight"). +// Return(func() (uint64, error) { +// return latestHeight, nil +// }). +// Once() // make sure this isn't called multiple times +// +// traces := &storageMock.TraceIndexer{} +// +// eventsChan := make(chan models.BlockEvents) +// +// subscriber := &mocks.EventSubscriber{} +// subscriber. +// On("Subscribe", mock.Anything). +// Return(func(ctx context.Context) <-chan models.BlockEvents { +// return eventsChan +// }) +// +// engine := NewEventIngestionEngine( +// subscriber, +// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), +// store, +// registerStore, +// blocks, +// receipts, +// transactions, +// traces, +// models.NewPublisher[*models.Block](zerolog.Nop()), +// models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()), +// zerolog.Nop(), +// metrics.NopCollector, +// defaultReplayerConfig(), +// ) +// +// done := make(chan struct{}) +// go func() { +// err := engine.Run(context.Background()) +// assert.NoError(t, err) +// close(done) +// }() +// +// storedCounter := 0 +// runs := uint64(20) +// for i := latestHeight + 1; i < latestHeight+runs; i++ { +// cadenceHeight := i + 10 +// blockCdc, block, blockEvent, err := newBlock(i, nil) +// require.NoError(t, err) +// +// blocks. +// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). +// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { +// assert.Equal(t, block, storeBlock) +// assert.Equal(t, cadenceHeight, h) +// storedCounter++ +// return nil +// }). +// Once() +// +// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ +// Events: []flow.Event{{ +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }}, +// Height: cadenceHeight, +// }) +// } +// +// close(eventsChan) +// <-done +// assert.Equal(t, runs-1, uint64(storedCounter)) +// }) +// +// t.Run("fail with events out of sequence", func(t *testing.T) { +// receipts := &storageMock.ReceiptIndexer{} +// transactions := &storageMock.TransactionIndexer{} +// latestHeight := uint64(10) +// +// store, registerStore := setupStore(t) +// +// blocks := &storageMock.BlockIndexer{} +// blocks. +// On("LatestCadenceHeight"). +// Return(func() (uint64, error) { +// return latestHeight, nil +// }). +// Once() // make sure this isn't called multiple times +// +// traces := &storageMock.TraceIndexer{} +// +// eventsChan := make(chan models.BlockEvents) +// subscriber := &mocks.EventSubscriber{} +// subscriber. +// On("Subscribe", mock.Anything). +// Return(func(ctx context.Context) <-chan models.BlockEvents { +// return eventsChan +// }) +// +// engine := NewEventIngestionEngine( +// subscriber, +// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), +// store, +// registerStore, +// blocks, +// receipts, +// transactions, +// traces, +// models.NewPublisher[*models.Block](), +// models.NewPublisher[[]*gethTypes.Log](), +// zerolog.Nop(), +// metrics.NopCollector, +// defaultReplayerConfig(), +// ) +// +// waitErr := make(chan struct{}) +// // catch eventual error due to out of sequence block height +// go func() { +// err := engine.Run(context.Background()) +// assert.ErrorIs(t, err, models.ErrInvalidHeight) +// assert.ErrorContains(t, err, "invalid height: received new block: 20, non-sequential of latest block: 11") +// close(waitErr) +// }() +// +// // first create one successful block event +// blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) +// cadenceHeight := latestHeight + 10 +// require.NoError(t, err) +// +// blocks. +// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). +// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { +// assert.Equal(t, block, storeBlock) +// assert.Equal(t, cadenceHeight, h) +// return nil +// }). +// Once() // this should only be called for first valid block +// +// cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ +// Events: []flow.Event{{ +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }}, +// Height: cadenceHeight, +// }) +// require.NoError(t, err) +// +// eventsChan <- models.BlockEvents{ +// Events: cadenceEvents, +// } +// +// // fail with next block height being incorrect +// blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height +// require.NoError(t, err) +// +// cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ +// Events: []flow.Event{{ +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }}, +// Height: cadenceHeight + 1, +// }) +// require.NoError(t, err) +// +// eventsChan <- models.BlockEvents{ +// Events: cadenceEvents, +// } +// +// close(eventsChan) +// <-waitErr +// }) +//} +// +//func TestBlockAndTransactionIngestion(t *testing.T) { +// +// t.Run("successfully ingest transaction and block", func(t *testing.T) { +// receipts := &storageMock.ReceiptIndexer{} +// transactions := &storageMock.TransactionIndexer{} +// latestHeight := uint64(10) +// nextHeight := latestHeight + 1 +// blockID := flow.Identifier{0x01} +// +// store, registerStore := setupStore(t) +// +// blocks := &storageMock.BlockIndexer{} +// blocks. +// On("LatestCadenceHeight"). +// Return(func() (uint64, error) { +// return latestHeight, nil +// }). +// Once() // make sure this isn't called multiple times +// +// blocks. +// On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). +// Return(func(h uint64, _ *pebbleDB.Batch) error { +// assert.Equal(t, nextHeight, h) +// return nil +// }) +// +// eventsChan := make(chan models.BlockEvents) +// subscriber := &mocks.EventSubscriber{} +// subscriber. +// On("Subscribe", mock.Anything). +// Return(func(ctx context.Context) <-chan models.BlockEvents { +// return eventsChan +// }) +// +// txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) +// require.NoError(t, err) +// blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) +// require.NoError(t, err) +// +// traces := &storageMock.TraceIndexer{} +// traces. +// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). +// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { +// assert.Equal(t, transaction.Hash(), txID) +// return nil +// }) +// +// engine := NewEventIngestionEngine( +// subscriber, +// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), +// store, +// registerStore, +// blocks, +// receipts, +// transactions, +// traces, +// models.NewPublisher[*models.Block](), +// models.NewPublisher[[]*gethTypes.Log](), +// zerolog.Nop(), +// metrics.NopCollector, +// defaultReplayerConfig(), +// ) +// +// done := make(chan struct{}) +// go func() { +// err := engine.Run(context.Background()) +// assert.NoError(t, err) +// close(done) +// }() +// +// blocks. +// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). +// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { +// assert.Equal(t, block, storeBlock) +// assert.Equal(t, blockID, id) +// assert.Equal(t, nextHeight, h) +// return nil +// }). +// Once() +// +// transactions. +// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). +// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { +// assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal +// return nil +// }). +// Once() +// +// receipts. +// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). +// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { +// assert.Len(t, receipts, 1) +// rcp := receipts[0] +// +// assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) +// assert.Len(t, rcp.Logs, len(result.Logs)) +// assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) +// return nil +// }). +// Once() +// +// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ +// Events: []flow.Event{{ +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }, { +// Type: string(txEvent.Etype), +// Value: txCdc, +// }}, +// Height: nextHeight, +// BlockID: blockID, +// }) +// +// close(eventsChan) +// <-done +// }) +// +// t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { +// receipts := &storageMock.ReceiptIndexer{} +// transactions := &storageMock.TransactionIndexer{} +// latestHeight := uint64(10) +// nextHeight := latestHeight + 1 +// +// store, registerStore := setupStore(t) +// +// blocks := &storageMock.BlockIndexer{} +// blocks. +// On("LatestCadenceHeight"). +// Return(func() (uint64, error) { +// return latestHeight, nil +// }). +// On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). +// Return(func(h uint64) error { return nil }) +// +// eventsChan := make(chan models.BlockEvents) +// subscriber := &mocks.EventSubscriber{} +// subscriber. +// On("Subscribe", mock.Anything). +// Return(func(ctx context.Context) <-chan models.BlockEvents { +// return eventsChan +// }) +// +// txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) +// require.NoError(t, err) +// blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) +// require.NoError(t, err) +// +// traces := &storageMock.TraceIndexer{} +// traces. +// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). +// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { +// assert.Equal(t, transaction.Hash(), txID) +// return nil +// }) +// +// engine := NewEventIngestionEngine( +// subscriber, +// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), +// store, +// registerStore, +// blocks, +// receipts, +// transactions, +// traces, +// models.NewPublisher[*models.Block](), +// models.NewPublisher[[]*gethTypes.Log](), +// zerolog.Nop(), +// metrics.NopCollector, +// defaultReplayerConfig(), +// ) +// +// done := make(chan struct{}) +// go func() { +// err := engine.Run(context.Background()) +// assert.NoError(t, err) +// close(done) +// }() +// +// blocksFirst := false // flag indicating we stored block first +// blocks. +// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). +// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { +// blocksFirst = true +// return nil +// }). +// Once() +// +// transactions. +// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). +// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { +// require.True(t, blocksFirst) +// return nil +// }). +// Once() +// +// receipts. +// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). +// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { +// require.True(t, blocksFirst) +// return nil +// }). +// Once() +// +// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ +// Events: []flow.Event{ +// // first transaction +// { +// Type: string(txEvent.Etype), +// Value: txCdc, +// }, +// // and then block (out-of-order) +// { +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }, +// }, +// Height: nextHeight, +// }) +// +// close(eventsChan) +// <-done +// }) +// +// t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { +// receipts := &storageMock.ReceiptIndexer{} +// transactions := &storageMock.TransactionIndexer{} +// latestCadenceHeight := uint64(0) +// +// store, registerStore := setupStore(t) +// +// blocks := &storageMock.BlockIndexer{} +// blocks. +// On("LatestCadenceHeight"). +// Return(func() (uint64, error) { +// return latestCadenceHeight, nil +// }). +// Once() // make sure this isn't called multiple times +// +// traces := &storageMock.TraceIndexer{} +// +// eventsChan := make(chan models.BlockEvents) +// subscriber := &mocks.EventSubscriber{} +// subscriber. +// On("Subscribe", mock.Anything). +// Return(func(ctx context.Context) <-chan models.BlockEvents { +// return eventsChan +// }). +// Once() +// +// engine := NewEventIngestionEngine( +// subscriber, +// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), +// store, +// registerStore, +// blocks, +// receipts, +// transactions, +// traces, +// models.NewPublisher[*models.Block](), +// models.NewPublisher[[]*gethTypes.Log](), +// zerolog.Nop(), +// metrics.NopCollector, +// defaultReplayerConfig(), +// ) +// +// done := make(chan struct{}) +// go func() { +// err := engine.Run(context.Background()) +// assert.NoError(t, err) +// close(done) +// }() +// +// evmHeight := uint64(0) +// events := make([]flow.Event, 0) +// blockIndexedFirst := false +// txsStored := 0 +// eventCount := 5 +// txHashes := make([]gethCommon.Hash, eventCount) +// +// for i := 0; i < eventCount; i++ { +// txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) +// txHashes[i] = res.TxHash +// require.NoError(t, err) +// +// // add a single transaction for each block +// transactions. +// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). +// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { +// assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal +// require.True(t, blockIndexedFirst) +// txsStored++ +// return nil +// }). +// Once() +// +// receipts. +// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). +// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). +// Once() +// +// traces. +// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). +// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { +// assert.Equal(t, transaction.Hash(), txID) +// return nil +// }) +// +// events = append(events, flow.Event{ +// Type: string(txEvent.Etype), +// Value: txCdc, +// }) +// } +// +// blocksStored := 0 +// blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) +// require.NoError(t, err) +// +// blocks. +// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). +// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { +// assert.Equal(t, block, storeBlock) +// assert.Equal(t, evmHeight, block.Height) +// assert.Equal(t, latestCadenceHeight+1, h) +// blockIndexedFirst = true +// blocksStored++ +// return nil +// }). +// Once() +// +// events = append(events, flow.Event{ +// Type: string(blockEvent.Etype), +// Value: blockCdc, +// }) +// +// // this messes up order of events to test if we still process events in-order +// // it will make transaction event first and then block event +// events[0], events[1] = events[1], events[0] +// // and it will make the first block be swapped with second block out-of-order +// events[1], events[2] = events[2], events[1] +// +// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ +// Events: events, +// Height: latestCadenceHeight + 1, +// }) +// +// close(eventsChan) +// <-done +// assert.Equal(t, eventCount, txsStored) +// assert.Equal(t, 1, blocksStored) +// }) +//} func newBlock(height uint64, txHashes []gethCommon.Hash) (cadence.Event, *models.Block, *events.Event, error) { gethBlock := types.NewBlock( @@ -626,41 +640,41 @@ func newTransaction(height uint64) (cadence.Event, *events.Event, models.Transac return cdcEv, ev, models.TransactionCall{Transaction: tx}, res, err } -func defaultReplayerConfig() replayer.Config { - return replayer.Config{ - ChainID: flowGo.Emulator, - RootAddr: evm.StorageAccountAddress(flowGo.Emulator), - CallTracerCollector: replayer.NopTracer, - ValidateResults: false, - } -} - -func setupStore(t *testing.T) (*pebble.Storage, *pebble.RegisterStorage) { - db, err := pebble.OpenDB(t.TempDir()) - require.NoError(t, err) - store := pebble.New(db, zerolog.Nop()) - - storageAddress := evm.StorageAccountAddress(flowGo.Emulator) - registerStore := pebble.NewRegisterStorage(store, storageAddress) - snapshot, err := registerStore.GetSnapshotAt(0) - require.NoError(t, err) - delta := storage.NewRegisterDelta(snapshot) - accountStatus := environment.NewAccountStatus() - err = delta.SetValue( - storageAddress[:], - []byte(flowGo.AccountStatusKey), - accountStatus.ToBytes(), - ) - require.NoError(t, err) - - batch := store.NewBatch() - defer func() { - require.NoError(t, batch.Close()) - }() - err = registerStore.Store(delta.GetUpdates(), 0, batch) - require.NoError(t, err) - err = batch.Commit(pebbleDB.Sync) - require.NoError(t, err) - - return store, registerStore -} +//func defaultReplayerConfig() replayer.Config { +// return replayer.Config{ +// ChainID: flowGo.Emulator, +// RootAddr: evm.StorageAccountAddress(flowGo.Emulator), +// CallTracerCollector: replayer.NopTracer, +// ValidateResults: false, +// } +//} +// +//func setupStore(t *testing.T) (*pebble.Storage, *pebble.RegisterStorage) { +// db, err := pebble.OpenDB(t.TempDir()) +// require.NoError(t, err) +// store := pebble.New(db, zerolog.Nop()) +// +// storageAddress := evm.StorageAccountAddress(flowGo.Emulator) +// registerStore := pebble.NewRegisterStorage(store, storageAddress) +// snapshot, err := registerStore.GetSnapshotAt(0) +// require.NoError(t, err) +// delta := storage.NewRegisterDelta(snapshot) +// accountStatus := environment.NewAccountStatus() +// err = delta.SetValue( +// storageAddress[:], +// []byte(flowGo.AccountStatusKey), +// accountStatus.ToBytes(), +// ) +// require.NoError(t, err) +// +// batch := store.NewBatch() +// defer func() { +// require.NoError(t, batch.Close()) +// }() +// err = registerStore.Store(delta.GetUpdates(), 0, batch) +// require.NoError(t, err) +// err = batch.Commit(pebbleDB.Sync) +// require.NoError(t, err) +// +// return store, registerStore +//} diff --git a/services/signer/signer.go b/services/signer/signer.go new file mode 100644 index 000000000..8bcbefd87 --- /dev/null +++ b/services/signer/signer.go @@ -0,0 +1,92 @@ +package signer + +import ( + "fmt" + + "github.com/onflow/flow-evm-gateway/config" + "github.com/onflow/flow-evm-gateway/services/requester" + "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/rs/zerolog" +) + +type Signer struct { + crypto.Signer + + log zerolog.Logger + config config.Config + + startupCompleted chan struct{} + closeKMSClient func() +} + +var _ component.Component = (*Signer)(nil) +var _ crypto.Signer = (*Signer)(nil) + +func NewSigner(log zerolog.Logger, config config.Config) *Signer { + return &Signer{ + log: log, + config: config, + startupCompleted: make(chan struct{}), + } +} + +func (s *Signer) Start(ctx irrecoverable.SignalerContext) { + cfg := s.config + defer close(s.startupCompleted) + + var err error + switch { + case cfg.COAKey != nil: + s.Signer, err = crypto.NewInMemorySigner(cfg.COAKey, crypto.SHA3_256) + case cfg.COAKeys != nil: + s.Signer, err = requester.NewKeyRotationSigner(cfg.COAKeys, crypto.SHA3_256) + case len(cfg.COACloudKMSKeys) > 0: + var signer *requester.KMSKeyRotationSigner + signer, err = requester.NewKMSKeyRotationSigner( + ctx, + cfg.COACloudKMSKeys, + s.log, + ) + s.closeKMSClient = func() { + // TODO(JanezP): this should definitely be a closer. Open a PR in the sdk + // signer.Close() + } + s.Signer = signer + default: + ctx.Throw(fmt.Errorf("must provide either single COA / keylist of COA keys / COA cloud KMS keys")) + return + } + if err != nil { + ctx.Throw(fmt.Errorf("failed to create a COA signer: %w", err)) + return + } +} + +func (s *Signer) Ready() <-chan struct{} { + ready := make(chan struct{}) + + go func() { + <-s.startupCompleted + close(ready) + }() + + return ready +} + +func (s *Signer) Done() <-chan struct{} { + done := make(chan struct{}) + + go func() { + <-s.startupCompleted + + if s.closeKMSClient != nil { + s.closeKMSClient() + } + + close(done) + }() + + return done +} From 78ccaf348e67c338166a42e073842b72188d103a Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Mon, 2 Dec 2024 16:11:43 +0100 Subject: [PATCH 02/12] fix stream tests --- bootstrap/bootstrap.go | 1135 +++++++++++++++++++++------------------- bootstrap/builder.go | 609 --------------------- go.mod | 6 +- go.sum | 4 +- models/stream_test.go | 112 ++-- tests/go.mod | 83 ++- tests/go.sum | 230 +++++++- 7 files changed, 979 insertions(+), 1200 deletions(-) delete mode 100644 bootstrap/builder.go diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index aa529df91..d389fb9c4 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -1,25 +1,46 @@ package bootstrap import ( + "context" + "errors" "fmt" + "math" + "time" "github.com/onflow/flow-evm-gateway/models" "github.com/onflow/flow-evm-gateway/services/requester" "github.com/onflow/flow-evm-gateway/storage" - "github.com/onflow/flow-evm-gateway/storage/pebble" "github.com/onflow/flow-go-sdk/access" "github.com/onflow/flow-go-sdk/access/grpc" - gethTypes "github.com/onflow/go-ethereum/core/types" - - "github.com/rs/zerolog" grpcOpts "google.golang.org/grpc" + "github.com/cockroachdb/pebble" + "github.com/hashicorp/go-multierror" + "github.com/onflow/flow-evm-gateway/api" "github.com/onflow/flow-evm-gateway/config" + "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/cmd" + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm" + flowGo "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/metrics" + gethTypes "github.com/onflow/go-ethereum/core/types" + "github.com/rs/zerolog" + "github.com/sethvargo/go-limiter/memorystore" + + metrics2 "github.com/onflow/flow-evm-gateway/metrics" + errs "github.com/onflow/flow-evm-gateway/models/errors" + "github.com/onflow/flow-evm-gateway/services/ingestion" + "github.com/onflow/flow-evm-gateway/services/replayer" + "github.com/onflow/flow-evm-gateway/services/signer" + pebble2 "github.com/onflow/flow-evm-gateway/storage/pebble" ) type Storages struct { - Storage *pebble.Storage - Registers *pebble.RegisterStorage + Storage *pebble2.Storage + Registers *pebble2.RegisterStorage Blocks storage.BlockIndexer Transactions storage.TransactionIndexer Receipts storage.ReceiptIndexer @@ -32,389 +53,586 @@ type Publishers struct { Logs *models.Publisher[[]*gethTypes.Log] } -// type Bootstrap struct { -// logger zerolog.Logger -// config config.Config -// client *requester.CrossSporkClient -// storages *Storages -// publishers *Publishers -// collector metrics.Collector -// server *api.Server -// metrics *flowMetrics.Server -// events *ingestion.Engine -// profiler *api.ProfileServer -// db *pebbleDB.DB -// } -// -// func New(config config.Config) (*Bootstrap, error) { -// logger := zerolog.New(config.LogWriter). -// With().Timestamp().Str("version", api.Version). -// Logger().Level(config.LogLevel) -// -// client, err := setupCrossSporkClient(config, logger) -// if err != nil { -// return nil, err -// } -// -// db, storages, err := setupStorage(config, client, logger) -// if err != nil { -// return nil, err -// } -// -// return &Bootstrap{ -// publishers: &Publishers{ -// Block: models.NewPublisher[*models.Block](), -// Transaction: models.NewPublisher[*gethTypes.Transaction](), -// Logs: models.NewPublisher[[]*gethTypes.Log](), -// }, -// db: db, -// storages: storages, -// logger: logger, -// config: config, -// client: client, -// collector: metrics.NewCollector(logger), -// }, nil -// } -// -// func (b *Bootstrap) StartEventIngestion(ctx context.Context) error { -// l := b.logger.With().Str("component", "bootstrap-ingestion").Logger() -// l.Info().Msg("bootstrap starting event ingestion") -// -// // get latest cadence block from the network and the database -// latestCadenceBlock, err := b.client.GetLatestBlock(context.Background(), true) -// if err != nil { -// return fmt.Errorf("failed to get latest cadence block: %w", err) -// } -// -// latestCadenceHeight, err := b.storages.Blocks.LatestCadenceHeight() -// if err != nil { -// return err -// } -// -// // make sure the provided block to start the indexing can be loaded -// _, err = b.client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) -// if err != nil { -// return fmt.Errorf( -// "failed to get provided cadence height %d: %w", -// latestCadenceHeight, -// err, -// ) -// } -// -// l.Info(). -// Uint64("start-cadence-height", latestCadenceHeight). -// Uint64("latest-cadence-height", latestCadenceBlock.Height). -// Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). -// Msg("indexing cadence height information") -// -// chainID := b.config.FlowNetworkID -// -// // create event subscriber -// subscriber := ingestion.NewRPCEventSubscriber( -// b.logger, -// b.client, -// chainID, -// latestCadenceHeight, -// ) -// -// callTracerCollector, err := replayer.NewCallTracerCollector(b.logger) -// if err != nil { -// return err -// } -// blocksProvider := replayer.NewBlocksProvider( -// b.storages.Blocks, -// chainID, -// callTracerCollector.TxTracer(), -// ) -// replayerConfig := replayer.Config{ -// ChainID: chainID, -// RootAddr: evm.StorageAccountAddress(chainID), -// CallTracerCollector: callTracerCollector, -// ValidateResults: true, -// } -// -// // initialize event ingestion engine -// b.events = ingestion.NewEventIngestionEngine( -// subscriber, -// blocksProvider, -// b.storages.Storage, -// b.storages.Registers, -// b.storages.Blocks, -// b.storages.Receipts, -// b.storages.Transactions, -// b.storages.Traces, -// b.publishers.Block, -// b.publishers.Logs, -// b.logger, -// b.collector, -// replayerConfig, -// ) -// -// StartEngine(ctx, b.events, l) -// return nil -// } -// -// func (b *Bootstrap) StopEventIngestion() { -// if b.events == nil { -// return -// } -// b.logger.Warn().Msg("stopping event ingestion engine") -// b.events.Stop() -// } -// -// func (b *Bootstrap) StartAPIServer(ctx context.Context) error { -// b.logger.Info().Msg("bootstrap starting Metrics server") -// -// b.server = api.NewServer(b.logger, b.collector, b.config) -// -// // create the signer based on either a single coa key being provided and using a simple in-memory -// // signer, or multiple keys being provided and using signer with key-rotation mechanism.bootstrap/bootstrap.go -// var signer crypto.Signer -// var err error -// switch { -// case b.config.COAKey != nil: -// signer, err = crypto.NewInMemorySigner(b.config.COAKey, crypto.SHA3_256) -// case b.config.COAKeys != nil: -// signer, err = requester.NewKeyRotationSigner(b.config.COAKeys, crypto.SHA3_256) -// case len(b.config.COACloudKMSKeys) > 0: -// signer, err = requester.NewKMSKeyRotationSigner( -// ctx, -// b.config.COACloudKMSKeys, -// b.logger, -// ) -// default: -// return fmt.Errorf("must provide either single COA / keylist of COA keys / COA cloud KMS keys") -// } -// if err != nil { -// return fmt.Errorf("failed to create a COA signer: %w", err) -// } -// -// // create transaction pool -// txPool := requester.NewTxPool( -// b.client, -// b.publishers.Transaction, -// b.logger, -// ) -// -// blocksProvider := replayer.NewBlocksProvider( -// b.storages.Blocks, -// b.config.FlowNetworkID, -// nil, -// ) -// -// evm, err := requester.NewEVM( -// b.storages.Registers, -// blocksProvider, -// b.client, -// b.config, -// signer, -// b.logger, -// b.storages.Blocks, -// txPool, -// b.collector, -// ) -// if err != nil { -// return fmt.Errorf("failed to create EVM requester: %w", err) -// } -// -// // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval -// // if no limit is defined we specify max value, effectively disabling rate-limiting -// rateLimit := b.config.RateLimit -// if rateLimit == 0 { -// b.logger.Warn().Msg("no rate-limiting is set") -// rateLimit = math.MaxInt -// } -// ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) -// if err != nil { -// return fmt.Errorf("failed to create rate limiter: %w", err) -// } -// -// blockchainAPI, err := api.NewBlockChainAPI( -// b.logger, -// b.config, -// evm, -// b.storages.Blocks, -// b.storages.Transactions, -// b.storages.Receipts, -// ratelimiter, -// b.collector, -// ) -// if err != nil { -// return err -// } -// -// streamAPI := api.NewStreamAPI( -// b.logger, -// b.config, -// b.storages.Blocks, -// b.storages.Transactions, -// b.storages.Receipts, -// b.publishers.Block, -// b.publishers.Transaction, -// b.publishers.Logs, -// ) -// -// pullAPI := api.NewPullAPI( -// b.logger, -// b.config, -// b.storages.Blocks, -// b.storages.Transactions, -// b.storages.Receipts, -// ratelimiter, -// ) -// -// debugAPI := api.NewDebugAPI( -// b.storages.Registers, -// b.storages.Traces, -// b.storages.Blocks, -// b.storages.Transactions, -// b.storages.Receipts, -// b.client, -// b.config, -// b.logger, -// b.collector, -// ) -// -// var walletAPI *api.WalletAPI -// if b.config.WalletEnabled { -// walletAPI = api.NewWalletAPI(b.config, blockchainAPI) -// } -// -// supportedAPIs := api.SupportedAPIs( -// blockchainAPI, -// streamAPI, -// pullAPI, -// debugAPI, -// walletAPI, -// b.config, -// ) -// -// if err := b.server.EnableRPC(supportedAPIs); err != nil { -// return err -// } -// -// if b.config.WSEnabled { -// if err := b.server.EnableWS(supportedAPIs); err != nil { -// return err -// } -// } -// -// if err := b.server.SetListenAddr(b.config.RPCHost, b.config.RPCPort); err != nil { -// return err -// } -// -// if err := b.server.Start(); err != nil { -// return err -// } -// -// b.logger.Info().Msgf("API server started: %s", b.server.ListenAddr()) -// return nil -// } -// -// func (b *Bootstrap) StopAPIServer() { -// if b.server == nil { -// return -// } -// b.logger.Warn().Msg("shutting down API server") -// b.server.Stop() -// } -// -// func (b *Bootstrap) StartMetricsServer(ctx context.Context) error { -// b.logger.Info().Msg("bootstrap starting Metrics server") -// -// b.metrics = flowMetrics.NewServer(b.logger, uint(b.config.MetricsPort)) -// err := util.WaitClosed(ctx, b.metrics.Ready()) -// if err != nil { -// return fmt.Errorf("failed to start Metrics server: %w", err) -// } -// -// return nil -// } -// -// func (b *Bootstrap) StopMetricsServer() { -// if b.metrics == nil { -// return -// } -// b.logger.Warn().Msg("shutting down Metrics server") -// <-b.metrics.Done() -// } -// -// func (b *Bootstrap) StartProfilerServer(_ context.Context) error { -// if !b.config.ProfilerEnabled { -// return nil -// } -// b.logger.Info().Msg("bootstrap starting profiler server") -// -// b.profiler = api.NewProfileServer(b.logger, b.config.ProfilerHost, b.config.ProfilerPort) -// -// b.profiler.Start() -// -// return nil -// } -// -// func (b *Bootstrap) StopProfilerServer() { -// if b.profiler == nil { -// return -// } -// -// b.logger.Warn().Msg("shutting down profiler server") -// -// err := b.profiler.Stop() -// if err != nil { -// if errors.Is(err, context.DeadlineExceeded) { -// b.logger.Warn().Msg("Profiler server graceful shutdown timed out") -// b.profiler.Close() -// } else { -// b.logger.Err(err).Msg("Profiler server graceful shutdown failed") -// } -// } -// } -// -// func (b *Bootstrap) StopDB() { -// if b.db == nil { -// return -// } -// err := b.db.Close() -// if err != nil { -// b.logger.Err(err).Msg("PebbleDB graceful shutdown failed") -// } -// } -// -// func (b *Bootstrap) StopClient() { -// if b.client == nil { -// return -// } -// err := b.client.Close() -// if err != nil { -// b.logger.Err(err).Msg("CrossSporkClient graceful shutdown failed") -// } -// } -// -// // StartEngine starts provided engine and panics if there are startup errors. -// func StartEngine( -// -// ctx context.Context, -// engine models.Engine, -// logger zerolog.Logger, -// -// ) { -// l := logger.With().Type("engine", engine).Logger() -// -// l.Info().Msg("starting engine") -// start := time.Now() -// go func() { -// err := engine.Run(ctx) -// if err != nil { -// l.Fatal().Err(err).Msg("engine failed to run") -// } -// }() -// -// <-engine.Ready() -// l.Info(). -// Dur("duration", time.Since(start)). -// Msg("engine started successfully") -// } -// +type EVMGatewayNodeImp struct { + cmd.NodeImp + config.Config +} + +// NewNode returns a new node instance +func NewNode( + component component.Component, + cfg config.Config, + logger zerolog.Logger, + cleanup func() error, + handleFatal func(error), +) *EVMGatewayNodeImp { + return &EVMGatewayNodeImp{ + Config: cfg, + NodeImp: cmd.NewBaseNode( + component, + logger.With(). + Str("node_role", "EVM Gateway"). + Logger(), + cleanup, + handleFatal, + ), + } +} + +type EVMGatewayNodeBuilder struct { + config.Config + Logger zerolog.Logger + componentBuilder component.ComponentManagerBuilder + components []cmd.NamedComponentFunc[config.Config] + postShutdownFns []func() error + modules []namedModuleFunc + + Metrics metrics2.Collector + DB *pebble.DB + Client *requester.CrossSporkClient + Storages *Storages + // Signer is used for signing flow transactions + Signer crypto.Signer + Publishers *Publishers +} + +func (fnb *EVMGatewayNodeBuilder) Build() (cmd.Node, error) { + // Run the prestart initialization. This includes anything that should be done before + // starting the components. + if err := fnb.onStart(); err != nil { + return nil, err + } + + return NewNode( + fnb.componentBuilder.Build(), + fnb.Config, + fnb.Logger, + fnb.postShutdown, + fnb.handleFatal, + ), nil +} + +func (fnb *EVMGatewayNodeBuilder) onStart() error { + + if err := fnb.initDB(); err != nil { + return err + } + + if err := fnb.initMetrics(); err != nil { + return err + } + + if err := fnb.initClient(); err != nil { + return err + } + + if err := fnb.initStorage(); err != nil { + return err + } + + // run all modules + if err := fnb.handleModules(); err != nil { + return fmt.Errorf("could not handle modules: %w", err) + } + + // run all components + return fnb.handleComponents() +} + +func (fnb *EVMGatewayNodeBuilder) initDB() error { + pebbleDB, err := pebble2.OpenDB(fnb.DatabaseDir) + if err != nil { + return fmt.Errorf("failed to open db for dir: %s, with: %w", fnb.DatabaseDir, err) + } + + fnb.DB = pebbleDB + + fnb.ShutdownFunc(func() error { + if err := fnb.DB.Close(); err != nil { + return fmt.Errorf("error closing pebble database: %w", err) + } + return nil + }) + + return err +} + +func (fnb *EVMGatewayNodeBuilder) Component(name string, f cmd.ReadyDoneFactory[config.Config]) *EVMGatewayNodeBuilder { + fnb.components = append(fnb.components, cmd.NamedComponentFunc[config.Config]{ + FN: f, + Name: name, + }) + return fnb +} + +// postShutdown is called by the node before exiting +// put any cleanup code here that should be run after all components have stopped +func (fnb *EVMGatewayNodeBuilder) postShutdown() error { + var errs *multierror.Error + + for _, fn := range fnb.postShutdownFns { + err := fn() + if err != nil { + errs = multierror.Append(errs, err) + } + } + fnb.Logger.Info().Msg("database has been closed") + return errs.ErrorOrNil() +} + +// handleFatal handles irrecoverable errors by logging them and exiting the process. +func (fnb *EVMGatewayNodeBuilder) handleFatal(err error) { + fnb.Logger.Fatal().Err(err).Msg("unhandled irrecoverable error") +} + +func NewEVMGatewayNodeBuilder( + config config.Config, +) *EVMGatewayNodeBuilder { + + logger := zerolog.New(config.LogWriter). + With().Timestamp().Str("version", api.Version). + Logger().Level(config.LogLevel) + + return &EVMGatewayNodeBuilder{ + Logger: logger, + Config: config, + componentBuilder: component.NewComponentManagerBuilder(), + } +} + +func (fnb *EVMGatewayNodeBuilder) Initialize() error { + fnb.PrintBuildDetails() + + fnb.EnqueueMetricsServerInit() + + return nil +} + +func (fnb *EVMGatewayNodeBuilder) LoadComponentsAndModules() { + fnb.initPublishers() + + fnb.Component("Transaction Signer", fnb.initSigner) + fnb.Component("API Server", fnb.apiServerComponent) + fnb.Component("Event Ingestion Engine", fnb.eventIngestionEngineComponent) + fnb.Component("Metrics Server", fnb.metricsServerComponent) + fnb.Component("Profiler Server", fnb.profilerServerComponent) +} + +func (fnb *EVMGatewayNodeBuilder) metricsServerComponent(config config.Config) (module.ReadyDoneAware, error) { + server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) profilerServerComponent(config config.Config) (module.ReadyDoneAware, error) { + server := api.NewProfileServer(fnb.Logger, config.ProfilerHost, config.ProfilerPort) + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module.ReadyDoneAware, error) { + log := fnb.Logger + + log.Info().Msg("bootstrap starting Metrics server") + + server := api.NewServer(log, fnb.Metrics, cfg) + + // create transaction pool + txPool := requester.NewTxPool( + fnb.Client, + fnb.Publishers.Transaction, + log, + ) + + blocksProvider := replayer.NewBlocksProvider( + fnb.Storages.Blocks, + cfg.FlowNetworkID, + nil, + ) + + evm, err := requester.NewEVM( + fnb.Storages.Registers, + blocksProvider, + fnb.Client, + cfg, + fnb.Signer, + log, + fnb.Storages.Blocks, + txPool, + fnb.Metrics, + ) + if err != nil { + return nil, fmt.Errorf("failed to create EVM requester: %w", err) + } + + // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval + // if no limit is defined we specify max value, effectively disabling rate-limiting + rateLimit := cfg.RateLimit + if rateLimit == 0 { + log.Warn().Msg("no rate-limiting is set") + rateLimit = math.MaxInt + } + ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) + if err != nil { + return nil, fmt.Errorf("failed to create rate limiter: %w", err) + } + + // get the height from which the indexing resumed since the last restart, + // this is needed for the `eth_syncing` endpoint. + indexingResumedHeight, err := fnb.Storages.Blocks.LatestEVMHeight() + if err != nil { + return nil, fmt.Errorf("failed to retrieve the indexing resumed height: %w", err) + } + + blockchainAPI := api.NewBlockChainAPI( + log, + cfg, + evm, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + ratelimiter, + fnb.Metrics, + indexingResumedHeight, + ) + + streamAPI := api.NewStreamAPI( + log, + cfg, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + fnb.Publishers.Block, + fnb.Publishers.Transaction, + fnb.Publishers.Logs, + ) + + pullAPI := api.NewPullAPI( + log, + cfg, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + ratelimiter, + ) + + debugAPI := api.NewDebugAPI( + fnb.Storages.Registers, + fnb.Storages.Traces, + fnb.Storages.Blocks, + fnb.Storages.Transactions, + fnb.Storages.Receipts, + fnb.Client, + cfg, + log, + fnb.Metrics, + ) + + var walletAPI *api.WalletAPI + if cfg.WalletEnabled { + walletAPI = api.NewWalletAPI(cfg, blockchainAPI) + } + + supportedAPIs := api.SupportedAPIs( + blockchainAPI, + streamAPI, + pullAPI, + debugAPI, + walletAPI, + cfg, + ) + + if err := server.EnableRPC(supportedAPIs); err != nil { + return nil, err + } + + if cfg.WSEnabled { + if err := server.EnableWS(supportedAPIs); err != nil { + return nil, err + } + } + + if err := server.SetListenAddr(cfg.RPCHost, cfg.RPCPort); err != nil { + return nil, err + } + + return server, nil +} + +func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Config) (module.ReadyDoneAware, error) { + l := fnb.Logger.With().Str("component", "bootstrap-ingestion").Logger() + l.Info().Msg("bootstrap starting event ingestion") + + // get latest cadence block from the network and the database + latestCadenceBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) + if err != nil { + return nil, fmt.Errorf("failed to get latest cadence block: %w", err) + } + + latestCadenceHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() + if err != nil { + return nil, err + } + + // make sure the provided block to start the indexing can be loaded + _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) + if err != nil { + return nil, fmt.Errorf( + "failed to get provided cadence height %d: %w", + latestCadenceHeight, + err, + ) + } + + l.Info(). + Uint64("start-cadence-height", latestCadenceHeight). + Uint64("latest-cadence-height", latestCadenceBlock.Height). + Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). + Msg("indexing cadence height information") + + chainID := cfg.FlowNetworkID + + // create event subscriber + subscriber := ingestion.NewRPCEventSubscriber( + fnb.Logger, + fnb.Client, + chainID, + latestCadenceHeight, + ) + + callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) + if err != nil { + return nil, err + } + blocksProvider := replayer.NewBlocksProvider( + fnb.Storages.Blocks, + chainID, + callTracerCollector.TxTracer(), + ) + replayerConfig := replayer.Config{ + ChainID: chainID, + RootAddr: evm.StorageAccountAddress(chainID), + CallTracerCollector: callTracerCollector, + ValidateResults: true, + } + + // initialize event ingestion engine + events := ingestion.NewEventIngestionEngine( + subscriber, + blocksProvider, + fnb.Storages.Storage, + fnb.Storages.Registers, + fnb.Storages.Blocks, + fnb.Storages.Receipts, + fnb.Storages.Transactions, + fnb.Storages.Traces, + fnb.Publishers.Block, + fnb.Publishers.Logs, + fnb.Logger, + fnb.Metrics, + replayerConfig, + ) + + return events, nil +} + +func (fnb *EVMGatewayNodeBuilder) PrintBuildDetails() { + fnb.Logger.Info().Str("version", api.Version).Msg("build details") +} + +// ShutdownFunc adds a callback function that is called after all components have exited. +func (fnb *EVMGatewayNodeBuilder) ShutdownFunc(fn func() error) *EVMGatewayNodeBuilder { + fnb.postShutdownFns = append(fnb.postShutdownFns, fn) + return fnb +} + +func (fnb *EVMGatewayNodeBuilder) EnqueueMetricsServerInit() { + fnb.Component("Metrics server", func(config config.Config) (module.ReadyDoneAware, error) { + server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) + return server, nil + }) +} + +func (fnb *EVMGatewayNodeBuilder) initMetrics() error { + fnb.Metrics = metrics2.NewCollector(fnb.Logger) + return nil +} + +func (fnb *EVMGatewayNodeBuilder) initStorage() error { + logger := fnb.Logger + cfg := fnb.Config + + store := pebble2.New(fnb.DB, fnb.Logger) + + blocks := pebble2.NewBlocks(store, cfg.FlowNetworkID) + storageAddress := evm.StorageAccountAddress(cfg.FlowNetworkID) + registerStore := pebble2.NewRegisterStorage(store, storageAddress) + + // hard set the start cadence height, this is used when force reindexing + if cfg.ForceStartCadenceHeight != 0 { + logger.Warn().Uint64("height", cfg.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") + if err := blocks.SetLatestCadenceHeight(cfg.ForceStartCadenceHeight, nil); err != nil { + return err + } + } + + // if database is not initialized require init height + if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { + // TODO(JanezP): move this to a separate function + err = func() (innerErr error) { + batch := store.NewBatch() + defer func(batch *pebble.Batch) { + innerErr = batch.Close() + }(batch) + + cadenceHeight := cfg.InitCadenceHeight + evmBlokcHeight := uint64(0) + cadenceBlock, err := fnb.Client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) + if err != nil { + return fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) + } + + snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) + if err != nil { + return fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) + } + + delta := storage.NewRegisterDelta(snapshot) + accountStatus := environment.NewAccountStatus() + err = delta.SetValue( + storageAddress[:], + []byte(flowGo.AccountStatusKey), + accountStatus.ToBytes(), + ) + if err != nil { + return fmt.Errorf("could not set account status: %w", err) + } + + err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) + if err != nil { + return fmt.Errorf("could not store register updates: %w", err) + } + + if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { + return fmt.Errorf( + "failed to init the database for block height: %d and ID: %s, with : %w", + cadenceHeight, + cadenceBlock.ID, + err, + ) + } + + err = batch.Commit(pebble.Sync) + if err != nil { + return fmt.Errorf("could not commit register updates: %w", err) + } + + logger.Info(). + Stringer("fvm_address_for_evm_storage_account", storageAddress). + Msgf("database initialized with cadence height: %d", cadenceHeight) + + return nil + }() + + if err != nil { + return fmt.Errorf("failed to init the database: %w", err) + } + } + //else { + // // TODO(JanezP): verify storage account owner is correct + //} + + fnb.Storages = &Storages{ + Storage: store, + Blocks: blocks, + Registers: registerStore, + Transactions: pebble2.NewTransactions(store), + Receipts: pebble2.NewReceipts(store), + Traces: pebble2.NewTraces(store), + } + + return nil +} +func (fnb *EVMGatewayNodeBuilder) initSigner(config config.Config) (module.ReadyDoneAware, error) { + sig := signer.NewSigner(fnb.Logger, config) + fnb.Signer = sig + return sig, nil +} + +func (fnb *EVMGatewayNodeBuilder) initPublishers() { + fnb.Publishers = &Publishers{} + + fnb.Component("Block Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[*models.Block](fnb.Logger) + fnb.Publishers.Block = p + return p, nil + }) + fnb.Component("Transaction Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[*gethTypes.Transaction](fnb.Logger) + fnb.Publishers.Transaction = p + return p, nil + }) + fnb.Component("Logs Publisher", func(config config.Config) (module.ReadyDoneAware, error) { + p := models.NewPublisher[[]*gethTypes.Log](fnb.Logger) + fnb.Publishers.Logs = p + return p, nil + }) +} + +func (fnb *EVMGatewayNodeBuilder) initClient() error { + logger := fnb.Logger + cfg := fnb.Config + + client, err := setupCrossSporkClient(cfg, logger) + if err != nil { + return fmt.Errorf("failed to create cross-spork client: %w", err) + } + + fnb.Client = client + + fnb.ShutdownFunc(func() error { + if err := fnb.Client.Close(); err != nil { + return fmt.Errorf("error closing cross-spork client: %w", err) + } + return nil + }) + + return nil +} + +type BuilderFunc func(nodeConfig config.Config) error + +type namedModuleFunc struct { + fn BuilderFunc + name string +} + +// handleModules initializes the given module. +func (fnb *EVMGatewayNodeBuilder) handleModule(v namedModuleFunc) error { + fnb.Logger.Info().Str("module", v.name).Msg("module initialization started") + err := v.fn(fnb.Config) + if err != nil { + return fmt.Errorf("module %s initialization failed: %w", v.name, err) + } + + fnb.Logger.Info().Str("module", v.name).Msg("module initialization complete") + return nil +} + +// handleModules initializes all modules that have been enqueued on this node builder. +func (fnb *EVMGatewayNodeBuilder) handleModules() error { + for _, f := range fnb.modules { + if err := fnb.handleModule(f); err != nil { + return err + } + } + + return nil +} + +func (fnb *EVMGatewayNodeBuilder) handleComponents() error { + cmd.AddWorkersFromComponents(fnb.Logger, fnb.Config, fnb.componentBuilder, fnb.components) + return nil +} + // setupCrossSporkClient sets up a cross-spork AN client. func setupCrossSporkClient(config config.Config, logger zerolog.Logger) (*requester.CrossSporkClient, error) { // create access client with cross-spork capabilities @@ -454,142 +672,3 @@ func setupCrossSporkClient(config config.Config, logger zerolog.Logger) (*reques return client, nil } - -// -//// setupStorage creates storage and initializes it with configured starting cadence height -//// in case such a height doesn't already exist in the database. -//func setupStorage( -// config config.Config, -// client *requester.CrossSporkClient, -// logger zerolog.Logger, -//) (*pebbleDB.DB, *Storages, error) { -// // create pebble storage from the provided database root directory -// db, err := pebble.OpenDB(config.DatabaseDir) -// if err != nil { -// return nil, nil, err -// } -// store := pebble.New(db, logger) -// -// blocks := pebble.NewBlocks(store, config.FlowNetworkID) -// storageAddress := evm.StorageAccountAddress(config.FlowNetworkID) -// registerStore := pebble.NewRegisterStorage(store, storageAddress) -// -// // hard set the start cadence height, this is used when force reindexing -// if config.ForceStartCadenceHeight != 0 { -// logger.Warn().Uint64("height", config.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") -// if err := blocks.SetLatestCadenceHeight(config.ForceStartCadenceHeight, nil); err != nil { -// return nil, nil, err -// } -// } -// -// // if database is not initialized require init height -// if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { -// batch := store.NewBatch() -// defer func(batch *pebbleDB.Batch) { -// err := batch.Close() -// if err != nil { -// // we don't know what went wrong, so this is fatal -// logger.Fatal().Err(err).Msg("failed to close batch") -// } -// }(batch) -// -// cadenceHeight := config.InitCadenceHeight -// evmBlokcHeight := uint64(0) -// cadenceBlock, err := client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) -// if err != nil { -// return nil, nil, fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) -// } -// -// snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) -// if err != nil { -// return nil, nil, fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) -// } -// -// delta := storage.NewRegisterDelta(snapshot) -// accountStatus := environment.NewAccountStatus() -// err = delta.SetValue( -// storageAddress[:], -// []byte(flowGo.AccountStatusKey), -// accountStatus.ToBytes(), -// ) -// if err != nil { -// return nil, nil, fmt.Errorf("could not set account status: %w", err) -// } -// -// err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) -// if err != nil { -// return nil, nil, fmt.Errorf("could not store register updates: %w", err) -// } -// -// if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { -// return nil, nil, fmt.Errorf( -// "failed to init the database for block height: %d and ID: %s, with : %w", -// cadenceHeight, -// cadenceBlock.ID, -// err, -// ) -// } -// -// err = batch.Commit(pebbleDB.Sync) -// if err != nil { -// return nil, nil, fmt.Errorf("could not commit register updates: %w", err) -// } -// -// logger.Info(). -// Stringer("fvm_address_for_evm_storage_account", storageAddress). -// Msgf("database initialized with cadence height: %d", cadenceHeight) -// } -// //else { -// // // TODO(JanezP): verify storage account owner is correct -// //} -// -// return db, &Storages{ -// Storage: store, -// Blocks: blocks, -// Registers: registerStore, -// Transactions: pebble.NewTransactions(store), -// Receipts: pebble.NewReceipts(store), -// Traces: pebble.NewTraces(store), -// }, nil -//} -// -//// Run will run complete bootstrap of the EVM gateway with all the engines. -//// Run is a blocking call, but it does signal readiness of the service -//// through a channel provided as an argument. -//func Run(ctx context.Context, cfg config.Config, ready component.ReadyFunc) error { -// boot, err := New(cfg) -// if err != nil { -// return err -// } -// -// if err := boot.StartEventIngestion(ctx); err != nil { -// return fmt.Errorf("failed to start event ingestion engine: %w", err) -// } -// -// if err := boot.StartAPIServer(ctx); err != nil { -// return fmt.Errorf("failed to start API server: %w", err) -// } -// -// if err := boot.StartMetricsServer(ctx); err != nil { -// return fmt.Errorf("failed to start Metrics server: %w", err) -// } -// -// if err := boot.StartProfilerServer(ctx); err != nil { -// return fmt.Errorf("failed to start profiler server: %w", err) -// } -// -// // mark ready -// ready() -// -// // if context is canceled start shutdown -// <-ctx.Done() -// boot.logger.Warn().Msg("bootstrap received context cancellation, stopping services") -// -// boot.StopEventIngestion() -// boot.StopMetricsServer() -// boot.StopAPIServer() -// boot.StopClient() -// boot.StopDB() -// -// return nil -//} diff --git a/bootstrap/builder.go b/bootstrap/builder.go deleted file mode 100644 index 58d75bc7d..000000000 --- a/bootstrap/builder.go +++ /dev/null @@ -1,609 +0,0 @@ -package bootstrap - -import ( - "context" - "errors" - "fmt" - "math" - "time" - - "github.com/cockroachdb/pebble" - "github.com/hashicorp/go-multierror" - "github.com/onflow/flow-evm-gateway/api" - "github.com/onflow/flow-evm-gateway/config" - metrics2 "github.com/onflow/flow-evm-gateway/metrics" - "github.com/onflow/flow-evm-gateway/models" - errs "github.com/onflow/flow-evm-gateway/models/errors" - "github.com/onflow/flow-evm-gateway/services/ingestion" - "github.com/onflow/flow-evm-gateway/services/replayer" - "github.com/onflow/flow-evm-gateway/services/requester" - "github.com/onflow/flow-evm-gateway/services/signer" - "github.com/onflow/flow-evm-gateway/storage" - pebble2 "github.com/onflow/flow-evm-gateway/storage/pebble" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/onflow/flow-go/cmd" - "github.com/onflow/flow-go/fvm/environment" - "github.com/onflow/flow-go/fvm/evm" - flowGo "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/metrics" - gethTypes "github.com/onflow/go-ethereum/core/types" - "github.com/rs/zerolog" - "github.com/sethvargo/go-limiter/memorystore" -) - -type EVMGatewayNodeImp struct { - cmd.NodeImp - config.Config -} - -// NewNode returns a new node instance -func NewNode( - component component.Component, - cfg config.Config, - logger zerolog.Logger, - cleanup func() error, - handleFatal func(error), -) *EVMGatewayNodeImp { - return &EVMGatewayNodeImp{ - Config: cfg, - NodeImp: cmd.NewBaseNode( - component, - logger.With(). - Str("node_role", "EVM Gateway"). - Logger(), - cleanup, - handleFatal, - ), - } -} - -type EVMGatewayNodeBuilder struct { - config.Config - Logger zerolog.Logger - componentBuilder component.ComponentManagerBuilder - components []cmd.NamedComponentFunc[config.Config] - postShutdownFns []func() error - modules []namedModuleFunc - - Metrics metrics2.Collector - DB *pebble.DB - Client *requester.CrossSporkClient - Storages *Storages - // Signer is used for signing flow transactions - Signer crypto.Signer - Publishers *Publishers -} - -func (fnb *EVMGatewayNodeBuilder) Build() (cmd.Node, error) { - // Run the prestart initialization. This includes anything that should be done before - // starting the components. - if err := fnb.onStart(); err != nil { - return nil, err - } - - return NewNode( - fnb.componentBuilder.Build(), - fnb.Config, - fnb.Logger, - fnb.postShutdown, - fnb.handleFatal, - ), nil -} - -func (fnb *EVMGatewayNodeBuilder) onStart() error { - - if err := fnb.initDB(); err != nil { - return err - } - - if err := fnb.initMetrics(); err != nil { - return err - } - - if err := fnb.initClient(); err != nil { - return err - } - - if err := fnb.initStorage(); err != nil { - return err - } - - // run all modules - if err := fnb.handleModules(); err != nil { - return fmt.Errorf("could not handle modules: %w", err) - } - - // run all components - return fnb.handleComponents() -} - -func (fnb *EVMGatewayNodeBuilder) initDB() error { - pebbleDB, err := pebble2.OpenDB(fnb.DatabaseDir) - if err != nil { - return fmt.Errorf("failed to open db for dir: %s, with: %w", fnb.DatabaseDir, err) - } - - fnb.DB = pebbleDB - - fnb.ShutdownFunc(func() error { - if err := fnb.DB.Close(); err != nil { - return fmt.Errorf("error closing pebble database: %w", err) - } - return nil - }) - - return err -} - -func (fnb *EVMGatewayNodeBuilder) Component(name string, f cmd.ReadyDoneFactory[config.Config]) *EVMGatewayNodeBuilder { - fnb.components = append(fnb.components, cmd.NamedComponentFunc[config.Config]{ - FN: f, - Name: name, - }) - return fnb -} - -// postShutdown is called by the node before exiting -// put any cleanup code here that should be run after all components have stopped -func (fnb *EVMGatewayNodeBuilder) postShutdown() error { - var errs *multierror.Error - - for _, fn := range fnb.postShutdownFns { - err := fn() - if err != nil { - errs = multierror.Append(errs, err) - } - } - fnb.Logger.Info().Msg("database has been closed") - return errs.ErrorOrNil() -} - -// handleFatal handles irrecoverable errors by logging them and exiting the process. -func (fnb *EVMGatewayNodeBuilder) handleFatal(err error) { - fnb.Logger.Fatal().Err(err).Msg("unhandled irrecoverable error") -} - -func NewEVMGatewayNodeBuilder( - config config.Config, -) *EVMGatewayNodeBuilder { - - logger := zerolog.New(config.LogWriter). - With().Timestamp().Str("version", api.Version). - Logger().Level(config.LogLevel) - - return &EVMGatewayNodeBuilder{ - Logger: logger, - Config: config, - componentBuilder: component.NewComponentManagerBuilder(), - } -} - -func (fnb *EVMGatewayNodeBuilder) Initialize() error { - fnb.PrintBuildDetails() - - fnb.EnqueueMetricsServerInit() - - return nil -} - -func (fnb *EVMGatewayNodeBuilder) LoadComponentsAndModules() { - fnb.initPublishers() - - fnb.Component("Transaction Signer", fnb.initSigner) - fnb.Component("API Server", fnb.apiServerComponent) - fnb.Component("Event Ingestion Engine", fnb.eventIngestionEngineComponent) - fnb.Component("Metrics Server", fnb.metricsServerComponent) - fnb.Component("Profiler Server", fnb.profilerServerComponent) -} - -func (fnb *EVMGatewayNodeBuilder) metricsServerComponent(config config.Config) (module.ReadyDoneAware, error) { - server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) - return server, nil -} - -func (fnb *EVMGatewayNodeBuilder) profilerServerComponent(config config.Config) (module.ReadyDoneAware, error) { - server := api.NewProfileServer(fnb.Logger, config.ProfilerHost, config.ProfilerPort) - return server, nil -} - -func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module.ReadyDoneAware, error) { - log := fnb.Logger - - log.Info().Msg("bootstrap starting Metrics server") - - server := api.NewServer(log, fnb.Metrics, cfg) - - // create transaction pool - txPool := requester.NewTxPool( - fnb.Client, - fnb.Publishers.Transaction, - log, - ) - - blocksProvider := replayer.NewBlocksProvider( - fnb.Storages.Blocks, - cfg.FlowNetworkID, - nil, - ) - - evm, err := requester.NewEVM( - fnb.Storages.Registers, - blocksProvider, - fnb.Client, - cfg, - fnb.Signer, - log, - fnb.Storages.Blocks, - txPool, - fnb.Metrics, - ) - if err != nil { - return nil, fmt.Errorf("failed to create EVM requester: %w", err) - } - - // create rate limiter for requests on the APIs. Tokens are number of requests allowed per 1 second interval - // if no limit is defined we specify max value, effectively disabling rate-limiting - rateLimit := cfg.RateLimit - if rateLimit == 0 { - log.Warn().Msg("no rate-limiting is set") - rateLimit = math.MaxInt - } - ratelimiter, err := memorystore.New(&memorystore.Config{Tokens: rateLimit, Interval: time.Second}) - if err != nil { - return nil, fmt.Errorf("failed to create rate limiter: %w", err) - } - - blockchainAPI, err := api.NewBlockChainAPI( - log, - cfg, - evm, - fnb.Storages.Blocks, - fnb.Storages.Transactions, - fnb.Storages.Receipts, - ratelimiter, - fnb.Metrics, - ) - if err != nil { - return nil, fmt.Errorf("failed to create blockchain API: %w", err) - } - - streamAPI := api.NewStreamAPI( - log, - cfg, - fnb.Storages.Blocks, - fnb.Storages.Transactions, - fnb.Storages.Receipts, - fnb.Publishers.Block, - fnb.Publishers.Transaction, - fnb.Publishers.Logs, - ) - - pullAPI := api.NewPullAPI( - log, - cfg, - fnb.Storages.Blocks, - fnb.Storages.Transactions, - fnb.Storages.Receipts, - ratelimiter, - ) - - debugAPI := api.NewDebugAPI( - fnb.Storages.Registers, - fnb.Storages.Traces, - fnb.Storages.Blocks, - fnb.Storages.Transactions, - fnb.Storages.Receipts, - fnb.Client, - cfg, - log, - fnb.Metrics, - ) - - var walletAPI *api.WalletAPI - if cfg.WalletEnabled { - walletAPI = api.NewWalletAPI(cfg, blockchainAPI) - } - - supportedAPIs := api.SupportedAPIs( - blockchainAPI, - streamAPI, - pullAPI, - debugAPI, - walletAPI, - cfg, - ) - - if err := server.EnableRPC(supportedAPIs); err != nil { - return nil, err - } - - if cfg.WSEnabled { - if err := server.EnableWS(supportedAPIs); err != nil { - return nil, err - } - } - - if err := server.SetListenAddr(cfg.RPCHost, cfg.RPCPort); err != nil { - return nil, err - } - - return server, nil -} - -func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Config) (module.ReadyDoneAware, error) { - l := fnb.Logger.With().Str("component", "bootstrap-ingestion").Logger() - l.Info().Msg("bootstrap starting event ingestion") - - // get latest cadence block from the network and the database - latestCadenceBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) - if err != nil { - return nil, fmt.Errorf("failed to get latest cadence block: %w", err) - } - - latestCadenceHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() - if err != nil { - return nil, err - } - - // make sure the provided block to start the indexing can be loaded - _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) - if err != nil { - return nil, fmt.Errorf( - "failed to get provided cadence height %d: %w", - latestCadenceHeight, - err, - ) - } - - l.Info(). - Uint64("start-cadence-height", latestCadenceHeight). - Uint64("latest-cadence-height", latestCadenceBlock.Height). - Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). - Msg("indexing cadence height information") - - chainID := cfg.FlowNetworkID - - // create event subscriber - subscriber := ingestion.NewRPCEventSubscriber( - fnb.Logger, - fnb.Client, - chainID, - latestCadenceHeight, - ) - - callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) - if err != nil { - return nil, err - } - blocksProvider := replayer.NewBlocksProvider( - fnb.Storages.Blocks, - chainID, - callTracerCollector.TxTracer(), - ) - replayerConfig := replayer.Config{ - ChainID: chainID, - RootAddr: evm.StorageAccountAddress(chainID), - CallTracerCollector: callTracerCollector, - ValidateResults: true, - } - - // initialize event ingestion engine - events := ingestion.NewEventIngestionEngine( - subscriber, - blocksProvider, - fnb.Storages.Storage, - fnb.Storages.Registers, - fnb.Storages.Blocks, - fnb.Storages.Receipts, - fnb.Storages.Transactions, - fnb.Storages.Traces, - fnb.Publishers.Block, - fnb.Publishers.Logs, - fnb.Logger, - fnb.Metrics, - replayerConfig, - ) - - return events, nil -} - -func (fnb *EVMGatewayNodeBuilder) PrintBuildDetails() { - fnb.Logger.Info().Str("version", api.Version).Msg("build details") -} - -// ShutdownFunc adds a callback function that is called after all components have exited. -func (fnb *EVMGatewayNodeBuilder) ShutdownFunc(fn func() error) *EVMGatewayNodeBuilder { - fnb.postShutdownFns = append(fnb.postShutdownFns, fn) - return fnb -} - -func (fnb *EVMGatewayNodeBuilder) EnqueueMetricsServerInit() { - fnb.Component("Metrics server", func(config config.Config) (module.ReadyDoneAware, error) { - server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) - return server, nil - }) -} - -func (fnb *EVMGatewayNodeBuilder) initMetrics() error { - fnb.Metrics = metrics2.NewCollector(fnb.Logger) - return nil -} - -func (fnb *EVMGatewayNodeBuilder) initStorage() error { - logger := fnb.Logger - cfg := fnb.Config - - store := pebble2.New(fnb.DB, fnb.Logger) - - blocks := pebble2.NewBlocks(store, cfg.FlowNetworkID) - storageAddress := evm.StorageAccountAddress(cfg.FlowNetworkID) - registerStore := pebble2.NewRegisterStorage(store, storageAddress) - - // hard set the start cadence height, this is used when force reindexing - if cfg.ForceStartCadenceHeight != 0 { - logger.Warn().Uint64("height", cfg.ForceStartCadenceHeight).Msg("force setting starting Cadence height!!!") - if err := blocks.SetLatestCadenceHeight(cfg.ForceStartCadenceHeight, nil); err != nil { - return err - } - } - - // if database is not initialized require init height - if _, err := blocks.LatestCadenceHeight(); errors.Is(err, errs.ErrStorageNotInitialized) { - // TODO(JanezP): move this to a separate function - err = func() (innerErr error) { - batch := store.NewBatch() - defer func(batch *pebble.Batch) { - innerErr = batch.Close() - }(batch) - - cadenceHeight := cfg.InitCadenceHeight - evmBlokcHeight := uint64(0) - cadenceBlock, err := fnb.Client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) - if err != nil { - return fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) - } - - snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) - if err != nil { - return fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) - } - - delta := storage.NewRegisterDelta(snapshot) - accountStatus := environment.NewAccountStatus() - err = delta.SetValue( - storageAddress[:], - []byte(flowGo.AccountStatusKey), - accountStatus.ToBytes(), - ) - if err != nil { - return fmt.Errorf("could not set account status: %w", err) - } - - err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) - if err != nil { - return fmt.Errorf("could not store register updates: %w", err) - } - - if err := blocks.InitHeights(cadenceHeight, cadenceBlock.ID, batch); err != nil { - return fmt.Errorf( - "failed to init the database for block height: %d and ID: %s, with : %w", - cadenceHeight, - cadenceBlock.ID, - err, - ) - } - - err = batch.Commit(pebble.Sync) - if err != nil { - return fmt.Errorf("could not commit register updates: %w", err) - } - - logger.Info(). - Stringer("fvm_address_for_evm_storage_account", storageAddress). - Msgf("database initialized with cadence height: %d", cadenceHeight) - - return nil - }() - - if err != nil { - return fmt.Errorf("failed to init the database: %w", err) - } - } - //else { - // // TODO(JanezP): verify storage account owner is correct - //} - - fnb.Storages = &Storages{ - Storage: store, - Blocks: blocks, - Registers: registerStore, - Transactions: pebble2.NewTransactions(store), - Receipts: pebble2.NewReceipts(store), - Traces: pebble2.NewTraces(store), - } - - return nil -} -func (fnb *EVMGatewayNodeBuilder) initSigner(config config.Config) (module.ReadyDoneAware, error) { - sig := signer.NewSigner(fnb.Logger, config) - fnb.Signer = sig - return sig, nil -} - -func (fnb *EVMGatewayNodeBuilder) initPublishers() { - fnb.Publishers = &Publishers{} - - fnb.Component("Block Publisher", func(config config.Config) (module.ReadyDoneAware, error) { - p := models.NewPublisher[*models.Block](fnb.Logger) - fnb.Publishers.Block = p - return p, nil - }) - fnb.Component("Transaction Publisher", func(config config.Config) (module.ReadyDoneAware, error) { - p := models.NewPublisher[*gethTypes.Transaction](fnb.Logger) - fnb.Publishers.Transaction = p - return p, nil - }) - fnb.Component("Logs Publisher", func(config config.Config) (module.ReadyDoneAware, error) { - p := models.NewPublisher[[]*gethTypes.Log](fnb.Logger) - fnb.Publishers.Logs = p - return p, nil - }) -} - -func (fnb *EVMGatewayNodeBuilder) initClient() error { - logger := fnb.Logger - cfg := fnb.Config - - client, err := setupCrossSporkClient(cfg, logger) - if err != nil { - return fmt.Errorf("failed to create cross-spork client: %w", err) - } - - fnb.Client = client - - fnb.ShutdownFunc(func() error { - if err := fnb.Client.Close(); err != nil { - return fmt.Errorf("error closing cross-spork client: %w", err) - } - return nil - }) - - return nil -} - -type BuilderFunc func(nodeConfig config.Config) error - -type namedModuleFunc struct { - fn BuilderFunc - name string -} - -// handleModules initializes the given module. -func (fnb *EVMGatewayNodeBuilder) handleModule(v namedModuleFunc) error { - fnb.Logger.Info().Str("module", v.name).Msg("module initialization started") - err := v.fn(fnb.Config) - if err != nil { - return fmt.Errorf("module %s initialization failed: %w", v.name, err) - } - - fnb.Logger.Info().Str("module", v.name).Msg("module initialization complete") - return nil -} - -// handleModules initializes all modules that have been enqueued on this node builder. -func (fnb *EVMGatewayNodeBuilder) handleModules() error { - for _, f := range fnb.modules { - if err := fnb.handleModule(f); err != nil { - return err - } - } - - return nil -} - -func (fnb *EVMGatewayNodeBuilder) handleComponents() error { - cmd.AddWorkersFromComponents(fnb.Logger, fnb.Config, fnb.componentBuilder, fnb.components) - return nil -} diff --git a/go.mod b/go.mod index ca78e9c8c..d444fddb6 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/hashicorp/go-multierror v1.1.1 github.com/onflow/atree v0.8.0 github.com/onflow/cadence v1.2.2 - github.com/onflow/flow-go v0.38.0-preview.0.4 + github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/prometheus/client_golang v1.18.0 @@ -22,6 +22,7 @@ require ( go.uber.org/ratelimit v0.3.1 golang.org/x/exp v0.0.0-20240119083558-1b970713d09a golang.org/x/sync v0.8.0 + google.golang.org/grpc v1.63.2 ) require ( @@ -300,7 +301,6 @@ require ( google.golang.org/genproto v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de // indirect - google.golang.org/grpc v1.63.2 // indirect google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 // indirect google.golang.org/protobuf v1.33.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect @@ -308,5 +308,3 @@ require ( lukechampine.com/blake3 v1.3.0 // indirect rsc.io/tmplfunc v0.0.3 // indirect ) - -replace github.com/onflow/flow-go => ../flow-go-2 diff --git a/go.sum b/go.sum index a9131103f..007493180 100644 --- a/go.sum +++ b/go.sum @@ -743,8 +743,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4 h1:vjnp6btehu3X/aYjsXYlA3r/GGYeB05so0d7ICtXbmg= -github.com/onflow/flow-go v0.38.0-preview.0.4/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d h1:HzvFJzdaRtVzJ0lXR8rvbT7maCjfX4CdZZqt/vD+7iM= +github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/models/stream_test.go b/models/stream_test.go index ded635dff..08e036e5e 100644 --- a/models/stream_test.go +++ b/models/stream_test.go @@ -1,13 +1,17 @@ package models_test import ( + "context" + "fmt" "sync" "sync/atomic" "testing" "time" "github.com/onflow/flow-evm-gateway/models" + "github.com/onflow/flow-go/module/irrecoverable" "github.com/rs/zerolog" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" ) @@ -15,7 +19,7 @@ func Test_Stream(t *testing.T) { t.Run("unsubscribe before subscribing", func(t *testing.T) { p := newMockPublisher() - s := newMockSubscription() + s := newMockSubscription(func(mockData) error { return nil }) require.NotPanics(t, func() { p.Unsubscribe(s) @@ -24,28 +28,45 @@ func Test_Stream(t *testing.T) { t.Run("subscribe, publish, unsubscribe, publish", func(t *testing.T) { p := newMockPublisher() - s1 := newMockSubscription() - s2 := newMockSubscription() + + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + p.Start(ictx) + + callbackChan := make(chan struct{}) + + f := func(mockData) error { + callbackChan <- struct{}{} + return nil + } + + s1 := newMockSubscription(f) + s2 := newMockSubscription(f) p.Subscribe(s1) p.Subscribe(s2) p.Publish(mockData{}) - require.Equal(t, uint64(1), s1.CallCount()) - require.Equal(t, uint64(1), s2.CallCount()) + <-callbackChan + <-callbackChan p.Unsubscribe(s1) p.Publish(mockData{}) - require.Equal(t, uint64(1), s1.CallCount()) - require.Equal(t, uint64(2), s2.CallCount()) + <-callbackChan }) t.Run("concurrent subscribe, publish, unsubscribe, publish", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() p := newMockPublisher() + p.Start(ictx) stopPublishing := make(chan struct{}) @@ -78,9 +99,17 @@ func Test_Stream(t *testing.T) { for i := 0; i < 10; i++ { go func() { subscriptions := make([]*mockSubscription, 10) + callCount := make([]atomic.Uint64, 10) for j := 0; j < 10; j++ { - s := newMockSubscription() + callCount[j].Store(0) + + s := newMockSubscription( + func(data mockData) error { + callCount[j].Add(1) + return nil + }, + ) subscriptions[j] = s p.Subscribe(s) @@ -99,7 +128,7 @@ func Test_Stream(t *testing.T) { // there should be at least 1 call for j := 0; j < 10; j++ { - require.GreaterOrEqual(t, subscriptions[j].CallCount(), uint64(10)) + require.GreaterOrEqual(t, callCount[j].Load(), uint64(10)) } waitAllUnsubscribed.Done() @@ -109,57 +138,44 @@ func Test_Stream(t *testing.T) { waitAllUnsubscribed.Wait() close(stopPublishing) }) - // - //t.Run("error handling", func(t *testing.T) { - // p := newMockPublisher() - // s := &mockSubscription{} - // errContent := fmt.Errorf("failed to process data") - // - // s.Subscription = models.NewSubscription[mockData](func(data mockData) error { - // s.callCount.Add(1) - // return errContent - // }) - // - // p.Subscribe(s) - // - // shouldReceiveError := make(chan struct{}) - // ready := make(chan struct{}) - // go func() { - // close(ready) - // select { - // case err := <-s.Error(): - // require.ErrorIs(t, err, errContent) - // case <-shouldReceiveError: - // require.Fail(t, "should have received error") - // } - // }() - // <-ready - // - // p.Publish(mockData{}) - // close(shouldReceiveError) - //}) + + t.Run("error handling", func(t *testing.T) { + errContent := fmt.Errorf("failed to process data") + gotError := make(chan struct{}) + + ctx := context.Background() + ictx := irrecoverable.NewMockSignalerContext(t, ctx) + ictx.On("Throw", errContent).Run(func(args mock.Arguments) { + close(gotError) + }) + + p := newMockPublisher() + p.Start(ictx) + + s := newMockSubscription(func(data mockData) error { + return errContent + }) + + p.Subscribe(s) + + p.Publish(mockData{}) + + <-gotError + }) } type mockData struct{} type mockSubscription struct { *models.Subscription[mockData] - callCount atomic.Uint64 } -func newMockSubscription() *mockSubscription { +func newMockSubscription(callback func(mockData) error) *mockSubscription { s := &mockSubscription{} - s.Subscription = models.NewSubscription[mockData](func(data mockData) error { - s.callCount.Add(1) - return nil - }) + s.Subscription = models.NewSubscription[mockData](callback) return s } -func (s *mockSubscription) CallCount() uint64 { - return s.callCount.Load() -} - func newMockPublisher() *models.Publisher[mockData] { return models.NewPublisher[mockData](zerolog.Nop()) } diff --git a/tests/go.mod b/tests/go.mod index 65b9c20e4..f93df8c28 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -8,7 +8,7 @@ require ( github.com/onflow/crypto v0.25.2 github.com/onflow/flow-emulator v1.1.1-0.20241125195348-4e121ffb12af github.com/onflow/flow-evm-gateway v0.0.0-20240201154855-4d4d3d3f19c7 - github.com/onflow/flow-go v0.38.0-preview.0.4 + github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/rs/zerolog v1.33.0 @@ -28,6 +28,22 @@ require ( github.com/StackExchange/wmi v1.2.1 // indirect github.com/VictoriaMetrics/fastcache v1.12.2 // indirect github.com/allegro/bigcache v1.2.1 // indirect + github.com/aws/aws-sdk-go-v2 v1.27.0 // indirect + github.com/aws/aws-sdk-go-v2/config v1.27.15 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.17.15 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.3 // 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.3.7 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.9 // 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.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.28.9 // indirect + github.com/aws/smithy-go v1.20.2 // indirect github.com/benbjohnson/clock v1.3.5 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bits-and-blooms/bitset v1.10.0 // indirect @@ -45,10 +61,14 @@ require ( 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-ipa v0.0.0-20240223125850-b1e8a79f509c // indirect github.com/crate-crypto/go-kzg-4844 v1.0.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 github.com/deckarep/golang-set/v2 v2.6.0 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect @@ -61,13 +81,19 @@ require ( github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 // indirect 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/c-kzg-4844 v1.0.0 // indirect github.com/ethereum/go-ethereum v1.13.10 // indirect github.com/ethereum/go-verkle v0.1.1-0.20240306133620-7d920df305f0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect + github.com/flynn/noise v1.0.1 // indirect + github.com/francoispqt/gojay v1.2.13 // indirect github.com/fsnotify/fsnotify v1.6.0 // indirect github.com/fxamacker/cbor/v2 v2.4.1-0.20230228173756-c0c9f774e40c // indirect github.com/fxamacker/circlehash v0.3.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.3 // indirect + github.com/gammazero/deque v0.1.0 // indirect + github.com/gammazero/workerpool v1.1.2 // indirect github.com/gballet/go-libpcsclite v0.0.0-20191108122812-4678299bea08 // indirect github.com/getsentry/sentry-go v0.27.0 // indirect github.com/glebarez/go-sqlite v1.22.0 // indirect @@ -77,8 +103,13 @@ require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.1 // indirect github.com/go-redis/redis/v8 v8.11.5 // indirect github.com/go-sourcemap/sourcemap v2.1.3+incompatible // indirect + 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/gogo/protobuf v1.3.2 // indirect github.com/golang/glog v1.2.0 // indirect @@ -87,6 +118,7 @@ require ( github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect github.com/google/go-cmp v0.6.0 // indirect github.com/google/go-dap v0.11.0 // indirect + github.com/google/gopacket v1.1.19 // indirect github.com/google/pprof v0.0.0-20231229205709-960ae82b1e42 // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/google/uuid v1.6.0 // indirect @@ -113,30 +145,57 @@ require ( github.com/ipfs/boxo v0.17.1-0.20240131173518-89bceff34bf1 // indirect github.com/ipfs/go-block-format v0.2.0 // indirect github.com/ipfs/go-cid v0.4.1 // indirect + github.com/ipfs/go-cidutil v0.1.0 // indirect github.com/ipfs/go-datastore v0.6.0 // indirect + github.com/ipfs/go-ds-badger2 v0.1.3 // indirect + github.com/ipfs/go-ipfs-delay v0.0.1 // indirect + github.com/ipfs/go-ipfs-pq v0.0.3 // indirect github.com/ipfs/go-ipfs-util v0.0.3 // indirect github.com/ipfs/go-ipld-format v0.6.0 // indirect + github.com/ipfs/go-log v1.0.5 // indirect github.com/ipfs/go-log/v2 v2.5.1 // indirect github.com/ipfs/go-metrics-interface v0.0.1 // indirect + github.com/ipfs/go-peertaskqueue v0.8.1 // indirect + github.com/ipld/go-ipld-prime v0.21.0 // indirect github.com/jackpal/go-nat-pmp v1.0.2 // indirect + github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect github.com/jbenet/goprocess v0.1.4 // indirect + github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/k0kubun/pp v3.0.1+incompatible // indirect github.com/kevinburke/go-bindata v3.24.0+incompatible // indirect github.com/klauspost/compress v1.17.4 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect + github.com/koron/go-ssdp v0.0.4 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/libp2p/go-buffer-pool v0.1.0 // indirect + github.com/libp2p/go-cidranger v1.1.0 // indirect + github.com/libp2p/go-flow-metrics v0.1.0 // indirect github.com/libp2p/go-libp2p v0.32.2 // indirect + github.com/libp2p/go-libp2p-asn-util v0.4.1 // indirect + github.com/libp2p/go-libp2p-kad-dht v0.25.2 // indirect + github.com/libp2p/go-libp2p-kbucket v0.6.3 // indirect github.com/libp2p/go-libp2p-pubsub v0.10.0 // indirect + github.com/libp2p/go-libp2p-record v0.2.0 // indirect + github.com/libp2p/go-libp2p-routing-helpers v0.7.3 // indirect github.com/libp2p/go-msgio v0.3.0 // indirect + github.com/libp2p/go-nat v0.2.0 // indirect + github.com/libp2p/go-netroute v0.2.1 // indirect + github.com/libp2p/go-reuseport v0.4.0 // indirect + github.com/libp2p/go-yamux/v4 v4.0.1 // indirect github.com/logrusorgru/aurora v2.0.3+incompatible // indirect github.com/logrusorgru/aurora/v4 v4.0.0 // indirect + github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/magiconair/properties v1.8.7 // indirect + github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.15 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect + github.com/miekg/dns v1.1.57 // indirect + github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b // indirect + 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/mitchellh/mapstructure v1.5.0 // indirect @@ -145,6 +204,8 @@ require ( github.com/multiformats/go-base32 v0.1.0 // indirect github.com/multiformats/go-base36 v0.2.0 // indirect github.com/multiformats/go-multiaddr v0.12.2 // indirect + github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect + github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect github.com/multiformats/go-multibase v0.2.0 // indirect github.com/multiformats/go-multicodec v0.9.0 // indirect github.com/multiformats/go-multihash v0.2.3 // indirect @@ -162,17 +223,27 @@ require ( github.com/onflow/nft-storefront/lib/go/contracts v1.0.0 // indirect github.com/onflow/sdks v0.6.0-preview.1 // indirect github.com/onflow/wal v1.0.2 // indirect + github.com/onsi/ginkgo/v2 v2.13.2 // indirect + github.com/opencontainers/runtime-spec v1.1.0 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 // indirect github.com/pelletier/go-toml/v2 v2.0.6 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/polydawn/refmt v0.89.0 // indirect + github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/prometheus/client_golang v1.18.0 // indirect github.com/prometheus/client_model v0.5.0 // indirect github.com/prometheus/common v0.45.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect github.com/psiemens/graceland v1.0.0 // indirect github.com/psiemens/sconfig v0.1.0 // indirect + github.com/quic-go/qpack v0.4.0 // indirect + github.com/quic-go/qtls-go1-20 v0.4.1 // indirect + github.com/quic-go/quic-go v0.40.1 // indirect + github.com/quic-go/webtransport-go v0.6.0 // indirect + github.com/raulk/go-watchdog v1.3.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rivo/uniseg v0.4.4 // indirect github.com/rogpeppe/go-internal v1.10.0 // indirect @@ -183,7 +254,9 @@ require ( github.com/sethvargo/go-limiter v1.0.0 // indirect github.com/sethvargo/go-retry v0.2.3 // indirect github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect + github.com/shirou/gopsutil/v3 v3.22.2 // indirect github.com/slok/go-http-metrics v0.10.0 // indirect + github.com/sony/gobreaker v0.5.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/spf13/afero v1.10.0 // indirect github.com/spf13/cast v1.5.0 // indirect @@ -204,7 +277,9 @@ require ( github.com/vmihailenco/msgpack v4.0.4+incompatible // indirect github.com/vmihailenco/msgpack/v4 v4.3.11 // 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 go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 // indirect @@ -217,11 +292,15 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.0.0 // indirect go.uber.org/atomic v1.11.0 // indirect + go.uber.org/dig v1.17.1 // indirect + go.uber.org/fx v1.20.1 // indirect + go.uber.org/mock v0.4.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/ratelimit v0.3.1 // indirect go.uber.org/zap v1.26.0 // indirect golang.org/x/crypto v0.28.0 // indirect golang.org/x/exp v0.0.0-20240119083558-1b970713d09a // indirect + golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.25.0 // indirect golang.org/x/oauth2 v0.17.0 // indirect golang.org/x/sync v0.8.0 // indirect @@ -229,6 +308,7 @@ require ( golang.org/x/term v0.25.0 // indirect golang.org/x/text v0.19.0 // indirect golang.org/x/time v0.5.0 // indirect + golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect gonum.org/v1/gonum v0.14.0 // indirect google.golang.org/api v0.162.0 // indirect @@ -237,6 +317,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de // indirect google.golang.org/grpc v1.63.2 // indirect + google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.2.0 // indirect google.golang.org/protobuf v1.33.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/tests/go.sum b/tests/go.sum index fc082fd7b..4739a5d6b 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -1,5 +1,7 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= @@ -35,6 +37,8 @@ cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= cloud.google.com/go/iam v1.1.6/go.mod h1:O0zxdPeGBoFdWW3HWmBxJsk0pfvNM/p/qa82rWOGTwI= cloud.google.com/go/kms v1.15.7 h1:7caV9K3yIxvlQPAcaFffhlT7d1qpxjB1wHBtjWa13SM= cloud.google.com/go/kms v1.15.7/go.mod h1:ub54lbsa6tDkUwnu4W7Yt1aAIFLnspgh0kPGToDukeI= +cloud.google.com/go/profiler v0.3.0 h1:R6y/xAeifaUXxd2x6w+jIwKxoKl8Cv5HJvcvASTPWJo= +cloud.google.com/go/profiler v0.3.0/go.mod h1:9wYk9eY4iZHsev8TQb61kh3wiOiSyz/xOYixWPzweCU= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -47,9 +51,15 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9 cloud.google.com/go/storage v1.14.0/go.mod h1:GrKmX003DSIwi9o29oFT7YDnHYwZoctc3fOKtUw0Xmo= cloud.google.com/go/storage v1.36.0 h1:P0mOkAcaJxhCTvAkMhxMfrTKiNcub4YmmPBtlhAyTr8= cloud.google.com/go/storage v1.36.0/go.mod h1:M6M/3V/D3KpzMTJyPOR/HU6n2Si5QdaXYEsng2xgOs8= +dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU= +dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4= +dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU= +git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/DataDog/zstd v1.5.2 h1:vUG4lAyuPCXO0TLbXvPv7EB7cNK1QV/luu55UHLrrn8= github.com/DataDog/zstd v1.5.2/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= @@ -77,6 +87,7 @@ github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk5 github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= github.com/allegro/bigcache v1.2.1 h1:hg1sY1raCwic3Vnsvje6TT7/pnZba83LeFck5NrFKSc= github.com/allegro/bigcache v1.2.1/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= +github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= @@ -87,21 +98,50 @@ github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6l github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= 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.27.0 h1:7bZWKoXhzI+mMR/HjdMx8ZCC5+6fY0lS5tr0bbgiLlo= github.com/aws/aws-sdk-go-v2 v1.27.0/go.mod h1:ffIFB97e2yNsv4aTSGkqtHnppsIJzw7G7BReUZ3jCXM= +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.27.15 h1:uNnGLZ+DutuNEkuPh6fwqK7LpEiPmzb7MIMA1mNWEUc= +github.com/aws/aws-sdk-go-v2/config v1.27.15/go.mod h1:7j7Kxx9/7kTmL7z4LlhwQe63MYEE5vkVV6nWg4ZAI8M= +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.17.15 h1:YDexlvDRCA8ems2T5IP1xkMtOZ1uLJOCJdTr0igs5zo= +github.com/aws/aws-sdk-go-v2/credentials v1.17.15/go.mod h1:vxHggqW6hFNaeNC0WyXS3VdyjcV0a4KMUY4dKJ96buU= +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.16.3 h1:dQLK4TjtnlRGb0czOht2CevZ5l6RSyRWAnKeGd7VAFE= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.3/go.mod h1:TL79f2P6+8Q7dTsILpiVST+AL9lkF6PPGI167Ny0Cjw= 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.3.7 h1:lf/8VTF2cM+N4SLzaYJERKEWAXq8MOMpZfU6wEPWsPk= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.7/go.mod h1:4SjkU7QiqK2M9oozyMzfZ/23LmUY+h3oFqhdeP5OMiI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7 h1:4OYVp0705xu8yjdyoWix0r9wPIRXnIzzOoUpQVHIJ/g= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.7/go.mod h1:vd7ESTEvI76T2Na050gODNmNU7+OyKrIKroYTu4ABiI= +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.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= +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/accept-encoding v1.11.2 h1:Ji0DY1xUsUr3I8cHps0G+XM3WWU16lP6yG8qu1GAZAs= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2/go.mod h1:5CsjAbs3NlGQyZNFACh+zztPDI7fU6eW9QsxjfnuBKg= +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.11.9 h1:Wx0rlZoEJR7JwlSZcHnEa7CNjrSIyVxMFWGAaXy4fJY= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.9/go.mod h1:aVMHdE0aHO3v+f/iw01fmXV/5DbfQ3Bi9nN7nd9bE9Y= 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/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.20.8 h1:Kv1hwNG6jHC/sxMTe5saMjH6t6ZLkgfvVxyEjfWL1ks= +github.com/aws/aws-sdk-go-v2/service/sso v1.20.8/go.mod h1:c1qtZUWtygI6ZdvKppzCSXsDOq5I4luJPZ0Ud3juFCA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2 h1:nWBZ1xHCF+A7vv9sDzJOq4NWIdzFYm0kH7Pr4OjHYsQ= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.2/go.mod h1:9lmoVDVLz/yUZwLaQ676TK02fhCu4+PgRSmMaKR1ozk= +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.28.9 h1:Qp6Boy0cGDloOE3zI6XhNLNZgjNS8YmiFQFHe71SaW0= +github.com/aws/aws-sdk-go-v2/service/sts v1.28.9/go.mod h1:0Aqn1MnEuitqfsCNyKsdKLhDUOr4txD/g19EfiUqgws= +github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.20.2 h1:tbp628ireGtzcHDDmLT/6ADHidqnwgF57XOXZe6tp4Q= github.com/aws/smithy-go v1.20.2/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= 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= github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o= github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -111,6 +151,7 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bits-and-blooms/bitset v1.10.0 h1:ePXTeiPEazB5+opbv5fr8umg2R/1NlzgDsyepwsSr88= github.com/bits-and-blooms/bitset v1.10.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g= github.com/btcsuite/btcd v0.20.1-beta/go.mod h1:wVuoA8VJLEcwgqHBwHmzLRazpKxTv13Px/pDuV7OomQ= github.com/btcsuite/btcd v0.21.0-beta h1:At9hIZdJW0s9E/fAz28nrz6AmcNlSVucCH796ZteX1M= github.com/btcsuite/btcd v0.21.0-beta/go.mod h1:ZSWyehm27aAuS9bvkATT+Xte3hjHZ+MRgMY/8NJ7K94= @@ -126,6 +167,7 @@ github.com/btcsuite/snappy-go v0.0.0-20151229074030-0bdef8d06723/go.mod h1:8woku github.com/btcsuite/snappy-go v1.0.0/go.mod h1:8woku9dyThutzjeg+3xrA5iCpBRH8XEEg3lh6TiUghc= github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtEyQwv5/p4Mg4C0fgbePVuGr935/5ddU9Z3TmDRY= 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/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/cenkalti/backoff/v4 v4.1.1/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= @@ -146,6 +188,7 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/readline v1.5.0/go.mod h1:x22KAscuvRqlLoK9CsoYsmxoXZMMFVyOl86cAH8qUic= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/chzyer/test v0.0.0-20210722231415-061457976a23/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= @@ -173,6 +216,7 @@ github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/Yj 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= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= @@ -182,8 +226,9 @@ github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3Ee github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e h1:Wf6HqHfScWJN9/ZjdUKyjop4mf3Qdd+1TvvltAvM3m8= +github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd/v22 v22.1.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= @@ -191,6 +236,7 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfc github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= 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.4 h1:wfIWP927BUkWJb2NmU/kNDYIBTh/ziUX91+lVfRxZq4= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/crate-crypto/go-ipa v0.0.0-20240223125850-b1e8a79f509c h1:uQYC5Z1mdLRPrZhHjHxufI8+2UG/i25QG92j0Er9p6I= @@ -199,6 +245,8 @@ github.com/crate-crypto/go-kzg-4844 v1.0.0 h1:TsSgHwrkTKecKJ4kadtHi4b3xHW5dCFUDF github.com/crate-crypto/go-kzg-4844 v1.0.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= +github.com/cskr/pubsub v1.0.2/go.mod h1:/8MzYXk/NJAz782G8RPkFzXTZVu63VotefPnR9TIRis= github.com/davecgh/go-spew v0.0.0-20171005155431-ecdeabc65495/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -214,6 +262,7 @@ github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3 github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0tEMk218= github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f h1:U5y3Y5UE0w7amNe7Z5G/twsBW0KEalRQXZzf8ufSh9I= github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f/go.mod h1:xH/i4TFMt8koVQZ6WFms69WAsDWr2XsYL3Hkl7jkoLE= +github.com/dgraph-io/badger/v2 v2.2007.3/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDmM2p1e+NhhnBCAE= 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.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= @@ -228,6 +277,7 @@ github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8 github.com/dlclark/regexp2 v1.4.1-0.20201116162257-a2a8dda75c91/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= github.com/dlclark/regexp2 v1.7.0 h1:7lJfhqlPssTb1WQx4yvTHN0uElPEv52sbaECrAQxjAo= github.com/dlclark/regexp2 v1.7.0/go.mod h1:DHkYz0B9wPfa6wondMfaivmHpzrQ3v9q8cnmRbL6yW8= +github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dop251/goja v0.0.0-20211022113120-dc8c55024d06/go.mod h1:R9ET47fwRVRPZnOGvHxxhuZcbrMCuiqOz3Rlrh4KSnk= @@ -245,6 +295,7 @@ github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFP github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= github.com/ef-ds/deque v1.0.4 h1:iFAZNmveMT9WERAkqLJ+oaABF9AcVQ5AjXem/hroniI= github.com/ef-ds/deque v1.0.4/go.mod h1:gXDnTC3yqvBcHbq2lcExjtAcVrOnJCbMcZXmuj8Z4tg= +github.com/elastic/gosigar v0.12.0/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4= github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= @@ -267,14 +318,15 @@ github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2 github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fjl/memsize v0.0.2 h1:27txuSD9or+NZlnOWdKUxeBzTAUkWCVh+4Gf2dWFOzA= github.com/fjl/memsize v0.0.2/go.mod h1:VvhXpOYNQvB+uIk2RvXzuaQtkQJzzIx6lSBe1xv7hi0= +github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= github.com/flynn/noise v1.0.1 h1:vPp/jdQLXC6ppsXSj/pM3W1BIJ5FEHE2TulSJBpb43Y= github.com/flynn/noise v1.0.1/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= -github.com/frankban/quicktest v1.14.4 h1:g2rn0vABPOOXmZUj+vbmUp0lPoXEMuhTpIluN0XL9UY= -github.com/frankban/quicktest v1.14.4/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= +github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHkI4W8= +github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= @@ -302,6 +354,8 @@ github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= github.com/glebarez/go-sqlite v1.22.0 h1:uAcMJhaA6r3LHMTFgP0SifzgXg46yJkgxqyuyec+ruQ= github.com/glebarez/go-sqlite v1.22.0/go.mod h1:PlBIdHe0+aUEFn+r2/uthrWq4FxbzugL0L8Li6yQJbc= +github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= @@ -325,9 +379,12 @@ github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ4 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-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= @@ -347,6 +404,7 @@ github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= 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-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/httphead v0.1.0 h1:exrUm0f4YX0L7EBwZHuCF4GDp8aJfVeBrlLQrs6NqWU= github.com/gobwas/httphead v0.1.0/go.mod h1:O/RXo79gxV8G+RqlR/otEwx4Q36zl9rqC5u12GKvMCM= @@ -358,6 +416,7 @@ github.com/gobwas/ws v1.2.1 h1:F2aeBZrm2NDsc7vbovKrWSogd4wvfAxg0FQ89/iqOTk= github.com/gobwas/ws v1.2.1/go.mod h1:hRKAFb8wOxFROYNsT1bqfWnhX+b5MFeJM9r2ZSwg/KY= 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= github.com/godbus/dbus/v5 v5.1.0/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= @@ -367,6 +426,7 @@ github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFG github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= @@ -381,6 +441,7 @@ github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -409,6 +470,7 @@ github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiu github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk= @@ -426,10 +488,14 @@ github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-dap v0.11.0 h1:SpAZJL41rOOvd85PuLCCLE1dteTQOyKNnn0H3DBHywo= github.com/google/go-dap v0.11.0/go.mod h1:HAeyoSd2WIfTfg+0GRXcFrb+RnojAtGNh+k+XTIxJDE= +github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -459,17 +525,22 @@ github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= 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.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= +github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= +github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c h1:7lF+Vz0LqiRidnzC1Oq86fpX1q/iEv2KJdrCtttYjT4= +github.com/gopherjs/gopherjs v0.0.0-20190430165422-3e4dfb77656c/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= 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= @@ -480,6 +551,7 @@ github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoA github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gorilla/websocket v1.5.1 h1:gmztn0JnHVt9JZquRuzLw3g4wouNVzKL15iLr/zn/QY= github.com/gorilla/websocket v1.5.1/go.mod h1:x3kM2JMyaluk02fnUJpQuwD2dCS5NDG2ZHL0uE0tcaY= +github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.2.2/go.mod h1:EaizFBKfUKtMIF5iaDEhniwNedqGo9FuLFzppDr3uwI= @@ -487,6 +559,7 @@ github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 h1:pRhl55Yx1eC7BZ1N+BBWwn github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0/go.mod h1:XKMd7iuf/RGPSMJ/U4HP0zS2Z9Fh8Ps9a+6X26m/tmI= 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= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 h1:Wqo399gCIufwto+VfwCSvsnfGpF/w5E9CNxSwbpD6No= @@ -556,20 +629,36 @@ github.com/ipfs/go-block-format v0.2.0 h1:ZqrkxBA2ICbDRbK8KJs/u0O3dlp6gmAuuXUJNi github.com/ipfs/go-block-format v0.2.0/go.mod h1:+jpL11nFx5A/SPpsoBn6Bzkra/zaArfSmsknbPMYgzM= github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= +github.com/ipfs/go-cidutil v0.1.0 h1:RW5hO7Vcf16dplUU60Hs0AKDkQAVPVplr7lk97CFL+Q= +github.com/ipfs/go-cidutil v0.1.0/go.mod h1:e7OEVBMIv9JaOxt9zaGEmAoSlXW9jdFZ5lP/0PwcfpA= +github.com/ipfs/go-datastore v0.5.1/go.mod h1:9zhEApYMTl17C8YDp7JmU7sQZi2/wqiYh73hakZ90Bk= github.com/ipfs/go-datastore v0.6.0 h1:JKyz+Gvz1QEZw0LsX1IBn+JFCJQH4SJVFtM4uWU0Myk= github.com/ipfs/go-datastore v0.6.0/go.mod h1:rt5M3nNbSO/8q1t4LNkLyUwRs8HupMeN/8O4Vn9YAT8= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= +github.com/ipfs/go-ds-badger2 v0.1.3 h1:Zo9JicXJ1DmXTN4KOw7oPXkspZ0AWHcAFCP1tQKnegg= +github.com/ipfs/go-ds-badger2 v0.1.3/go.mod h1:TPhhljfrgewjbtuL/tczP8dNrBYwwk+SdPYbms/NO9w= +github.com/ipfs/go-ipfs-blocksutil v0.0.1 h1:Eh/H4pc1hsvhzsQoMEP3Bke/aW5P5rVM1IWFJMcGIPQ= +github.com/ipfs/go-ipfs-blocksutil v0.0.1/go.mod h1:Yq4M86uIOmxmGPUHv/uI7uKqZNtLb449gwKqXjIsnRk= +github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= +github.com/ipfs/go-ipfs-delay v0.0.1 h1:r/UXYyRcddO6thwOnhiznIAiSvxMECGgtv35Xs1IeRQ= +github.com/ipfs/go-ipfs-delay v0.0.1/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= +github.com/ipfs/go-ipfs-pq v0.0.3 h1:YpoHVJB+jzK15mr/xsWC574tyDLkezVrDNeaalQBsTE= +github.com/ipfs/go-ipfs-pq v0.0.3/go.mod h1:btNw5hsHBpRcSSgZtiNm/SLj5gYIZ18AKtv3kERkRb4= github.com/ipfs/go-ipfs-util v0.0.3 h1:2RFdGez6bu2ZlZdI+rWfIdbQb1KudQp3VGwPtdNCmE0= github.com/ipfs/go-ipfs-util v0.0.3/go.mod h1:LHzG1a0Ig4G+iZ26UUOMjHd+lfM84LZCrn17xAKWBvs= github.com/ipfs/go-ipld-format v0.6.0 h1:VEJlA2kQ3LqFSIm5Vu6eIlSxD/Ze90xtc4Meten1F5U= github.com/ipfs/go-ipld-format v0.6.0/go.mod h1:g4QVMTn3marU3qXchwjpKPKgJv+zF+OlaKMyhJ4LHPg= github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= +github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= +github.com/ipfs/go-log/v2 v2.5.0/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= github.com/ipfs/go-log/v2 v2.5.1 h1:1XdUzF7048prq4aBjDQQ4SL5RxftpRGdXhNRwKSAlcY= github.com/ipfs/go-log/v2 v2.5.1/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= +github.com/ipfs/go-peertaskqueue v0.8.1 h1:YhxAs1+wxb5jk7RvS0LHdyiILpNmRIRnZVztekOF0pg= +github.com/ipfs/go-peertaskqueue v0.8.1/go.mod h1:Oxxd3eaK279FxeydSPPVGHzbwVeHjatZ2GA8XD+KbPU= github.com/ipld/go-ipld-prime v0.21.0 h1:n4JmcpOlPDIxBcY037SVfpd1G+Sj1nKZah0m6QH9C2E= github.com/ipld/go-ipld-prime v0.21.0/go.mod h1:3RLqy//ERg/y5oShXXdx5YIp50cFGOanyMctpPjsvxQ= github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= @@ -579,11 +668,14 @@ github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABo github.com/jbenet/go-temp-err-catcher v0.1.0/go.mod h1:0kJRvmDZXNMIiJirNPEYfhpPwbGVtZVWC34vc5WLsDk= github.com/jbenet/goprocess v0.1.4 h1:DRGOFReOMqqDNXwW70QkacFW0YN9QnwLV0Vqk+3oU0o= github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= +github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU= github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= +github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= +github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= @@ -596,6 +688,7 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= @@ -607,6 +700,7 @@ github.com/k0kubun/pp v3.0.1+incompatible/go.mod h1:GWse8YhT0p8pT4ir3ZgBbfZild3t github.com/kevinburke/go-bindata v3.24.0+incompatible h1:qajFA3D0pH94OTLU4zcCCKCDgR+Zr2cZK/RPJHDdFoY= github.com/kevinburke/go-bindata v3.24.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= @@ -625,11 +719,13 @@ github.com/koron/go-ssdp v0.0.4/go.mod h1:oDXq+E5IL5q0U8uSBcoAXzTzInwy5lEgC91HoK github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/pty v1.1.3/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= @@ -681,10 +777,14 @@ github.com/logrusorgru/aurora v2.0.3+incompatible h1:tOpm7WcpBTn4fjmVfgpQq0EfczG github.com/logrusorgru/aurora v2.0.3+incompatible/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= github.com/logrusorgru/aurora/v4 v4.0.0 h1:sRjfPpun/63iADiSvGGjgA1cAYegEWMPCJdUpJYn9JA= github.com/logrusorgru/aurora/v4 v4.0.0/go.mod h1:lP0iIa2nrnT/qoFXcOZSrZQpJ1o6n2CUf/hyHi2Q4ZQ= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= +github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= +github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk= github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= @@ -708,13 +808,19 @@ github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= +github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= github.com/miekg/dns v1.1.57 h1:Jzi7ApEIzwEPLHWRcafCN9LZSBbqQpxjt/wpgvg7wcM= github.com/miekg/dns v1.1.57/go.mod h1:uqRjCRUuEAA6qsOiJvDd+CFo/vW+y5WR6SNmHE55hZk= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= +github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc h1:PTfri+PuQmWDqERdnNMiD9ZejrlswWrCpBEZgWOiTrc= github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= +github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= +github.com/minio/sha256-simd v0.1.1-0.20190913151208-6de447530771/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl0J58iy0KM= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= @@ -741,12 +847,15 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/mr-tron/base58 v1.1.2/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o= github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc= github.com/multiformats/go-base32 v0.1.0 h1:pVx9xoSPqEIQG8o+UbAe7DNi51oej1NtK+aGkbLYxPE= github.com/multiformats/go-base32 v0.1.0/go.mod h1:Kj3tFY6zNr+ABYMqeUNeGvkIC/UYgtWibDcT0rExnbI= github.com/multiformats/go-base36 v0.2.0 h1:lFsAbNOGeKtuKozrtBsAkSVhv1p9D0/qedU9rQyccr0= github.com/multiformats/go-base36 v0.2.0/go.mod h1:qvnKE++v+2MWCfePClUEjE78Z7P2a1UV0xHgWc0hkp4= +github.com/multiformats/go-multiaddr v0.1.1/go.mod h1:aMKBKNEYmzmDmxfX88/vz+J5IU55txyt0p4aiWVohjo= +github.com/multiformats/go-multiaddr v0.2.0/go.mod h1:0nO36NvPpyV4QzvTLi/lafl2y95ncPj0vFwVF6k6wJ4= github.com/multiformats/go-multiaddr v0.12.2 h1:9G9sTY/wCYajKa9lyfWPmpZAwe6oV+Wb1zcmMS1HG24= github.com/multiformats/go-multiaddr v0.12.2/go.mod h1:GKyaTYjZRdcUhyOetrxTk9z0cW+jA/YrnqTOvKgi44M= github.com/multiformats/go-multiaddr-dns v0.3.1 h1:QgQgR+LQVt3NPTjbrLLpsaT2ufAA2y0Mkk+QRVJbW3A= @@ -757,10 +866,12 @@ github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivnc github.com/multiformats/go-multibase v0.2.0/go.mod h1:bFBZX4lKCA/2lyOFSAoKH5SS6oPyjtnzK/XTFDPkNuk= github.com/multiformats/go-multicodec v0.9.0 h1:pb/dlPnzee/Sxv/j4PmkDRxCOi3hXTz3IbPKOXWJkmg= github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= +github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U= github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM= github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE= github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA= +github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8= github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -774,6 +885,8 @@ github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzE github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +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/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= @@ -799,8 +912,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4 h1:vjnp6btehu3X/aYjsXYlA3r/GGYeB05so0d7ICtXbmg= -github.com/onflow/flow-go v0.38.0-preview.0.4/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d h1:HzvFJzdaRtVzJ0lXR8rvbT7maCjfX4CdZZqt/vD+7iM= +github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= @@ -829,9 +942,10 @@ github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5 github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= -github.com/onsi/gomega v1.18.1/go.mod h1:0q+aL8jAiMXy9hbwj2mr5GziHiwhAIQpFmmtT5hitRs= +github.com/onsi/gomega v1.29.0 h1:KIA/t2t5UBzoirT4H9tsML45GEbo3ouUnBHsCfD2tVg= +github.com/onsi/gomega v1.29.0/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= +github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opencontainers/runtime-spec v1.1.0 h1:HHUyrt9mwHUjtasSbXSMvs4cyFxh+Bll4AjJ9odEGpg= github.com/opencontainers/runtime-spec v1.1.0/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= @@ -841,6 +955,7 @@ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA= +github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4= @@ -871,6 +986,9 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/polydawn/refmt v0.89.0 h1:ADJTApkvkeBZsN0tBTx8QjpD9JkmxbKp0cxfr9qszm4= github.com/polydawn/refmt v0.89.0/go.mod h1:/zvteZs/GwLtCgZ4BL6CBsk9IKIlexP43ObX9AxTqTw= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= +github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= @@ -887,6 +1005,7 @@ github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6T github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= +github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= @@ -896,6 +1015,7 @@ github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB8 github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= +github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= @@ -952,13 +1072,38 @@ github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0 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= +github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sethvargo/go-limiter v1.0.0 h1:JqW13eWEMn0VFv86OKn8wiYJY/m250WoXdrjRV0kLe4= github.com/sethvargo/go-limiter v1.0.0/go.mod h1:01b6tW25Ap+MeLYBuD4aHunMrJoNO5PVUFdS9rac3II= github.com/sethvargo/go-retry v0.2.3 h1:oYlgvIvsju3jNbottWABtbnoLC+GDtLdBHxKWxQm/iU= github.com/sethvargo/go-retry v0.2.3/go.mod h1:1afjQuvh7s4gflMObvjLPaWgluLLyhA1wmVZ6KLpICw= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU= github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil/v3 v3.22.2 h1:wCrArWFkHYIdDxx/FSfF5RB4dpJYW6t7rcp3+zL8uks= +github.com/shirou/gopsutil/v3 v3.22.2/go.mod h1:WapW1AOOPlHyXr+yOyw3uYx36enocrtSoSBy0L5vUHY= +github.com/shurcooL/component v0.0.0-20170202220835-f88ec8f54cc4/go.mod h1:XhFIlyj5a1fBNx5aJTbKoIq0mNaPvOagO+HjB3EtxrY= +github.com/shurcooL/events v0.0.0-20181021180414-410e4ca65f48/go.mod h1:5u70Mqkb5O5cxEA8nxTsgrgLehJeAw6Oc4Ab1c/P1HM= +github.com/shurcooL/github_flavored_markdown v0.0.0-20181002035957-2122de532470/go.mod h1:2dOwnU2uBioM+SGy2aZoq1f/Sd1l9OkAeAUvjSyvgU0= +github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= +github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= +github.com/shurcooL/gofontwoff v0.0.0-20180329035133-29b52fc0a18d/go.mod h1:05UtEgK5zq39gLST6uB0cf3NEHjETfB4Fgr3Gx5R9Vw= +github.com/shurcooL/gopherjslib v0.0.0-20160914041154-feb6d3990c2c/go.mod h1:8d3azKNyqcHP1GaQE/c6dDgjkgSx2BZ4IoEi4F1reUI= +github.com/shurcooL/highlight_diff v0.0.0-20170515013008-09bb4053de1b/go.mod h1:ZpfEhSmds4ytuByIcDnOLkTHGUI6KNqRNPDLHDk+mUU= +github.com/shurcooL/highlight_go v0.0.0-20181028180052-98c3abbbae20/go.mod h1:UDKB5a1T23gOMUJrI+uSuH0VRDStOiUVSjBTRDVBVag= +github.com/shurcooL/home v0.0.0-20181020052607-80b7ffcb30f9/go.mod h1:+rgNQw2P9ARFAs37qieuu7ohDNQ3gds9msbT2yn85sg= +github.com/shurcooL/htmlg v0.0.0-20170918183704-d01228ac9e50/go.mod h1:zPn1wHpTIePGnXSHpsVPWEktKXHr6+SS6x/IKRb7cpw= +github.com/shurcooL/httperror v0.0.0-20170206035902-86b7830d14cc/go.mod h1:aYMfkZ6DWSJPJ6c4Wwz3QtW22G7mf/PEgaB9k/ik5+Y= +github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +github.com/shurcooL/httpgzip v0.0.0-20180522190206-b1c53ac65af9/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= +github.com/shurcooL/issues v0.0.0-20181008053335-6292fdc1e191/go.mod h1:e2qWDig5bLteJ4fwvDAc2NHzqFEthkqn7aOZAOpj+PQ= +github.com/shurcooL/issuesapp v0.0.0-20180602232740-048589ce2241/go.mod h1:NPpHK2TI7iSaM0buivtFUc9offApnI0Alt/K8hcHy0I= +github.com/shurcooL/notifications v0.0.0-20181007000457-627ab5aea122/go.mod h1:b5uSkrEVM1jQUspwbixRBhaIjIzL2xazXp6kntxYle0= +github.com/shurcooL/octicon v0.0.0-20181028054416-fa4f57f9efb2/go.mod h1:eWdoE5JD4R5UVWDucdOPg1g2fqQRq78IQa9zlOV1vpQ= +github.com/shurcooL/reactions v0.0.0-20181006231557-f2e0b4ca5b82/go.mod h1:TCR1lToEk4d2s07G3XGfz2QrgHXg4RJBvjrOozvoWfk= +github.com/shurcooL/sanitized_anchor_name v0.0.0-20170918181015-86672fcb3f95/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4= +github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= @@ -968,11 +1113,17 @@ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVs github.com/slok/go-http-metrics v0.10.0 h1:rh0LaYEKza5eaYRGDXujKrOln57nHBi4TtVhmNEpbgM= github.com/slok/go-http-metrics v0.10.0/go.mod h1:lFqdaS4kWMfUKCSukjC47PdCeTk+hXDUVm8kLHRqJ38= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= +github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg= +github.com/smartystreets/goconvey v1.7.2/go.mod h1:Vw0tHAZW6lzCRk3xgdin6fKYcG+G3Pg9vgXWeJpQFMM= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/sony/gobreaker v0.5.0 h1:dRCvqm0P490vZPmy7ppEk2qCnCieBooFJ+YoXGYB+yg= github.com/sony/gobreaker v0.5.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE= +github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod h1:HuIsMU8RRBOtsCgI77wP899iHVBQpCmg4ErYMZB+2IA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= @@ -1012,10 +1163,12 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/subosito/gotenv v1.4.2 h1:X1TuBLAMDFbaTAChgCBLu3DU3UPyELpnF2jjJ2cz/S8= @@ -1024,10 +1177,13 @@ github.com/supranational/blst v0.3.11 h1:LyU6FolezeWAhvQk0k6O/d49jqgO52MSDDfYgbe github.com/supranational/blst v0.3.11/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc= +github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA= 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/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= 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= @@ -1044,16 +1200,24 @@ github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLY github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli v1.22.1 h1:+mkCCcOFKPnCmVYVcURKps1Xe+3zP90gSYGNfRkjoIY= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli v1.22.10 h1:p8Fspmz3iTctJstry1PYS3HVdllxnEzTEsgIgtxTrCk= +github.com/urfave/cli v1.22.10/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/cli/v2 v2.25.7 h1:VAzn5oq403l5pHjc4OhD54+XGO9cdKVL/7lDjF+iKUs= github.com/urfave/cli/v2 v2.25.7/go.mod h1:8qnjx1vcq5s2/wpsqoZFndg2CE5tNFyrTvS6SinrnYQ= +github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU= +github.com/viant/toolbox v0.24.0/go.mod h1:OxMCG57V0PXuIP2HNQrtJf2CjqdmbrOx5EkMILuUhzM= github.com/vmihailenco/msgpack v4.0.4+incompatible h1:dSLoQfGFAo3F6OoNhwUmLwVgaUXK79GlxNBwueZn0xI= github.com/vmihailenco/msgpack v4.0.4+incompatible/go.mod h1:fy3FlTQTDXWkZ7Bh6AcGMlsjHatGryHQYUTf1ShIgkk= github.com/vmihailenco/msgpack/v4 v4.3.11 h1:Q47CePddpNGNhk4GCnAx9DDtASi2rasatE0cd26cZoE= github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/tagparser v0.1.1 h1:quXMXlA39OCbd2wAdTsGDlK9RkOk6Wuw+x37wVyIuWY= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= +github.com/warpfork/go-testmark v0.12.1 h1:rMgCpJfwy1sJ50x0M0NgyphxYYPMOODIJHhsXyEHU0s= +github.com/warpfork/go-testmark v0.12.1/go.mod h1:kHwy7wfvGSPh1rQJYKayD4AbtNaeyZdcGi9tNJTaa5Y= +github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0 h1:GDDkbFiaK8jsSDJfjId/PEGEShv6ugrt4kYsC5UIDaQ= +github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= @@ -1068,6 +1232,8 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/zeebo/assert v1.1.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= @@ -1078,6 +1244,7 @@ github.com/zeebo/pcg v1.0.1/go.mod h1:09F0S9iiKrwn9rlI5yjLkmrug154/YRW6KnnXVDM/l go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= +go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= @@ -1109,6 +1276,7 @@ go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v8 go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= @@ -1123,6 +1291,7 @@ go.uber.org/mock v0.4.0 h1:VcM4ZOtdbR4f6VXfiOpwpVJDL6lCReaZ6mw31wqh7KU= go.uber.org/mock v0.4.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= +go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= @@ -1131,21 +1300,29 @@ go.uber.org/ratelimit v0.3.1/go.mod h1:6euWsTB6U/Nb3X++xEUXA8ciPJvr19Q/0h1+oDcJh go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= +go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= +golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200115085410-6d4e4cb37c7d/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= @@ -1166,6 +1343,7 @@ golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRj golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= 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= +golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -1196,6 +1374,8 @@ golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181029044818-c44066c5c816/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1203,6 +1383,7 @@ golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190313220215-9f648a60d977/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -1235,6 +1416,7 @@ golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= @@ -1243,6 +1425,8 @@ golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= 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= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1253,6 +1437,7 @@ golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.17.0 h1:6m3ZPmLEFdVxKKWnKq4VqZ60gutO35zm+zrAHVmHyDQ= golang.org/x/oauth2 v0.17.0/go.mod h1:OzPDGQiuQMguemayvdylqddI7qcD9lnSDb+1FiwQ5HA= +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= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1267,17 +1452,20 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.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= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181029174526-d69651ed3497/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190316082340-a2f829d7f35f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1301,6 +1489,7 @@ golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1313,6 +1502,7 @@ golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1321,9 +1511,11 @@ golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210225134936-a50acf3fe073/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423185535-09eb48e85fd7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1331,6 +1523,9 @@ golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220111092808-5a964db01320/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1370,6 +1565,8 @@ golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030000716-a0a13e073c7b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= @@ -1434,6 +1631,9 @@ golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSm golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.14.0 h1:2NiG67LD1tEH0D7kM+ps2V+fXmsAnpUeec7n8tcr4S0= gonum.org/v1/gonum v0.14.0/go.mod h1:AoWeoz0becf9QMWtE8iWXNXc27fK4fNeHNf/oMejGfU= +google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +google.golang.org/api v0.0.0-20181030000543-1d582fd0359e/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= @@ -1458,6 +1658,7 @@ google.golang.org/api v0.162.0 h1:Vhs54HkaEpkMBdgGdOT2P6F0csGG/vxDS0hWHJzmmps= google.golang.org/api v0.162.0/go.mod h1:6SulDkfoBIg4NFmCuZ39XeeAgSHCPecfSUuDyYlAHs0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= @@ -1467,6 +1668,10 @@ google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCID google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181029155118-b69ba1387ce2/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181202183823-bd91e49a0898/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= +google.golang.org/genproto v0.0.0-20190306203927-b5d61aea6440/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1513,6 +1718,8 @@ google.golang.org/genproto/googleapis/bytestream v0.0.0-20240125205218-1f4bbc51b google.golang.org/genproto/googleapis/bytestream v0.0.0-20240125205218-1f4bbc51befe/go.mod h1:SCz6T5xjNXM4QFPRwxHcfChp7V+9DcXR3ay2TkHR8Tg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de h1:cZGRis4/ot9uVm639a+rHCUaG0JJHEsdyzSQTMX+suY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:H4O17MA/PE9BsGx3w+a+W2VOLLD1Qf7oJneAoU6WktY= +google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM= @@ -1537,6 +1744,8 @@ google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA5 google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +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= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -1549,6 +1758,7 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= @@ -1563,6 +1773,7 @@ gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMy gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= 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/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= @@ -1586,6 +1797,7 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= +grpc.go4.org v0.0.0-20170609214715-11d0a25b4919/go.mod h1:77eQGdRu53HpSqPFJFmuJdjuHRquDANNeA4x7B8WQ9o= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= @@ -1616,3 +1828,5 @@ 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= +sourcegraph.com/sqs/pbtypes v0.0.0-20180604144634-d3ebe8f20ae4/go.mod h1:ketZ/q3QxT9HOBeFhu6RdvsftgpsbFHBF5Cas6cDKZ0= From 86aa27ca8c28432d41302cf77df019fdf5175774 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Thu, 12 Dec 2024 16:15:54 +0100 Subject: [PATCH 03/12] update flow-go --- bootstrap/bootstrap.go | 8 ++++---- go.mod | 2 +- go.sum | 4 ++-- tests/go.mod | 4 ++-- tests/go.sum | 8 ++++---- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index d389fb9c4..2d8d0cf34 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -83,7 +83,7 @@ type EVMGatewayNodeBuilder struct { config.Config Logger zerolog.Logger componentBuilder component.ComponentManagerBuilder - components []cmd.NamedComponentFunc[config.Config] + components []cmd.NamedComponentFactory[config.Config] postShutdownFns []func() error modules []namedModuleFunc @@ -158,9 +158,9 @@ func (fnb *EVMGatewayNodeBuilder) initDB() error { } func (fnb *EVMGatewayNodeBuilder) Component(name string, f cmd.ReadyDoneFactory[config.Config]) *EVMGatewayNodeBuilder { - fnb.components = append(fnb.components, cmd.NamedComponentFunc[config.Config]{ - FN: f, - Name: name, + fnb.components = append(fnb.components, cmd.NamedComponentFactory[config.Config]{ + ComponentFactory: f, + Name: name, }) return fnb } diff --git a/go.mod b/go.mod index d444fddb6..bae916b0d 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/hashicorp/go-multierror v1.1.1 github.com/onflow/atree v0.8.0 github.com/onflow/cadence v1.2.2 - github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/prometheus/client_golang v1.18.0 diff --git a/go.sum b/go.sum index 007493180..c7002907d 100644 --- a/go.sum +++ b/go.sum @@ -743,8 +743,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d h1:HzvFJzdaRtVzJ0lXR8rvbT7maCjfX4CdZZqt/vD+7iM= -github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 h1:9JCb0qdirFgkTOQ+iKrnX+3Q5Xh9/hFCP9yM7Ky/QKA= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/tests/go.mod b/tests/go.mod index f93df8c28..8b05bd79f 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -6,9 +6,9 @@ require ( github.com/goccy/go-json v0.10.2 github.com/onflow/cadence v1.2.2 github.com/onflow/crypto v0.25.2 - github.com/onflow/flow-emulator v1.1.1-0.20241125195348-4e121ffb12af + github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f github.com/onflow/flow-evm-gateway v0.0.0-20240201154855-4d4d3d3f19c7 - github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/rs/zerolog v1.33.0 diff --git a/tests/go.sum b/tests/go.sum index 4739a5d6b..3930d8c40 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -906,14 +906,14 @@ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0 h1:R86HaOuk6vpuECZ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0/go.mod h1:9asTBnB6Tw2UlVVtQKyS/egYv3xr4zVlJnJ75z1dfac= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0 h1:u2DAG8pk0xFH7TwS70t1gSZ/FtIIZWMSNyiu4SeXBYg= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0/go.mod h1:pN768Al/wLRlf3bwugv9TyxniqJxMu4sxnX9eQJam64= -github.com/onflow/flow-emulator v1.1.1-0.20241125195348-4e121ffb12af h1:VO9GygdHSeO6UpN8PHHOIXJ5vVG/1LZd2KqrH6elPL8= -github.com/onflow/flow-emulator v1.1.1-0.20241125195348-4e121ffb12af/go.mod h1:8cKIt/iHkdsYmOETXe2D28UC3DvOSC/zpX6bi6DcNyE= +github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f h1:u102x6G1Y2TDH20ssNi4idh81ntP3EOgM7IWPrNlEnA= +github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f/go.mod h1:D/qlW9iRE2k1Nmhu1wGl/z5MXBOm+ysVODlIs3PLSJU= github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3SsEftzXG2JlmSe24= github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d h1:HzvFJzdaRtVzJ0lXR8rvbT7maCjfX4CdZZqt/vD+7iM= -github.com/onflow/flow-go v0.38.0-preview.0.0.20241127200547-bacae248590d/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 h1:9JCb0qdirFgkTOQ+iKrnX+3Q5Xh9/hFCP9yM7Ky/QKA= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= From f07606793b3b8688778d3cf709c460539dbcbd9f Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Fri, 13 Dec 2024 17:27:27 +0100 Subject: [PATCH 04/12] update tests --- bootstrap/bootstrap.go | 9 - cmd/run/cmd.go | 2 +- go.mod | 2 +- go.sum | 4 +- services/ingestion/engine.go | 2 +- services/ingestion/engine_test.go | 1205 ++++++++++++++--------------- tests/go.mod | 2 +- tests/go.sum | 4 +- tests/helpers.go | 23 +- tests/integration_test.go | 30 +- 10 files changed, 621 insertions(+), 662 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 2d8d0cf34..c0f4c5aa6 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -203,8 +203,6 @@ func NewEVMGatewayNodeBuilder( func (fnb *EVMGatewayNodeBuilder) Initialize() error { fnb.PrintBuildDetails() - fnb.EnqueueMetricsServerInit() - return nil } @@ -444,13 +442,6 @@ func (fnb *EVMGatewayNodeBuilder) ShutdownFunc(fn func() error) *EVMGatewayNodeB return fnb } -func (fnb *EVMGatewayNodeBuilder) EnqueueMetricsServerInit() { - fnb.Component("Metrics server", func(config config.Config) (module.ReadyDoneAware, error) { - server := metrics.NewServer(fnb.Logger, uint(config.MetricsPort)) - return server, nil - }) -} - func (fnb *EVMGatewayNodeBuilder) initMetrics() error { fnb.Metrics = metrics2.NewCollector(fnb.Logger) return nil diff --git a/cmd/run/cmd.go b/cmd/run/cmd.go index a25f18ead..e26e37dc1 100644 --- a/cmd/run/cmd.go +++ b/cmd/run/cmd.go @@ -49,7 +49,7 @@ var Cmd = &cobra.Command{ if err != nil { builder.Logger.Fatal().Err(err).Send() } - node.Run() + node.Run(command.Context()) return nil }, } diff --git a/go.mod b/go.mod index bae916b0d..3cb7df86d 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/hashicorp/go-multierror v1.1.1 github.com/onflow/atree v0.8.0 github.com/onflow/cadence v1.2.2 - github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/prometheus/client_golang v1.18.0 diff --git a/go.sum b/go.sum index c7002907d..bd23d04c6 100644 --- a/go.sum +++ b/go.sum @@ -743,8 +743,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 h1:9JCb0qdirFgkTOQ+iKrnX+3Q5Xh9/hFCP9yM7Ky/QKA= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 h1:kcAoHB/uEGP3wTJcv2aVtCHSgwY0bdn8qjw7GvvHfM8= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/services/ingestion/engine.go b/services/ingestion/engine.go index 8f1d4c6ec..94535c6af 100644 --- a/services/ingestion/engine.go +++ b/services/ingestion/engine.go @@ -135,7 +135,7 @@ func (e *Engine) run(ctx irrecoverable.SignalerContext, ready component.ReadyFun err := e.processEvents(events.Events) if err != nil { e.log.Error().Err(err).Msg("failed to process EVM events") - return + ctx.Throw(err) } } } diff --git a/services/ingestion/engine_test.go b/services/ingestion/engine_test.go index da77d6633..27987697a 100644 --- a/services/ingestion/engine_test.go +++ b/services/ingestion/engine_test.go @@ -1,587 +1,574 @@ package ingestion import ( + "context" "encoding/hex" "math/big" + "testing" + + "github.com/onflow/flow-go/module/irrecoverable" + + "github.com/onflow/flow-evm-gateway/storage" + + pebbleDB "github.com/cockroachdb/pebble" + "github.com/onflow/flow-go/fvm/environment" + "github.com/onflow/flow-go/fvm/evm" + "github.com/onflow/flow-go/fvm/evm/events" + flowGo "github.com/onflow/flow-go/model/flow" + + "github.com/onflow/flow-evm-gateway/metrics" + "github.com/onflow/flow-evm-gateway/services/ingestion/mocks" + "github.com/onflow/flow-evm-gateway/services/replayer" + "github.com/onflow/flow-evm-gateway/storage/pebble" "github.com/onflow/cadence" + "github.com/onflow/flow-evm-gateway/models" - "github.com/onflow/flow-go/fvm/evm/events" + + "github.com/onflow/flow-go-sdk" "github.com/onflow/flow-go/fvm/evm/types" - flowGo "github.com/onflow/flow-go/model/flow" gethCommon "github.com/onflow/go-ethereum/common" gethTypes "github.com/onflow/go-ethereum/core/types" + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + storageMock "github.com/onflow/flow-evm-gateway/storage/mocks" ) -// -//import ( -// "context" -// "encoding/hex" -// "encoding/json" -// "math/big" -// "testing" -// -// "github.com/onflow/flow-evm-gateway/storage" -// -// pebbleDB "github.com/cockroachdb/pebble" -// "github.com/onflow/flow-go/fvm/environment" -// "github.com/onflow/flow-go/fvm/evm" -// "github.com/onflow/flow-go/fvm/evm/events" -// flowGo "github.com/onflow/flow-go/model/flow" -// -// "github.com/onflow/flow-evm-gateway/metrics" -// "github.com/onflow/flow-evm-gateway/services/ingestion/mocks" -// "github.com/onflow/flow-evm-gateway/services/replayer" -// "github.com/onflow/flow-evm-gateway/storage/pebble" -// -// "github.com/onflow/cadence" -// -// "github.com/onflow/flow-evm-gateway/models" -// -// "github.com/onflow/flow-go-sdk" -// "github.com/onflow/flow-go/fvm/evm/types" -// gethCommon "github.com/onflow/go-ethereum/common" -// gethTypes "github.com/onflow/go-ethereum/core/types" -// "github.com/rs/zerolog" -// "github.com/stretchr/testify/assert" -// "github.com/stretchr/testify/mock" -// "github.com/stretchr/testify/require" -// -// storageMock "github.com/onflow/flow-evm-gateway/storage/mocks" -//) -// -//func TestSerialBlockIngestion(t *testing.T) { -// -// t.Run("successfully ingest serial blocks", func(t *testing.T) { -// receipts := &storageMock.ReceiptIndexer{} -// transactions := &storageMock.TransactionIndexer{} -// latestHeight := uint64(10) -// -// store, registerStore := setupStore(t) -// -// blocks := &storageMock.BlockIndexer{} -// blocks. -// On("LatestCadenceHeight"). -// Return(func() (uint64, error) { -// return latestHeight, nil -// }). -// Once() // make sure this isn't called multiple times -// -// traces := &storageMock.TraceIndexer{} -// -// eventsChan := make(chan models.BlockEvents) -// -// subscriber := &mocks.EventSubscriber{} -// subscriber. -// On("Subscribe", mock.Anything). -// Return(func(ctx context.Context) <-chan models.BlockEvents { -// return eventsChan -// }) -// -// engine := NewEventIngestionEngine( -// subscriber, -// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), -// store, -// registerStore, -// blocks, -// receipts, -// transactions, -// traces, -// models.NewPublisher[*models.Block](zerolog.Nop()), -// models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()), -// zerolog.Nop(), -// metrics.NopCollector, -// defaultReplayerConfig(), -// ) -// -// done := make(chan struct{}) -// go func() { -// err := engine.Run(context.Background()) -// assert.NoError(t, err) -// close(done) -// }() -// -// storedCounter := 0 -// runs := uint64(20) -// for i := latestHeight + 1; i < latestHeight+runs; i++ { -// cadenceHeight := i + 10 -// blockCdc, block, blockEvent, err := newBlock(i, nil) -// require.NoError(t, err) -// -// blocks. -// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). -// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { -// assert.Equal(t, block, storeBlock) -// assert.Equal(t, cadenceHeight, h) -// storedCounter++ -// return nil -// }). -// Once() -// -// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ -// Events: []flow.Event{{ -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }}, -// Height: cadenceHeight, -// }) -// } -// -// close(eventsChan) -// <-done -// assert.Equal(t, runs-1, uint64(storedCounter)) -// }) -// -// t.Run("fail with events out of sequence", func(t *testing.T) { -// receipts := &storageMock.ReceiptIndexer{} -// transactions := &storageMock.TransactionIndexer{} -// latestHeight := uint64(10) -// -// store, registerStore := setupStore(t) -// -// blocks := &storageMock.BlockIndexer{} -// blocks. -// On("LatestCadenceHeight"). -// Return(func() (uint64, error) { -// return latestHeight, nil -// }). -// Once() // make sure this isn't called multiple times -// -// traces := &storageMock.TraceIndexer{} -// -// eventsChan := make(chan models.BlockEvents) -// subscriber := &mocks.EventSubscriber{} -// subscriber. -// On("Subscribe", mock.Anything). -// Return(func(ctx context.Context) <-chan models.BlockEvents { -// return eventsChan -// }) -// -// engine := NewEventIngestionEngine( -// subscriber, -// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), -// store, -// registerStore, -// blocks, -// receipts, -// transactions, -// traces, -// models.NewPublisher[*models.Block](), -// models.NewPublisher[[]*gethTypes.Log](), -// zerolog.Nop(), -// metrics.NopCollector, -// defaultReplayerConfig(), -// ) -// -// waitErr := make(chan struct{}) -// // catch eventual error due to out of sequence block height -// go func() { -// err := engine.Run(context.Background()) -// assert.ErrorIs(t, err, models.ErrInvalidHeight) -// assert.ErrorContains(t, err, "invalid height: received new block: 20, non-sequential of latest block: 11") -// close(waitErr) -// }() -// -// // first create one successful block event -// blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) -// cadenceHeight := latestHeight + 10 -// require.NoError(t, err) -// -// blocks. -// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). -// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { -// assert.Equal(t, block, storeBlock) -// assert.Equal(t, cadenceHeight, h) -// return nil -// }). -// Once() // this should only be called for first valid block -// -// cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ -// Events: []flow.Event{{ -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }}, -// Height: cadenceHeight, -// }) -// require.NoError(t, err) -// -// eventsChan <- models.BlockEvents{ -// Events: cadenceEvents, -// } -// -// // fail with next block height being incorrect -// blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height -// require.NoError(t, err) -// -// cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ -// Events: []flow.Event{{ -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }}, -// Height: cadenceHeight + 1, -// }) -// require.NoError(t, err) -// -// eventsChan <- models.BlockEvents{ -// Events: cadenceEvents, -// } -// -// close(eventsChan) -// <-waitErr -// }) -//} -// -//func TestBlockAndTransactionIngestion(t *testing.T) { -// -// t.Run("successfully ingest transaction and block", func(t *testing.T) { -// receipts := &storageMock.ReceiptIndexer{} -// transactions := &storageMock.TransactionIndexer{} -// latestHeight := uint64(10) -// nextHeight := latestHeight + 1 -// blockID := flow.Identifier{0x01} -// -// store, registerStore := setupStore(t) -// -// blocks := &storageMock.BlockIndexer{} -// blocks. -// On("LatestCadenceHeight"). -// Return(func() (uint64, error) { -// return latestHeight, nil -// }). -// Once() // make sure this isn't called multiple times -// -// blocks. -// On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). -// Return(func(h uint64, _ *pebbleDB.Batch) error { -// assert.Equal(t, nextHeight, h) -// return nil -// }) -// -// eventsChan := make(chan models.BlockEvents) -// subscriber := &mocks.EventSubscriber{} -// subscriber. -// On("Subscribe", mock.Anything). -// Return(func(ctx context.Context) <-chan models.BlockEvents { -// return eventsChan -// }) -// -// txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) -// require.NoError(t, err) -// blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) -// require.NoError(t, err) -// -// traces := &storageMock.TraceIndexer{} -// traces. -// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). -// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { -// assert.Equal(t, transaction.Hash(), txID) -// return nil -// }) -// -// engine := NewEventIngestionEngine( -// subscriber, -// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), -// store, -// registerStore, -// blocks, -// receipts, -// transactions, -// traces, -// models.NewPublisher[*models.Block](), -// models.NewPublisher[[]*gethTypes.Log](), -// zerolog.Nop(), -// metrics.NopCollector, -// defaultReplayerConfig(), -// ) -// -// done := make(chan struct{}) -// go func() { -// err := engine.Run(context.Background()) -// assert.NoError(t, err) -// close(done) -// }() -// -// blocks. -// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). -// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { -// assert.Equal(t, block, storeBlock) -// assert.Equal(t, blockID, id) -// assert.Equal(t, nextHeight, h) -// return nil -// }). -// Once() -// -// transactions. -// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). -// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { -// assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal -// return nil -// }). -// Once() -// -// receipts. -// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). -// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { -// assert.Len(t, receipts, 1) -// rcp := receipts[0] -// -// assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) -// assert.Len(t, rcp.Logs, len(result.Logs)) -// assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) -// return nil -// }). -// Once() -// -// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ -// Events: []flow.Event{{ -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }, { -// Type: string(txEvent.Etype), -// Value: txCdc, -// }}, -// Height: nextHeight, -// BlockID: blockID, -// }) -// -// close(eventsChan) -// <-done -// }) -// -// t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { -// receipts := &storageMock.ReceiptIndexer{} -// transactions := &storageMock.TransactionIndexer{} -// latestHeight := uint64(10) -// nextHeight := latestHeight + 1 -// -// store, registerStore := setupStore(t) -// -// blocks := &storageMock.BlockIndexer{} -// blocks. -// On("LatestCadenceHeight"). -// Return(func() (uint64, error) { -// return latestHeight, nil -// }). -// On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). -// Return(func(h uint64) error { return nil }) -// -// eventsChan := make(chan models.BlockEvents) -// subscriber := &mocks.EventSubscriber{} -// subscriber. -// On("Subscribe", mock.Anything). -// Return(func(ctx context.Context) <-chan models.BlockEvents { -// return eventsChan -// }) -// -// txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) -// require.NoError(t, err) -// blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) -// require.NoError(t, err) -// -// traces := &storageMock.TraceIndexer{} -// traces. -// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). -// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { -// assert.Equal(t, transaction.Hash(), txID) -// return nil -// }) -// -// engine := NewEventIngestionEngine( -// subscriber, -// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), -// store, -// registerStore, -// blocks, -// receipts, -// transactions, -// traces, -// models.NewPublisher[*models.Block](), -// models.NewPublisher[[]*gethTypes.Log](), -// zerolog.Nop(), -// metrics.NopCollector, -// defaultReplayerConfig(), -// ) -// -// done := make(chan struct{}) -// go func() { -// err := engine.Run(context.Background()) -// assert.NoError(t, err) -// close(done) -// }() -// -// blocksFirst := false // flag indicating we stored block first -// blocks. -// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). -// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { -// blocksFirst = true -// return nil -// }). -// Once() -// -// transactions. -// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). -// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { -// require.True(t, blocksFirst) -// return nil -// }). -// Once() -// -// receipts. -// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). -// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { -// require.True(t, blocksFirst) -// return nil -// }). -// Once() -// -// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ -// Events: []flow.Event{ -// // first transaction -// { -// Type: string(txEvent.Etype), -// Value: txCdc, -// }, -// // and then block (out-of-order) -// { -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }, -// }, -// Height: nextHeight, -// }) -// -// close(eventsChan) -// <-done -// }) -// -// t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { -// receipts := &storageMock.ReceiptIndexer{} -// transactions := &storageMock.TransactionIndexer{} -// latestCadenceHeight := uint64(0) -// -// store, registerStore := setupStore(t) -// -// blocks := &storageMock.BlockIndexer{} -// blocks. -// On("LatestCadenceHeight"). -// Return(func() (uint64, error) { -// return latestCadenceHeight, nil -// }). -// Once() // make sure this isn't called multiple times -// -// traces := &storageMock.TraceIndexer{} -// -// eventsChan := make(chan models.BlockEvents) -// subscriber := &mocks.EventSubscriber{} -// subscriber. -// On("Subscribe", mock.Anything). -// Return(func(ctx context.Context) <-chan models.BlockEvents { -// return eventsChan -// }). -// Once() -// -// engine := NewEventIngestionEngine( -// subscriber, -// replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), -// store, -// registerStore, -// blocks, -// receipts, -// transactions, -// traces, -// models.NewPublisher[*models.Block](), -// models.NewPublisher[[]*gethTypes.Log](), -// zerolog.Nop(), -// metrics.NopCollector, -// defaultReplayerConfig(), -// ) -// -// done := make(chan struct{}) -// go func() { -// err := engine.Run(context.Background()) -// assert.NoError(t, err) -// close(done) -// }() -// -// evmHeight := uint64(0) -// events := make([]flow.Event, 0) -// blockIndexedFirst := false -// txsStored := 0 -// eventCount := 5 -// txHashes := make([]gethCommon.Hash, eventCount) -// -// for i := 0; i < eventCount; i++ { -// txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) -// txHashes[i] = res.TxHash -// require.NoError(t, err) -// -// // add a single transaction for each block -// transactions. -// On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). -// Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { -// assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal -// require.True(t, blockIndexedFirst) -// txsStored++ -// return nil -// }). -// Once() -// -// receipts. -// On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). -// Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). -// Once() -// -// traces. -// On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). -// Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { -// assert.Equal(t, transaction.Hash(), txID) -// return nil -// }) -// -// events = append(events, flow.Event{ -// Type: string(txEvent.Etype), -// Value: txCdc, -// }) -// } -// -// blocksStored := 0 -// blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) -// require.NoError(t, err) -// -// blocks. -// On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). -// Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { -// assert.Equal(t, block, storeBlock) -// assert.Equal(t, evmHeight, block.Height) -// assert.Equal(t, latestCadenceHeight+1, h) -// blockIndexedFirst = true -// blocksStored++ -// return nil -// }). -// Once() -// -// events = append(events, flow.Event{ -// Type: string(blockEvent.Etype), -// Value: blockCdc, -// }) -// -// // this messes up order of events to test if we still process events in-order -// // it will make transaction event first and then block event -// events[0], events[1] = events[1], events[0] -// // and it will make the first block be swapped with second block out-of-order -// events[1], events[2] = events[2], events[1] -// -// eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ -// Events: events, -// Height: latestCadenceHeight + 1, -// }) -// -// close(eventsChan) -// <-done -// assert.Equal(t, eventCount, txsStored) -// assert.Equal(t, 1, blocksStored) -// }) -//} +func TestSerialBlockIngestion(t *testing.T) { + + t.Run("successfully ingest serial blocks", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + receipts := &storageMock.ReceiptIndexer{} + transactions := &storageMock.TransactionIndexer{} + latestHeight := uint64(10) + + store, registerStore := setupStore(t) + + blocks := &storageMock.BlockIndexer{} + blocks. + On("LatestCadenceHeight"). + Return(func() (uint64, error) { + return latestHeight, nil + }). + Once() // make sure this isn't called multiple times + + traces := &storageMock.TraceIndexer{} + + eventsChan := make(chan models.BlockEvents) + + subscriber := &mocks.EventSubscriber{} + subscriber. + On("Subscribe", mock.Anything). + Return(func(ctx context.Context) <-chan models.BlockEvents { + return eventsChan + }) + + engine := NewEventIngestionEngine( + subscriber, + replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + store, + registerStore, + blocks, + receipts, + transactions, + traces, + models.NewPublisher[*models.Block](zerolog.Nop()), + models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()), + zerolog.Nop(), + metrics.NopCollector, + defaultReplayerConfig(), + ) + + engine.Start(ictx) + <-engine.Ready() + + storedCounter := 0 + runs := uint64(20) + for i := latestHeight + 1; i < latestHeight+runs; i++ { + cadenceHeight := i + 10 + blockCdc, block, blockEvent, err := newBlock(i, nil) + require.NoError(t, err) + + blocks. + On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + assert.Equal(t, block, storeBlock) + assert.Equal(t, cadenceHeight, h) + storedCounter++ + return nil + }). + Once() + + eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + Events: []flow.Event{{ + Type: string(blockEvent.Etype), + Value: blockCdc, + }}, + Height: cadenceHeight, + }) + } + + close(eventsChan) + <-engine.Done() + assert.Equal(t, runs-1, uint64(storedCounter)) + }) + + // t.Run("fail with events out of sequence", func(t *testing.T) { + // receipts := &storageMock.ReceiptIndexer{} + // transactions := &storageMock.TransactionIndexer{} + // latestHeight := uint64(10) + // + // store, registerStore := setupStore(t) + // + // blocks := &storageMock.BlockIndexer{} + // blocks. + // On("LatestCadenceHeight"). + // Return(func() (uint64, error) { + // return latestHeight, nil + // }). + // Once() // make sure this isn't called multiple times + // + // traces := &storageMock.TraceIndexer{} + // + // eventsChan := make(chan models.BlockEvents) + // subscriber := &mocks.EventSubscriber{} + // subscriber. + // On("Subscribe", mock.Anything). + // Return(func(ctx context.Context) <-chan models.BlockEvents { + // return eventsChan + // }) + // + // engine := NewEventIngestionEngine( + // subscriber, + // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + // store, + // registerStore, + // blocks, + // receipts, + // transactions, + // traces, + // models.NewPublisher[*models.Block](), + // models.NewPublisher[[]*gethTypes.Log](), + // zerolog.Nop(), + // metrics.NopCollector, + // defaultReplayerConfig(), + // ) + // + // waitErr := make(chan struct{}) + // // catch eventual error due to out of sequence block height + // go func() { + // err := engine.Run(context.Background()) + // assert.ErrorIs(t, err, models.ErrInvalidHeight) + // assert.ErrorContains(t, err, "invalid height: received new block: 20, non-sequential of latest block: 11") + // close(waitErr) + // }() + // + // // first create one successful block event + // blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) + // cadenceHeight := latestHeight + 10 + // require.NoError(t, err) + // + // blocks. + // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + // assert.Equal(t, block, storeBlock) + // assert.Equal(t, cadenceHeight, h) + // return nil + // }). + // Once() // this should only be called for first valid block + // + // cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ + // Events: []flow.Event{{ + // Type: string(blockEvent.Etype), + // Value: blockCdc, + // }}, + // Height: cadenceHeight, + // }) + // require.NoError(t, err) + // + // eventsChan <- models.BlockEvents{ + // Events: cadenceEvents, + // } + // + // // fail with next block height being incorrect + // blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height + // require.NoError(t, err) + // + // cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ + // Events: []flow.Event{{ + // Type: string(blockEvent.Etype), + // Value: blockCdc, + // }}, + // Height: cadenceHeight + 1, + // }) + // require.NoError(t, err) + // + // eventsChan <- models.BlockEvents{ + // Events: cadenceEvents, + // } + // + // close(eventsChan) + // <-waitErr + // }) + //} + // + //func TestBlockAndTransactionIngestion(t *testing.T) { + // + // t.Run("successfully ingest transaction and block", func(t *testing.T) { + // receipts := &storageMock.ReceiptIndexer{} + // transactions := &storageMock.TransactionIndexer{} + // latestHeight := uint64(10) + // nextHeight := latestHeight + 1 + // blockID := flow.Identifier{0x01} + // + // store, registerStore := setupStore(t) + // + // blocks := &storageMock.BlockIndexer{} + // blocks. + // On("LatestCadenceHeight"). + // Return(func() (uint64, error) { + // return latestHeight, nil + // }). + // Once() // make sure this isn't called multiple times + // + // blocks. + // On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). + // Return(func(h uint64, _ *pebbleDB.Batch) error { + // assert.Equal(t, nextHeight, h) + // return nil + // }) + // + // eventsChan := make(chan models.BlockEvents) + // subscriber := &mocks.EventSubscriber{} + // subscriber. + // On("Subscribe", mock.Anything). + // Return(func(ctx context.Context) <-chan models.BlockEvents { + // return eventsChan + // }) + // + // txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) + // require.NoError(t, err) + // blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) + // require.NoError(t, err) + // + // traces := &storageMock.TraceIndexer{} + // traces. + // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + // assert.Equal(t, transaction.Hash(), txID) + // return nil + // }) + // + // engine := NewEventIngestionEngine( + // subscriber, + // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + // store, + // registerStore, + // blocks, + // receipts, + // transactions, + // traces, + // models.NewPublisher[*models.Block](), + // models.NewPublisher[[]*gethTypes.Log](), + // zerolog.Nop(), + // metrics.NopCollector, + // defaultReplayerConfig(), + // ) + // + // done := make(chan struct{}) + // go func() { + // err := engine.Run(context.Background()) + // assert.NoError(t, err) + // close(done) + // }() + // + // blocks. + // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + // assert.Equal(t, block, storeBlock) + // assert.Equal(t, blockID, id) + // assert.Equal(t, nextHeight, h) + // return nil + // }). + // Once() + // + // transactions. + // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + // assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal + // return nil + // }). + // Once() + // + // receipts. + // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { + // assert.Len(t, receipts, 1) + // rcp := receipts[0] + // + // assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) + // assert.Len(t, rcp.Logs, len(result.Logs)) + // assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) + // return nil + // }). + // Once() + // + // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + // Events: []flow.Event{{ + // Type: string(blockEvent.Etype), + // Value: blockCdc, + // }, { + // Type: string(txEvent.Etype), + // Value: txCdc, + // }}, + // Height: nextHeight, + // BlockID: blockID, + // }) + // + // close(eventsChan) + // <-done + // }) + // + // t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { + // receipts := &storageMock.ReceiptIndexer{} + // transactions := &storageMock.TransactionIndexer{} + // latestHeight := uint64(10) + // nextHeight := latestHeight + 1 + // + // store, registerStore := setupStore(t) + // + // blocks := &storageMock.BlockIndexer{} + // blocks. + // On("LatestCadenceHeight"). + // Return(func() (uint64, error) { + // return latestHeight, nil + // }). + // On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). + // Return(func(h uint64) error { return nil }) + // + // eventsChan := make(chan models.BlockEvents) + // subscriber := &mocks.EventSubscriber{} + // subscriber. + // On("Subscribe", mock.Anything). + // Return(func(ctx context.Context) <-chan models.BlockEvents { + // return eventsChan + // }) + // + // txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) + // require.NoError(t, err) + // blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) + // require.NoError(t, err) + // + // traces := &storageMock.TraceIndexer{} + // traces. + // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + // assert.Equal(t, transaction.Hash(), txID) + // return nil + // }) + // + // engine := NewEventIngestionEngine( + // subscriber, + // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + // store, + // registerStore, + // blocks, + // receipts, + // transactions, + // traces, + // models.NewPublisher[*models.Block](), + // models.NewPublisher[[]*gethTypes.Log](), + // zerolog.Nop(), + // metrics.NopCollector, + // defaultReplayerConfig(), + // ) + // + // done := make(chan struct{}) + // go func() { + // err := engine.Run(context.Background()) + // assert.NoError(t, err) + // close(done) + // }() + // + // blocksFirst := false // flag indicating we stored block first + // blocks. + // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + // blocksFirst = true + // return nil + // }). + // Once() + // + // transactions. + // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + // require.True(t, blocksFirst) + // return nil + // }). + // Once() + // + // receipts. + // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { + // require.True(t, blocksFirst) + // return nil + // }). + // Once() + // + // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + // Events: []flow.Event{ + // // first transaction + // { + // Type: string(txEvent.Etype), + // Value: txCdc, + // }, + // // and then block (out-of-order) + // { + // Type: string(blockEvent.Etype), + // Value: blockCdc, + // }, + // }, + // Height: nextHeight, + // }) + // + // close(eventsChan) + // <-done + // }) + // + // t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { + // receipts := &storageMock.ReceiptIndexer{} + // transactions := &storageMock.TransactionIndexer{} + // latestCadenceHeight := uint64(0) + // + // store, registerStore := setupStore(t) + // + // blocks := &storageMock.BlockIndexer{} + // blocks. + // On("LatestCadenceHeight"). + // Return(func() (uint64, error) { + // return latestCadenceHeight, nil + // }). + // Once() // make sure this isn't called multiple times + // + // traces := &storageMock.TraceIndexer{} + // + // eventsChan := make(chan models.BlockEvents) + // subscriber := &mocks.EventSubscriber{} + // subscriber. + // On("Subscribe", mock.Anything). + // Return(func(ctx context.Context) <-chan models.BlockEvents { + // return eventsChan + // }). + // Once() + // + // engine := NewEventIngestionEngine( + // subscriber, + // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + // store, + // registerStore, + // blocks, + // receipts, + // transactions, + // traces, + // models.NewPublisher[*models.Block](), + // models.NewPublisher[[]*gethTypes.Log](), + // zerolog.Nop(), + // metrics.NopCollector, + // defaultReplayerConfig(), + // ) + // + // done := make(chan struct{}) + // go func() { + // err := engine.Run(context.Background()) + // assert.NoError(t, err) + // close(done) + // }() + // + // evmHeight := uint64(0) + // events := make([]flow.Event, 0) + // blockIndexedFirst := false + // txsStored := 0 + // eventCount := 5 + // txHashes := make([]gethCommon.Hash, eventCount) + // + // for i := 0; i < eventCount; i++ { + // txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) + // txHashes[i] = res.TxHash + // require.NoError(t, err) + // + // // add a single transaction for each block + // transactions. + // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + // assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal + // require.True(t, blockIndexedFirst) + // txsStored++ + // return nil + // }). + // Once() + // + // receipts. + // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). + // Once() + // + // traces. + // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + // assert.Equal(t, transaction.Hash(), txID) + // return nil + // }) + // + // events = append(events, flow.Event{ + // Type: string(txEvent.Etype), + // Value: txCdc, + // }) + // } + // + // blocksStored := 0 + // blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) + // require.NoError(t, err) + // + // blocks. + // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + // assert.Equal(t, block, storeBlock) + // assert.Equal(t, evmHeight, block.Height) + // assert.Equal(t, latestCadenceHeight+1, h) + // blockIndexedFirst = true + // blocksStored++ + // return nil + // }). + // Once() + // + // events = append(events, flow.Event{ + // Type: string(blockEvent.Etype), + // Value: blockCdc, + // }) + // + // // this messes up order of events to test if we still process events in-order + // // it will make transaction event first and then block event + // events[0], events[1] = events[1], events[0] + // // and it will make the first block be swapped with second block out-of-order + // events[1], events[2] = events[2], events[1] + // + // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + // Events: events, + // Height: latestCadenceHeight + 1, + // }) + // + // close(eventsChan) + // <-done + // assert.Equal(t, eventCount, txsStored) + // assert.Equal(t, 1, blocksStored) + // }) +} func newBlock(height uint64, txHashes []gethCommon.Hash) (cadence.Event, *models.Block, *events.Event, error) { gethBlock := types.NewBlock( @@ -640,41 +627,41 @@ func newTransaction(height uint64) (cadence.Event, *events.Event, models.Transac return cdcEv, ev, models.TransactionCall{Transaction: tx}, res, err } -//func defaultReplayerConfig() replayer.Config { -// return replayer.Config{ -// ChainID: flowGo.Emulator, -// RootAddr: evm.StorageAccountAddress(flowGo.Emulator), -// CallTracerCollector: replayer.NopTracer, -// ValidateResults: false, -// } -//} -// -//func setupStore(t *testing.T) (*pebble.Storage, *pebble.RegisterStorage) { -// db, err := pebble.OpenDB(t.TempDir()) -// require.NoError(t, err) -// store := pebble.New(db, zerolog.Nop()) -// -// storageAddress := evm.StorageAccountAddress(flowGo.Emulator) -// registerStore := pebble.NewRegisterStorage(store, storageAddress) -// snapshot, err := registerStore.GetSnapshotAt(0) -// require.NoError(t, err) -// delta := storage.NewRegisterDelta(snapshot) -// accountStatus := environment.NewAccountStatus() -// err = delta.SetValue( -// storageAddress[:], -// []byte(flowGo.AccountStatusKey), -// accountStatus.ToBytes(), -// ) -// require.NoError(t, err) -// -// batch := store.NewBatch() -// defer func() { -// require.NoError(t, batch.Close()) -// }() -// err = registerStore.Store(delta.GetUpdates(), 0, batch) -// require.NoError(t, err) -// err = batch.Commit(pebbleDB.Sync) -// require.NoError(t, err) -// -// return store, registerStore -//} +func defaultReplayerConfig() replayer.Config { + return replayer.Config{ + ChainID: flowGo.Emulator, + RootAddr: evm.StorageAccountAddress(flowGo.Emulator), + CallTracerCollector: replayer.NopTracer, + ValidateResults: false, + } +} + +func setupStore(t *testing.T) (*pebble.Storage, *pebble.RegisterStorage) { + db, err := pebble.OpenDB(t.TempDir()) + require.NoError(t, err) + store := pebble.New(db, zerolog.Nop()) + + storageAddress := evm.StorageAccountAddress(flowGo.Emulator) + registerStore := pebble.NewRegisterStorage(store, storageAddress) + snapshot, err := registerStore.GetSnapshotAt(0) + require.NoError(t, err) + delta := storage.NewRegisterDelta(snapshot) + accountStatus := environment.NewAccountStatus() + err = delta.SetValue( + storageAddress[:], + []byte(flowGo.AccountStatusKey), + accountStatus.ToBytes(), + ) + require.NoError(t, err) + + batch := store.NewBatch() + defer func() { + require.NoError(t, batch.Close()) + }() + err = registerStore.Store(delta.GetUpdates(), 0, batch) + require.NoError(t, err) + err = batch.Commit(pebbleDB.Sync) + require.NoError(t, err) + + return store, registerStore +} diff --git a/tests/go.mod b/tests/go.mod index 8b05bd79f..767287e78 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -8,7 +8,7 @@ require ( github.com/onflow/crypto v0.25.2 github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f github.com/onflow/flow-evm-gateway v0.0.0-20240201154855-4d4d3d3f19c7 - github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/rs/zerolog v1.33.0 diff --git a/tests/go.sum b/tests/go.sum index 3930d8c40..4a453f5eb 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -912,8 +912,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649 h1:9JCb0qdirFgkTOQ+iKrnX+3Q5Xh9/hFCP9yM7Ky/QKA= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241211211009-759daa544649/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 h1:kcAoHB/uEGP3wTJcv2aVtCHSgwY0bdn8qjw7GvvHfM8= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/tests/helpers.go b/tests/helpers.go index 883ec1a25..58923ab54 100644 --- a/tests/helpers.go +++ b/tests/helpers.go @@ -176,15 +176,7 @@ func servicesSetup(t *testing.T) (emulator.Emulator, func()) { TxStateValidation: config.LocalIndexValidation, } - bootstrapDone := make(chan struct{}) - go func() { - err = bootstrap.Run(ctx, cfg, func() { - close(bootstrapDone) - }) - require.NoError(t, err) - }() - - <-bootstrapDone + startGateway(t, ctx, cfg) return emu, func() { cancel() @@ -192,6 +184,19 @@ func servicesSetup(t *testing.T) (emulator.Emulator, func()) { } } +func startGateway(t *testing.T, ctx context.Context, cfg config.Config) { + builder := bootstrap.NewEVMGatewayNodeBuilder(cfg) + + err := builder.Initialize() + require.NoError(t, err) + + builder.LoadComponentsAndModules() + + node, err := builder.Build() + require.NoError(t, err) + go node.Run(ctx) +} + // executeTest will run the provided JS test file using mocha // and will report failure or success of the test. func executeTest(t *testing.T, testFile string) { diff --git a/tests/integration_test.go b/tests/integration_test.go index 5c9e31693..396021e62 100644 --- a/tests/integration_test.go +++ b/tests/integration_test.go @@ -82,15 +82,7 @@ func Test_ConcurrentTransactionSubmissionWithTxSeal(t *testing.T) { url: fmt.Sprintf("%s:%d", cfg.RPCHost, cfg.RPCPort), } - ready := make(chan struct{}) - go func() { - err := bootstrap.Run(ctx, cfg, func() { - close(ready) - }) - require.NoError(t, err) - }() - - <-ready + startGateway(t, ctx, cfg) eoaKey, err := crypto.HexToECDSA(eoaTestPrivateKey) require.NoError(t, err) @@ -289,15 +281,7 @@ func Test_EthClientTest(t *testing.T) { LogWriter: testLogWriter(), } - ready := make(chan struct{}) - go func() { - err := bootstrap.Run(ctx, cfg, func() { - close(ready) - }) - require.NoError(t, err) - }() - - <-ready + startGateway(t, ctx, cfg) ethClient, err := ethclient.Dial("http://127.0.0.1:8545") require.NoError(t, err) @@ -391,15 +375,7 @@ func Test_CloudKMSConcurrentTransactionSubmission(t *testing.T) { url: fmt.Sprintf("%s:%d", cfg.RPCHost, cfg.RPCPort), } - ready := make(chan struct{}) - go func() { - err := bootstrap.Run(ctx, cfg, func() { - close(ready) - }) - require.NoError(t, err) - }() - - <-ready + startGateway(t, ctx, cfg) eoaKey, err := crypto.HexToECDSA(eoaTestPrivateKey) require.NoError(t, err) From 91199a16f2935c71b07d226b5282f3648b5b1a62 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Fri, 13 Dec 2024 21:27:15 +0100 Subject: [PATCH 05/12] fix tests and rebase --- Makefile | 1 - api/server.go | 2 + bootstrap/bootstrap.go | 20 ++-- bootstrap/utils.go | 40 ------- cmd/run/cmd.go | 1 - models/mocks/Engine.go | 91 ---------------- services/ingestion/engine_test.go | 9 +- services/requester/key_store_component.go | 125 ++++++++++++++++++++++ services/signer/signer.go | 92 ---------------- tests/integration_test.go | 10 +- 10 files changed, 146 insertions(+), 245 deletions(-) delete mode 100644 bootstrap/utils.go delete mode 100644 models/mocks/Engine.go create mode 100644 services/requester/key_store_component.go delete mode 100644 services/signer/signer.go diff --git a/Makefile b/Makefile index 0b07c6797..4fbc9fc3f 100644 --- a/Makefile +++ b/Makefile @@ -103,7 +103,6 @@ generate: mockery --dir=storage --name=TraceIndexer --output=storage/mocks mockery --all --dir=services/traces --output=services/traces/mocks mockery --all --dir=services/ingestion --output=services/ingestion/mocks - mockery --dir=models --name=Engine --output=models/mocks .PHONY: ci ci: check-tidy test e2e-test diff --git a/api/server.go b/api/server.go index f964da028..be303c923 100644 --- a/api/server.go +++ b/api/server.go @@ -18,6 +18,8 @@ import ( "strings" "time" + "github.com/onflow/go-ethereum/core" + "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index c0f4c5aa6..aa72d087e 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -18,7 +18,6 @@ import ( "github.com/hashicorp/go-multierror" "github.com/onflow/flow-evm-gateway/api" "github.com/onflow/flow-evm-gateway/config" - "github.com/onflow/flow-go-sdk/crypto" "github.com/onflow/flow-go/cmd" "github.com/onflow/flow-go/fvm/environment" "github.com/onflow/flow-go/fvm/evm" @@ -34,7 +33,6 @@ import ( errs "github.com/onflow/flow-evm-gateway/models/errors" "github.com/onflow/flow-evm-gateway/services/ingestion" "github.com/onflow/flow-evm-gateway/services/replayer" - "github.com/onflow/flow-evm-gateway/services/signer" pebble2 "github.com/onflow/flow-evm-gateway/storage/pebble" ) @@ -92,8 +90,8 @@ type EVMGatewayNodeBuilder struct { Client *requester.CrossSporkClient Storages *Storages // Signer is used for signing flow transactions - Signer crypto.Signer Publishers *Publishers + Keystore *requester.KeyStoreComponent } func (fnb *EVMGatewayNodeBuilder) Build() (cmd.Node, error) { @@ -209,7 +207,7 @@ func (fnb *EVMGatewayNodeBuilder) Initialize() error { func (fnb *EVMGatewayNodeBuilder) LoadComponentsAndModules() { fnb.initPublishers() - fnb.Component("Transaction Signer", fnb.initSigner) + fnb.Component("Key Store", fnb.initKeyStore) fnb.Component("API Server", fnb.apiServerComponent) fnb.Component("Event Ingestion Engine", fnb.eventIngestionEngineComponent) fnb.Component("Metrics Server", fnb.metricsServerComponent) @@ -238,6 +236,7 @@ func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module. fnb.Client, fnb.Publishers.Transaction, log, + cfg, ) blocksProvider := replayer.NewBlocksProvider( @@ -251,11 +250,11 @@ func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module. blocksProvider, fnb.Client, cfg, - fnb.Signer, log, fnb.Storages.Blocks, txPool, fnb.Metrics, + fnb.Keystore.KeyStore, ) if err != nil { return nil, fmt.Errorf("failed to create EVM requester: %w", err) @@ -393,6 +392,7 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi fnb.Logger, fnb.Client, chainID, + fnb.Keystore, latestCadenceHeight, ) @@ -542,10 +542,12 @@ func (fnb *EVMGatewayNodeBuilder) initStorage() error { return nil } -func (fnb *EVMGatewayNodeBuilder) initSigner(config config.Config) (module.ReadyDoneAware, error) { - sig := signer.NewSigner(fnb.Logger, config) - fnb.Signer = sig - return sig, nil +func (fnb *EVMGatewayNodeBuilder) initKeyStore(cfg config.Config) (module.ReadyDoneAware, error) { + keystore := requester.NewKeyStoreComponent(fnb.Logger, cfg, fnb.Client) + + fnb.Keystore = keystore + + return keystore, nil } func (fnb *EVMGatewayNodeBuilder) initPublishers() { diff --git a/bootstrap/utils.go b/bootstrap/utils.go deleted file mode 100644 index 0c7d741c5..000000000 --- a/bootstrap/utils.go +++ /dev/null @@ -1,40 +0,0 @@ -package bootstrap - -import ( - "context" - "fmt" - - "github.com/onflow/flow-evm-gateway/config" - "github.com/onflow/flow-evm-gateway/services/requester" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/rs/zerolog" -) - -// createSigner creates the signer based on either a single coa key being -// provided and using a simple in-memory signer, or a Cloud KMS key being -// provided and using a Cloud KMS signer. -func createSigner( - ctx context.Context, - config config.Config, - logger zerolog.Logger, -) (crypto.Signer, error) { - var signer crypto.Signer - var err error - switch { - case config.COAKey != nil: - signer, err = crypto.NewInMemorySigner(config.COAKey, crypto.SHA3_256) - case config.COACloudKMSKey != nil: - signer, err = requester.NewKMSKeySigner( - ctx, - *config.COACloudKMSKey, - logger, - ) - default: - return nil, fmt.Errorf("must provide either single COA / Cloud KMS key") - } - if err != nil { - return nil, fmt.Errorf("failed to create a COA signer: %w", err) - } - - return signer, nil -} diff --git a/cmd/run/cmd.go b/cmd/run/cmd.go index e26e37dc1..c590a93af 100644 --- a/cmd/run/cmd.go +++ b/cmd/run/cmd.go @@ -1,7 +1,6 @@ package run import ( - "encoding/json" "fmt" "math/big" "os" diff --git a/models/mocks/Engine.go b/models/mocks/Engine.go deleted file mode 100644 index 5373c9a69..000000000 --- a/models/mocks/Engine.go +++ /dev/null @@ -1,91 +0,0 @@ -// Code generated by mockery v2.43.2. DO NOT EDIT. - -package mocks - -import ( - context "context" - - mock "github.com/stretchr/testify/mock" -) - -// Engine is an autogenerated mock type for the Engine type -type Engine struct { - mock.Mock -} - -// Done provides a mock function with given fields: -func (_m *Engine) Done() <-chan struct{} { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for Done") - } - - var r0 <-chan struct{} - if rf, ok := ret.Get(0).(func() <-chan struct{}); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(<-chan struct{}) - } - } - - return r0 -} - -// Ready provides a mock function with given fields: -func (_m *Engine) Ready() <-chan struct{} { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for Ready") - } - - var r0 <-chan struct{} - if rf, ok := ret.Get(0).(func() <-chan struct{}); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(<-chan struct{}) - } - } - - return r0 -} - -// Run provides a mock function with given fields: ctx -func (_m *Engine) Run(ctx context.Context) error { - ret := _m.Called(ctx) - - if len(ret) == 0 { - panic("no return value specified for Run") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context) error); ok { - r0 = rf(ctx) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Stop provides a mock function with given fields: -func (_m *Engine) Stop() { - _m.Called() -} - -// NewEngine creates a new instance of Engine. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -// The first argument is typically a *testing.T value. -func NewEngine(t interface { - mock.TestingT - Cleanup(func()) -}) *Engine { - mock := &Engine{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/services/ingestion/engine_test.go b/services/ingestion/engine_test.go index 27987697a..950f52262 100644 --- a/services/ingestion/engine_test.go +++ b/services/ingestion/engine_test.go @@ -69,6 +69,11 @@ func TestSerialBlockIngestion(t *testing.T) { return eventsChan }) + blocksPublisher := models.NewPublisher[*models.Block](zerolog.Nop()) + blocksPublisher.Start(ictx) + logsPublisher := models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()) + logsPublisher.Start(ictx) + engine := NewEventIngestionEngine( subscriber, replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), @@ -78,8 +83,8 @@ func TestSerialBlockIngestion(t *testing.T) { receipts, transactions, traces, - models.NewPublisher[*models.Block](zerolog.Nop()), - models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()), + blocksPublisher, + logsPublisher, zerolog.Nop(), metrics.NopCollector, defaultReplayerConfig(), diff --git a/services/requester/key_store_component.go b/services/requester/key_store_component.go new file mode 100644 index 000000000..8324c827a --- /dev/null +++ b/services/requester/key_store_component.go @@ -0,0 +1,125 @@ +package requester + +import ( + "context" + "fmt" + + "github.com/onflow/flow-evm-gateway/config" + "github.com/onflow/flow-go-sdk/access" + "github.com/onflow/flow-go-sdk/crypto" + "github.com/onflow/flow-go/module/component" + "github.com/onflow/flow-go/module/irrecoverable" + "github.com/rs/zerolog" +) + +type KeyStoreComponent struct { + *KeyStore + + log zerolog.Logger + config config.Config + client access.Client + startupCompleted chan struct{} +} + +var _ component.Component = (*KeyStoreComponent)(nil) + +func NewKeyStoreComponent(log zerolog.Logger, config config.Config, client access.Client) *KeyStoreComponent { + ks := &KeyStoreComponent{ + log: log, + config: config, + client: client, + startupCompleted: make(chan struct{}), + } + + return ks +} + +func (k *KeyStoreComponent) Start(ctx irrecoverable.SignalerContext) { + defer close(k.startupCompleted) + + k.log.Info().Msg("starting key store component") + + accountKeys := make([]*AccountKey, 0) + account, err := k.client.GetAccount(ctx, k.config.COAAddress) + if err != nil { + ctx.Throw(fmt.Errorf( + "failed to get signer info account for address: %s, with: %w", + k.config.COAAddress, + err, + )) + + return + } + signer, err := createSigner(ctx, k.config, k.log) + + if err != nil { + ctx.Throw(err) + + return + } + for _, key := range account.Keys { + accountKeys = append(accountKeys, &AccountKey{ + AccountKey: *key, + Address: k.config.COAAddress, + Signer: signer, + }) + } + + k.KeyStore = NewKeyStore(accountKeys) + +} + +func (k *KeyStoreComponent) Ready() <-chan struct{} { + ready := make(chan struct{}) + + go func() { + <-k.startupCompleted + close(ready) + }() + + return ready +} + +func (k *KeyStoreComponent) Done() <-chan struct{} { + done := make(chan struct{}) + + go func() { + <-k.startupCompleted + + // This is where we would close the KMS client connection, + // but it currently does not have a close method + + close(done) + }() + + return done +} + +// createSigner creates the signer based on either a single coa key being +// provided and using a simple in-memory signer, or a Cloud KMS key being +// provided and using a Cloud KMS signer. +func createSigner( + ctx context.Context, + config config.Config, + logger zerolog.Logger, +) (crypto.Signer, error) { + var signer crypto.Signer + var err error + switch { + case config.COAKey != nil: + signer, err = crypto.NewInMemorySigner(config.COAKey, crypto.SHA3_256) + case config.COACloudKMSKey != nil: + signer, err = NewKMSKeySigner( + ctx, + *config.COACloudKMSKey, + logger, + ) + default: + return nil, fmt.Errorf("must provide either single COA / Cloud KMS key") + } + if err != nil { + return nil, fmt.Errorf("failed to create a COA signer: %w", err) + } + + return signer, nil +} diff --git a/services/signer/signer.go b/services/signer/signer.go deleted file mode 100644 index 8bcbefd87..000000000 --- a/services/signer/signer.go +++ /dev/null @@ -1,92 +0,0 @@ -package signer - -import ( - "fmt" - - "github.com/onflow/flow-evm-gateway/config" - "github.com/onflow/flow-evm-gateway/services/requester" - "github.com/onflow/flow-go-sdk/crypto" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/rs/zerolog" -) - -type Signer struct { - crypto.Signer - - log zerolog.Logger - config config.Config - - startupCompleted chan struct{} - closeKMSClient func() -} - -var _ component.Component = (*Signer)(nil) -var _ crypto.Signer = (*Signer)(nil) - -func NewSigner(log zerolog.Logger, config config.Config) *Signer { - return &Signer{ - log: log, - config: config, - startupCompleted: make(chan struct{}), - } -} - -func (s *Signer) Start(ctx irrecoverable.SignalerContext) { - cfg := s.config - defer close(s.startupCompleted) - - var err error - switch { - case cfg.COAKey != nil: - s.Signer, err = crypto.NewInMemorySigner(cfg.COAKey, crypto.SHA3_256) - case cfg.COAKeys != nil: - s.Signer, err = requester.NewKeyRotationSigner(cfg.COAKeys, crypto.SHA3_256) - case len(cfg.COACloudKMSKeys) > 0: - var signer *requester.KMSKeyRotationSigner - signer, err = requester.NewKMSKeyRotationSigner( - ctx, - cfg.COACloudKMSKeys, - s.log, - ) - s.closeKMSClient = func() { - // TODO(JanezP): this should definitely be a closer. Open a PR in the sdk - // signer.Close() - } - s.Signer = signer - default: - ctx.Throw(fmt.Errorf("must provide either single COA / keylist of COA keys / COA cloud KMS keys")) - return - } - if err != nil { - ctx.Throw(fmt.Errorf("failed to create a COA signer: %w", err)) - return - } -} - -func (s *Signer) Ready() <-chan struct{} { - ready := make(chan struct{}) - - go func() { - <-s.startupCompleted - close(ready) - }() - - return ready -} - -func (s *Signer) Done() <-chan struct{} { - done := make(chan struct{}) - - go func() { - <-s.startupCompleted - - if s.closeKMSClient != nil { - s.closeKMSClient() - } - - close(done) - }() - - return done -} diff --git a/tests/integration_test.go b/tests/integration_test.go index 396021e62..4ef66dd2c 100644 --- a/tests/integration_test.go +++ b/tests/integration_test.go @@ -184,15 +184,7 @@ func Test_ConcurrentTransactionSubmissionWithLocalIndex(t *testing.T) { url: fmt.Sprintf("%s:%d", cfg.RPCHost, cfg.RPCPort), } - ready := make(chan struct{}) - go func() { - err = bootstrap.Run(ctx, cfg, func() { - close(ready) - }) - require.NoError(t, err) - }() - - <-ready + startGateway(t, ctx, cfg) time.Sleep(3 * time.Second) // some time to startup From a333610220dd2c88dae8bcfc54314e88a629647d Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Fri, 13 Dec 2024 21:38:55 +0100 Subject: [PATCH 06/12] fix rebase --- bootstrap/bootstrap.go | 1 + 1 file changed, 1 insertion(+) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index aa72d087e..77ed3bf21 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -321,6 +321,7 @@ func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module. cfg, log, fnb.Metrics, + ratelimiter, ) var walletAPI *api.WalletAPI From 2e04cb509fa7979a150e7b9136ce250edca3994a Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Mon, 16 Dec 2024 15:56:24 +0100 Subject: [PATCH 07/12] fix remaining tests --- services/ingestion/engine_test.go | 922 +++++++++++++++--------------- tests/go.mod | 2 +- tests/go.sum | 4 +- 3 files changed, 476 insertions(+), 452 deletions(-) diff --git a/services/ingestion/engine_test.go b/services/ingestion/engine_test.go index 950f52262..fd2c62d69 100644 --- a/services/ingestion/engine_test.go +++ b/services/ingestion/engine_test.go @@ -3,6 +3,7 @@ package ingestion import ( "context" "encoding/hex" + "encoding/json" "math/big" "testing" @@ -124,455 +125,478 @@ func TestSerialBlockIngestion(t *testing.T) { assert.Equal(t, runs-1, uint64(storedCounter)) }) - // t.Run("fail with events out of sequence", func(t *testing.T) { - // receipts := &storageMock.ReceiptIndexer{} - // transactions := &storageMock.TransactionIndexer{} - // latestHeight := uint64(10) - // - // store, registerStore := setupStore(t) - // - // blocks := &storageMock.BlockIndexer{} - // blocks. - // On("LatestCadenceHeight"). - // Return(func() (uint64, error) { - // return latestHeight, nil - // }). - // Once() // make sure this isn't called multiple times - // - // traces := &storageMock.TraceIndexer{} - // - // eventsChan := make(chan models.BlockEvents) - // subscriber := &mocks.EventSubscriber{} - // subscriber. - // On("Subscribe", mock.Anything). - // Return(func(ctx context.Context) <-chan models.BlockEvents { - // return eventsChan - // }) - // - // engine := NewEventIngestionEngine( - // subscriber, - // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - // store, - // registerStore, - // blocks, - // receipts, - // transactions, - // traces, - // models.NewPublisher[*models.Block](), - // models.NewPublisher[[]*gethTypes.Log](), - // zerolog.Nop(), - // metrics.NopCollector, - // defaultReplayerConfig(), - // ) - // - // waitErr := make(chan struct{}) - // // catch eventual error due to out of sequence block height - // go func() { - // err := engine.Run(context.Background()) - // assert.ErrorIs(t, err, models.ErrInvalidHeight) - // assert.ErrorContains(t, err, "invalid height: received new block: 20, non-sequential of latest block: 11") - // close(waitErr) - // }() - // - // // first create one successful block event - // blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) - // cadenceHeight := latestHeight + 10 - // require.NoError(t, err) - // - // blocks. - // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - // assert.Equal(t, block, storeBlock) - // assert.Equal(t, cadenceHeight, h) - // return nil - // }). - // Once() // this should only be called for first valid block - // - // cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ - // Events: []flow.Event{{ - // Type: string(blockEvent.Etype), - // Value: blockCdc, - // }}, - // Height: cadenceHeight, - // }) - // require.NoError(t, err) - // - // eventsChan <- models.BlockEvents{ - // Events: cadenceEvents, - // } - // - // // fail with next block height being incorrect - // blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height - // require.NoError(t, err) - // - // cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ - // Events: []flow.Event{{ - // Type: string(blockEvent.Etype), - // Value: blockCdc, - // }}, - // Height: cadenceHeight + 1, - // }) - // require.NoError(t, err) - // - // eventsChan <- models.BlockEvents{ - // Events: cadenceEvents, - // } - // - // close(eventsChan) - // <-waitErr - // }) - //} - // - //func TestBlockAndTransactionIngestion(t *testing.T) { - // - // t.Run("successfully ingest transaction and block", func(t *testing.T) { - // receipts := &storageMock.ReceiptIndexer{} - // transactions := &storageMock.TransactionIndexer{} - // latestHeight := uint64(10) - // nextHeight := latestHeight + 1 - // blockID := flow.Identifier{0x01} - // - // store, registerStore := setupStore(t) - // - // blocks := &storageMock.BlockIndexer{} - // blocks. - // On("LatestCadenceHeight"). - // Return(func() (uint64, error) { - // return latestHeight, nil - // }). - // Once() // make sure this isn't called multiple times - // - // blocks. - // On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). - // Return(func(h uint64, _ *pebbleDB.Batch) error { - // assert.Equal(t, nextHeight, h) - // return nil - // }) - // - // eventsChan := make(chan models.BlockEvents) - // subscriber := &mocks.EventSubscriber{} - // subscriber. - // On("Subscribe", mock.Anything). - // Return(func(ctx context.Context) <-chan models.BlockEvents { - // return eventsChan - // }) - // - // txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) - // require.NoError(t, err) - // blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) - // require.NoError(t, err) - // - // traces := &storageMock.TraceIndexer{} - // traces. - // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - // assert.Equal(t, transaction.Hash(), txID) - // return nil - // }) - // - // engine := NewEventIngestionEngine( - // subscriber, - // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - // store, - // registerStore, - // blocks, - // receipts, - // transactions, - // traces, - // models.NewPublisher[*models.Block](), - // models.NewPublisher[[]*gethTypes.Log](), - // zerolog.Nop(), - // metrics.NopCollector, - // defaultReplayerConfig(), - // ) - // - // done := make(chan struct{}) - // go func() { - // err := engine.Run(context.Background()) - // assert.NoError(t, err) - // close(done) - // }() - // - // blocks. - // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - // assert.Equal(t, block, storeBlock) - // assert.Equal(t, blockID, id) - // assert.Equal(t, nextHeight, h) - // return nil - // }). - // Once() - // - // transactions. - // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - // assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal - // return nil - // }). - // Once() - // - // receipts. - // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { - // assert.Len(t, receipts, 1) - // rcp := receipts[0] - // - // assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) - // assert.Len(t, rcp.Logs, len(result.Logs)) - // assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) - // return nil - // }). - // Once() - // - // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - // Events: []flow.Event{{ - // Type: string(blockEvent.Etype), - // Value: blockCdc, - // }, { - // Type: string(txEvent.Etype), - // Value: txCdc, - // }}, - // Height: nextHeight, - // BlockID: blockID, - // }) - // - // close(eventsChan) - // <-done - // }) - // - // t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { - // receipts := &storageMock.ReceiptIndexer{} - // transactions := &storageMock.TransactionIndexer{} - // latestHeight := uint64(10) - // nextHeight := latestHeight + 1 - // - // store, registerStore := setupStore(t) - // - // blocks := &storageMock.BlockIndexer{} - // blocks. - // On("LatestCadenceHeight"). - // Return(func() (uint64, error) { - // return latestHeight, nil - // }). - // On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). - // Return(func(h uint64) error { return nil }) - // - // eventsChan := make(chan models.BlockEvents) - // subscriber := &mocks.EventSubscriber{} - // subscriber. - // On("Subscribe", mock.Anything). - // Return(func(ctx context.Context) <-chan models.BlockEvents { - // return eventsChan - // }) - // - // txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) - // require.NoError(t, err) - // blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) - // require.NoError(t, err) - // - // traces := &storageMock.TraceIndexer{} - // traces. - // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - // assert.Equal(t, transaction.Hash(), txID) - // return nil - // }) - // - // engine := NewEventIngestionEngine( - // subscriber, - // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - // store, - // registerStore, - // blocks, - // receipts, - // transactions, - // traces, - // models.NewPublisher[*models.Block](), - // models.NewPublisher[[]*gethTypes.Log](), - // zerolog.Nop(), - // metrics.NopCollector, - // defaultReplayerConfig(), - // ) - // - // done := make(chan struct{}) - // go func() { - // err := engine.Run(context.Background()) - // assert.NoError(t, err) - // close(done) - // }() - // - // blocksFirst := false // flag indicating we stored block first - // blocks. - // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - // blocksFirst = true - // return nil - // }). - // Once() - // - // transactions. - // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - // require.True(t, blocksFirst) - // return nil - // }). - // Once() - // - // receipts. - // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { - // require.True(t, blocksFirst) - // return nil - // }). - // Once() - // - // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - // Events: []flow.Event{ - // // first transaction - // { - // Type: string(txEvent.Etype), - // Value: txCdc, - // }, - // // and then block (out-of-order) - // { - // Type: string(blockEvent.Etype), - // Value: blockCdc, - // }, - // }, - // Height: nextHeight, - // }) - // - // close(eventsChan) - // <-done - // }) - // - // t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { - // receipts := &storageMock.ReceiptIndexer{} - // transactions := &storageMock.TransactionIndexer{} - // latestCadenceHeight := uint64(0) - // - // store, registerStore := setupStore(t) - // - // blocks := &storageMock.BlockIndexer{} - // blocks. - // On("LatestCadenceHeight"). - // Return(func() (uint64, error) { - // return latestCadenceHeight, nil - // }). - // Once() // make sure this isn't called multiple times - // - // traces := &storageMock.TraceIndexer{} - // - // eventsChan := make(chan models.BlockEvents) - // subscriber := &mocks.EventSubscriber{} - // subscriber. - // On("Subscribe", mock.Anything). - // Return(func(ctx context.Context) <-chan models.BlockEvents { - // return eventsChan - // }). - // Once() - // - // engine := NewEventIngestionEngine( - // subscriber, - // replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), - // store, - // registerStore, - // blocks, - // receipts, - // transactions, - // traces, - // models.NewPublisher[*models.Block](), - // models.NewPublisher[[]*gethTypes.Log](), - // zerolog.Nop(), - // metrics.NopCollector, - // defaultReplayerConfig(), - // ) - // - // done := make(chan struct{}) - // go func() { - // err := engine.Run(context.Background()) - // assert.NoError(t, err) - // close(done) - // }() - // - // evmHeight := uint64(0) - // events := make([]flow.Event, 0) - // blockIndexedFirst := false - // txsStored := 0 - // eventCount := 5 - // txHashes := make([]gethCommon.Hash, eventCount) - // - // for i := 0; i < eventCount; i++ { - // txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) - // txHashes[i] = res.TxHash - // require.NoError(t, err) - // - // // add a single transaction for each block - // transactions. - // On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). - // Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { - // assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal - // require.True(t, blockIndexedFirst) - // txsStored++ - // return nil - // }). - // Once() - // - // receipts. - // On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). - // Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). - // Once() - // - // traces. - // On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). - // Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { - // assert.Equal(t, transaction.Hash(), txID) - // return nil - // }) - // - // events = append(events, flow.Event{ - // Type: string(txEvent.Etype), - // Value: txCdc, - // }) - // } - // - // blocksStored := 0 - // blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) - // require.NoError(t, err) - // - // blocks. - // On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). - // Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { - // assert.Equal(t, block, storeBlock) - // assert.Equal(t, evmHeight, block.Height) - // assert.Equal(t, latestCadenceHeight+1, h) - // blockIndexedFirst = true - // blocksStored++ - // return nil - // }). - // Once() - // - // events = append(events, flow.Event{ - // Type: string(blockEvent.Etype), - // Value: blockCdc, - // }) - // - // // this messes up order of events to test if we still process events in-order - // // it will make transaction event first and then block event - // events[0], events[1] = events[1], events[0] - // // and it will make the first block be swapped with second block out-of-order - // events[1], events[2] = events[2], events[1] - // - // eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ - // Events: events, - // Height: latestCadenceHeight + 1, - // }) - // - // close(eventsChan) - // <-done - // assert.Equal(t, eventCount, txsStored) - // assert.Equal(t, 1, blocksStored) - // }) + t.Run("fail with events out of sequence", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + ictx.On("Throw", mock.Anything).Run(func(args mock.Arguments) { + err := args.Get(0).(error) + require.ErrorContains(t, err, "failed to index events for cadence block 21: invalid height: received new block: 20, non-sequential of latest block: 11") + }) + + receipts := &storageMock.ReceiptIndexer{} + transactions := &storageMock.TransactionIndexer{} + latestHeight := uint64(10) + + store, registerStore := setupStore(t) + + blocks := &storageMock.BlockIndexer{} + blocks. + On("LatestCadenceHeight"). + Return(func() (uint64, error) { + return latestHeight, nil + }). + Once() // make sure this isn't called multiple times + + traces := &storageMock.TraceIndexer{} + + eventsChan := make(chan models.BlockEvents) + subscriber := &mocks.EventSubscriber{} + subscriber. + On("Subscribe", mock.Anything). + Return(func(ctx context.Context) <-chan models.BlockEvents { + return eventsChan + }) + + blocksPublisher := models.NewPublisher[*models.Block](zerolog.Nop()) + blocksPublisher.Start(ictx) + logsPublisher := models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()) + logsPublisher.Start(ictx) + + engine := NewEventIngestionEngine( + subscriber, + replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + store, + registerStore, + blocks, + receipts, + transactions, + traces, + blocksPublisher, + logsPublisher, + zerolog.Nop(), + metrics.NopCollector, + defaultReplayerConfig(), + ) + + engine.Start(ictx) + <-engine.Ready() + + // first create one successful block event + blockCdc, block, blockEvent, err := newBlock(latestHeight+1, nil) + cadenceHeight := latestHeight + 10 + require.NoError(t, err) + + blocks. + On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + assert.Equal(t, block, storeBlock) + assert.Equal(t, cadenceHeight, h) + return nil + }). + Once() // this should only be called for first valid block + + cadenceEvents, err := models.NewCadenceEvents(flow.BlockEvents{ + Events: []flow.Event{{ + Type: string(blockEvent.Etype), + Value: blockCdc, + }}, + Height: cadenceHeight, + }) + require.NoError(t, err) + + eventsChan <- models.BlockEvents{ + Events: cadenceEvents, + } + + // fail with next block height being incorrect + blockCdc, _, blockEvent, err = newBlock(latestHeight+10, nil) // not sequential next block height + require.NoError(t, err) + + cadenceEvents, err = models.NewCadenceEvents(flow.BlockEvents{ + Events: []flow.Event{{ + Type: string(blockEvent.Etype), + Value: blockCdc, + }}, + Height: cadenceHeight + 1, + }) + require.NoError(t, err) + + eventsChan <- models.BlockEvents{ + Events: cadenceEvents, + } + + close(eventsChan) + <-engine.Done() + }) +} + +func TestBlockAndTransactionIngestion(t *testing.T) { + + t.Run("successfully ingest transaction and block", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + receipts := &storageMock.ReceiptIndexer{} + transactions := &storageMock.TransactionIndexer{} + latestHeight := uint64(10) + nextHeight := latestHeight + 1 + blockID := flow.Identifier{0x01} + + store, registerStore := setupStore(t) + + blocks := &storageMock.BlockIndexer{} + blocks. + On("LatestCadenceHeight"). + Return(func() (uint64, error) { + return latestHeight, nil + }). + Once() // make sure this isn't called multiple times + + blocks. + On("SetLatestCadenceHeight", mock.AnythingOfType("uint64"), mock.Anything). + Return(func(h uint64, _ *pebbleDB.Batch) error { + assert.Equal(t, nextHeight, h) + return nil + }) + + eventsChan := make(chan models.BlockEvents) + subscriber := &mocks.EventSubscriber{} + subscriber. + On("Subscribe", mock.Anything). + Return(func(ctx context.Context) <-chan models.BlockEvents { + return eventsChan + }) + + txCdc, txEvent, transaction, result, err := newTransaction(nextHeight) + require.NoError(t, err) + blockCdc, block, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{result.TxHash}) + require.NoError(t, err) + + traces := &storageMock.TraceIndexer{} + traces. + On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + assert.Equal(t, transaction.Hash(), txID) + return nil + }) + + blocksPublisher := models.NewPublisher[*models.Block](zerolog.Nop()) + blocksPublisher.Start(ictx) + logsPublisher := models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()) + logsPublisher.Start(ictx) + + engine := NewEventIngestionEngine( + subscriber, + replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + store, + registerStore, + blocks, + receipts, + transactions, + traces, + blocksPublisher, + logsPublisher, + zerolog.Nop(), + metrics.NopCollector, + defaultReplayerConfig(), + ) + + engine.Start(ictx) + <-engine.Ready() + + blocks. + On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + assert.Equal(t, block, storeBlock) + assert.Equal(t, blockID, id) + assert.Equal(t, nextHeight, h) + return nil + }). + Once() + + transactions. + On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal + return nil + }). + Once() + + receipts. + On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { + assert.Len(t, receipts, 1) + rcp := receipts[0] + + assert.Equal(t, nextHeight, rcp.BlockNumber.Uint64()) + assert.Len(t, rcp.Logs, len(result.Logs)) + assert.Equal(t, result.DeployedContractAddress.ToCommon().String(), rcp.ContractAddress.String()) + return nil + }). + Once() + + eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + Events: []flow.Event{{ + Type: string(blockEvent.Etype), + Value: blockCdc, + }, { + Type: string(txEvent.Etype), + Value: txCdc, + }}, + Height: nextHeight, + BlockID: blockID, + }) + + close(eventsChan) + <-engine.Done() + }) + + t.Run("ingest block first and then transaction even if received out-of-order", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + receipts := &storageMock.ReceiptIndexer{} + transactions := &storageMock.TransactionIndexer{} + latestHeight := uint64(10) + nextHeight := latestHeight + 1 + + store, registerStore := setupStore(t) + + blocks := &storageMock.BlockIndexer{} + blocks. + On("LatestCadenceHeight"). + Return(func() (uint64, error) { + return latestHeight, nil + }). + On("SetLatestCadenceHeight", mock.AnythingOfType("uint64")). + Return(func(h uint64) error { return nil }) + + eventsChan := make(chan models.BlockEvents) + subscriber := &mocks.EventSubscriber{} + subscriber. + On("Subscribe", mock.Anything). + Return(func(ctx context.Context) <-chan models.BlockEvents { + return eventsChan + }) + + txCdc, txEvent, transaction, res, err := newTransaction(nextHeight) + require.NoError(t, err) + blockCdc, _, blockEvent, err := newBlock(nextHeight, []gethCommon.Hash{res.TxHash}) + require.NoError(t, err) + + traces := &storageMock.TraceIndexer{} + traces. + On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + assert.Equal(t, transaction.Hash(), txID) + return nil + }) + + blocksPublisher := models.NewPublisher[*models.Block](zerolog.Nop()) + blocksPublisher.Start(ictx) + logsPublisher := models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()) + logsPublisher.Start(ictx) + + engine := NewEventIngestionEngine( + subscriber, + replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + store, + registerStore, + blocks, + receipts, + transactions, + traces, + blocksPublisher, + logsPublisher, + zerolog.Nop(), + metrics.NopCollector, + defaultReplayerConfig(), + ) + + engine.Start(ictx) + <-engine.Ready() + + blocksFirst := false // flag indicating we stored block first + blocks. + On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + blocksFirst = true + return nil + }). + Once() + + transactions. + On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + require.True(t, blocksFirst) + return nil + }). + Once() + + receipts. + On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { + require.True(t, blocksFirst) + return nil + }). + Once() + + eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + Events: []flow.Event{ + // first transaction + { + Type: string(txEvent.Etype), + Value: txCdc, + }, + // and then block (out-of-order) + { + Type: string(blockEvent.Etype), + Value: blockCdc, + }, + }, + Height: nextHeight, + }) + + close(eventsChan) + <-engine.Done() + }) + + t.Run("ingest block and multiple transactions in same block event, even if out-of-order", func(t *testing.T) { + ctx := context.Background() + ictx, cancel := irrecoverable.NewMockSignalerContextWithCancel(t, ctx) + defer cancel() + + receipts := &storageMock.ReceiptIndexer{} + transactions := &storageMock.TransactionIndexer{} + latestCadenceHeight := uint64(0) + + store, registerStore := setupStore(t) + + blocks := &storageMock.BlockIndexer{} + blocks. + On("LatestCadenceHeight"). + Return(func() (uint64, error) { + return latestCadenceHeight, nil + }). + Once() // make sure this isn't called multiple times + + traces := &storageMock.TraceIndexer{} + + eventsChan := make(chan models.BlockEvents) + subscriber := &mocks.EventSubscriber{} + subscriber. + On("Subscribe", mock.Anything). + Return(func(ctx context.Context) <-chan models.BlockEvents { + return eventsChan + }). + Once() + + blocksPublisher := models.NewPublisher[*models.Block](zerolog.Nop()) + blocksPublisher.Start(ictx) + logsPublisher := models.NewPublisher[[]*gethTypes.Log](zerolog.Nop()) + logsPublisher.Start(ictx) + + engine := NewEventIngestionEngine( + subscriber, + replayer.NewBlocksProvider(blocks, flowGo.Emulator, nil), + store, + registerStore, + blocks, + receipts, + transactions, + traces, + blocksPublisher, + logsPublisher, + zerolog.Nop(), + metrics.NopCollector, + defaultReplayerConfig(), + ) + + engine.Start(ictx) + <-engine.Ready() + + evmHeight := uint64(0) + events := make([]flow.Event, 0) + blockIndexedFirst := false + txsStored := 0 + eventCount := 5 + txHashes := make([]gethCommon.Hash, eventCount) + + for i := 0; i < eventCount; i++ { + txCdc, txEvent, transaction, res, err := newTransaction(evmHeight) + txHashes[i] = res.TxHash + require.NoError(t, err) + + // add a single transaction for each block + transactions. + On("Store", mock.AnythingOfType("models.TransactionCall"), mock.Anything). + Return(func(tx models.Transaction, _ *pebbleDB.Batch) error { + assert.Equal(t, transaction.Hash(), tx.Hash()) // if hashes are equal tx is equal + require.True(t, blockIndexedFirst) + txsStored++ + return nil + }). + Once() + + receipts. + On("Store", mock.AnythingOfType("[]*models.Receipt"), mock.Anything). + Return(func(receipts []*models.Receipt, _ *pebbleDB.Batch) error { return nil }). + Once() + + traces. + On("StoreTransaction", mock.AnythingOfType("common.Hash"), mock.AnythingOfType("json.RawMessage"), mock.Anything). + Return(func(txID gethCommon.Hash, trace json.RawMessage, batch *pebbleDB.Batch) error { + assert.Equal(t, transaction.Hash(), txID) + return nil + }) + + events = append(events, flow.Event{ + Type: string(txEvent.Etype), + Value: txCdc, + }) + } + + blocksStored := 0 + blockCdc, block, blockEvent, err := newBlock(evmHeight, txHashes) + require.NoError(t, err) + + blocks. + On("Store", mock.AnythingOfType("uint64"), mock.Anything, mock.AnythingOfType("*models.Block"), mock.Anything). + Return(func(h uint64, id flow.Identifier, storeBlock *models.Block, _ *pebbleDB.Batch) error { + assert.Equal(t, block, storeBlock) + assert.Equal(t, evmHeight, block.Height) + assert.Equal(t, latestCadenceHeight+1, h) + blockIndexedFirst = true + blocksStored++ + return nil + }). + Once() + + events = append(events, flow.Event{ + Type: string(blockEvent.Etype), + Value: blockCdc, + }) + + // this messes up order of events to test if we still process events in-order + // it will make transaction event first and then block event + events[0], events[1] = events[1], events[0] + // and it will make the first block be swapped with second block out-of-order + events[1], events[2] = events[2], events[1] + + eventsChan <- models.NewSingleBlockEvents(flow.BlockEvents{ + Events: events, + Height: latestCadenceHeight + 1, + }) + + close(eventsChan) + <-engine.Done() + assert.Equal(t, eventCount, txsStored) + assert.Equal(t, 1, blocksStored) + }) } func newBlock(height uint64, txHashes []gethCommon.Hash) (cadence.Event, *models.Block, *events.Event, error) { diff --git a/tests/go.mod b/tests/go.mod index 767287e78..15feb0f9d 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -6,7 +6,7 @@ require ( github.com/goccy/go-json v0.10.2 github.com/onflow/cadence v1.2.2 github.com/onflow/crypto v0.25.2 - github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f + github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47 github.com/onflow/flow-evm-gateway v0.0.0-20240201154855-4d4d3d3f19c7 github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 github.com/onflow/flow-go-sdk v1.2.3 diff --git a/tests/go.sum b/tests/go.sum index 4a453f5eb..20299d662 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -906,8 +906,8 @@ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0 h1:R86HaOuk6vpuECZ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0/go.mod h1:9asTBnB6Tw2UlVVtQKyS/egYv3xr4zVlJnJ75z1dfac= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0 h1:u2DAG8pk0xFH7TwS70t1gSZ/FtIIZWMSNyiu4SeXBYg= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0/go.mod h1:pN768Al/wLRlf3bwugv9TyxniqJxMu4sxnX9eQJam64= -github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f h1:u102x6G1Y2TDH20ssNi4idh81ntP3EOgM7IWPrNlEnA= -github.com/onflow/flow-emulator v1.1.1-0.20241212150837-28eb6a46d91f/go.mod h1:D/qlW9iRE2k1Nmhu1wGl/z5MXBOm+ysVODlIs3PLSJU= +github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47 h1:OjYlGVC+rLxP7rCS74U0aqilChCjJrtvU52Bt7rZ62w= +github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47/go.mod h1:45cGyqSvI7o/oOHuYZkMc917f5JSKiCBQh30NV3w6Ks= github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3SsEftzXG2JlmSe24= github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= From 4984428e4a161527701ede08ca0adb939d1bff71 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Wed, 18 Dec 2024 17:30:50 +0100 Subject: [PATCH 08/12] apply review comments --- bootstrap/bootstrap.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 77ed3bf21..b521a973f 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -85,11 +85,10 @@ type EVMGatewayNodeBuilder struct { postShutdownFns []func() error modules []namedModuleFunc - Metrics metrics2.Collector - DB *pebble.DB - Client *requester.CrossSporkClient - Storages *Storages - // Signer is used for signing flow transactions + Metrics metrics2.Collector + DB *pebble.DB + Client *requester.CrossSporkClient + Storages *Storages Publishers *Publishers Keystore *requester.KeyStoreComponent } @@ -227,7 +226,7 @@ func (fnb *EVMGatewayNodeBuilder) profilerServerComponent(config config.Config) func (fnb *EVMGatewayNodeBuilder) apiServerComponent(cfg config.Config) (module.ReadyDoneAware, error) { log := fnb.Logger - log.Info().Msg("bootstrap starting Metrics server") + log.Info().Msg("bootstrap starting API server") server := api.NewServer(log, fnb.Metrics, cfg) From e9e537f835e6991b73ebef762610f043638cc5e4 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Fri, 3 Jan 2025 16:08:17 +0100 Subject: [PATCH 09/12] update dependencies and address review comments --- api/profiler.go | 105 ++++++++++++++++++----------------- bootstrap/bootstrap.go | 25 +++++---- cmd/run/cmd.go | 4 +- go.mod | 2 +- go.sum | 4 +- services/ingestion/engine.go | 2 + tests/go.mod | 4 +- tests/go.sum | 8 +-- tests/helpers.go | 1 + 9 files changed, 80 insertions(+), 75 deletions(-) diff --git a/api/profiler.go b/api/profiler.go index 973ba349b..efb366693 100644 --- a/api/profiler.go +++ b/api/profiler.go @@ -9,6 +9,8 @@ import ( "strconv" "time" + "github.com/rs/zerolog/log" + "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" @@ -16,11 +18,11 @@ import ( ) type ProfileServer struct { + component.Component + log zerolog.Logger server *http.Server endpoint string - - startupCompleted chan struct{} } var _ component.Component = (*ProfileServer)(nil) @@ -31,70 +33,69 @@ func NewProfileServer( port int, ) *ProfileServer { endpoint := net.JoinHostPort(host, strconv.Itoa(port)) - return &ProfileServer{ - log: logger, - server: &http.Server{Addr: endpoint}, - endpoint: endpoint, - startupCompleted: make(chan struct{}), + + s := &ProfileServer{ + log: logger, + server: &http.Server{Addr: endpoint}, + endpoint: endpoint, } + + s.Component = component.NewComponentManagerBuilder(). + AddWorker(s.serve). + AddWorker(s.shutdownOnContextDone). + Build() + + return s } -func (s *ProfileServer) Start(ctx irrecoverable.SignalerContext) { - defer close(s.startupCompleted) +func (s *ProfileServer) serve(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { + s.log.Info().Msg("starting profiler server on address") + + l, err := net.Listen("tcp", s.endpoint) + if err != nil { + s.log.Err(err).Msg("failed to start the metrics server") + ctx.Throw(err) + return + } + + ready() + // pass the signaler context to the server so that the signaler context + // can control the server's lifetime s.server.BaseContext = func(_ net.Listener) context.Context { return ctx } - go func() { - s.log.Info().Msgf("Profiler server started: %s", s.endpoint) - - if err := s.server.ListenAndServe(); err != nil { - // http.ErrServerClosed is returned when Close or Shutdown is called - // we don't consider this an error, so print this with debug level instead - if errors.Is(err, http.ErrServerClosed) { - s.log.Debug().Err(err).Msg("Profiler server shutdown") - } else { - s.log.Err(err).Msg("error running profiler server") - } + err = s.server.Serve(l) // blocking call + if err != nil { + if errors.Is(err, http.ErrServerClosed) { + return } - }() -} - -func (s *ProfileServer) Ready() <-chan struct{} { - ready := make(chan struct{}) - - go func() { - <-s.startupCompleted - close(ready) - }() - return ready + log.Err(err).Msg("fatal error in the metrics server") + ctx.Throw(err) + } } -func (s *ProfileServer) Done() <-chan struct{} { - done := make(chan struct{}) - go func() { - <-s.startupCompleted - defer close(done) +func (s *ProfileServer) shutdownOnContextDone(ictx irrecoverable.SignalerContext, ready component.ReadyFunc) { + ready() + <-ictx.Done() - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() - err := s.server.Shutdown(ctx) - if err == nil { - s.log.Info().Msg("Profiler server graceful shutdown completed") - } + err := s.server.Shutdown(ctx) + if err == nil { + s.log.Info().Msg("Profiler server graceful shutdown completed") + } - if errors.Is(err, ctx.Err()) { - s.log.Warn().Msg("Profiler server graceful shutdown timed out") - err := s.server.Close() - if err != nil { - s.log.Err(err).Msg("error closing profiler server") - } - } else { - s.log.Err(err).Msg("error shutting down profiler server") + if errors.Is(err, ctx.Err()) { + s.log.Warn().Msg("Profiler server graceful shutdown timed out") + err := s.server.Close() + if err != nil { + s.log.Err(err).Msg("error closing profiler server") } - }() - return done + } else { + s.log.Err(err).Msg("error shutting down profiler server") + } } diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index b521a973f..696a584e1 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -148,10 +148,12 @@ func (fnb *EVMGatewayNodeBuilder) initDB() error { if err := fnb.DB.Close(); err != nil { return fmt.Errorf("error closing pebble database: %w", err) } + + fnb.Logger.Info().Msg("database has been closed") return nil }) - return err + return nil } func (fnb *EVMGatewayNodeBuilder) Component(name string, f cmd.ReadyDoneFactory[config.Config]) *EVMGatewayNodeBuilder { @@ -173,7 +175,6 @@ func (fnb *EVMGatewayNodeBuilder) postShutdown() error { errs = multierror.Append(errs, err) } } - fnb.Logger.Info().Msg("database has been closed") return errs.ErrorOrNil() } @@ -206,11 +207,11 @@ func (fnb *EVMGatewayNodeBuilder) Initialize() error { func (fnb *EVMGatewayNodeBuilder) LoadComponentsAndModules() { fnb.initPublishers() + fnb.Component("Profiler Server", fnb.profilerServerComponent) + fnb.Component("Metrics Server", fnb.metricsServerComponent) fnb.Component("Key Store", fnb.initKeyStore) fnb.Component("API Server", fnb.apiServerComponent) fnb.Component("Event Ingestion Engine", fnb.eventIngestionEngineComponent) - fnb.Component("Metrics Server", fnb.metricsServerComponent) - fnb.Component("Profiler Server", fnb.profilerServerComponent) } func (fnb *EVMGatewayNodeBuilder) metricsServerComponent(config config.Config) (module.ReadyDoneAware, error) { @@ -359,30 +360,30 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi l.Info().Msg("bootstrap starting event ingestion") // get latest cadence block from the network and the database - latestCadenceBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) + gatewayLatestBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) if err != nil { return nil, fmt.Errorf("failed to get latest cadence block: %w", err) } - latestCadenceHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() + chainLatestHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() if err != nil { return nil, err } // make sure the provided block to start the indexing can be loaded - _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), latestCadenceHeight) + _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), chainLatestHeight) if err != nil { return nil, fmt.Errorf( "failed to get provided cadence height %d: %w", - latestCadenceHeight, + chainLatestHeight, err, ) } l.Info(). - Uint64("start-cadence-height", latestCadenceHeight). - Uint64("latest-cadence-height", latestCadenceBlock.Height). - Uint64("missed-heights", latestCadenceBlock.Height-latestCadenceHeight). + Uint64("chain-cadence-height", chainLatestHeight). + Uint64("gateway-cadence-height", gatewayLatestBlock.Height). + Uint64("missed-heights", gatewayLatestBlock.Height-chainLatestHeight). Msg("indexing cadence height information") chainID := cfg.FlowNetworkID @@ -393,7 +394,7 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi fnb.Client, chainID, fnb.Keystore, - latestCadenceHeight, + chainLatestHeight, ) callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) diff --git a/cmd/run/cmd.go b/cmd/run/cmd.go index c590a93af..1cc988806 100644 --- a/cmd/run/cmd.go +++ b/cmd/run/cmd.go @@ -39,14 +39,14 @@ var Cmd = &cobra.Command{ builder := bootstrap.NewEVMGatewayNodeBuilder(cfg) if err := builder.Initialize(); err != nil { - builder.Logger.Fatal().Err(err).Send() + builder.Logger.Fatal().Err(err).Msg("failed to initialize") } builder.LoadComponentsAndModules() node, err := builder.Build() if err != nil { - builder.Logger.Fatal().Err(err).Send() + builder.Logger.Fatal().Err(err).Msg("failed to build node") } node.Run(command.Context()) return nil diff --git a/go.mod b/go.mod index 3cb7df86d..abe9f4155 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/hashicorp/go-multierror v1.1.1 github.com/onflow/atree v0.8.0 github.com/onflow/cadence v1.2.2 - github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/prometheus/client_golang v1.18.0 diff --git a/go.sum b/go.sum index bd23d04c6..db2915323 100644 --- a/go.sum +++ b/go.sum @@ -743,8 +743,8 @@ github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3 github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 h1:kcAoHB/uEGP3wTJcv2aVtCHSgwY0bdn8qjw7GvvHfM8= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4 h1:PSxMlqAgTzeTb/J6nknx3q74MNtUFnKWfWEneE+HLX8= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/services/ingestion/engine.go b/services/ingestion/engine.go index 94535c6af..339861b26 100644 --- a/services/ingestion/engine.go +++ b/services/ingestion/engine.go @@ -129,6 +129,7 @@ func (e *Engine) run(ctx irrecoverable.SignalerContext, ready component.ReadyFun return } if events.Err != nil { + ctx.Throw(fmt.Errorf("failure in event subscription with: %w", events.Err)) return } @@ -136,6 +137,7 @@ func (e *Engine) run(ctx irrecoverable.SignalerContext, ready component.ReadyFun if err != nil { e.log.Error().Err(err).Msg("failed to process EVM events") ctx.Throw(err) + return } } } diff --git a/tests/go.mod b/tests/go.mod index 15feb0f9d..957b3914b 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -6,9 +6,9 @@ require ( github.com/goccy/go-json v0.10.2 github.com/onflow/cadence v1.2.2 github.com/onflow/crypto v0.25.2 - github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47 + github.com/onflow/flow-emulator v1.1.1-0.20250103150158-e029ea3471fe github.com/onflow/flow-evm-gateway v0.0.0-20240201154855-4d4d3d3f19c7 - github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 + github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4 github.com/onflow/flow-go-sdk v1.2.3 github.com/onflow/go-ethereum v1.14.7 github.com/rs/zerolog v1.33.0 diff --git a/tests/go.sum b/tests/go.sum index 20299d662..08c27c08d 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -906,14 +906,14 @@ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0 h1:R86HaOuk6vpuECZ github.com/onflow/flow-core-contracts/lib/go/contracts v1.4.0/go.mod h1:9asTBnB6Tw2UlVVtQKyS/egYv3xr4zVlJnJ75z1dfac= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0 h1:u2DAG8pk0xFH7TwS70t1gSZ/FtIIZWMSNyiu4SeXBYg= github.com/onflow/flow-core-contracts/lib/go/templates v1.4.0/go.mod h1:pN768Al/wLRlf3bwugv9TyxniqJxMu4sxnX9eQJam64= -github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47 h1:OjYlGVC+rLxP7rCS74U0aqilChCjJrtvU52Bt7rZ62w= -github.com/onflow/flow-emulator v1.1.1-0.20241216151608-ed31000dff47/go.mod h1:45cGyqSvI7o/oOHuYZkMc917f5JSKiCBQh30NV3w6Ks= +github.com/onflow/flow-emulator v1.1.1-0.20250103150158-e029ea3471fe h1:0O/VrpF4ujhCqj5pzYmrb5qvghzf+1fxVxbp4jlneM4= +github.com/onflow/flow-emulator v1.1.1-0.20250103150158-e029ea3471fe/go.mod h1:6Uj/ftISX4Su7blfRKTdENdyr29XAp364lT1ESgrh7s= github.com/onflow/flow-ft/lib/go/contracts v1.0.1 h1:Ts5ob+CoCY2EjEd0W6vdLJ7hLL3SsEftzXG2JlmSe24= github.com/onflow/flow-ft/lib/go/contracts v1.0.1/go.mod h1:PwsL8fC81cjnUnTfmyL/HOIyHnyaw/JA474Wfj2tl6A= github.com/onflow/flow-ft/lib/go/templates v1.0.1 h1:FDYKAiGowABtoMNusLuRCILIZDtVqJ/5tYI4VkF5zfM= github.com/onflow/flow-ft/lib/go/templates v1.0.1/go.mod h1:uQ8XFqmMK2jxyBSVrmyuwdWjTEb+6zGjRYotfDJ5pAE= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49 h1:kcAoHB/uEGP3wTJcv2aVtCHSgwY0bdn8qjw7GvvHfM8= -github.com/onflow/flow-go v0.38.0-preview.0.4.0.20241213150609-85fd812a7e49/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4 h1:PSxMlqAgTzeTb/J6nknx3q74MNtUFnKWfWEneE+HLX8= +github.com/onflow/flow-go v0.38.0-preview.0.4.0.20250102180624-72adf9e522c4/go.mod h1:c4ubAQ2WIMYY/TOaBvbajROEFWv2HwhKeGOsEdLPIM0= github.com/onflow/flow-go-sdk v1.2.3 h1:jb+0dIXBO12Zt8x3c2xDXYPv6k3sRTUvhe59M+EcXTI= github.com/onflow/flow-go-sdk v1.2.3/go.mod h1:jMaffBTlAIdutx+pBhRIigLZFIBYSDDST0Uax1rW2qo= github.com/onflow/flow-nft/lib/go/contracts v1.2.2 h1:XFERNVUDGbZ4ViZjt7P1cGD80mO1PzUJYPfdhXFsGbQ= diff --git a/tests/helpers.go b/tests/helpers.go index 58923ab54..47e5e3366 100644 --- a/tests/helpers.go +++ b/tests/helpers.go @@ -195,6 +195,7 @@ func startGateway(t *testing.T, ctx context.Context, cfg config.Config) { node, err := builder.Build() require.NoError(t, err) go node.Run(ctx) + <-node.Ready() } // executeTest will run the provided JS test file using mocha From 3e04bc30f7095ba1967aff271a3cfc5f60670990 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Wed, 8 Jan 2025 16:27:11 +0100 Subject: [PATCH 10/12] Apply review comments --- bootstrap/bootstrap.go | 16 ++++++++-------- services/requester/key_store_component.go | 6 ++++++ 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 696a584e1..746871bd6 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -360,30 +360,30 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi l.Info().Msg("bootstrap starting event ingestion") // get latest cadence block from the network and the database - gatewayLatestBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) + chainLatestBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) if err != nil { return nil, fmt.Errorf("failed to get latest cadence block: %w", err) } - chainLatestHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() + gatewayLatestHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() if err != nil { return nil, err } // make sure the provided block to start the indexing can be loaded - _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), chainLatestHeight) + _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), chainLatestBlock.Height) if err != nil { return nil, fmt.Errorf( "failed to get provided cadence height %d: %w", - chainLatestHeight, + chainLatestBlock.Height, err, ) } l.Info(). - Uint64("chain-cadence-height", chainLatestHeight). - Uint64("gateway-cadence-height", gatewayLatestBlock.Height). - Uint64("missed-heights", gatewayLatestBlock.Height-chainLatestHeight). + Uint64("chain-cadence-height", chainLatestBlock.Height). + Uint64("gateway-cadence-height", gatewayLatestHeight). + Uint64("missed-heights", chainLatestBlock.Height-gatewayLatestHeight). Msg("indexing cadence height information") chainID := cfg.FlowNetworkID @@ -394,7 +394,7 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi fnb.Client, chainID, fnb.Keystore, - chainLatestHeight, + gatewayLatestHeight, ) callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) diff --git a/services/requester/key_store_component.go b/services/requester/key_store_component.go index 8324c827a..9cab8affb 100644 --- a/services/requester/key_store_component.go +++ b/services/requester/key_store_component.go @@ -58,6 +58,12 @@ func (k *KeyStoreComponent) Start(ctx irrecoverable.SignalerContext) { return } for _, key := range account.Keys { + // Skip account keys that do not use the same Public Key as the + // configured crypto.Signer object. + if !key.PublicKey.Equals(signer.PublicKey()) { + continue + } + accountKeys = append(accountKeys, &AccountKey{ AccountKey: *key, Address: k.config.COAAddress, From 681347311fd5f3df053c3af31b87feb07e754772 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Wed, 8 Jan 2025 17:19:25 +0100 Subject: [PATCH 11/12] move logging for missed heights --- bootstrap/bootstrap.go | 26 +++++++------------------- services/ingestion/event_subscriber.go | 16 ++++++++++++++++ 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index 746871bd6..a4eb78724 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -359,42 +359,30 @@ func (fnb *EVMGatewayNodeBuilder) eventIngestionEngineComponent(cfg config.Confi l := fnb.Logger.With().Str("component", "bootstrap-ingestion").Logger() l.Info().Msg("bootstrap starting event ingestion") - // get latest cadence block from the network and the database - chainLatestBlock, err := fnb.Client.GetLatestBlock(context.Background(), true) - if err != nil { - return nil, fmt.Errorf("failed to get latest cadence block: %w", err) - } - - gatewayLatestHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() + latestIndexedHeight, err := fnb.Storages.Blocks.LatestCadenceHeight() if err != nil { return nil, err } - - // make sure the provided block to start the indexing can be loaded - _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), chainLatestBlock.Height) + // make sure the latest indexed block can be retrieved from the network + _, err = fnb.Client.GetBlockHeaderByHeight(context.Background(), latestIndexedHeight) if err != nil { return nil, fmt.Errorf( - "failed to get provided cadence height %d: %w", - chainLatestBlock.Height, + "failed to get header for last indexed cadence height %d: %w", + latestIndexedHeight, err, ) } - l.Info(). - Uint64("chain-cadence-height", chainLatestBlock.Height). - Uint64("gateway-cadence-height", gatewayLatestHeight). - Uint64("missed-heights", chainLatestBlock.Height-gatewayLatestHeight). - Msg("indexing cadence height information") - chainID := cfg.FlowNetworkID // create event subscriber + // continuing from the latest indexed height subscriber := ingestion.NewRPCEventSubscriber( fnb.Logger, fnb.Client, chainID, fnb.Keystore, - gatewayLatestHeight, + latestIndexedHeight, ) callTracerCollector, err := replayer.NewCallTracerCollector(fnb.Logger) diff --git a/services/ingestion/event_subscriber.go b/services/ingestion/event_subscriber.go index bc9d37084..39f4c958c 100644 --- a/services/ingestion/event_subscriber.go +++ b/services/ingestion/event_subscriber.go @@ -74,6 +74,22 @@ func (r *RPCEventSubscriber) Subscribe(ctx context.Context) <-chan models.BlockE close(eventsChan) }() + // get latest cadence block from the network and the database + chainLatestBlockHeader, err := r.client.GetLatestBlockHeader(ctx, true) + if err != nil { + eventsChan <- models.NewBlockEventsError(fmt.Errorf("failed to get latest cadence block: %w", err)) + return + } + latestOnChainHeight := chainLatestBlockHeader.Height + + blocksToCatchUp := latestOnChainHeight - r.height + + r.logger.Info(). + Uint64("chain-cadence-height", latestOnChainHeight). + Uint64("latest-indexed-height", r.height). + Uint64("missed-heights", blocksToCatchUp). + Msg("indexing cadence height information") + // if the height is from the previous spork, backfill all the eventsChan from previous sporks first if r.client.IsPastSpork(r.height) { r.logger.Info(). From fc2f6538bf2771f7efcabfa31a7a01913dc6bfd9 Mon Sep 17 00:00:00 2001 From: Janez Podhostnik Date: Thu, 9 Jan 2025 17:02:58 +0100 Subject: [PATCH 12/12] review fixes --- bootstrap/bootstrap.go | 6 +++--- services/ingestion/event_subscriber.go | 9 +++++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/bootstrap/bootstrap.go b/bootstrap/bootstrap.go index a4eb78724..c1a5403bb 100644 --- a/bootstrap/bootstrap.go +++ b/bootstrap/bootstrap.go @@ -464,13 +464,13 @@ func (fnb *EVMGatewayNodeBuilder) initStorage() error { }(batch) cadenceHeight := cfg.InitCadenceHeight - evmBlokcHeight := uint64(0) + evmBlockHeight := uint64(0) cadenceBlock, err := fnb.Client.GetBlockHeaderByHeight(context.Background(), cadenceHeight) if err != nil { return fmt.Errorf("could not fetch provided cadence height, make sure it's correct: %w", err) } - snapshot, err := registerStore.GetSnapshotAt(evmBlokcHeight) + snapshot, err := registerStore.GetSnapshotAt(evmBlockHeight) if err != nil { return fmt.Errorf("could not get register snapshot at block height %d: %w", 0, err) } @@ -486,7 +486,7 @@ func (fnb *EVMGatewayNodeBuilder) initStorage() error { return fmt.Errorf("could not set account status: %w", err) } - err = registerStore.Store(delta.GetUpdates(), evmBlokcHeight, batch) + err = registerStore.Store(delta.GetUpdates(), evmBlockHeight, batch) if err != nil { return fmt.Errorf("could not store register updates: %w", err) } diff --git a/services/ingestion/event_subscriber.go b/services/ingestion/event_subscriber.go index 39f4c958c..857a5bdc1 100644 --- a/services/ingestion/event_subscriber.go +++ b/services/ingestion/event_subscriber.go @@ -82,12 +82,17 @@ func (r *RPCEventSubscriber) Subscribe(ctx context.Context) <-chan models.BlockE } latestOnChainHeight := chainLatestBlockHeader.Height - blocksToCatchUp := latestOnChainHeight - r.height + blocksToCatchUp := int64(0) + if latestOnChainHeight > r.height { + blocksToCatchUp = int64(latestOnChainHeight - r.height) + } else { + blocksToCatchUp = int64(r.height - latestOnChainHeight) + } r.logger.Info(). Uint64("chain-cadence-height", latestOnChainHeight). Uint64("latest-indexed-height", r.height). - Uint64("missed-heights", blocksToCatchUp). + Int64("missed-heights", blocksToCatchUp). Msg("indexing cadence height information") // if the height is from the previous spork, backfill all the eventsChan from previous sporks first