diff --git a/CHANGELOG.md b/CHANGELOG.md index b428c37..0f85eb3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,25 +10,42 @@ If you were at `firehose-core` version `1.0.0` and are bumping to `1.1.0`, you s ## Unreleased +### Tools + * Enhanced `firecore tools print merged-blocks` with various small quality of life improvements: - Now accepts a block range instead of a single start block. - Passing a single block as the block range will print this single block alone. - Block range is now optional, defaulting to run until there is no more files to read. - It's possible to pass a merged blocks file directly, with or without an optional range. +### Firehose + +> [!IMPORTANT] +> This release will reject firehose connections from clients that don't support GZIP or ZSTD compression. Use `--firehose-enforce-compression=false` to keep previous behavior, then check the logs for `incoming Substreams Blocks request` logs with the value `compressed: false` to track users who are not using compressed HTTP connections. + +> [!IMPORTANT] +> This release removes the old `sf.firehose.v1` protocol (replaced by `sf.firehose.v2` in 2022, this should not affect any reasonably recent client) + +* Add support for ConnectWeb firehose requests. +* Always use gzip compression on firehose requests for clients that support it (instead of always answering with the same compression as the request). + ## v1.6.9 ### Substreams * Fix an issue preventing proper detection of gzip compression when multiple headers are set (ex: python grpc client) +* Add support for zstd compression on server * Fix an issue preventing some tier2 requests on last-stage from correctly generating stores. This could lead to some missing "backfilling" jobs and slower time to first block on reconnection. * Fix a thread leak on cursor resolution resulting in bad counter for active connections -* Add support for zstd encoding on server ## v1.6.8 > [!NOTE] +<<<<<<< HEAD > This release will reject connections from clients that don't support GZIP compression. Use `--substreams-tier1-enforce-compression=false` to keep previous behavior, then check the logs for `incoming Substreams Blocks request` logs with the value `compressed: false` to track users who are not using compressed HTTP connections. +======= +> This release will reject substreams connections from clients that don't support GZIP compression. Use `--substreams-tier1-enforce-compression=false` to keep previous behavior, then check the logs for `incoming Substreams Blocks request` logs with the value `compressed: false` to track users who are not using compressed HTTP connections. +>>>>>>> 380546c (switch firehose to connectweb server, disable proxy for firehoseV1, add --firehose-enforce-compression true by default) * Substreams: add `--substreams-tier1-enforce-compression` to reject connections from clients that do not support GZIP compression * Substreams performance: reduced the number of `mallocs` (patching some third-party libraries) diff --git a/cmd/apps/firehose.go b/cmd/apps/firehose.go index de38dee..b2752de 100644 --- a/cmd/apps/firehose.go +++ b/cmd/apps/firehose.go @@ -35,6 +35,7 @@ func RegisterFirehoseApp[B firecore.Block](chain *firecore.Chain[B], rootLog *za cmd.Flags().String("firehose-discovery-service-url", "", "Url to configure the gRPC discovery service") //traffic-director://xds?vpc_network=vpc-global&use_xds_reds=true cmd.Flags().Int("firehose-rate-limit-bucket-size", -1, "Rate limit bucket size (default: no rate limit)") cmd.Flags().Duration("firehose-rate-limit-bucket-fill-rate", 10*time.Second, "Rate limit bucket refill rate (default: 10s)") + cmd.Flags().Bool("firehose-enforce-compression", true, "Reject any request that does not accept gzip or zstd encoding in their GRPC/Connect header") return nil }, @@ -80,6 +81,10 @@ func RegisterFirehoseApp[B firecore.Block](chain *firecore.Chain[B], rootLog *za var serverOptions []server.Option + if viper.GetBool("firehose-enforce-compression") { + serverOptions = append(serverOptions, server.WithEnforceCompression(true)) + } + limiterSize := viper.GetInt("firehose-rate-limit-bucket-size") limiterRefillRate := viper.GetDuration("firehose-rate-limit-bucket-fill-rate") if limiterSize > 0 { diff --git a/cmd/apps/substreams_tier1.go b/cmd/apps/substreams_tier1.go index abe60d2..04d9eff 100644 --- a/cmd/apps/substreams_tier1.go +++ b/cmd/apps/substreams_tier1.go @@ -49,7 +49,7 @@ func RegisterSubstreamsTier1App[B firecore.Block](chain *firecore.Chain[B], root cmd.Flags().String("substreams-tier1-discovery-service-url", "", "URL to configure the grpc discovery service, used for communication with tier2") //traffic-director://xds?vpc_network=vpc-global&use_xds_reds=true cmd.Flags().Bool("substreams-tier1-subrequests-insecure", false, "Connect to tier2 without checking certificate validity") cmd.Flags().Bool("substreams-tier1-subrequests-plaintext", true, "Connect to tier2 without client in plaintext mode") - cmd.Flags().Bool("substreams-tier1-enforce-compression", true, "Reject any request that does not accept gzip encoding in their GRPC/Connect header") + cmd.Flags().Bool("substreams-tier1-enforce-compression", true, "Reject any request that does not accept gzip or zstd encoding in their GRPC/Connect header") cmd.Flags().Int("substreams-tier1-max-subrequests", 4, "number of parallel subrequests that the tier1 can make to the tier2 per request") cmd.Flags().String("substreams-tier1-block-type", "", "Block type to use for the substreams tier1 (Ex: sf.ethereum.type.v2.Block)") diff --git a/firehose/server/blocks.go b/firehose/server/blocks.go index 7e57fe9..7858bce 100644 --- a/firehose/server/blocks.go +++ b/firehose/server/blocks.go @@ -5,9 +5,12 @@ import ( "errors" "fmt" "math/rand" + "net/http" "os" + "strings" "time" + connect "connectrpc.com/connect" "github.com/streamingfast/bstream" pbbstream "github.com/streamingfast/bstream/pb/sf/bstream/v1" "github.com/streamingfast/bstream/stream" @@ -19,16 +22,15 @@ import ( pbfirehose "github.com/streamingfast/pbgo/sf/firehose/v2" "go.uber.org/zap" "google.golang.org/grpc/codes" - "google.golang.org/grpc/metadata" "google.golang.org/grpc/status" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/anypb" ) -func (s *Server) Block(ctx context.Context, request *pbfirehose.SingleBlockRequest) (*pbfirehose.SingleBlockResponse, error) { +func (s *Server) Block(ctx context.Context, request *connect.Request[pbfirehose.SingleBlockRequest]) (*connect.Response[pbfirehose.SingleBlockResponse], error) { var blockNum uint64 var blockHash string - switch ref := request.Reference.(type) { + switch ref := request.Msg.Reference.(type) { case *pbfirehose.SingleBlockRequest_BlockHashAndNumber_: blockNum = ref.BlockHashAndNumber.Num blockHash = ref.BlockHashAndNumber.Hash @@ -71,15 +73,22 @@ func (s *Server) Block(ctx context.Context, request *pbfirehose.SingleBlockReque auth := dauth.FromContext(ctx) metering.Send(ctx, meter, auth.UserID(), auth.APIKeyID(), auth.RealIP(), auth.Meta(), "sf.firehose.v2.Firehose/Block", resp) - return resp, nil + return connect.NewResponse(resp), nil } -func (s *Server) Blocks(request *pbfirehose.Request, streamSrv pbfirehose.Stream_BlocksServer) error { - ctx := streamSrv.Context() +// Blocks(context.Context, *connect.Request[v2.Request], *connect.ServerStream[v2.Response]) error +func (s *Server) Blocks(ctx context.Context, request *connect.Request[pbfirehose.Request], streamSrv *connect.ServerStream[pbfirehose.Response]) error { metrics.RequestCounter.Inc() logger := logging.Logger(ctx, s.logger) + if !matchHeader(request.Header(), acceptedCompressionValues) { + if s.enforceCompression { + return status.Error(codes.InvalidArgument, "client does not support compression") + } + logger.Info("client does not support compression") + } + if s.rateLimiter != nil { rlCtx, cancel := context.WithTimeout(ctx, 30*time.Second) defer cancel() @@ -102,10 +111,7 @@ func (s *Server) Blocks(request *pbfirehose.Request, streamSrv pbfirehose.Stream hostname = "unknown" logger.Warn("cannot determine hostname, using 'unknown'", zap.Error(err)) } - md := metadata.New(map[string]string{"hostname": hostname}) - if err := streamSrv.SendHeader(md); err != nil { - logger.Warn("cannot send metadata header", zap.Error(err)) - } + streamSrv.ResponseHeader().Add("hostname", hostname) } var blockCount uint64 @@ -123,7 +129,7 @@ func (s *Server) Blocks(request *pbfirehose.Request, streamSrv pbfirehose.Stream obj = block.Payload } - protoStep, skip := stepToProto(step, request.FinalBlocksOnly) + protoStep, skip := stepToProto(step, request.Msg.FinalBlocksOnly) if skip { return nil } @@ -175,7 +181,7 @@ func (s *Server) Blocks(request *pbfirehose.Request, streamSrv pbfirehose.Stream return nil }) - if len(request.Transforms) > 0 && s.transformRegistry == nil { + if len(request.Msg.Transforms) > 0 && s.transformRegistry == nil { return status.Errorf(codes.Unimplemented, "no transforms registry configured within this instance") } @@ -209,11 +215,11 @@ func (s *Server) Blocks(request *pbfirehose.Request, streamSrv pbfirehose.Stream }) } - ctx = s.initFunc(ctx, request) + ctx = s.initFunc(ctx, request.Msg) str, err := s.streamFactory.New( ctx, handlerFunc, - request, + request.Msg, logger, stream.WithLiveSourceHandlerMiddleware(liveSourceMiddlewareHandler), stream.WithFileSourceHandlerMiddleware(fileSourceMiddlewareHandler), @@ -294,3 +300,22 @@ func stepToProto(step bstream.StepType, finalBlocksOnly bool) (outStep pbfirehos } return 0, true // simply skip irreversible or stalled here } + +// must be lowercase +var compressionHeader = map[string]bool{"grpc-accept-encoding": true, "connect-accept-encoding": true} +var acceptedCompressionValues = map[string]bool{"gzip": true, "zstd": true} + +func matchHeader(headers http.Header, expected map[string]bool) bool { + for k, v := range headers { + if compressionHeader[strings.ToLower(k)] { + for _, vv := range v { + for _, vvv := range strings.Split(vv, ",") { + if expected[strings.TrimSpace(strings.ToLower(vvv))] { + return true + } + } + } + } + } + return false +} diff --git a/firehose/server/local.go b/firehose/server/local.go deleted file mode 100644 index 43f0db2..0000000 --- a/firehose/server/local.go +++ /dev/null @@ -1,77 +0,0 @@ -package server - -import ( - "context" - - pbfirehose "github.com/streamingfast/pbgo/sf/firehose/v2" - "google.golang.org/grpc" - "google.golang.org/grpc/metadata" -) - -type BlocksPipe struct { - //grpc.ServerStream - grpc.ClientStream - ctx context.Context - pipeChan chan *pbfirehose.Response - err error -} - -func (p *BlocksPipe) SendHeader(metadata.MD) error { - return nil -} -func (p *BlocksPipe) SetHeader(metadata.MD) error { - return nil -} -func (p *BlocksPipe) SetTrailer(metadata.MD) { - return -} - -func (p *BlocksPipe) Context() context.Context { - return p.ctx -} - -func (p *BlocksPipe) Send(resp *pbfirehose.Response) error { - select { - case <-p.ctx.Done(): - return p.ctx.Err() - case p.pipeChan <- resp: - } - return nil -} - -func (p *BlocksPipe) Recv() (*pbfirehose.Response, error) { - select { - case resp, ok := <-p.pipeChan: - if !ok { - return resp, p.err - } - return resp, nil - case <-p.ctx.Done(): - select { - // ensure we empty the pipeChan - case resp, ok := <-p.pipeChan: - if !ok { - return resp, p.err - } - return resp, nil - default: - return nil, p.err - } - } -} - -func (s *Server) BlocksFromLocal(ctx context.Context, req *pbfirehose.Request) pbfirehose.Stream_BlocksClient { - cctx, cancel := context.WithCancel(ctx) - - pipe := &BlocksPipe{ - ctx: cctx, - pipeChan: make(chan *pbfirehose.Response), - } - go func() { - err := s.Blocks(req, pipe) - pipe.err = err - cancel() - }() - - return pipe -} diff --git a/firehose/server/server.go b/firehose/server/server.go index a2f8099..e7e6354 100644 --- a/firehose/server/server.go +++ b/firehose/server/server.go @@ -2,24 +2,28 @@ package server import ( "context" + "net/http" "net/url" "strings" "sync" "time" + connect "connectrpc.com/connect" + connect_go "connectrpc.com/connect" _ "github.com/mostynb/go-grpc-compression/zstd" "github.com/streamingfast/bstream/transform" "github.com/streamingfast/dauth" dauthgrpc "github.com/streamingfast/dauth/middleware/grpc" dgrpcserver "github.com/streamingfast/dgrpc/server" - "github.com/streamingfast/dgrpc/server/factory" + connectweb "github.com/streamingfast/dgrpc/server/connectrpc" "github.com/streamingfast/dmetering" firecore "github.com/streamingfast/firehose-core" "github.com/streamingfast/firehose-core/firehose" "github.com/streamingfast/firehose-core/firehose/info" "github.com/streamingfast/firehose-core/firehose/rate" "github.com/streamingfast/firehose-core/metering" - pbfirehoseV1 "github.com/streamingfast/pbgo/sf/firehose/v1" + fhconnect "github.com/streamingfast/pbgo/sf/firehose/v2/pbfirehoseconnect" + pbfirehoseV2 "github.com/streamingfast/pbgo/sf/firehose/v2" "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc" "go.opentelemetry.io/otel" @@ -37,14 +41,15 @@ type Server struct { initFunc func(context.Context, *pbfirehoseV2.Request) context.Context postHookFunc func(context.Context, *pbfirehoseV2.Response) - servers []*wrappedServer - logger *zap.Logger + servers []*wrappedServer + enforceCompression bool + logger *zap.Logger rateLimiter rate.Limiter } type wrappedServer struct { - dgrpcserver.Server + *connectweb.ConnectWebServer listenAddr string } @@ -56,6 +61,12 @@ func WithLeakyBucketLimiter(size int, dripRate time.Duration) Option { } } +func WithEnforceCompression(enforce bool) Option { + return func(s *Server) { + s.enforceCompression = enforce + } +} + func New( transformRegistry *transform.Registry, streamFactory *firecore.StreamFactory, @@ -86,7 +97,16 @@ func New( tracerProvider := otel.GetTracerProvider() - var servers []*wrappedServer + s := &Server{ + transformRegistry: transformRegistry, + blockGetter: blockGetter, + streamFactory: streamFactory, + initFunc: initFunc, + postHookFunc: postHookFunc, + logger: logger, + servers: []*wrappedServer{}, + } + for _, addr := range strings.Split(listenAddr, ",") { options := []dgrpcserver.Option{ dgrpcserver.WithLogger(logger), @@ -96,6 +116,7 @@ func New( dgrpcserver.WithGRPCServerOptions(grpc.MaxRecvMsgSize(25 * 1024 * 1024)), dgrpcserver.WithPostUnaryInterceptor(dauthgrpc.UnaryAuthChecker(authenticator, logger)), dgrpcserver.WithPostStreamInterceptor(dauthgrpc.StreamAuthChecker(authenticator, logger)), + dgrpcserver.WithConnectPermissiveCORS(), } if serviceDiscoveryURL != nil { @@ -104,40 +125,39 @@ func New( if strings.Contains(addr, "*") { options = append(options, dgrpcserver.WithInsecureServer()) - addr = strings.ReplaceAll(addr, "*", "") } else { options = append(options, dgrpcserver.WithPlainTextServer()) } - srv := factory.ServerFromOptions(options...) + streamHandlerGetter := func(opts ...connect_go.HandlerOption) (string, http.Handler) { + return fhconnect.NewStreamHandler(s, opts...) + } + options = append(options, dgrpcserver.WithConnectReflection(pbfirehoseV2.Stream_ServiceDesc.ServiceName)) - servers = append(servers, &wrappedServer{ - Server: srv, - listenAddr: addr, - }) + fetchHandlerGetter := func(opts ...connect_go.HandlerOption) (string, http.Handler) { + return fhconnect.NewFetchHandler(s, opts...) + } + options = append(options, dgrpcserver.WithConnectReflection(pbfirehoseV2.Fetch_ServiceDesc.ServiceName)) - } + handlerGetters := []connectweb.HandlerGetter{streamHandlerGetter, fetchHandlerGetter} + if infoServer != nil { + infoHandlerGetter := func(opts ...connect_go.HandlerOption) (string, http.Handler) { + out, outh := fhconnect.NewEndpointInfoHandler(&InfoServerWrapper{rpcInfoServer: infoServer}, opts...) + return out, outh + } + handlerGetters = append(handlerGetters, infoHandlerGetter) + options = append(options, dgrpcserver.WithConnectReflection(pbfirehoseV2.EndpointInfo_ServiceDesc.ServiceName)) + } - s := &Server{ - servers: servers, - transformRegistry: transformRegistry, - blockGetter: blockGetter, - streamFactory: streamFactory, - initFunc: initFunc, - postHookFunc: postHookFunc, - logger: logger, - } + cleanAddr := strings.ReplaceAll(addr, "*", "") - logger.Info("registering grpc services") - for _, srv := range servers { - srv.RegisterService(func(gs grpc.ServiceRegistrar) { - if blockGetter != nil { - pbfirehoseV2.RegisterFetchServer(gs, s) - } - pbfirehoseV2.RegisterEndpointInfoServer(gs, infoServer) - pbfirehoseV2.RegisterStreamServer(gs, s) - pbfirehoseV1.RegisterStreamServer(gs, NewFirehoseProxyV1ToV2(s)) // compatibility with firehose + srv := connectweb.New(handlerGetters, options...) + + s.servers = append(s.servers, &wrappedServer{ + ConnectWebServer: srv, + listenAddr: cleanAddr, }) + } for _, opt := range opts { @@ -154,8 +174,9 @@ func (s *Server) OnTerminated(f func(error)) { } func (s *Server) Shutdown(timeout time.Duration) { + // FIXME we need to implement the timeout here for _, server := range s.servers { - server.Shutdown(timeout) + server.Shutdown(nil) } } @@ -166,7 +187,7 @@ func (s *Server) Launch() { go func() { server.Launch(server.listenAddr) for _, srv := range s.servers { - srv.Shutdown(0) // immediately shutdown all other servers when one terminates, in case a single one failed + srv.Shutdown(nil) // immediately shutdown all other servers when one terminates, in case a single one failed } wg.Done() }() @@ -201,3 +222,16 @@ func withRequestMeter(ctx context.Context) context.Context { } return context.WithValue(ctx, requestMeterKey, &requestMeter{}) } + +type InfoServerWrapper struct { + rpcInfoServer *info.InfoServer +} + +// Info implements pbsubstreamsrpcconnect.EndpointInfoHandler. +func (i *InfoServerWrapper) Info(ctx context.Context, req *connect.Request[pbfirehoseV2.InfoRequest]) (*connect.Response[pbfirehoseV2.InfoResponse], error) { + resp, err := i.rpcInfoServer.Info(ctx, req.Msg) + if err != nil { + return nil, err + } + return connect.NewResponse(resp), nil +} diff --git a/firehose/server/v1proxy.go b/firehose/server/v1proxy.go deleted file mode 100644 index a6724d5..0000000 --- a/firehose/server/v1proxy.go +++ /dev/null @@ -1,86 +0,0 @@ -package server - -import ( - "fmt" - - pbfirehoseV1 "github.com/streamingfast/pbgo/sf/firehose/v1" - pbfirehoseV2 "github.com/streamingfast/pbgo/sf/firehose/v2" - "google.golang.org/grpc" -) - -type FirehoseProxyV1ToV2 struct { - server *Server -} - -func NewFirehoseProxyV1ToV2(server *Server) *FirehoseProxyV1ToV2 { - return &FirehoseProxyV1ToV2{ - server: server, - } -} - -func (s *FirehoseProxyV1ToV2) Blocks(req *pbfirehoseV1.Request, streamSrv pbfirehoseV1.Stream_BlocksServer) error { - - var finalBlocksOnly bool - var validSteps bool - var withUndo bool - switch len(req.ForkSteps) { - case 1: - if req.ForkSteps[0] == pbfirehoseV1.ForkStep_STEP_IRREVERSIBLE { - finalBlocksOnly = true - validSteps = true - } - if req.ForkSteps[0] == pbfirehoseV1.ForkStep_STEP_NEW { - validSteps = true - } - case 2: - if (req.ForkSteps[0] == pbfirehoseV1.ForkStep_STEP_NEW && req.ForkSteps[1] == pbfirehoseV1.ForkStep_STEP_UNDO) || - (req.ForkSteps[1] == pbfirehoseV1.ForkStep_STEP_NEW && req.ForkSteps[0] == pbfirehoseV1.ForkStep_STEP_UNDO) { - validSteps = true - withUndo = true - } else if req.ForkSteps[0] == pbfirehoseV1.ForkStep_STEP_NEW && req.ForkSteps[1] == pbfirehoseV1.ForkStep_STEP_IRREVERSIBLE { - validSteps = true - // compatibility hack. you won't receive IRREVERSIBLE here - } - } - if !validSteps { - return fmt.Errorf("invalid parameter for ForkSteps: this server implements firehose v2 operation and only supports [NEW,UNDO] or [IRREVERSIBLE]") - } - - reqV2 := &pbfirehoseV2.Request{ - StartBlockNum: req.StartBlockNum, - Cursor: req.StartCursor, - StopBlockNum: req.StopBlockNum, - FinalBlocksOnly: finalBlocksOnly, - Transforms: req.Transforms, - } - - wrapper := streamWrapper{ServerStream: streamSrv, next: streamSrv, withUndo: withUndo} - - return s.server.Blocks(reqV2, wrapper) -} - -type streamWrapper struct { - grpc.ServerStream - next pbfirehoseV1.Stream_BlocksServer - withUndo bool -} - -func (w streamWrapper) Send(response *pbfirehoseV2.Response) error { - return w.next.Send(&pbfirehoseV1.Response{ - Block: response.Block, - Step: convertForkStep(response.Step), - Cursor: response.Cursor, - }) -} - -func convertForkStep(in pbfirehoseV2.ForkStep) pbfirehoseV1.ForkStep { - switch in { - case pbfirehoseV2.ForkStep_STEP_FINAL: - return pbfirehoseV1.ForkStep_STEP_IRREVERSIBLE - case pbfirehoseV2.ForkStep_STEP_NEW: - return pbfirehoseV1.ForkStep_STEP_NEW - case pbfirehoseV2.ForkStep_STEP_UNDO: - return pbfirehoseV1.ForkStep_STEP_UNDO - } - return pbfirehoseV1.ForkStep_STEP_UNKNOWN -} diff --git a/firehose/tests/integration_test.go b/firehose/tests/integration_test.go deleted file mode 100644 index 1e42891..0000000 --- a/firehose/tests/integration_test.go +++ /dev/null @@ -1,236 +0,0 @@ -package firehose - -//import ( -// "context" -// "encoding/json" -// "fmt" -// "testing" -// "time" -// -// "github.com/alicebob/miniredis/v2/server" -// "github.com/streamingfast/bstream" -// "github.com/streamingfast/dstore" -// pbbstream "github.com/streamingfast/bstream/pb/sf/bstream/v1" -// pbfirehose "github.com/streamingfast/pbgo/sf/firehose/v1" -// "github.com/stretchr/testify/assert" -// "github.com/stretchr/testify/require" -// "go.uber.org/zap" -// "google.golang.org/protobuf/proto" -//) -// -//func TestFullFlow(t *testing.T) { -// -// stepNew := pbfirehose.ForkStep_STEP_NEW -// stepIrr := pbfirehose.ForkStep_STEP_IRREVERSIBLE -// stepUndo := pbfirehose.ForkStep_STEP_UNDO -// _ = stepUndo -// -// type expectedResp struct { -// num uint64 -// id string -// step pbfirehose.ForkStep -// } -// -// tests := []struct { -// name string -// files map[int][]byte -// irreversibleBlocksIndexes map[int]map[int]string -// startBlockNum uint64 -// stopBlockNum uint64 -// cursor *bstream.LastFiredBlock -// expectedResponses []expectedResp -// }{ -// { -// "scenario 1 -- irreversible index, no cursor", -// map[int][]byte{ -// 0: testBlocks( -// 4, "4a", "3a", 0, -// 6, "6a", "4a", 0, -// ), -// 100: testBlocks( -// 100, "100a", "6a", 6, -// 102, "102a", "100a", 6, -// 103, "103a", "102a", 100, // moves LIB from 6 to 100 -// ), -// 200: testBlocks( -// 204, "204b", "103a", 102, // moves LIB from 100 to 102 -// 205, "205b", "103b", 100, //unlinkable -// ), -// }, -// map[int]map[int]string{ -// 0: { -// 4: "4a", -// 6: "6a", -// }, -// 200: { // this hould not be used -// 204: "204a", -// 206: "206a", -// }, -// }, -// 5, -// 0, -// nil, -// []expectedResp{ -// {6, "6a", stepNew}, -// {6, "6a", stepIrr}, -// {100, "100a", stepNew}, -// {102, "102a", stepNew}, -// {103, "103a", stepNew}, -// {100, "100a", stepIrr}, -// {204, "204b", stepNew}, -// {102, "102a", stepIrr}, -// }, -// }, -// { -// "scenario 2 -- no irreversible index, start->stop with some libs", -// map[int][]byte{ -// 0: testBlocks( -// 4, "4a", "3a", 0, -// 6, "6a", "4a", 4, -// ), -// 100: testBlocks( -// 100, "100a", "6a", 6, -// 102, "102a", "100a", 6, -// 103, "103a", "102a", 100, // triggers StepIrr -// 104, "104a", "103a", 100, // after stop block -// ), -// }, -// nil, -// 6, -// 103, -// nil, -// []expectedResp{ -// {6, "6a", stepNew}, -// {100, "100a", stepNew}, -// {6, "6a", stepIrr}, -// {102, "102a", stepNew}, -// {103, "103a", stepNew}, -// }, -// }, -// } -// -// for _, c := range tests { -// t.Run(c.name, func(t *testing.T) { -// -// logger := zap.NewNop() -// bs := dstore.NewMockStore(nil) -// for i, data := range c.files { -// bs.SetFile(base(i), data) -// } -// -// irrStore := getIrrStore(c.irreversibleBlocksIndexes) -// -// // fake block decoder func to return pbbstream.Block -// bstream.GetBlockDecoder = bstream.BlockDecoderFunc(func(blk *pbbstream.Block) (interface{}, error) { -// block := new(pbbstream.Block) -// block.Number = blk.Number -// block.Id = blk.Id -// block.PreviousId = blk.PreviousId -// return block, nil -// }) -// -// tracker := bstream.NewTracker(0) // 0 value not used -// fmt.Println(bstream.GetProtocolFirstStreamableBlock) -// tracker.AddResolver(bstream.OffsetStartBlockResolver(200)) -// -// i := NewStreamFactory( -// []dstore.Store{bs}, -// irrStore, -// []uint64{10000, 1000, 100}, -// nil, -// nil, -// tracker, -// ) -// -// s := server.NewServer( -// logger, -// nil, -// i, -// ) -// -// ctx, cancelCtx := context.WithCancel(context.Background()) -// defer cancelCtx() -// localClient := s.BlocksFromLocal(ctx, &pbfirehose.Request{ -// StartBlockNum: int64(c.startBlockNum), -// StopBlockNum: c.stopBlockNum, -// }) -// -// for _, r := range c.expectedResponses { -// resp, err := localClient.Recv() -// require.NotNil(t, resp) -// require.NoError(t, err) -// -// fmt.Println(resp.LastFiredBlock) -// cursor, err := bstream.CursorFromOpaque(resp.LastFiredBlock) -// require.NoError(t, err, "cursor sent from firehose should always be valid") -// require.False(t, cursor.IsEmpty()) -// -// b := &pbbstream.Block{} -// err = proto.Unmarshal(resp.Block.Value, b) -// require.NoError(t, err) -// -// require.Equal(t, r.num, b.Number) -// require.Equal(t, r.id, b.Id) -// require.Equal(t, r.step, resp.Step) -// } -// -// // catchExtraBlock -// moreChan := make(chan *pbbstream.Block) -// go func() { -// resp, err := localClient.Recv() -// require.NoError(t, err) -// if resp == nil { -// return -// } -// -// b := &pbbstream.Block{} -// err = proto.Unmarshal(resp.Block.Value, b) -// require.NoError(t, err) -// moreChan <- b -// }() -// -// select { -// case resp := <-moreChan: -// assert.Falsef(t, true, "an extra block was seen: %s", resp.String()) -// case <-time.After(time.Millisecond * 50): -// } -// -// }) -// } -// -//} -// -//func base(in int) string { -// return fmt.Sprintf("%010d", in) -//} -// -//func testBlocks(in ...interface{}) (out []byte) { -// var blks []bstream.ParsableTestBlock -// for i := 0; i < len(in); i += 4 { -// blks = append(blks, bstream.ParsableTestBlock{ -// Number: uint64(in[i].(int)), -// ID: in[i+1].(string), -// PreviousID: in[i+2].(string), -// LIBNum: uint64(in[i+3].(int)), -// }) -// } -// -// for _, blk := range blks { -// b, err := json.Marshal(blk) -// if err != nil { -// panic(err) -// } -// out = append(out, b...) -// out = append(out, '\n') -// } -// return -//} -// -//func getIrrStore(irrBlkIdxs map[int]map[int]string) (irrStore *dstore.MockStore) { -// irrStore = dstore.NewMockStore(nil) -// for j, n := range irrBlkIdxs { -// filename, cnt := bstream.TestIrrBlocksIdx(j, 100, n) -// irrStore.SetFile(filename, cnt) -// } -// return -//} diff --git a/firehose/tests/stream_blocks_test.go b/firehose/tests/stream_blocks_test.go deleted file mode 100644 index 3e2dd0a..0000000 --- a/firehose/tests/stream_blocks_test.go +++ /dev/null @@ -1,103 +0,0 @@ -package firehose - -//import ( -// "context" -// "strings" -// "testing" -// -// pbfirehose "github.com/streamingfast/pbgo/sf/firehose/v1" -// -// "github.com/streamingfast/bstream" -// "github.com/streamingfast/dstore" -// pbbstream "github.com/streamingfast/bstream/pb/sf/bstream/v1" -// "github.com/stretchr/testify/assert" -// "github.com/stretchr/testify/require" -// "go.uber.org/zap" -// "google.golang.org/protobuf/proto" -//) -// -//func TestLocalBlocks(t *testing.T) { -// -// store := dstore.NewMockStore(nil) -// idxStore := dstore.NewMockStore(nil) -// blocksStores := []dstore.Store{store} -// logger := zap.NewNop() -// -// i := NewStreamFactory( -// blocksStores, -// idxStore, -// []uint64{10000, 1000, 100}, -// nil, -// nil, -// nil, -// ) -// -// s := NewServer( -// logger, -// nil, -// i, -// ) -// -// // fake block decoder func to return bstream.Block -// bstream.GetBlockDecoder = bstream.BlockDecoderFunc(func(blk *pbbstream.Block) (interface{}, error) { -// block := new(pbbstream.Block) -// block.Number = blk.Number -// block.Id = blk.Id -// block.PreviousId = blk.PreviousId -// return block, nil -// }) -// -// blocks := strings.Join([]string{ -// bstream.TestJSONBlockWithLIBNum("00000002a", "00000001a", 1), -// bstream.TestJSONBlockWithLIBNum("00000003a", "00000002a", 2), -// bstream.TestJSONBlockWithLIBNum("00000004a", "00000003a", 3), // last one closes on endblock -// }, "\n") -// -// store.SetFile("0000000000", []byte(blocks)) -// -// localClient := s.BlocksFromLocal(context.Background(), &pbfirehose.Request{ -// StartBlockNum: 2, -// StopBlockNum: 4, -// }) -// -// // ---- -// blk, err := localClient.Recv() -// require.NoError(t, err) -// b := &pbbstream.Block{} -// err = proto.Unmarshal(blk.Block.Value, b) -// require.NoError(t, err) -// require.Equal(t, uint64(2), b.Number) -// require.Equal(t, blk.Step, pbfirehose.ForkStep_STEP_NEW) -// -// // ---- -// blk, err = localClient.Recv() -// require.NoError(t, err) -// b = &pbbstream.Block{} -// err = proto.Unmarshal(blk.Block.Value, b) -// require.NoError(t, err) -// assert.Equal(t, uint64(3), b.Number) -// assert.Equal(t, blk.Step, pbfirehose.ForkStep_STEP_NEW) -// -// // ---- -// blk, err = localClient.Recv() -// require.NoError(t, err) -// b = &pbbstream.Block{} -// err = proto.Unmarshal(blk.Block.Value, b) -// require.NoError(t, err) -// assert.Equal(t, uint64(2), b.Number) -// assert.Equal(t, blk.Step, pbfirehose.ForkStep_STEP_IRREVERSIBLE) -// -// // ---- -// blk, err = localClient.Recv() -// require.NoError(t, err) -// b = &pbbstream.Block{} -// err = proto.Unmarshal(blk.Block.Value, b) -// require.NoError(t, err) -// assert.Equal(t, uint64(4), b.Number) -// assert.Equal(t, blk.Step, pbfirehose.ForkStep_STEP_NEW) -// -// // ---- -// blk, err = localClient.Recv() -// require.NoError(t, err) -// require.Nil(t, blk) -//} diff --git a/go.mod b/go.mod index 75cd261..39d8fb9 100644 --- a/go.mod +++ b/go.mod @@ -5,7 +5,7 @@ go 1.22.0 require ( buf.build/gen/go/bufbuild/reflect/connectrpc/go v1.16.1-20240117202343-bf8f65e8876c.1 buf.build/gen/go/bufbuild/reflect/protocolbuffers/go v1.33.0-20240117202343-bf8f65e8876c.1 - connectrpc.com/connect v1.16.1 + connectrpc.com/connect v1.18.1 github.com/ShinyTrinkets/overseer v0.3.0 github.com/dustin/go-humanize v1.0.1 github.com/go-json-experiment/json v0.0.0-20231013223334-54c864be5b8d @@ -29,7 +29,7 @@ require ( github.com/streamingfast/dstore v0.1.1-0.20241011152904-9acd6205dc14 github.com/streamingfast/logging v0.0.0-20230608130331-f22c91403091 github.com/streamingfast/payment-gateway v0.0.0-20240426151444-581e930c76e2 - github.com/streamingfast/pbgo v0.0.6-0.20240823134334-812f6a16c5cb + github.com/streamingfast/pbgo v0.0.6-0.20250114182320-0b43084f4000 github.com/streamingfast/snapshotter v0.0.0-20230316190750-5bcadfde44d0 github.com/streamingfast/substreams v1.11.4-0.20250113142113-36c2750be692 github.com/stretchr/testify v1.9.0 @@ -37,11 +37,12 @@ require ( go.uber.org/multierr v1.10.0 go.uber.org/zap v1.26.0 golang.org/x/exp v0.0.0-20231006140011-7918f672742d - google.golang.org/grpc v1.64.0 - google.golang.org/protobuf v1.34.2 + google.golang.org/grpc v1.69.2 + google.golang.org/protobuf v1.35.1 ) require ( + cel.dev/expr v0.16.2 // indirect cloud.google.com/go/auth v0.6.1 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect connectrpc.com/grpchealth v1.3.0 // indirect @@ -60,17 +61,18 @@ require ( github.com/lucasb-eyer/go-colorful v1.2.0 // indirect github.com/mattn/go-isatty v0.0.20 // indirect github.com/muesli/termenv v0.15.3-0.20240618155329-98d742f6907a // indirect + github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/protocolbuffers/protoscope v0.0.0-20221109213918-8e7a6aafa2c9 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/sercand/kuberesolver/v5 v5.1.1 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect ) require ( cloud.google.com/go v0.115.0 // indirect - cloud.google.com/go/compute/metadata v0.3.0 // indirect + cloud.google.com/go/compute/metadata v0.5.2 // indirect cloud.google.com/go/iam v1.1.8 // indirect cloud.google.com/go/monitoring v1.19.0 // indirect cloud.google.com/go/storage v1.42.0 // indirect @@ -79,7 +81,7 @@ require ( contrib.go.opencensus.io/exporter/zipkin v0.1.1 // indirect github.com/Azure/azure-pipeline-go v0.2.3 // indirect github.com/Azure/azure-storage-blob-go v0.14.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v0.32.3 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.15.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.39.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/propagator v0.0.0-20221018185641-36f91511cfd7 // indirect @@ -94,19 +96,19 @@ require ( github.com/bits-and-blooms/bitset v1.12.0 // indirect github.com/blendle/zapdriver v1.3.2-0.20200203083823-9200777f8a3d // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/chzyer/readline v1.5.0 // indirect github.com/cilium/ebpf v0.4.0 // indirect - github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect + github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 // indirect github.com/containerd/cgroups v1.0.4 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/emicklei/go-restful/v3 v3.8.0 // indirect - github.com/envoyproxy/go-control-plane v0.12.0 // indirect - github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect + github.com/envoyproxy/go-control-plane v0.13.1 // indirect + github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect github.com/fsnotify/fsnotify v1.6.0 // indirect - github.com/go-logr/logr v1.4.1 // indirect + github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect github.com/go-openapi/jsonreference v0.19.5 // indirect @@ -150,7 +152,7 @@ require ( github.com/paulbellamy/ratecounter v0.2.0 // indirect github.com/pelletier/go-toml/v2 v2.0.6 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/client_model v0.6.0 // indirect github.com/prometheus/common v0.44.0 // indirect github.com/prometheus/procfs v0.11.0 // indirect github.com/rs/cors v1.10.0 // indirect @@ -170,24 +172,24 @@ require ( github.com/tetratelabs/wazero v1.8.0 // indirect github.com/yourbasic/graph v0.0.0-20210606180040-8ecfec1c2869 // indirect go.opencensus.io v0.24.0 - go.opentelemetry.io/contrib/detectors/gcp v1.9.0 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.31.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 - go.opentelemetry.io/otel v1.24.0 + go.opentelemetry.io/otel v1.31.0 go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.23.1 // indirect go.opentelemetry.io/otel/exporters/zipkin v1.23.1 // indirect - go.opentelemetry.io/otel/metric v1.24.0 // indirect - go.opentelemetry.io/otel/sdk v1.24.0 // indirect - go.opentelemetry.io/otel/trace v1.24.0 // indirect + go.opentelemetry.io/otel/metric v1.31.0 // indirect + go.opentelemetry.io/otel/sdk v1.31.0 // indirect + go.opentelemetry.io/otel/trace v1.31.0 // indirect go.uber.org/atomic v1.10.0 go.uber.org/automaxprocs v1.5.1 - golang.org/x/crypto v0.24.0 // indirect + golang.org/x/crypto v0.28.0 // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.26.0 - golang.org/x/oauth2 v0.21.0 + golang.org/x/net v0.30.0 + golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.25.0 // indirect - golang.org/x/term v0.21.0 // indirect - golang.org/x/text v0.18.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 google.golang.org/api v0.187.0 // indirect google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d // indirect diff --git a/go.sum b/go.sum index 6555f0d..99f5082 100644 --- a/go.sum +++ b/go.sum @@ -2,6 +2,8 @@ buf.build/gen/go/bufbuild/reflect/connectrpc/go v1.16.1-20240117202343-bf8f65e88 buf.build/gen/go/bufbuild/reflect/connectrpc/go v1.16.1-20240117202343-bf8f65e8876c.1/go.mod h1:8aC0AUYVzAH5wP6/43Z89/0un0nvZyf+PVSeeaHyYyg= buf.build/gen/go/bufbuild/reflect/protocolbuffers/go v1.33.0-20240117202343-bf8f65e8876c.1 h1:9ROfgUJtdplIn+2PvUB+Z7HMRVBIsU0uCPAcPfes98I= buf.build/gen/go/bufbuild/reflect/protocolbuffers/go v1.33.0-20240117202343-bf8f65e8876c.1/go.mod h1:TF9ggHlVzYnMjBa4v+ghV6daBmQ0AU4KaAMezoCmX9c= +cel.dev/expr v0.16.2 h1:RwRhoH17VhAu9U5CMvMhH1PDVgf0tuz9FT+24AfMLfU= +cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= @@ -363,8 +365,9 @@ cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= +cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= +cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= @@ -1516,8 +1519,8 @@ cloud.google.com/go/workflows v1.12.3/go.mod h1:fmOUeeqEwPzIU81foMjTRQIdwQHADi/v cloud.google.com/go/workflows v1.12.4/go.mod h1:yQ7HUqOkdJK4duVtMeBCAOPiN1ZF1E9pAMX51vpwB/w= cloud.google.com/go/workflows v1.12.5/go.mod h1:KbK5/Ef28G8MKLXcsvt/laH1Vka4CKeQj0I1/wEiByo= cloud.google.com/go/workflows v1.12.6/go.mod h1:oDbEHKa4otYg4abwdw2Z094jB0TLLiFGAPA78EDAKag= -connectrpc.com/connect v1.16.1 h1:rOdrK/RTI/7TVnn3JsVxt3n028MlTRwmK5Q4heSpjis= -connectrpc.com/connect v1.16.1/go.mod h1:XpZAduBQUySsb4/KO5JffORVkDI4B6/EYPi7N8xpNZw= +connectrpc.com/connect v1.18.1 h1:PAg7CjSAGvscaf6YZKUefjoih5Z/qYkyaTrBW8xvYPw= +connectrpc.com/connect v1.18.1/go.mod h1:0292hj1rnx8oFrStN7cB4jjVBeqs+Yx5yDIC2prWDO8= connectrpc.com/grpchealth v1.3.0 h1:FA3OIwAvuMokQIXQrY5LbIy8IenftksTP/lG4PbYN+E= connectrpc.com/grpchealth v1.3.0/go.mod h1:3vpqmX25/ir0gVgW6RdnCPPZRcR6HvqtXX5RNPmDXHM= connectrpc.com/grpcreflect v1.2.0 h1:Q6og1S7HinmtbEuBvARLNwYmTbhEGRpHDhqrPNlmK+U= @@ -1552,8 +1555,8 @@ github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBp 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/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0/go.mod h1:dppbR7CwXD4pgtV9t3wD1812RaLDcBjtblcDF5f1vI0= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v0.32.3 h1:fiyErF/p5fz79DvMCca9ayvYiWYsFP1oJbskt9fjo8I= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v0.32.3/go.mod h1:s7Gpwj0tk7XnVCm4BQEmx/mbS36SuTCY/vMB2SNxe8o= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 h1:cZpsGsWTIFKymTA0je7IIvi1O7Es7apb9CF3EQlOcfE= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.15.0 h1:5uR5WqunMUqN5Z+USN/N25aM7zWd0JUCIfz1B/w0HtA= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.15.0/go.mod h1:LTScD9l1w6+z1IB3FKtXxS4oenRlIJQQrIiV/Iq1Bsw= github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.39.0 h1:RDD62LpQbuv4rpLOm0w1zlLIcIo7k+zi3EZV5nVyAo8= @@ -1632,8 +1635,9 @@ github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMr github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/charmbracelet/lipgloss v1.0.0 h1:O7VkGDvqEdGi93X+DeqsQ7PKHDgtQfF8j8/O2qFMQNg= github.com/charmbracelet/lipgloss v1.0.0/go.mod h1:U5fy9Z+C38obMs+T+tJqst9VGzlOYGj4ri9reL3qUlo= github.com/charmbracelet/x/ansi v0.4.2 h1:0JM6Aj/g/KC154/gOP4vfxun0ff6itogDYk41kof+qk= @@ -1667,8 +1671,9 @@ github.com/cncf/xds/go v0.0.0-20230428030218-4003588d1b74/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq8dk6e9PdstVsDgu9RuyIIJqAaF//0IM= -github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 h1:DBmgJDC9dTfkVyGgipamEh2BpGYxScCH1TOF1LL1cXc= github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50/go.mod h1:5e1+Vvlzido69INQaVO6d87Qn543Xr6nooe9Kz7oBFM= +github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 h1:QVw89YDxXxEe+l8gU8ETbOasdwEV+avkR75ZzsVV9WI= +github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= github.com/containerd/cgroups v1.0.4 h1:jN/mbWBEaz+T1pi5OFtnkQ+8qnmEbAr1Oo1FRm5B0dA= github.com/containerd/cgroups v1.0.4/go.mod h1:nLNQtsF7Sl2HxNebu77i1R0oDlhiTG+kO4JTrUzo6IA= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= @@ -1702,8 +1707,9 @@ github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJ github.com/envoyproxy/go-control-plane v0.11.0/go.mod h1:VnHyVMpzcLvCFt9yUz1UnCwHLhwx1WguiVDV7pTG/tI= github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q= github.com/envoyproxy/go-control-plane v0.11.1/go.mod h1:uhMcXKCQMEJHiAb0w+YGefQLaTEw+YhGluxZkrTmD0g= -github.com/envoyproxy/go-control-plane v0.12.0 h1:4X+VP1GHd1Mhj6IB5mMeGbLCleqxjletLK6K0rbxyZI= github.com/envoyproxy/go-control-plane v0.12.0/go.mod h1:ZBTaoJ23lqITozF0M6G4/IragXCQKCnYbmlmtHvwRG0= +github.com/envoyproxy/go-control-plane v0.13.1 h1:vPfJZCkob6yTMEgS+0TwfTUfbHjfy/6vOJ8hUWX/uXE= +github.com/envoyproxy/go-control-plane v0.13.1/go.mod h1:X45hY0mufo6Fd0KW3rqsGvQMw58jvjymeCzBU3mWyHw= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= @@ -1711,8 +1717,9 @@ github.com/envoyproxy/protoc-gen-validate v0.10.0/go.mod h1:DRjgyB0I43LtJapqN6Ni github.com/envoyproxy/protoc-gen-validate v0.10.1/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= github.com/envoyproxy/protoc-gen-validate v1.0.1/go.mod h1:0vj8bNkYbSTNS2PIyH87KZaeN4x9zpL9Qt8fQC7d+vs= github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= -github.com/envoyproxy/protoc-gen-validate v1.0.4 h1:gVPz/FMfvh57HdSJQyvBtF00j8JU4zdyUgIUNhlgg0A= github.com/envoyproxy/protoc-gen-validate v1.0.4/go.mod h1:qys6tmnRsYrQqIhm2bvKZH4Blx/1gTIZ2UKVY1M+Yew= +github.com/envoyproxy/protoc-gen-validate v1.1.0 h1:tntQDh69XqOCOZsDz0lVJQez/2L6Uu2PdjCQwWCJ3bM= +github.com/envoyproxy/protoc-gen-validate v1.1.0/go.mod h1:sXRDRVmzEbkM7CVcM06s9shE/m23dg3wzjl0UWqJ2q4= github.com/ettle/strcase v0.1.1/go.mod h1:hzDLsPC7/lwKyBOywSHEP89nt2pDgdy+No1NBA9o9VY= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= @@ -1747,8 +1754,9 @@ github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbV github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= -github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= 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-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= @@ -2061,6 +2069,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 h1:GFCKgmp0tecUJ0sJuv4pzYCqS9+RGSn52M3FUwPs+uo= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1:t/avpk3KcrXxUnYOhZhMXJlSEyie6gQbtLq5NM3loB8= 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/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= @@ -2071,8 +2081,9 @@ github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1: github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= -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/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= +github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= @@ -2160,8 +2171,8 @@ github.com/streamingfast/overseer v0.2.1-0.20210326144022-ee491780e3ef h1:9IVFHR github.com/streamingfast/overseer v0.2.1-0.20210326144022-ee491780e3ef/go.mod h1:cq8CvbZ3ioFmGrHokSAJalS0lC+pVXLKhITScItUGXY= github.com/streamingfast/payment-gateway v0.0.0-20240426151444-581e930c76e2 h1:bliib3pAObbM+6cKYQFa8axbCY/x6RczQZrOxdM7OZA= github.com/streamingfast/payment-gateway v0.0.0-20240426151444-581e930c76e2/go.mod h1:DsnLrpKZ3DIDL6FmYVuxbC44fXvQdY7aCdSLMpbqZ8Q= -github.com/streamingfast/pbgo v0.0.6-0.20240823134334-812f6a16c5cb h1:Xqt4ned9ELmQMKcg7cFbm56MKG2gBjnE1M+2HObOs6w= -github.com/streamingfast/pbgo v0.0.6-0.20240823134334-812f6a16c5cb/go.mod h1:eDQjKBYg9BWE2BTaV3UZeLZ5xw05+ywA9RCFTmM1w5Y= +github.com/streamingfast/pbgo v0.0.6-0.20250114182320-0b43084f4000 h1:1YdNfouYACMSCCWddsyBWZzX7c9s0tBUIoAbMb7mSAM= +github.com/streamingfast/pbgo v0.0.6-0.20250114182320-0b43084f4000/go.mod h1:MHb9nR5zyNyvA1Zi2wud+hWpoILpll7vpLUijvh3YV0= github.com/streamingfast/protoreflect v0.0.0-20231205191344-4b629d20ce8d h1:33VIARqUqBUKXJcuQoOS1rVSms54tgxhhNCmrLptpLg= github.com/streamingfast/protoreflect v0.0.0-20231205191344-4b629d20ce8d/go.mod h1:aBJivEdekmFWYSQ29EE/fN9IanJWJXbtjy3ky0XD/jE= github.com/streamingfast/sf-tracing v0.0.0-20240430173521-888827872b90 h1:94HllkX4ttYVilo8ZJv05b5z8JiMmqBvv4+Jdgk/+2A= @@ -2223,8 +2234,8 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.9.0 h1:en6EnI47A3nrVtKCIgwFS5SUAhYW8LHn4Rkmm6HGhzg= -go.opentelemetry.io/contrib/detectors/gcp v1.9.0/go.mod h1:OqG0FEnmWeJWYVrEovaHXHXY4bVTnp/WfTzhwrsGWlw= +go.opentelemetry.io/contrib/detectors/gcp v1.31.0 h1:G1JQOreVrfhRkner+l4mrGxmfqYCAuy76asTDAo0xsA= +go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0/go.mod h1:r9vWsPS/3AQItv3OSlEJ/E4mbrhUbbw18meOjArPtKQ= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.48.0/go.mod h1:tIKj3DbO8N9Y2xo52og3irLsPI4GW02DSMtrVgNMgxg= @@ -2239,8 +2250,9 @@ go.opentelemetry.io/otel v1.19.0/go.mod h1:i0QyjOq3UPoTzff0PJB2N66fb4S0+rSbSB15/ go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= go.opentelemetry.io/otel v1.23.0/go.mod h1:YCycw9ZeKhcJFrb34iVSkyT0iczq/zYDtZYFufObyB0= -go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= +go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= +go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.23.1 h1:IqmsDcJnxQSs6W+1TMSqpYO7VY4ZuEKJGYlSBPUlT1s= go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.23.1/go.mod h1:VMZ84RYOd4Lrp0+09mckDvqBj2PXWDwOFaxb1P5uO8g= go.opentelemetry.io/otel/exporters/zipkin v1.23.1 h1:goka4KdsPPpHHQnzp1/XE1wVpk2oQO9RXCOH4MZWSyg= @@ -2249,21 +2261,24 @@ go.opentelemetry.io/otel/metric v1.19.0/go.mod h1:L5rUsV9kM1IxCj1MmSdS+JQAcVm319 go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= go.opentelemetry.io/otel/metric v1.23.0/go.mod h1:MqUW2X2a6Q8RN96E2/nqNoT+z9BSms20Jb7Bbp+HiTo= -go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI= go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= +go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= +go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= go.opentelemetry.io/otel/sdk v1.19.0/go.mod h1:NedEbbS4w3C6zElbLdPJKOpJQOrGUJ+GfzpjUvI0v1A= go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= go.opentelemetry.io/otel/sdk v1.22.0/go.mod h1:iu7luyVGYovrRpe2fmj3CVKouQNdTOkxtLzPvPz1DOc= -go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35HoYiQWGDFg= -go.opentelemetry.io/otel/sdk/metric v1.19.0 h1:EJoTO5qysMsYCa+w4UghwFV/ptQgqSL/8Ni+hx+8i1k= -go.opentelemetry.io/otel/sdk/metric v1.19.0/go.mod h1:XjG0jQyFJrv2PbMvwND7LwCEhsJzCzV5210euduKcKY= +go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= +go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= +go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= go.opentelemetry.io/otel/trace v1.19.0/go.mod h1:mfaSyvGyEJEI0nyV2I4qhNQnbBOUUmYZpYojqMnX2vo= go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= go.opentelemetry.io/otel/trace v1.23.0/go.mod h1:GSGTbIClEsuZrGIzoEHqsVfxgn5UkggkflQwDScNUsk= -go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= +go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= +go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -2318,8 +2333,9 @@ golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDf golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/crypto v0.22.0/go.mod h1:vr6Su+7cTlO45qkww3VDJlzDn0ctJvRgYbC2NvXHt+M= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= -golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= +golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= +golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2433,8 +2449,9 @@ golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/net v0.24.0/go.mod h1:2Q7sJY5mzlzWjKtYUEXSlBWCdyaioyXzRB2RtU8KVE8= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= -golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= +golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= +golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -2468,8 +2485,9 @@ golang.org/x/oauth2 v0.17.0/go.mod h1:OzPDGQiuQMguemayvdylqddI7qcD9lnSDb+1FiwQ5H golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/oauth2 v0.19.0/go.mod h1:vYi7skDa1x015PmRRYZ7+s1cWyPgrPiSYRe4rnsexc8= golang.org/x/oauth2 v0.20.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= -golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= +golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= 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= @@ -2573,8 +2591,8 @@ golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= -golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +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/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= 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= @@ -2598,8 +2616,9 @@ golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.19.0/go.mod h1:2CuTdWZ7KHSQwUzKva0cbMg6q2DMI3Mmxp+gKJbskEk= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= -golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= +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.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -2621,8 +2640,8 @@ golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= -golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= -golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +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-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2926,8 +2945,9 @@ google.golang.org/genproto/googleapis/api v0.0.0-20240521202816-d264139d666e/go. google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= google.golang.org/genproto/googleapis/api v0.0.0-20240604185151-ef581f913117/go.mod h1:OimBR/bc1wPO9iV4NC2bpyjy3VnAwZh5EBPQdtaE5oo= google.golang.org/genproto/googleapis/api v0.0.0-20240610135401-a8a62080eff3/go.mod h1:kdrSS/OiLkPrNUpzD4aHgCq2rVuC/YRxok32HXZ4vRE= -google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4 h1:MuYw1wJzT+ZkybKfaOXKp5hJiZDn2iHaXRw0mRYdHSc= google.golang.org/genproto/googleapis/api v0.0.0-20240617180043-68d350f18fd4/go.mod h1:px9SlOOZBg1wM1zdnr8jEL4CNGUBZ+ZKYtNPApNQc4c= +google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= +google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230807174057-1744710a1577/go.mod h1:NjCQG/D8JandXxM57PZbAJL1DCNL6EypA0vPPwfsc7c= google.golang.org/genproto/googleapis/bytestream v0.0.0-20231030173426-d783a09b4405/go.mod h1:GRUCuLdzVqZte8+Dl/D4N25yLzcGqqWaYkeVOwulFqw= @@ -2993,8 +3013,9 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240604185151-ef581f913117/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240610135401-a8a62080eff3/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240617180043-68d350f18fd4/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d h1:k3zyW3BYYR30e8v3x0bTDdE9vpYFjZHK+HcyqkrppWk= google.golang.org/genproto/googleapis/rpc v0.0.0-20240624140628-dc46fd24d27d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= 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= @@ -3045,8 +3066,9 @@ google.golang.org/grpc v1.62.0/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJai google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= google.golang.org/grpc v1.63.0/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= -google.golang.org/grpc v1.64.0 h1:KH3VH9y/MgNQg1dE7b3XfVK0GsPSIzJwdF617gUSbvY= google.golang.org/grpc v1.64.0/go.mod h1:oxjF8E3FBnjp+/gVFYdWacaLDx9na1aqy9oovLpxQYg= +google.golang.org/grpc v1.69.2 h1:U3S9QEtbXC0bYNvRtcoklF3xGtLViumSYxWykJS+7AU= +google.golang.org/grpc v1.69.2/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= 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= @@ -3070,8 +3092,9 @@ google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHh google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= +google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=