From e9534b093575f4d7edb201ba9b2c3738c7e4ad14 Mon Sep 17 00:00:00 2001 From: Aaron Craelius Date: Mon, 27 Jul 2020 13:57:15 -0400 Subject: [PATCH] Add gRPC server & reflection (#6463) * Add gRPC proxy * Make GRPC disabled by default * WIP on integration tests * WIP on integration tests * Start setting up in process tests * Start setting up in process tests * Make it compile * Add start server to network util * Add Println * Use go routine * Fix scopelint * Move to proxy_test * Add response type cache * Remove proxy * Tweaks * Use channel to handle error * Use error chan * Update server/start.go Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com> * Use %w * Add sdk.Context * Add comments * Fix lint * Add header and tests * Address comments * Factorize some code * Fix lint * Add height and prove in req metadata * Add reflection test * Fix lint * Put grpc test in server/grpc * Update baseapp/grpcserver.go * Update baseapp/grpcserver.go * Remove proof header Co-authored-by: Amaury Martiny Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk Co-authored-by: SaReN --- baseapp/abci.go | 18 ++++--- baseapp/baseapp.go | 11 ++-- baseapp/grpcrouter.go | 12 +++++ baseapp/grpcserver.go | 86 ++++++++++++++++++++++++++++++ server/config/config.go | 18 +++++++ server/constructors.go | 35 ------------- server/export.go | 3 +- server/grpc/server.go | 47 +++++++++++++++++ server/grpc/server_test.go | 101 ++++++++++++++++++++++++++++++++++++ server/pruning.go | 13 ++--- server/start.go | 21 ++++++-- server/types/app.go | 48 +++++++++++++++++ server/util.go | 3 +- simapp/simd/cmd/root.go | 3 +- testutil/network/network.go | 18 +++++-- testutil/network/util.go | 10 ++++ types/context.go | 7 +-- 17 files changed, 384 insertions(+), 70 deletions(-) create mode 100644 baseapp/grpcserver.go create mode 100644 server/grpc/server.go create mode 100644 server/grpc/server_test.go create mode 100644 server/types/app.go diff --git a/baseapp/abci.go b/baseapp/abci.go index e5784479e7be..be37b9a4bc4b 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -349,7 +349,7 @@ func (app *BaseApp) Query(req abci.RequestQuery) abci.ResponseQuery { } func (app *BaseApp) handleQueryGRPC(handler GRPCQueryHandler, req abci.RequestQuery) abci.ResponseQuery { - ctx, err := app.createQueryContext(req) + ctx, err := app.createQueryContext(req.Height, req.Prove) if err != nil { return sdkerrors.QueryResult(err) } @@ -364,13 +364,15 @@ func (app *BaseApp) handleQueryGRPC(handler GRPCQueryHandler, req abci.RequestQu return res } -func (app *BaseApp) createQueryContext(req abci.RequestQuery) (sdk.Context, error) { +// createQueryContext creates a new sdk.Context for a query, taking as args +// the block height and whether the query needs a proof or not. +func (app *BaseApp) createQueryContext(height int64, prove bool) (sdk.Context, error) { // when a client did not provide a query height, manually inject the latest - if req.Height == 0 { - req.Height = app.LastBlockHeight() + if height == 0 { + height = app.LastBlockHeight() } - if req.Height <= 1 && req.Prove { + if height <= 1 && prove { return sdk.Context{}, sdkerrors.Wrap( sdkerrors.ErrInvalidRequest, @@ -378,12 +380,12 @@ func (app *BaseApp) createQueryContext(req abci.RequestQuery) (sdk.Context, erro ) } - cacheMS, err := app.cms.CacheMultiStoreWithVersion(req.Height) + cacheMS, err := app.cms.CacheMultiStoreWithVersion(height) if err != nil { return sdk.Context{}, sdkerrors.Wrapf( sdkerrors.ErrInvalidRequest, - "failed to load state at height %d; %s (latest height: %d)", req.Height, err, app.LastBlockHeight(), + "failed to load state at height %d; %s (latest height: %d)", height, err, app.LastBlockHeight(), ) } @@ -517,7 +519,7 @@ func handleQueryCustom(app *BaseApp, path []string, req abci.RequestQuery) abci. return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "no custom querier found for route %s", path[1])) } - ctx, err := app.createQueryContext(req) + ctx, err := app.createQueryContext(req.Height, req.Prove) if err != nil { return sdkerrors.QueryResult(err) } diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 281c869e72b0..0cedd43f6c17 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -172,8 +172,8 @@ func (app *BaseApp) MountStores(keys ...sdk.StoreKey) { } } -// MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp -// multistore. +// MountKVStores mounts all IAVL or DB stores to the provided keys in the +// BaseApp multistore. func (app *BaseApp) MountKVStores(keys map[string]*sdk.KVStoreKey) { for _, key := range keys { if !app.fauxMerkleMode { @@ -186,8 +186,8 @@ func (app *BaseApp) MountKVStores(keys map[string]*sdk.KVStoreKey) { } } -// MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp -// multistore. +// MountTransientStores mounts all IAVL or DB stores to the provided keys in +// the BaseApp multistore. func (app *BaseApp) MountTransientStores(keys map[string]*sdk.TransientStoreKey) { for _, key := range keys { app.MountStore(key, sdk.StoreTypeTransient) @@ -297,9 +297,6 @@ func (app *BaseApp) Router() sdk.Router { // QueryRouter returns the QueryRouter of a BaseApp. func (app *BaseApp) QueryRouter() sdk.QueryRouter { return app.queryRouter } -// GRPCQueryRouter returns the GRPCQueryRouter of a BaseApp. -func (app *BaseApp) GRPCQueryRouter() *GRPCQueryRouter { return app.grpcQueryRouter } - // Seal seals a BaseApp. It prohibits any further modifications to a BaseApp. func (app *BaseApp) Seal() { app.sealed = true } diff --git a/baseapp/grpcrouter.go b/baseapp/grpcrouter.go index 8bb9fcdaa36d..9ed6f7cd44de 100644 --- a/baseapp/grpcrouter.go +++ b/baseapp/grpcrouter.go @@ -20,6 +20,13 @@ var protoCodec = encoding.GetCodec(proto.Name) type GRPCQueryRouter struct { routes map[string]GRPCQueryHandler anyUnpacker types.AnyUnpacker + serviceData []serviceData +} + +// serviceData represents a gRPC service, along with its handler. +type serviceData struct { + serviceDesc *grpc.ServiceDesc + handler interface{} } var _ gogogrpc.Server @@ -83,6 +90,11 @@ func (qrt *GRPCQueryRouter) RegisterService(sd *grpc.ServiceDesc, handler interf }, nil } } + + qrt.serviceData = append(qrt.serviceData, serviceData{ + serviceDesc: sd, + handler: handler, + }) } // AnyUnpacker returns the AnyUnpacker for the router diff --git a/baseapp/grpcserver.go b/baseapp/grpcserver.go new file mode 100644 index 000000000000..0ff2b5407df8 --- /dev/null +++ b/baseapp/grpcserver.go @@ -0,0 +1,86 @@ +package baseapp + +import ( + "context" + "strconv" + + gogogrpc "github.com/gogo/protobuf/grpc" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" + + servergrpc "github.com/cosmos/cosmos-sdk/server/grpc" + sdk "github.com/cosmos/cosmos-sdk/types" +) + +// GRPCQueryRouter returns the GRPCQueryRouter of a BaseApp. +func (app *BaseApp) GRPCQueryRouter() *GRPCQueryRouter { return app.grpcQueryRouter } + +// RegisterGRPCServer registers gRPC services directly with the gRPC server. +func (app *BaseApp) RegisterGRPCServer(server gogogrpc.Server) { + // Define an interceptor for all gRPC queries: this interceptor will create + // a new sdk.Context, and pass it into the query handler. + interceptor := func(grpcCtx context.Context, req interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (resp interface{}, err error) { + // If there's some metadata in the context, retrieve it. + md, ok := metadata.FromIncomingContext(grpcCtx) + if !ok { + return nil, status.Error(codes.Internal, "unable to retrieve metadata") + } + + // Get height header from the request context, if present. + var height int64 + if heightHeaders := md.Get(servergrpc.GRPCBlockHeightHeader); len(heightHeaders) > 0 { + height, err = strconv.ParseInt(heightHeaders[0], 10, 64) + if err != nil { + return nil, err + } + } + + // Create the sdk.Context. Passing false as 2nd arg, as we can't + // actually support proofs with gRPC right now. + sdkCtx, err := app.createQueryContext(height, false) + if err != nil { + return nil, err + } + + // Attach the sdk.Context into the gRPC's context.Context. + grpcCtx = context.WithValue(grpcCtx, sdk.SdkContextKey, sdkCtx) + + // Add relevant gRPC headers + if height == 0 { + height = sdkCtx.BlockHeight() // If height was not set in the request, set it to the latest + } + md = metadata.Pairs(servergrpc.GRPCBlockHeightHeader, strconv.FormatInt(height, 10)) + grpc.SetHeader(grpcCtx, md) + + return handler(grpcCtx, req) + } + + // Loop through all services and methods, add the interceptor, and register + // the service. + for _, data := range app.GRPCQueryRouter().serviceData { + desc := data.serviceDesc + newMethods := make([]grpc.MethodDesc, len(desc.Methods)) + + for i, method := range desc.Methods { + methodHandler := method.Handler + newMethods[i] = grpc.MethodDesc{ + MethodName: method.MethodName, + Handler: func(srv interface{}, ctx context.Context, dec func(interface{}) error, _ grpc.UnaryServerInterceptor) (interface{}, error) { + return methodHandler(srv, ctx, dec, interceptor) + }, + } + } + + newDesc := &grpc.ServiceDesc{ + ServiceName: desc.ServiceName, + HandlerType: desc.HandlerType, + Methods: newMethods, + Streams: desc.Streams, + Metadata: desc.Metadata, + } + + server.RegisterService(newDesc, data.handler) + } +} diff --git a/server/config/config.go b/server/config/config.go index d78072d88494..801aaa06d0ab 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -75,6 +75,15 @@ type APIConfig struct { // Ref: https://github.com/cosmos/cosmos-sdk/issues/6420 } +// GRPCConfig defines configuration for the gRPC server. +type GRPCConfig struct { + // Enable defines if the gRPC server should be enabled. + Enable bool `mapstructure:"enable"` + + // Address defines the API server to listen on + Address string `mapstructure:"address"` +} + // Config defines the server's top level configuration type Config struct { BaseConfig `mapstructure:",squash"` @@ -82,6 +91,7 @@ type Config struct { // Telemetry defines the application telemetry configuration Telemetry telemetry.Config `mapstructure:"telemetry"` API APIConfig `mapstructure:"api"` + GRPC GRPCConfig `mapstructure:"grpc"` } // SetMinGasPrices sets the validator's minimum gas prices. @@ -134,6 +144,10 @@ func DefaultConfig() *Config { RPCReadTimeout: 10, RPCMaxBodyBytes: 1000000, }, + GRPC: GRPCConfig{ + Enable: false, + Address: "0.0.0.0:9090", + }, } } @@ -177,5 +191,9 @@ func GetConfig(v *viper.Viper) Config { RPCMaxBodyBytes: v.GetUint("api.rpc-max-body-bytes"), EnableUnsafeCORS: v.GetBool("api.enabled-unsafe-cors"), }, + GRPC: GRPCConfig{ + Enable: v.GetBool("grpc.enable"), + Address: v.GetString("grpc.address"), + }, } } diff --git a/server/constructors.go b/server/constructors.go index 6ff44a440228..7f6f38b55ce3 100644 --- a/server/constructors.go +++ b/server/constructors.go @@ -1,50 +1,15 @@ package server import ( - "encoding/json" "io" "os" "path/filepath" - abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/libs/log" - tmtypes "github.com/tendermint/tendermint/types" dbm "github.com/tendermint/tm-db" - "github.com/cosmos/cosmos-sdk/server/api" sdk "github.com/cosmos/cosmos-sdk/types" ) -type ( - // AppOptions defines an interface that is passed into an application - // constructor, typically used to set BaseApp options that are either supplied - // via config file or through CLI arguments/flags. The underlying implementation - // is defined by the server package and is typically implemented via a Viper - // literal defined on the server Context. Note, casting Get calls may not yield - // the expected types and could result in type assertion errors. It is recommend - // to either use the cast package or perform manual conversion for safety. - AppOptions interface { - Get(string) interface{} - } - - // Application defines an application interface that wraps abci.Application. - // The interface defines the necessary contracts to be implemented in order - // to fully bootstrap and start an application. - Application interface { - abci.Application - - RegisterAPIRoutes(*api.Server) - } - - // AppCreator is a function that allows us to lazily initialize an - // application using various configurations. - AppCreator func(log.Logger, dbm.DB, io.Writer, AppOptions) Application - - // AppExporter is a function that dumps all app state to - // JSON-serializable structure and returns the current validator set. - AppExporter func(log.Logger, dbm.DB, io.Writer, int64, bool, []string) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error) -) - func openDB(rootDir string) (dbm.DB, error) { dataDir := filepath.Join(rootDir, "data") db, err := sdk.NewLevelDB("application", dataDir) diff --git a/server/export.go b/server/export.go index 0fc94a06a5bd..5be74127d789 100644 --- a/server/export.go +++ b/server/export.go @@ -13,6 +13,7 @@ import ( "github.com/cosmos/cosmos-sdk/client" "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/codec" + "github.com/cosmos/cosmos-sdk/server/types" sdk "github.com/cosmos/cosmos-sdk/types" ) @@ -23,7 +24,7 @@ const ( ) // ExportCmd dumps app state to JSON. -func ExportCmd(appExporter AppExporter, defaultNodeHome string) *cobra.Command { +func ExportCmd(appExporter types.AppExporter, defaultNodeHome string) *cobra.Command { cmd := &cobra.Command{ Use: "export", Short: "Export state to JSON", diff --git a/server/grpc/server.go b/server/grpc/server.go new file mode 100644 index 000000000000..c081133dac22 --- /dev/null +++ b/server/grpc/server.go @@ -0,0 +1,47 @@ +package grpc + +import ( + "fmt" + "net" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/reflection" + + "github.com/cosmos/cosmos-sdk/server/types" +) + +const ( + // GRPCBlockHeightHeader is the gRPC header for block height. + GRPCBlockHeightHeader = "x-cosmos-block-height" +) + +// StartGRPCServer starts a gRPC server on the given address. +func StartGRPCServer(app types.Application, address string) (*grpc.Server, error) { + grpcSrv := grpc.NewServer() + app.RegisterGRPCServer(grpcSrv) + + // Reflection allows external clients to see what services and methods + // the gRPC server exposes. + reflection.Register(grpcSrv) + + listener, err := net.Listen("tcp", address) + if err != nil { + return nil, err + } + + errCh := make(chan error) + go func() { + err = grpcSrv.Serve(listener) + if err != nil { + errCh <- fmt.Errorf("failed to serve: %w", err) + } + }() + + select { + case err := <-errCh: + return nil, err + case <-time.After(5 * time.Second): // assume server started successfully + return grpcSrv, nil + } +} diff --git a/server/grpc/server_test.go b/server/grpc/server_test.go new file mode 100644 index 000000000000..bd8c6979efc3 --- /dev/null +++ b/server/grpc/server_test.go @@ -0,0 +1,101 @@ +package grpc_test + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "google.golang.org/grpc/metadata" + + servergrpc "github.com/cosmos/cosmos-sdk/server/grpc" + "github.com/cosmos/cosmos-sdk/testutil/network" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + banktypes "github.com/cosmos/cosmos-sdk/x/bank/types" + rpb "google.golang.org/grpc/reflection/grpc_reflection_v1alpha" +) + +type IntegrationTestSuite struct { + suite.Suite + + network *network.Network +} + +func (s *IntegrationTestSuite) SetupSuite() { + s.T().Log("setting up integration test suite") + + s.network = network.New(s.T(), network.DefaultConfig()) + s.Require().NotNil(s.network) + + _, err := s.network.WaitForHeight(2) + s.Require().NoError(err) +} + +func (s *IntegrationTestSuite) TearDownSuite() { + s.T().Log("tearing down integration test suite") + s.network.Cleanup() +} + +func (s *IntegrationTestSuite) TestGRPC() { + val0 := s.network.Validators[0] + conn, err := grpc.Dial( + val0.AppConfig.GRPC.Address, + grpc.WithInsecure(), // Or else we get "no transport security set" + ) + s.Require().NoError(err) + + // gRPC query to test service should work + testClient := testdata.NewTestServiceClient(conn) + testRes, err := testClient.Echo(context.Background(), &testdata.EchoRequest{Message: "hello"}) + s.Require().NoError(err) + s.Require().Equal("hello", testRes.Message) + + // gRPC query to bank service should work + denom := fmt.Sprintf("%stoken", val0.Moniker) + bankClient := banktypes.NewQueryClient(conn) + var header metadata.MD + bankRes, err := bankClient.Balance( + context.Background(), + &banktypes.QueryBalanceRequest{Address: val0.Address, Denom: denom}, + grpc.Header(&header), // Also fetch grpc header + ) + s.Require().NoError(err) + s.Require().Equal( + sdk.NewCoin(denom, s.network.Config.AccountTokens), + *bankRes.GetBalance(), + ) + blockHeight := header.Get(servergrpc.GRPCBlockHeightHeader) + s.Require().Equal([]string{"2"}, blockHeight) + + // Request metadata should work + bankRes, err = bankClient.Balance( + metadata.AppendToOutgoingContext(context.Background(), servergrpc.GRPCBlockHeightHeader, "1"), // Add metadata to request + &banktypes.QueryBalanceRequest{Address: val0.Address, Denom: denom}, + grpc.Header(&header), + ) + blockHeight = header.Get(servergrpc.GRPCBlockHeightHeader) + s.Require().Equal([]string{"1"}, blockHeight) + + // Test server reflection + reflectClient := rpb.NewServerReflectionClient(conn) + stream, err := reflectClient.ServerReflectionInfo(context.Background(), grpc.WaitForReady(true)) + s.Require().NoError(err) + s.Require().NoError(stream.Send(&rpb.ServerReflectionRequest{ + MessageRequest: &rpb.ServerReflectionRequest_ListServices{}, + })) + res, err := stream.Recv() + s.Require().NoError(err) + services := res.GetListServicesResponse().Service + servicesMap := make(map[string]bool) + for _, s := range services { + servicesMap[s.Name] = true + } + // Make sure the following services are present + s.Require().True(servicesMap["cosmos.bank.Query"]) +} + +func TestIntegrationTestSuite(t *testing.T) { + suite.Run(t, new(IntegrationTestSuite)) +} diff --git a/server/pruning.go b/server/pruning.go index ca4f9d6e57ad..44aa4ba2d1e6 100644 --- a/server/pruning.go +++ b/server/pruning.go @@ -6,22 +6,23 @@ import ( "github.com/spf13/cast" + "github.com/cosmos/cosmos-sdk/server/types" "github.com/cosmos/cosmos-sdk/store" - "github.com/cosmos/cosmos-sdk/store/types" + storetypes "github.com/cosmos/cosmos-sdk/store/types" ) // GetPruningOptionsFromFlags parses command flags and returns the correct // PruningOptions. If a pruning strategy is provided, that will be parsed and // returned, otherwise, it is assumed custom pruning options are provided. -func GetPruningOptionsFromFlags(appOpts AppOptions) (types.PruningOptions, error) { +func GetPruningOptionsFromFlags(appOpts types.AppOptions) (storetypes.PruningOptions, error) { strategy := strings.ToLower(cast.ToString(appOpts.Get(FlagPruning))) switch strategy { - case types.PruningOptionDefault, types.PruningOptionNothing, types.PruningOptionEverything: - return types.NewPruningOptionsFromString(strategy), nil + case storetypes.PruningOptionDefault, storetypes.PruningOptionNothing, storetypes.PruningOptionEverything: + return storetypes.NewPruningOptionsFromString(strategy), nil - case types.PruningOptionCustom: - opts := types.NewPruningOptions( + case storetypes.PruningOptionCustom: + opts := storetypes.NewPruningOptions( cast.ToUint64(appOpts.Get(FlagPruningKeepRecent)), cast.ToUint64(appOpts.Get(FlagPruningKeepEvery)), cast.ToUint64(appOpts.Get(FlagPruningInterval)), diff --git a/server/start.go b/server/start.go index e02933412812..810080f6f7f2 100644 --- a/server/start.go +++ b/server/start.go @@ -17,12 +17,15 @@ import ( pvm "github.com/tendermint/tendermint/privval" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/rpc/client/local" + "google.golang.org/grpc" "github.com/cosmos/cosmos-sdk/client" "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/server/api" "github.com/cosmos/cosmos-sdk/server/config" + servergrpc "github.com/cosmos/cosmos-sdk/server/grpc" + "github.com/cosmos/cosmos-sdk/server/types" storetypes "github.com/cosmos/cosmos-sdk/store/types" ) @@ -49,7 +52,7 @@ const ( // StartCmd runs the service passed in, either stand-alone or in-process with // Tendermint. -func StartCmd(appCreator AppCreator, defaultNodeHome string) *cobra.Command { +func StartCmd(appCreator types.AppCreator, defaultNodeHome string) *cobra.Command { cmd := &cobra.Command{ Use: "start", Short: "Run the full node", @@ -125,7 +128,7 @@ which accepts a path for the resulting pprof file. return cmd } -func startStandAlone(ctx *Context, appCreator AppCreator) error { +func startStandAlone(ctx *Context, appCreator types.AppCreator) error { addr := ctx.Viper.GetString(flagAddress) transport := ctx.Viper.GetString(flagTransport) home := ctx.Viper.GetString(flags.FlagHome) @@ -165,7 +168,7 @@ func startStandAlone(ctx *Context, appCreator AppCreator) error { select {} } -func startInProcess(ctx *Context, cdc codec.JSONMarshaler, appCreator AppCreator) error { +func startInProcess(ctx *Context, cdc codec.JSONMarshaler, appCreator types.AppCreator) error { cfg := ctx.Config home := cfg.RootDir @@ -239,6 +242,14 @@ func startInProcess(ctx *Context, cdc codec.JSONMarshaler, appCreator AppCreator } } + var grpcSrv *grpc.Server + if config.GRPC.Enable { + grpcSrv, err = servergrpc.StartGRPCServer(app, config.GRPC.Address) + if err != nil { + return err + } + } + var cpuProfileCleanup func() if cpuProfile := ctx.Viper.GetString(flagCPUProfile); cpuProfile != "" { @@ -272,6 +283,10 @@ func startInProcess(ctx *Context, cdc codec.JSONMarshaler, appCreator AppCreator _ = apiSrv.Close() } + if grpcSrv != nil { + grpcSrv.Stop() + } + ctx.Logger.Info("exiting...") }) diff --git a/server/types/app.go b/server/types/app.go new file mode 100644 index 000000000000..f68a93207630 --- /dev/null +++ b/server/types/app.go @@ -0,0 +1,48 @@ +package types + +import ( + "encoding/json" + "io" + + "github.com/gogo/protobuf/grpc" + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + tmtypes "github.com/tendermint/tendermint/types" + dbm "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/server/api" +) + +type ( + // AppOptions defines an interface that is passed into an application + // constructor, typically used to set BaseApp options that are either supplied + // via config file or through CLI arguments/flags. The underlying implementation + // is defined by the server package and is typically implemented via a Viper + // literal defined on the server Context. Note, casting Get calls may not yield + // the expected types and could result in type assertion errors. It is recommend + // to either use the cast package or perform manual conversion for safety. + AppOptions interface { + Get(string) interface{} + } + + // Application defines an application interface that wraps abci.Application. + // The interface defines the necessary contracts to be implemented in order + // to fully bootstrap and start an application. + Application interface { + abci.Application + + RegisterAPIRoutes(*api.Server) + + // RegisterGRPCServer registers gRPC services directly with the gRPC + // server. + RegisterGRPCServer(grpc.Server) + } + + // AppCreator is a function that allows us to lazily initialize an + // application using various configurations. + AppCreator func(log.Logger, dbm.DB, io.Writer, AppOptions) Application + + // AppExporter is a function that dumps all app state to + // JSON-serializable structure and returns the current validator set. + AppExporter func(log.Logger, dbm.DB, io.Writer, int64, bool, []string) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error) +) diff --git a/server/util.go b/server/util.go index 7b329e49c09c..842bfbc837d8 100644 --- a/server/util.go +++ b/server/util.go @@ -23,6 +23,7 @@ import ( "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/server/config" + "github.com/cosmos/cosmos-sdk/server/types" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/version" ) @@ -167,7 +168,7 @@ func interceptConfigs(ctx *Context, rootViper *viper.Viper) (*tmcfg.Config, erro } // add server commands -func AddCommands(rootCmd *cobra.Command, appCreator AppCreator, appExport AppExporter) { +func AddCommands(rootCmd *cobra.Command, appCreator types.AppCreator, appExport types.AppExporter) { tendermintCmd := &cobra.Command{ Use: "tendermint", Short: "Tendermint subcommands", diff --git a/simapp/simd/cmd/root.go b/simapp/simd/cmd/root.go index 48695b04f480..10f47c3b7194 100644 --- a/simapp/simd/cmd/root.go +++ b/simapp/simd/cmd/root.go @@ -21,6 +21,7 @@ import ( "github.com/cosmos/cosmos-sdk/client/keys" "github.com/cosmos/cosmos-sdk/client/rpc" "github.com/cosmos/cosmos-sdk/server" + servertypes "github.com/cosmos/cosmos-sdk/server/types" "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/store" sdk "github.com/cosmos/cosmos-sdk/types" @@ -148,7 +149,7 @@ func txCommand() *cobra.Command { return cmd } -func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, appOpts server.AppOptions) server.Application { +func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, appOpts servertypes.AppOptions) servertypes.Application { var cache sdk.MultiStorePersistentCache if cast.ToBool(appOpts.Get(server.FlagInterBlockCache)) { diff --git a/testutil/network/network.go b/testutil/network/network.go index af561b79f48f..81668ae49666 100644 --- a/testutil/network/network.go +++ b/testutil/network/network.go @@ -24,6 +24,7 @@ import ( "github.com/tendermint/tendermint/node" tmclient "github.com/tendermint/tendermint/rpc/client" dbm "github.com/tendermint/tm-db" + "google.golang.org/grpc" "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/client" @@ -33,6 +34,7 @@ import ( "github.com/cosmos/cosmos-sdk/server" "github.com/cosmos/cosmos-sdk/server/api" srvconfig "github.com/cosmos/cosmos-sdk/server/config" + servertypes "github.com/cosmos/cosmos-sdk/server/types" "github.com/cosmos/cosmos-sdk/simapp" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" @@ -47,9 +49,9 @@ var lock = new(sync.Mutex) // AppConstructor defines a function which accepts a network configuration and // creates an ABCI Application to provide to Tendermint. -type AppConstructor = func(val Validator) server.Application +type AppConstructor = func(val Validator) servertypes.Application -func NewSimApp(val Validator) server.Application { +func NewSimApp(val Validator) servertypes.Application { return simapp.NewSimApp( val.Ctx.Logger, dbm.NewMemDB(), nil, true, make(map[int64]bool), val.Ctx.Config.RootDir, 0, baseapp.SetPruning(storetypes.NewPruningOptionsFromString(val.AppConfig.Pruning)), @@ -120,7 +122,7 @@ type ( BaseDir string Validators []*Validator - config Config + Config Config } // Validator defines an in-process Tendermint validator node. Through this object, @@ -143,6 +145,7 @@ type ( tmNode *node.Node api *api.Server + grpc *grpc.Server } ) @@ -159,7 +162,7 @@ func New(t *testing.T, cfg Config) *Network { T: t, BaseDir: baseDir, Validators: make([]*Validator, cfg.NumValidators), - config: cfg, + Config: cfg, } t.Log("preparing test network...") @@ -205,6 +208,11 @@ func New(t *testing.T, cfg Config) *Network { rpcAddr, _, err := server.FreeTCPAddr() require.NoError(t, err) tmCfg.RPC.ListenAddress = rpcAddr + + _, grpcPort, err := server.FreeTCPAddr() + require.NoError(t, err) + appCfg.GRPC.Address = fmt.Sprintf("0.0.0.0:%s", grpcPort) + appCfg.GRPC.Enable = true } logger := log.NewNopLogger() @@ -431,7 +439,7 @@ func (n *Network) Cleanup() { } } - if n.config.CleanupDir { + if n.Config.CleanupDir { _ = os.RemoveAll(n.BaseDir) } diff --git a/testutil/network/util.go b/testutil/network/util.go index 86bed50eb34e..4754bc7959a5 100644 --- a/testutil/network/util.go +++ b/testutil/network/util.go @@ -15,6 +15,7 @@ import ( "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/server/api" + servergrpc "github.com/cosmos/cosmos-sdk/server/grpc" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" banktypes "github.com/cosmos/cosmos-sdk/x/bank/types" "github.com/cosmos/cosmos-sdk/x/genutil" @@ -82,6 +83,15 @@ func startInProcess(cfg Config, val *Validator) error { val.api = apiSrv } + if val.AppConfig.GRPC.Enable { + grpcSrv, err := servergrpc.StartGRPCServer(app, val.AppConfig.GRPC.Address) + if err != nil { + return err + } + + val.grpc = grpcSrv + } + return nil } diff --git a/types/context.go b/types/context.go index 0cc01091c1f2..251ce6828b69 100644 --- a/types/context.go +++ b/types/context.go @@ -245,19 +245,20 @@ func (c Context) CacheContext() (cc Context, writeCache func()) { // ContextKey defines a type alias for a stdlib Context key. type ContextKey string -const sdkContextKey ContextKey = "sdk-context" +// SdkContextKey is the key in the context.Context which holds the sdk.Context. +const SdkContextKey ContextKey = "sdk-context" // WrapSDKContext returns a stdlib context.Context with the provided sdk.Context's internal // context as a value. It is useful for passing an sdk.Context through methods that take a // stdlib context.Context parameter such as generated gRPC methods. To get the original // sdk.Context back, call UnwrapSDKContext. func WrapSDKContext(ctx Context) context.Context { - return context.WithValue(ctx.ctx, sdkContextKey, ctx) + return context.WithValue(ctx.ctx, SdkContextKey, ctx) } // UnwrapSDKContext retrieves a Context from a context.Context instance // attached with WrapSDKContext. It panics if a Context was not properly // attached func UnwrapSDKContext(ctx context.Context) Context { - return ctx.Value(sdkContextKey).(Context) + return ctx.Value(SdkContextKey).(Context) }