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 <amaury.martiny@protonmail.com>
Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com>
Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com>
Co-authored-by: SaReN <sahithnarahari@gmail.com>
This commit is contained in:
Aaron Craelius 2020-07-27 13:57:15 -04:00 committed by GitHub
parent 20488b4f8e
commit e9534b0935
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 384 additions and 70 deletions

View File

@ -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)
}

View File

@ -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 }

View File

@ -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

86
baseapp/grpcserver.go Normal file
View File

@ -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)
}
}

View File

@ -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"),
},
}
}

View File

@ -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)

View File

@ -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",

47
server/grpc/server.go Normal file
View File

@ -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
}
}

101
server/grpc/server_test.go Normal file
View File

@ -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))
}

View File

@ -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)),

View File

@ -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...")
})

48
server/types/app.go Normal file
View File

@ -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)
)

View File

@ -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",

View File

@ -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)) {

View File

@ -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)
}

View File

@ -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
}

View File

@ -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)
}