diff --git a/client/client.go b/client/client.go index b4adfbcda..3afb838f8 100644 --- a/client/client.go +++ b/client/client.go @@ -1,8 +1,10 @@ package tmspcli import ( - "github.com/tendermint/tmsp/types" + "fmt" "sync" + + "github.com/tendermint/tmsp/types" ) type Client interface { @@ -39,6 +41,21 @@ type Client interface { //---------------------------------------- +func NewClient(addr, transport string, mustConnect bool) (client Client, err error) { + switch transport { + case "socket": + client, err = NewSocketClient(addr, mustConnect) + case "grpc": + client, err = NewGRPCClient(addr, mustConnect) + default: + err = fmt.Errorf("Unknown tmsp transport %s", transport) + + } + return +} + +//---------------------------------------- + type Callback func(*types.Request, *types.Response) //---------------------------------------- diff --git a/client/grpc_client.go b/client/grpc_client.go new file mode 100644 index 000000000..3fbd000b0 --- /dev/null +++ b/client/grpc_client.go @@ -0,0 +1,296 @@ +package tmspcli + +import ( + "fmt" + "net" + "sync" + "time" + + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" + + . "github.com/tendermint/go-common" + "github.com/tendermint/tmsp/types" +) + +// A stripped copy of the remoteClient that makes +// synchronous calls using grpc +type grpcClient struct { + QuitService + mustConnect bool + + client types.TMSPApplicationClient + + mtx sync.Mutex + addr string + err error + resCb func(*types.Request, *types.Response) // listens to all callbacks +} + +func NewGRPCClient(addr string, mustConnect bool) (*grpcClient, error) { + cli := &grpcClient{ + addr: addr, + mustConnect: mustConnect, + } + cli.QuitService = *NewQuitService(nil, "grpcClient", cli) + _, err := cli.Start() // Just start it, it's confusing for callers to remember to start. + return cli, err +} + +func dialerFunc(addr string, timeout time.Duration) (net.Conn, error) { + return Connect(addr) +} + +func (cli *grpcClient) OnStart() error { + cli.QuitService.OnStart() +RETRY_LOOP: + for { + conn, err := grpc.Dial(cli.addr, grpc.WithInsecure(), grpc.WithDialer(dialerFunc)) + if err != nil { + if cli.mustConnect { + return err + } else { + fmt.Printf("tmsp.grpcClient failed to connect to %v. Retrying...\n", cli.addr) + time.Sleep(time.Second * 3) + continue RETRY_LOOP + } + } + cli.client = types.NewTMSPApplicationClient(conn) + return nil + } +} + +func (cli *grpcClient) OnStop() { + cli.QuitService.OnStop() + // TODO: how to close when TMSPApplicationClient interface doesn't expose Close ? +} + +// Set listener for all responses +// NOTE: callback may get internally generated flush responses. +func (cli *grpcClient) SetResponseCallback(resCb Callback) { + cli.mtx.Lock() + defer cli.mtx.Unlock() + cli.resCb = resCb +} + +func (cli *grpcClient) StopForError(err error) { + cli.mtx.Lock() + fmt.Printf("Stopping tmsp.grpcClient for error: %v\n", err.Error()) + if cli.err == nil { + cli.err = err + } + cli.mtx.Unlock() + cli.Stop() +} + +func (cli *grpcClient) Error() error { + cli.mtx.Lock() + defer cli.mtx.Unlock() + return cli.err +} + +//---------------------------------------- +// async calls are really sync. +// maybe one day, if people really want it, we use grpc streams, +// but hopefully not :D + +func (cli *grpcClient) EchoAsync(msg string) *ReqRes { + req := types.ToRequestEcho(msg) + res, err := cli.client.Echo(context.Background(), req.GetEcho()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_Echo{res}}) +} + +func (cli *grpcClient) FlushAsync() *ReqRes { + req := types.ToRequestFlush() + res, err := cli.client.Flush(context.Background(), req.GetFlush()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_Flush{res}}) +} + +func (cli *grpcClient) InfoAsync() *ReqRes { + req := types.ToRequestInfo() + res, err := cli.client.Info(context.Background(), req.GetInfo()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_Info{res}}) +} + +func (cli *grpcClient) SetOptionAsync(key string, value string) *ReqRes { + req := types.ToRequestSetOption(key, value) + res, err := cli.client.SetOption(context.Background(), req.GetSetOption()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_SetOption{res}}) +} + +func (cli *grpcClient) AppendTxAsync(tx []byte) *ReqRes { + req := types.ToRequestAppendTx(tx) + res, err := cli.client.AppendTx(context.Background(), req.GetAppendTx()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_AppendTx{res}}) +} + +func (cli *grpcClient) CheckTxAsync(tx []byte) *ReqRes { + req := types.ToRequestCheckTx(tx) + res, err := cli.client.CheckTx(context.Background(), req.GetCheckTx()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_CheckTx{res}}) +} + +func (cli *grpcClient) QueryAsync(query []byte) *ReqRes { + req := types.ToRequestQuery(query) + res, err := cli.client.Query(context.Background(), req.GetQuery()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_Query{res}}) +} + +func (cli *grpcClient) CommitAsync() *ReqRes { + req := types.ToRequestCommit() + res, err := cli.client.Commit(context.Background(), req.GetCommit()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_Commit{res}}) +} + +func (cli *grpcClient) InitChainAsync(validators []*types.Validator) *ReqRes { + req := types.ToRequestInitChain(validators) + res, err := cli.client.InitChain(context.Background(), req.GetInitChain()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_InitChain{res}}) +} + +func (cli *grpcClient) BeginBlockAsync(height uint64) *ReqRes { + req := types.ToRequestBeginBlock(height) + res, err := cli.client.BeginBlock(context.Background(), req.GetBeginBlock()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_BeginBlock{res}}) +} + +func (cli *grpcClient) EndBlockAsync(height uint64) *ReqRes { + req := types.ToRequestEndBlock(height) + res, err := cli.client.EndBlock(context.Background(), req.GetEndBlock()) + if err != nil { + cli.err = err + } + return cli.finishAsyncCall(req, &types.Response{&types.Response_EndBlock{res}}) +} + +func (cli *grpcClient) finishAsyncCall(req *types.Request, res *types.Response) *ReqRes { + reqres := NewReqRes(req) + reqres.Response = res // Set response + reqres.Done() // Release waiters + + // Notify reqRes listener if set + if cb := reqres.GetCallback(); cb != nil { + cb(res) + } + + // Notify client listener if set + if cli.resCb != nil { + cli.resCb(reqres.Request, res) + } + return reqres +} + +//---------------------------------------- + +func (cli *grpcClient) EchoSync(msg string) (res types.Result) { + r := cli.EchoAsync(msg).Response.GetEcho() + return types.NewResultOK([]byte(r.Message), LOG) +} + +func (cli *grpcClient) FlushSync() error { + return nil +} + +func (cli *grpcClient) InfoSync() (res types.Result) { + r := cli.InfoAsync().Response.GetInfo() + return types.NewResultOK([]byte(r.Info), LOG) +} + +func (cli *grpcClient) SetOptionSync(key string, value string) (res types.Result) { + reqres := cli.SetOptionAsync(key, value) + if cli.err != nil { + return types.ErrInternalError.SetLog(cli.err.Error()) + } + resp := reqres.Response.GetSetOption() + return types.Result{Code: OK, Data: nil, Log: resp.Log} +} + +func (cli *grpcClient) AppendTxSync(tx []byte) (res types.Result) { + reqres := cli.AppendTxAsync(tx) + if cli.err != nil { + return types.ErrInternalError.SetLog(cli.err.Error()) + } + resp := reqres.Response.GetAppendTx() + return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} +} + +func (cli *grpcClient) CheckTxSync(tx []byte) (res types.Result) { + reqres := cli.CheckTxAsync(tx) + if cli.err != nil { + return types.ErrInternalError.SetLog(cli.err.Error()) + } + resp := reqres.Response.GetCheckTx() + return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} +} + +func (cli *grpcClient) QuerySync(query []byte) (res types.Result) { + reqres := cli.QueryAsync(query) + if cli.err != nil { + return types.ErrInternalError.SetLog(cli.err.Error()) + } + resp := reqres.Response.GetQuery() + return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} +} + +func (cli *grpcClient) CommitSync() (res types.Result) { + reqres := cli.CommitAsync() + if cli.err != nil { + return types.ErrInternalError.SetLog(cli.err.Error()) + } + resp := reqres.Response.GetCommit() + return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} +} + +func (cli *grpcClient) InitChainSync(validators []*types.Validator) (err error) { + cli.InitChainAsync(validators) + if cli.err != nil { + return cli.err + } + return nil +} + +func (cli *grpcClient) BeginBlockSync(height uint64) (err error) { + cli.BeginBlockAsync(height) + if cli.err != nil { + return cli.err + } + return nil +} + +func (cli *grpcClient) EndBlockSync(height uint64) (validators []*types.Validator, err error) { + reqres := cli.EndBlockAsync(height) + if cli.err != nil { + return nil, cli.err + } + return reqres.Response.GetEndBlock().Diffs, nil +} diff --git a/client/local_client.go b/client/local_client.go index 28ef6f329..06db859b4 100644 --- a/client/local_client.go +++ b/client/local_client.go @@ -38,13 +38,13 @@ func (app *localClient) Stop() bool { func (app *localClient) FlushAsync() *ReqRes { // Do nothing - return newLocalReqRes(types.RequestFlush(), nil) + return newLocalReqRes(types.ToRequestFlush(), nil) } func (app *localClient) EchoAsync(msg string) *ReqRes { return app.callback( - types.RequestEcho(msg), - types.ResponseEcho(msg), + types.ToRequestEcho(msg), + types.ToResponseEcho(msg), ) } @@ -53,8 +53,8 @@ func (app *localClient) InfoAsync() *ReqRes { info := app.Application.Info() app.mtx.Unlock() return app.callback( - types.RequestInfo(), - types.ResponseInfo(info), + types.ToRequestInfo(), + types.ToResponseInfo(info), ) } @@ -63,8 +63,8 @@ func (app *localClient) SetOptionAsync(key string, value string) *ReqRes { log := app.Application.SetOption(key, value) app.mtx.Unlock() return app.callback( - types.RequestSetOption(key, value), - types.ResponseSetOption(log), + types.ToRequestSetOption(key, value), + types.ToResponseSetOption(log), ) } @@ -73,8 +73,8 @@ func (app *localClient) AppendTxAsync(tx []byte) *ReqRes { res := app.Application.AppendTx(tx) app.mtx.Unlock() return app.callback( - types.RequestAppendTx(tx), - types.ResponseAppendTx(res.Code, res.Data, res.Log), + types.ToRequestAppendTx(tx), + types.ToResponseAppendTx(res.Code, res.Data, res.Log), ) } @@ -83,8 +83,8 @@ func (app *localClient) CheckTxAsync(tx []byte) *ReqRes { res := app.Application.CheckTx(tx) app.mtx.Unlock() return app.callback( - types.RequestCheckTx(tx), - types.ResponseCheckTx(res.Code, res.Data, res.Log), + types.ToRequestCheckTx(tx), + types.ToResponseCheckTx(res.Code, res.Data, res.Log), ) } @@ -93,8 +93,8 @@ func (app *localClient) QueryAsync(tx []byte) *ReqRes { res := app.Application.Query(tx) app.mtx.Unlock() return app.callback( - types.RequestQuery(tx), - types.ResponseQuery(res.Code, res.Data, res.Log), + types.ToRequestQuery(tx), + types.ToResponseQuery(res.Code, res.Data, res.Log), ) } @@ -103,8 +103,8 @@ func (app *localClient) CommitAsync() *ReqRes { res := app.Application.Commit() app.mtx.Unlock() return app.callback( - types.RequestCommit(), - types.ResponseCommit(res.Code, res.Data, res.Log), + types.ToRequestCommit(), + types.ToResponseCommit(res.Code, res.Data, res.Log), ) } @@ -114,8 +114,8 @@ func (app *localClient) InitChainAsync(validators []*types.Validator) *ReqRes { bcApp.InitChain(validators) } reqRes := app.callback( - types.RequestInitChain(validators), - types.ResponseInitChain(), + types.ToRequestInitChain(validators), + types.ToResponseInitChain(), ) app.mtx.Unlock() return reqRes @@ -128,8 +128,8 @@ func (app *localClient) BeginBlockAsync(height uint64) *ReqRes { } app.mtx.Unlock() return app.callback( - types.RequestBeginBlock(height), - types.ResponseBeginBlock(), + types.ToRequestBeginBlock(height), + types.ToResponseBeginBlock(), ) } @@ -141,8 +141,8 @@ func (app *localClient) EndBlockAsync(height uint64) *ReqRes { } app.mtx.Unlock() return app.callback( - types.RequestEndBlock(height), - types.ResponseEndBlock(validators), + types.ToRequestEndBlock(height), + types.ToResponseEndBlock(validators), ) } diff --git a/client/remote_client.go b/client/remote_client.go index d0aff152b..3c6803b7e 100644 --- a/client/remote_client.go +++ b/client/remote_client.go @@ -6,6 +6,7 @@ import ( "errors" "fmt" "net" + "reflect" "sync" "time" @@ -13,6 +14,11 @@ import ( "github.com/tendermint/tmsp/types" ) +const ( + OK = types.CodeType_OK + LOG = "" +) + const reqQueueSize = 256 // TODO make configurable const maxResponseSize = 1048576 // 1MB TODO make configurable const flushThrottleMS = 20 // Don't wait longer than... @@ -36,7 +42,7 @@ type remoteClient struct { resCb func(*types.Request, *types.Response) // listens to all callbacks } -func NewClient(addr string, mustConnect bool) (*remoteClient, error) { +func NewSocketClient(addr string, mustConnect bool) (*remoteClient, error) { cli := &remoteClient{ reqQueue: make(chan *ReqRes, reqQueueSize), flushTimer: NewThrottleTimer("remoteClient", flushThrottleMS), @@ -48,39 +54,28 @@ func NewClient(addr string, mustConnect bool) (*remoteClient, error) { } cli.QuitService = *NewQuitService(nil, "remoteClient", cli) _, err := cli.Start() // Just start it, it's confusing for callers to remember to start. - if mustConnect { - return nil, err - } else { - return cli, nil - } + return cli, err } -func (cli *remoteClient) OnStart() (err error) { +func (cli *remoteClient) OnStart() error { cli.QuitService.OnStart() - doneCh := make(chan struct{}) - go func() { - RETRY_LOOP: - for { - conn, err_ := Connect(cli.addr) - if err_ != nil { - if cli.mustConnect { - err = err_ // OnStart() will return this. - close(doneCh) - return - } else { - fmt.Printf("tmsp.remoteClient failed to connect to %v. Retrying...\n", cli.addr) - time.Sleep(time.Second * 3) - continue RETRY_LOOP - } +RETRY_LOOP: + for { + conn, err := Connect(cli.addr) + if err != nil { + if cli.mustConnect { + return err + } else { + fmt.Printf("tmsp.remoteClient failed to connect to %v. Retrying...\n", cli.addr) + time.Sleep(time.Second * 3) + continue RETRY_LOOP } - go cli.sendRequestsRoutine(conn) - go cli.recvResponseRoutine(conn) - close(doneCh) // OnStart() will return no error. - return } - }() - <-doneCh - return // err + go cli.sendRequestsRoutine(conn) + go cli.recvResponseRoutine(conn) + return err + } + return nil // never happens } func (cli *remoteClient) OnStop() { @@ -122,7 +117,7 @@ func (cli *remoteClient) sendRequestsRoutine(conn net.Conn) { select { case <-cli.flushTimer.Ch: select { - case cli.reqQueue <- NewReqRes(types.RequestFlush()): + case cli.reqQueue <- NewReqRes(types.ToRequestFlush()): // cant this block ? default: // Probably will fill the buffer, or retry later. } @@ -136,7 +131,7 @@ func (cli *remoteClient) sendRequestsRoutine(conn net.Conn) { return } // log.Debug("Sent request", "requestType", reflect.TypeOf(reqres.Request), "request", reqres.Request) - if reqres.Request.Type == types.MessageType_Flush { + if _, ok := reqres.Request.Value.(*types.Request_Flush); ok { err = w.Flush() if err != nil { cli.StopForError(err) @@ -156,10 +151,10 @@ func (cli *remoteClient) recvResponseRoutine(conn net.Conn) { cli.StopForError(err) return } - switch res.Type { - case types.MessageType_Exception: + switch r := res.Value.(type) { + case *types.Response_Exception: // XXX After setting cli.err, release waiters (e.g. reqres.Done()) - cli.StopForError(errors.New(res.Error)) + cli.StopForError(errors.New(r.Exception.Error)) default: // log.Debug("Received response", "responseType", reflect.TypeOf(res), "response", res) err := cli.didRecvResponse(res) @@ -183,12 +178,12 @@ func (cli *remoteClient) didRecvResponse(res *types.Response) error { // Get the first ReqRes next := cli.reqSent.Front() if next == nil { - return fmt.Errorf("Unexpected result type %v when nothing expected", res.Type) + return fmt.Errorf("Unexpected result type %v when nothing expected", reflect.TypeOf(res.Value)) } reqres := next.Value.(*ReqRes) if !resMatchesReq(reqres.Request, res) { return fmt.Errorf("Unexpected result type %v when response to %v expected", - res.Type, reqres.Request.Type) + reflect.TypeOf(res.Value), reflect.TypeOf(reqres.Request.Value)) } reqres.Response = res // Set response @@ -211,128 +206,128 @@ func (cli *remoteClient) didRecvResponse(res *types.Response) error { //---------------------------------------- func (cli *remoteClient) EchoAsync(msg string) *ReqRes { - return cli.queueRequest(types.RequestEcho(msg)) + return cli.queueRequest(types.ToRequestEcho(msg)) } func (cli *remoteClient) FlushAsync() *ReqRes { - return cli.queueRequest(types.RequestFlush()) + return cli.queueRequest(types.ToRequestFlush()) } func (cli *remoteClient) InfoAsync() *ReqRes { - return cli.queueRequest(types.RequestInfo()) + return cli.queueRequest(types.ToRequestInfo()) } func (cli *remoteClient) SetOptionAsync(key string, value string) *ReqRes { - return cli.queueRequest(types.RequestSetOption(key, value)) + return cli.queueRequest(types.ToRequestSetOption(key, value)) } func (cli *remoteClient) AppendTxAsync(tx []byte) *ReqRes { - return cli.queueRequest(types.RequestAppendTx(tx)) + return cli.queueRequest(types.ToRequestAppendTx(tx)) } func (cli *remoteClient) CheckTxAsync(tx []byte) *ReqRes { - return cli.queueRequest(types.RequestCheckTx(tx)) + return cli.queueRequest(types.ToRequestCheckTx(tx)) } func (cli *remoteClient) QueryAsync(query []byte) *ReqRes { - return cli.queueRequest(types.RequestQuery(query)) + return cli.queueRequest(types.ToRequestQuery(query)) } func (cli *remoteClient) CommitAsync() *ReqRes { - return cli.queueRequest(types.RequestCommit()) + return cli.queueRequest(types.ToRequestCommit()) } func (cli *remoteClient) InitChainAsync(validators []*types.Validator) *ReqRes { - return cli.queueRequest(types.RequestInitChain(validators)) + return cli.queueRequest(types.ToRequestInitChain(validators)) } func (cli *remoteClient) BeginBlockAsync(height uint64) *ReqRes { - return cli.queueRequest(types.RequestBeginBlock(height)) + return cli.queueRequest(types.ToRequestBeginBlock(height)) } func (cli *remoteClient) EndBlockAsync(height uint64) *ReqRes { - return cli.queueRequest(types.RequestEndBlock(height)) + return cli.queueRequest(types.ToRequestEndBlock(height)) } //---------------------------------------- func (cli *remoteClient) EchoSync(msg string) (res types.Result) { - reqres := cli.queueRequest(types.RequestEcho(msg)) + reqres := cli.queueRequest(types.ToRequestEcho(msg)) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response - return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} + resp := reqres.Response.GetEcho() + return types.Result{Code: OK, Data: []byte(resp.Message), Log: LOG} } func (cli *remoteClient) FlushSync() error { - cli.queueRequest(types.RequestFlush()).Wait() + cli.queueRequest(types.ToRequestFlush()).Wait() return cli.err } func (cli *remoteClient) InfoSync() (res types.Result) { - reqres := cli.queueRequest(types.RequestInfo()) + reqres := cli.queueRequest(types.ToRequestInfo()) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response - return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} + resp := reqres.Response.GetInfo() + return types.Result{Code: OK, Data: []byte(resp.Info), Log: LOG} } func (cli *remoteClient) SetOptionSync(key string, value string) (res types.Result) { - reqres := cli.queueRequest(types.RequestSetOption(key, value)) + reqres := cli.queueRequest(types.ToRequestSetOption(key, value)) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response - return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} + resp := reqres.Response.GetSetOption() + return types.Result{Code: OK, Data: nil, Log: resp.Log} } func (cli *remoteClient) AppendTxSync(tx []byte) (res types.Result) { - reqres := cli.queueRequest(types.RequestAppendTx(tx)) + reqres := cli.queueRequest(types.ToRequestAppendTx(tx)) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response + resp := reqres.Response.GetAppendTx() return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} } func (cli *remoteClient) CheckTxSync(tx []byte) (res types.Result) { - reqres := cli.queueRequest(types.RequestCheckTx(tx)) + reqres := cli.queueRequest(types.ToRequestCheckTx(tx)) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response + resp := reqres.Response.GetCheckTx() return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} } func (cli *remoteClient) QuerySync(query []byte) (res types.Result) { - reqres := cli.queueRequest(types.RequestQuery(query)) + reqres := cli.queueRequest(types.ToRequestQuery(query)) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response + resp := reqres.Response.GetQuery() return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} } func (cli *remoteClient) CommitSync() (res types.Result) { - reqres := cli.queueRequest(types.RequestCommit()) + reqres := cli.queueRequest(types.ToRequestCommit()) cli.FlushSync() if cli.err != nil { return types.ErrInternalError.SetLog(cli.err.Error()) } - resp := reqres.Response + resp := reqres.Response.GetCommit() return types.Result{Code: resp.Code, Data: resp.Data, Log: resp.Log} } func (cli *remoteClient) InitChainSync(validators []*types.Validator) (err error) { - cli.queueRequest(types.RequestInitChain(validators)) + cli.queueRequest(types.ToRequestInitChain(validators)) cli.FlushSync() if cli.err != nil { return cli.err @@ -341,7 +336,7 @@ func (cli *remoteClient) InitChainSync(validators []*types.Validator) (err error } func (cli *remoteClient) BeginBlockSync(height uint64) (err error) { - cli.queueRequest(types.RequestBeginBlock(height)) + cli.queueRequest(types.ToRequestBeginBlock(height)) cli.FlushSync() if cli.err != nil { return cli.err @@ -350,24 +345,25 @@ func (cli *remoteClient) BeginBlockSync(height uint64) (err error) { } func (cli *remoteClient) EndBlockSync(height uint64) (validators []*types.Validator, err error) { - reqres := cli.queueRequest(types.RequestEndBlock(height)) + reqres := cli.queueRequest(types.ToRequestEndBlock(height)) cli.FlushSync() if cli.err != nil { return nil, cli.err } - return reqres.Response.Validators, nil + return reqres.Response.GetEndBlock().Diffs, nil } //---------------------------------------- func (cli *remoteClient) queueRequest(req *types.Request) *ReqRes { reqres := NewReqRes(req) + // TODO: set cli.err if reqQueue times out cli.reqQueue <- reqres // Maybe auto-flush, or unset auto-flush - switch req.Type { - case types.MessageType_Flush: + switch req.Value.(type) { + case *types.Request_Flush: cli.flushTimer.Unset() default: cli.flushTimer.Set() @@ -379,5 +375,27 @@ func (cli *remoteClient) queueRequest(req *types.Request) *ReqRes { //---------------------------------------- func resMatchesReq(req *types.Request, res *types.Response) (ok bool) { - return req.Type == res.Type + switch req.Value.(type) { + case *types.Request_Echo: + _, ok = res.Value.(*types.Response_Echo) + case *types.Request_Flush: + _, ok = res.Value.(*types.Response_Flush) + case *types.Request_Info: + _, ok = res.Value.(*types.Response_Info) + case *types.Request_SetOption: + _, ok = res.Value.(*types.Response_SetOption) + case *types.Request_AppendTx: + _, ok = res.Value.(*types.Response_AppendTx) + case *types.Request_CheckTx: + _, ok = res.Value.(*types.Response_CheckTx) + case *types.Request_Commit: + _, ok = res.Value.(*types.Response_Commit) + case *types.Request_Query: + _, ok = res.Value.(*types.Response_Query) + case *types.Request_InitChain: + _, ok = res.Value.(*types.Response_InitChain) + case *types.Request_EndBlock: + _, ok = res.Value.(*types.Response_EndBlock) + } + return ok } diff --git a/cmd/counter/main.go b/cmd/counter/main.go index fc003fb88..c04a40e0f 100644 --- a/cmd/counter/main.go +++ b/cmd/counter/main.go @@ -11,12 +11,13 @@ import ( func main() { addrPtr := flag.String("addr", "tcp://0.0.0.0:46658", "Listen address") + tmspPtr := flag.String("tmsp", "socket", "TMSP server: socket | grpc") serialPtr := flag.Bool("serial", false, "Enforce incrementing (serial) txs") flag.Parse() app := counter.NewCounterApplication(*serialPtr) // Start the listener - _, err := server.NewServer(*addrPtr, app) + _, err := server.NewServer(*addrPtr, *tmspPtr, app) if err != nil { Exit(err.Error()) } diff --git a/cmd/dummy/main.go b/cmd/dummy/main.go index 25486e44f..8efa69e6b 100644 --- a/cmd/dummy/main.go +++ b/cmd/dummy/main.go @@ -11,10 +11,11 @@ import ( func main() { addrPtr := flag.String("addr", "tcp://0.0.0.0:46658", "Listen address") + tmspPtr := flag.String("tmsp", "socket", "socket | grpc") flag.Parse() // Start the listener - _, err := server.NewServer(*addrPtr, dummy.NewDummyApplication()) + _, err := server.NewServer(*addrPtr, *tmspPtr, dummy.NewDummyApplication()) if err != nil { Exit(err.Error()) } diff --git a/cmd/tmsp-cli/tmsp-cli.go b/cmd/tmsp-cli/tmsp-cli.go index 275bb23a0..373266efc 100644 --- a/cmd/tmsp-cli/tmsp-cli.go +++ b/cmd/tmsp-cli/tmsp-cli.go @@ -5,18 +5,18 @@ import ( "errors" "fmt" "io" - "net" "os" "strings" "github.com/codegangsta/cli" . "github.com/tendermint/go-common" "github.com/tendermint/go-wire/expr" + "github.com/tendermint/tmsp/client" "github.com/tendermint/tmsp/types" ) -// connection is a global variable so it can be reused by the console -var conn net.Conn +// client is a global variable so it can be reused by the console +var client tmspcli.Client func main() { app := cli.NewApp() @@ -28,6 +28,11 @@ func main() { Value: "tcp://127.0.0.1:46658", Usage: "address of application socket", }, + cli.StringFlag{ + Name: "tmsp", + Value: "socket", + Usage: "socket or grpc", + }, } app.Commands = []cli.Command{ { @@ -103,9 +108,9 @@ func main() { } func before(c *cli.Context) error { - if conn == nil { + if client == nil { var err error - conn, err = Connect(c.GlobalString("address")) + client, err = tmspcli.NewClient(c.GlobalString("address"), c.GlobalString("tmsp"), false) if err != nil { Exit(err.Error()) } @@ -148,7 +153,9 @@ func cmdConsole(app *cli.App, c *cli.Context) error { args := []string{"tmsp"} args = append(args, strings.Split(string(line), " ")...) - app.Run(args) + if err := app.Run(args); err != nil { + return err + } } return nil } @@ -159,21 +166,15 @@ func cmdEcho(c *cli.Context) error { if len(args) != 1 { return errors.New("Command echo takes 1 argument") } - res, err := makeRequest(conn, types.RequestEcho(args[0])) - if err != nil { - return err - } - printResponse(res, string(res.Data)) + res := client.EchoSync(args[0]) + printResponse(res, string(res.Data), false) return nil } // Get some info from the application func cmdInfo(c *cli.Context) error { - res, err := makeRequest(conn, types.RequestInfo()) - if err != nil { - return err - } - printResponse(res, string(res.Data)) + res := client.InfoSync() + printResponse(res, string(res.Data), false) return nil } @@ -183,11 +184,8 @@ func cmdSetOption(c *cli.Context) error { if len(args) != 2 { return errors.New("Command set_option takes 2 arguments (key, value)") } - res, err := makeRequest(conn, types.RequestSetOption(args[0], args[1])) - if err != nil { - return err - } - printResponse(res, Fmt("%s=%s", args[0], args[1])) + res := client.SetOptionSync(args[0], args[1]) + printResponse(res, Fmt("%s=%s", args[0], args[1]), false) return nil } @@ -203,11 +201,8 @@ func cmdAppendTx(c *cli.Context) error { return err } - res, err := makeRequest(conn, types.RequestAppendTx(txBytes)) - if err != nil { - return err - } - printResponse(res, string(res.Data)) + res := client.AppendTxSync(txBytes) + printResponse(res, string(res.Data), true) return nil } @@ -223,21 +218,15 @@ func cmdCheckTx(c *cli.Context) error { return err } - res, err := makeRequest(conn, types.RequestCheckTx(txBytes)) - if err != nil { - return err - } - printResponse(res, string(res.Data)) + res := client.CheckTxSync(txBytes) + printResponse(res, string(res.Data), true) return nil } // Get application Merkle root hash func cmdCommit(c *cli.Context) error { - res, err := makeRequest(conn, types.RequestCommit()) - if err != nil { - return err - } - printResponse(res, Fmt("%X", res.Data)) + res := client.CommitSync() + printResponse(res, Fmt("%X", res.Data), false) return nil } @@ -253,24 +242,20 @@ func cmdQuery(c *cli.Context) error { return err } - res, err := makeRequest(conn, types.RequestQuery(queryBytes)) - if err != nil { - return err - } - printResponse(res, string(res.Data)) + res := client.QuerySync(queryBytes) + printResponse(res, string(res.Data), true) return nil } //-------------------------------------------------------------------------------- -func printResponse(res *types.Response, s string) { - switch res.Type { - case types.MessageType_AppendTx, types.MessageType_CheckTx, types.MessageType_Query: +func printResponse(res types.Result, s string, printCode bool) { + if printCode { fmt.Printf("-> code: %s\n", res.Code.String()) } - if res.Error != "" { + /*if res.Error != "" { fmt.Printf("-> error: %s\n", res.Error) - } + }*/ if s != "" { fmt.Printf("-> data: {%s}\n", s) } @@ -279,41 +264,3 @@ func printResponse(res *types.Response, s string) { } } - -func responseString(res *types.Response) string { - return Fmt("type: %v\tdata: %v\tcode: %v", res.Type, res.Data, res.Code) -} - -func makeRequest(conn net.Conn, req *types.Request) (*types.Response, error) { - - // Write desired request - err := types.WriteMessage(req, conn) - if err != nil { - return nil, err - } - - // Write flush request - err = types.WriteMessage(types.RequestFlush(), conn) - if err != nil { - return nil, err - } - - // Read desired response - var res = &types.Response{} - err = types.ReadMessage(conn, res) - if err != nil { - return nil, err - } - - // Read flush response - var resFlush = &types.Response{} - err = types.ReadMessage(conn, resFlush) - if err != nil { - return nil, err - } - if resFlush.Type != types.MessageType_Flush { - return nil, errors.New(Fmt("Expected types.MessageType_Flush but got %v instead", resFlush.Type)) - } - - return res, nil -} diff --git a/example/dummy/dummy_test.go b/example/dummy/dummy_test.go index 22338b00c..c5ac2d271 100644 --- a/example/dummy/dummy_test.go +++ b/example/dummy/dummy_test.go @@ -1,6 +1,7 @@ package dummy import ( + "reflect" "testing" "time" @@ -14,7 +15,7 @@ func TestStream(t *testing.T) { numAppendTxs := 200000 // Start the listener - server, err := server.NewServer("unix://test.sock", NewDummyApplication()) + server, err := server.NewSocketServer("unix://test.sock", NewDummyApplication()) if err != nil { Exit(err.Error()) } @@ -39,11 +40,11 @@ func TestStream(t *testing.T) { } // Process response - switch res.Type { - case types.MessageType_AppendTx: + switch r := res.Value.(type) { + case *types.Response_AppendTx: counter += 1 - if res.Code != types.CodeType_OK { - t.Error("AppendTx failed with ret_code", res.Code) + if r.AppendTx.Code != types.CodeType_OK { + t.Error("AppendTx failed with ret_code", r.AppendTx.Code) } if counter > numAppendTxs { t.Fatal("Too many AppendTx responses") @@ -55,10 +56,10 @@ func TestStream(t *testing.T) { close(done) }() } - case types.MessageType_Flush: + case *types.Response_Flush: // ignore default: - t.Error("Unexpected response type", res.Type) + t.Error("Unexpected response type", reflect.TypeOf(res.Value)) } } }() @@ -66,7 +67,7 @@ func TestStream(t *testing.T) { // Write requests for counter := 0; counter < numAppendTxs; counter++ { // Send request - var req = types.RequestAppendTx([]byte("test")) + var req = types.ToRequestAppendTx([]byte("test")) err := types.WriteMessage(req, conn) if err != nil { t.Fatal(err.Error()) @@ -75,7 +76,7 @@ func TestStream(t *testing.T) { // Sometimes send flush messages if counter%123 == 0 { t.Log("flush") - err := types.WriteMessage(types.RequestFlush(), conn) + err := types.WriteMessage(types.ToRequestFlush(), conn) if err != nil { t.Fatal(err.Error()) } @@ -83,7 +84,7 @@ func TestStream(t *testing.T) { } // Send final flush message - err = types.WriteMessage(types.RequestFlush(), conn) + err = types.WriteMessage(types.ToRequestFlush(), conn) if err != nil { t.Fatal(err.Error()) } diff --git a/example/nil/nil_test.go b/example/nil/nil_test.go index 0f3f6adb4..ed7a36ddb 100644 --- a/example/nil/nil_test.go +++ b/example/nil/nil_test.go @@ -1,9 +1,14 @@ package nilapp import ( + "net" + "reflect" "testing" "time" + "golang.org/x/net/context" + "google.golang.org/grpc" + . "github.com/tendermint/go-common" "github.com/tendermint/tmsp/server" "github.com/tendermint/tmsp/types" @@ -14,7 +19,7 @@ func TestStream(t *testing.T) { numAppendTxs := 200000 // Start the listener - server, err := server.NewServer("unix://test.sock", NewNilApplication()) + server, err := server.NewSocketServer("unix://test.sock", NewNilApplication()) if err != nil { Exit(err.Error()) } @@ -39,11 +44,11 @@ func TestStream(t *testing.T) { } // Process response - switch res.Type { - case types.MessageType_AppendTx: + switch r := res.Value.(type) { + case *types.Response_AppendTx: counter += 1 - if res.Code != types.CodeType_OK { - t.Error("AppendTx failed with ret_code", res.Code) + if r.AppendTx.Code != types.CodeType_OK { + t.Error("AppendTx failed with ret_code", r.AppendTx.Code) } if counter > numAppendTxs { t.Fatal("Too many AppendTx responses") @@ -55,10 +60,10 @@ func TestStream(t *testing.T) { close(done) }() } - case types.MessageType_Flush: + case *types.Response_Flush: // ignore default: - t.Error("Unexpected response type", res.Type) + t.Error("Unexpected response type", reflect.TypeOf(res.Value)) } } }() @@ -66,7 +71,7 @@ func TestStream(t *testing.T) { // Write requests for counter := 0; counter < numAppendTxs; counter++ { // Send request - var req = types.RequestAppendTx([]byte("test")) + var req = types.ToRequestAppendTx([]byte("test")) err := types.WriteMessage(req, conn) if err != nil { t.Fatal(err.Error()) @@ -75,7 +80,7 @@ func TestStream(t *testing.T) { // Sometimes send flush messages if counter%123 == 0 { t.Log("flush") - err := types.WriteMessage(types.RequestFlush(), conn) + err := types.WriteMessage(types.ToRequestFlush(), conn) if err != nil { t.Fatal(err.Error()) } @@ -83,10 +88,61 @@ func TestStream(t *testing.T) { } // Send final flush message - err = types.WriteMessage(types.RequestFlush(), conn) + err = types.WriteMessage(types.ToRequestFlush(), conn) if err != nil { t.Fatal(err.Error()) } <-done } + +//------------------------- +// test grpc + +func dialerFunc(addr string, timeout time.Duration) (net.Conn, error) { + return Connect(addr) +} + +func TestGRPCSync(t *testing.T) { + + numAppendTxs := 2000 + + // Start the listener + server, err := server.NewGRPCServer("unix://test.sock", types.NewGRPCApplication(NewNilApplication())) + if err != nil { + Exit(err.Error()) + } + defer server.Stop() + + // Connect to the socket + conn, err := grpc.Dial("unix://test.sock", grpc.WithInsecure(), grpc.WithDialer(dialerFunc)) + if err != nil { + Exit(err.Error()) + } + defer conn.Close() + + client := types.NewTMSPApplicationClient(conn) + + // Write requests + for counter := 0; counter < numAppendTxs; counter++ { + // Send request + response, err := client.AppendTx(context.Background(), &types.RequestAppendTx{[]byte("test")}) + if err != nil { + t.Fatal(err.Error()) + } + counter += 1 + if response.Code != types.CodeType_OK { + t.Error("AppendTx failed with ret_code", response.Code) + } + if counter > numAppendTxs { + t.Fatal("Too many AppendTx responses") + } + t.Log("response", counter) + if counter == numAppendTxs { + go func() { + time.Sleep(time.Second * 2) // Wait for a bit to allow counter overflow + }() + } + + } +} diff --git a/server/grpc_server.go b/server/grpc_server.go new file mode 100644 index 000000000..3a21e3fb9 --- /dev/null +++ b/server/grpc_server.go @@ -0,0 +1,55 @@ +package server + +import ( + "net" + "strings" + + "google.golang.org/grpc" + + . "github.com/tendermint/go-common" + "github.com/tendermint/tmsp/types" +) + +// var maxNumberConnections = 2 + +type GRPCServer struct { + QuitService + + proto string + addr string + listener net.Listener + + app types.TMSPApplicationServer +} + +func NewGRPCServer(protoAddr string, app types.TMSPApplicationServer) (Service, error) { + parts := strings.SplitN(protoAddr, "://", 2) + proto, addr := parts[0], parts[1] + s := &GRPCServer{ + proto: proto, + addr: addr, + listener: nil, + app: app, + } + s.QuitService = *NewQuitService(nil, "TMSPServer", s) + _, err := s.Start() // Just start it + return s, err +} + +func (s *GRPCServer) OnStart() error { + s.QuitService.OnStart() + ln, err := net.Listen(s.proto, s.addr) + if err != nil { + return err + } + s.listener = ln + grpcServer := grpc.NewServer() + types.RegisterTMSPApplicationServer(grpcServer, s.app) + go grpcServer.Serve(ln) + return nil +} + +func (s *GRPCServer) OnStop() { + s.QuitService.OnStop() + s.listener.Close() +} diff --git a/server/server.go b/server/server.go index e64ab30b5..f2b9d0af4 100644 --- a/server/server.go +++ b/server/server.go @@ -1,190 +1,22 @@ package server import ( - "bufio" "fmt" - "io" - "net" - "strings" - "sync" . "github.com/tendermint/go-common" "github.com/tendermint/tmsp/types" ) -// var maxNumberConnections = 2 - -type Server struct { - QuitService - - proto string - addr string - listener net.Listener - - appMtx sync.Mutex - app types.Application -} - -func NewServer(protoAddr string, app types.Application) (*Server, error) { - parts := strings.SplitN(protoAddr, "://", 2) - proto, addr := parts[0], parts[1] - s := &Server{ - proto: proto, - addr: addr, - listener: nil, - app: app, - } - s.QuitService = *NewQuitService(nil, "TMSPServer", s) - _, err := s.Start() // Just start it - return s, err -} - -func (s *Server) OnStart() error { - s.QuitService.OnStart() - ln, err := net.Listen(s.proto, s.addr) - if err != nil { - return err - } - s.listener = ln - go s.acceptConnectionsRoutine() - return nil -} - -func (s *Server) OnStop() { - s.QuitService.OnStop() - s.listener.Close() -} - -func (s *Server) acceptConnectionsRoutine() { - // semaphore := make(chan struct{}, maxNumberConnections) - - for { - // semaphore <- struct{}{} - - // Accept a connection - fmt.Println("Waiting for new connection...") - conn, err := s.listener.Accept() - if err != nil { - if !s.IsRunning() { - return // Ignore error from listener closing. - } - Exit("Failed to accept connection: " + err.Error()) - } else { - fmt.Println("Accepted a new connection") - } - - closeConn := make(chan error, 2) // Push to signal connection closed - responses := make(chan *types.Response, 1000) // A channel to buffer responses - - // Read requests from conn and deal with them - go s.handleRequests(closeConn, conn, responses) - // Pull responses from 'responses' and write them to conn. - go s.handleResponses(closeConn, responses, conn) - - go func() { - // Wait until signal to close connection - errClose := <-closeConn - if errClose != nil { - fmt.Printf("Connection error: %v\n", errClose) - } else { - fmt.Println("Connection was closed.") - } - - // Close the connection - err := conn.Close() - if err != nil { - fmt.Printf("Error in closing connection: %v\n", err) - } - - // <-semaphore - }() - } -} - -// Read requests from conn and deal with them -func (s *Server) handleRequests(closeConn chan error, conn net.Conn, responses chan<- *types.Response) { - var count int - var bufReader = bufio.NewReader(conn) - for { - - var req = &types.Request{} - err := types.ReadMessage(bufReader, req) - if err != nil { - if err == io.EOF { - closeConn <- fmt.Errorf("Connection closed by client") - } else { - closeConn <- fmt.Errorf("Error in handleRequests: %v", err.Error()) - } - return - } - s.appMtx.Lock() - count++ - s.handleRequest(req, responses) - s.appMtx.Unlock() - } -} - -func (s *Server) handleRequest(req *types.Request, responses chan<- *types.Response) { - switch req.Type { - case types.MessageType_Echo: - responses <- types.ResponseEcho(string(req.Data)) - case types.MessageType_Flush: - responses <- types.ResponseFlush() - case types.MessageType_Info: - data := s.app.Info() - responses <- types.ResponseInfo(data) - case types.MessageType_SetOption: - logStr := s.app.SetOption(req.Key, req.Value) - responses <- types.ResponseSetOption(logStr) - case types.MessageType_AppendTx: - res := s.app.AppendTx(req.Data) - responses <- types.ResponseAppendTx(res.Code, res.Data, res.Log) - case types.MessageType_CheckTx: - res := s.app.CheckTx(req.Data) - responses <- types.ResponseCheckTx(res.Code, res.Data, res.Log) - case types.MessageType_Commit: - res := s.app.Commit() - responses <- types.ResponseCommit(res.Code, res.Data, res.Log) - case types.MessageType_Query: - res := s.app.Query(req.Data) - responses <- types.ResponseQuery(res.Code, res.Data, res.Log) - case types.MessageType_InitChain: - if app, ok := s.app.(types.BlockchainAware); ok { - app.InitChain(req.Validators) - responses <- types.ResponseInitChain() - } else { - responses <- types.ResponseInitChain() - } - case types.MessageType_EndBlock: - if app, ok := s.app.(types.BlockchainAware); ok { - validators := app.EndBlock(req.Height) - responses <- types.ResponseEndBlock(validators) - } else { - responses <- types.ResponseEndBlock(nil) - } +func NewServer(protoAddr, transport string, app types.Application) (Service, error) { + var s Service + var err error + switch transport { + case "socket": + s, err = NewSocketServer(protoAddr, app) + case "grpc": + s, err = NewGRPCServer(protoAddr, types.NewGRPCApplication(app)) default: - responses <- types.ResponseException("Unknown request") - } -} - -// Pull responses from 'responses' and write them to conn. -func (s *Server) handleResponses(closeConn chan error, responses <-chan *types.Response, conn net.Conn) { - var count int - var bufWriter = bufio.NewWriter(conn) - for { - var res = <-responses - err := types.WriteMessage(res, bufWriter) - if err != nil { - closeConn <- fmt.Errorf("Error in handleResponses: %v", err.Error()) - return - } - if res.Type == types.MessageType_Flush { - err = bufWriter.Flush() - if err != nil { - closeConn <- fmt.Errorf("Error in handleResponses: %v", err.Error()) - return - } - } - count++ + err = fmt.Errorf("Unknown server type %s", transport) } + return s, err } diff --git a/server/socket_server.go b/server/socket_server.go new file mode 100644 index 000000000..7969974e4 --- /dev/null +++ b/server/socket_server.go @@ -0,0 +1,191 @@ +package server + +import ( + "bufio" + "fmt" + "io" + "net" + "strings" + "sync" + + . "github.com/tendermint/go-common" + "github.com/tendermint/tmsp/types" +) + +// var maxNumberConnections = 2 + +type SocketServer struct { + QuitService + + proto string + addr string + listener net.Listener + + appMtx sync.Mutex + app types.Application +} + +func NewSocketServer(protoAddr string, app types.Application) (Service, error) { + parts := strings.SplitN(protoAddr, "://", 2) + proto, addr := parts[0], parts[1] + s := &SocketServer{ + proto: proto, + addr: addr, + listener: nil, + app: app, + } + s.QuitService = *NewQuitService(nil, "TMSPServer", s) + _, err := s.Start() // Just start it + return s, err +} + +func (s *SocketServer) OnStart() error { + s.QuitService.OnStart() + ln, err := net.Listen(s.proto, s.addr) + if err != nil { + return err + } + s.listener = ln + go s.acceptConnectionsRoutine() + return nil +} + +func (s *SocketServer) OnStop() { + s.QuitService.OnStop() + s.listener.Close() +} + +func (s *SocketServer) acceptConnectionsRoutine() { + // semaphore := make(chan struct{}, maxNumberConnections) + + for { + // semaphore <- struct{}{} + + // Accept a connection + fmt.Println("Waiting for new connection...") + conn, err := s.listener.Accept() + if err != nil { + if !s.IsRunning() { + return // Ignore error from listener closing. + } + Exit("Failed to accept connection: " + err.Error()) + } else { + fmt.Println("Accepted a new connection") + } + + closeConn := make(chan error, 2) // Push to signal connection closed + responses := make(chan *types.Response, 1000) // A channel to buffer responses + + // Read requests from conn and deal with them + go s.handleRequests(closeConn, conn, responses) + // Pull responses from 'responses' and write them to conn. + go s.handleResponses(closeConn, responses, conn) + + go func() { + // Wait until signal to close connection + errClose := <-closeConn + if errClose != nil { + fmt.Printf("Connection error: %v\n", errClose) + } else { + fmt.Println("Connection was closed.") + } + + // Close the connection + err := conn.Close() + if err != nil { + fmt.Printf("Error in closing connection: %v\n", err) + } + + // <-semaphore + }() + } +} + +// Read requests from conn and deal with them +func (s *SocketServer) handleRequests(closeConn chan error, conn net.Conn, responses chan<- *types.Response) { + var count int + var bufReader = bufio.NewReader(conn) + for { + + var req = &types.Request{} + err := types.ReadMessage(bufReader, req) + if err != nil { + if err == io.EOF { + closeConn <- fmt.Errorf("Connection closed by client") + } else { + closeConn <- fmt.Errorf("Error in handleValue: %v", err.Error()) + } + return + } + s.appMtx.Lock() + count++ + s.handleRequest(req, responses) + s.appMtx.Unlock() + } +} + +func (s *SocketServer) handleRequest(req *types.Request, responses chan<- *types.Response) { + switch r := req.Value.(type) { + case *types.Request_Echo: + responses <- types.ToResponseEcho(r.Echo.Message) + case *types.Request_Flush: + responses <- types.ToResponseFlush() + case *types.Request_Info: + data := s.app.Info() + responses <- types.ToResponseInfo(data) + case *types.Request_SetOption: + so := r.SetOption + logStr := s.app.SetOption(so.Key, so.Value) + responses <- types.ToResponseSetOption(logStr) + case *types.Request_AppendTx: + res := s.app.AppendTx(r.AppendTx.Tx) + responses <- types.ToResponseAppendTx(res.Code, res.Data, res.Log) + case *types.Request_CheckTx: + res := s.app.CheckTx(r.CheckTx.Tx) + responses <- types.ToResponseCheckTx(res.Code, res.Data, res.Log) + case *types.Request_Commit: + res := s.app.Commit() + responses <- types.ToResponseCommit(res.Code, res.Data, res.Log) + case *types.Request_Query: + res := s.app.Query(r.Query.Query) + responses <- types.ToResponseQuery(res.Code, res.Data, res.Log) + case *types.Request_InitChain: + if app, ok := s.app.(types.BlockchainAware); ok { + app.InitChain(r.InitChain.Validators) + responses <- types.ToResponseInitChain() + } else { + responses <- types.ToResponseInitChain() + } + case *types.Request_EndBlock: + if app, ok := s.app.(types.BlockchainAware); ok { + validators := app.EndBlock(r.EndBlock.Height) + responses <- types.ToResponseEndBlock(validators) + } else { + responses <- types.ToResponseEndBlock(nil) + } + default: + responses <- types.ToResponseException("Unknown request") + } +} + +// Pull responses from 'responses' and write them to conn. +func (s *SocketServer) handleResponses(closeConn chan error, responses <-chan *types.Response, conn net.Conn) { + var count int + var bufWriter = bufio.NewWriter(conn) + for { + var res = <-responses + err := types.WriteMessage(res, bufWriter) + if err != nil { + closeConn <- fmt.Errorf("Error in handleValue: %v", err.Error()) + return + } + if _, ok := res.Value.(*types.Response_Flush); ok { + err = bufWriter.Flush() + if err != nil { + closeConn <- fmt.Errorf("Error in handleValue: %v", err.Error()) + return + } + } + count++ + } +} diff --git a/tests/benchmarks/parallel/parallel.go b/tests/benchmarks/parallel/parallel.go index 7cab2eb26..db57e19bf 100644 --- a/tests/benchmarks/parallel/parallel.go +++ b/tests/benchmarks/parallel/parallel.go @@ -36,7 +36,7 @@ func main() { counter := 0 for i := 0; ; i++ { var bufWriter = bufio.NewWriter(conn) - var req = types.RequestEcho("foobar") + var req = types.ToRequestEcho("foobar") err := types.WriteMessage(req, bufWriter) if err != nil { diff --git a/tests/benchmarks/simple/simple.go b/tests/benchmarks/simple/simple.go index 8fb02585e..585ebd2b5 100644 --- a/tests/benchmarks/simple/simple.go +++ b/tests/benchmarks/simple/simple.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "net" + "reflect" //"encoding/hex" . "github.com/tendermint/go-common" @@ -21,7 +22,7 @@ func main() { // Make a bunch of requests counter := 0 for i := 0; ; i++ { - req := types.RequestEcho("foobar") + req := types.ToRequestEcho("foobar") _, err := makeRequest(conn, req) if err != nil { Exit(err.Error()) @@ -41,7 +42,7 @@ func makeRequest(conn net.Conn, req *types.Request) (*types.Response, error) { if err != nil { return nil, err } - err = types.WriteMessage(types.RequestFlush(), bufWriter) + err = types.WriteMessage(types.ToRequestFlush(), bufWriter) if err != nil { return nil, err } @@ -61,8 +62,8 @@ func makeRequest(conn net.Conn, req *types.Request) (*types.Response, error) { if err != nil { return nil, err } - if resFlush.Type != types.MessageType_Flush { - return nil, errors.New(Fmt("Expected flush response but got something else: %v", resFlush.Type)) + if _, ok := resFlush.Value.(*types.Response_Flush); !ok { + return nil, errors.New(Fmt("Expected flush response but got something else: %v", reflect.TypeOf(resFlush))) } return res, nil diff --git a/tests/test.sh b/tests/test.sh index d375eafae..ebdab0c80 100755 --- a/tests/test.sh +++ b/tests/test.sh @@ -5,5 +5,8 @@ cd $ROOT # test golang counter COUNTER_APP="counter" go run $ROOT/tests/test_counter.go +# test golang counter via grpc +COUNTER_APP="counter -tmsp=grpc" go run $ROOT/tests/test_counter.go -tmsp=grpc + # test nodejs counter COUNTER_APP="node ../js-tmsp/example/app.js" go run $ROOT/tests/test_counter.go diff --git a/tests/test_counter.go b/tests/test_counter.go index 44adc3207..2f78b6964 100644 --- a/tests/test_counter.go +++ b/tests/test_counter.go @@ -2,6 +2,7 @@ package main import ( "bytes" + "flag" "fmt" "os" "time" @@ -12,7 +13,10 @@ import ( "github.com/tendermint/tmsp/types" ) +var tmspPtr = flag.String("tmsp", "socket", "socket or grpc") + func main() { + flag.Parse() // Run tests testBasic() @@ -70,7 +74,7 @@ func startApp() *process.Process { func startClient() tmspcli.Client { // Start client - client, err := tmspcli.NewClient("tcp://127.0.0.1:46658", true) + client, err := tmspcli.NewClient("tcp://127.0.0.1:46658", *tmspPtr, true) if err != nil { panic("connecting to counter_app: " + err.Error()) } @@ -100,9 +104,6 @@ func commit(client tmspcli.Client, hashExp []byte) { func appendTx(client tmspcli.Client, txBytes []byte, codeExp types.CodeType, dataExp []byte) { res := client.AppendTxSync(txBytes) code, data, log := res.Code, res.Data, res.Log - if res.IsErr() { - panic(Fmt("appending tx %X: %v\nlog: %v", txBytes, log)) - } if code != codeExp { panic(Fmt("AppendTx response code was unexpected. Got %v expected %v. Log: %v", code, codeExp, log)) diff --git a/types/application.go b/types/application.go index 65ca9ab01..c6e6f24b8 100644 --- a/types/application.go +++ b/types/application.go @@ -1,5 +1,9 @@ package types +import ( + context "golang.org/x/net/context" +) + // Applications type Application interface { @@ -36,3 +40,70 @@ type BlockchainAware interface { // diffs: changed validators from app to TendermintCore EndBlock(height uint64) (diffs []*Validator) } + +//------------------------------------ +type GRPCApplication struct { + app Application +} + +func NewGRPCApplication(app Application) *GRPCApplication { + return &GRPCApplication{app} +} + +func (app *GRPCApplication) Echo(ctx context.Context, req *RequestEcho) (*ResponseEcho, error) { + return &ResponseEcho{req.Message}, nil +} + +func (app *GRPCApplication) Flush(ctx context.Context, req *RequestFlush) (*ResponseFlush, error) { + return &ResponseFlush{}, nil +} + +func (app *GRPCApplication) Info(ctx context.Context, req *RequestInfo) (*ResponseInfo, error) { + return &ResponseInfo{app.app.Info()}, nil +} + +func (app *GRPCApplication) SetOption(ctx context.Context, req *RequestSetOption) (*ResponseSetOption, error) { + return &ResponseSetOption{app.app.SetOption(req.Key, req.Value)}, nil +} + +func (app *GRPCApplication) AppendTx(ctx context.Context, req *RequestAppendTx) (*ResponseAppendTx, error) { + r := app.app.AppendTx(req.Tx) + return &ResponseAppendTx{r.Code, r.Data, r.Log}, nil +} + +func (app *GRPCApplication) CheckTx(ctx context.Context, req *RequestCheckTx) (*ResponseCheckTx, error) { + r := app.app.CheckTx(req.Tx) + return &ResponseCheckTx{r.Code, r.Data, r.Log}, nil +} + +func (app *GRPCApplication) Query(ctx context.Context, req *RequestQuery) (*ResponseQuery, error) { + r := app.app.Query(req.Query) + return &ResponseQuery{r.Code, r.Data, r.Log}, nil +} + +func (app *GRPCApplication) Commit(ctx context.Context, req *RequestCommit) (*ResponseCommit, error) { + r := app.app.Commit() + return &ResponseCommit{r.Code, r.Data, r.Log}, nil +} + +func (app *GRPCApplication) InitChain(ctx context.Context, req *RequestInitChain) (*ResponseInitChain, error) { + if chainAware, ok := app.app.(BlockchainAware); ok { + chainAware.InitChain(req.Validators) + } + return &ResponseInitChain{}, nil +} + +func (app *GRPCApplication) BeginBlock(ctx context.Context, req *RequestBeginBlock) (*ResponseBeginBlock, error) { + if chainAware, ok := app.app.(BlockchainAware); ok { + chainAware.BeginBlock(req.Height) + } + return &ResponseBeginBlock{}, nil +} + +func (app *GRPCApplication) EndBlock(ctx context.Context, req *RequestEndBlock) (*ResponseEndBlock, error) { + if chainAware, ok := app.app.(BlockchainAware); ok { + diffs := chainAware.EndBlock(req.Height) + return &ResponseEndBlock{diffs}, nil + } + return &ResponseEndBlock{}, nil +} diff --git a/types/messages.go b/types/messages.go index facc3bfc0..6b047d47b 100644 --- a/types/messages.go +++ b/types/messages.go @@ -7,169 +7,143 @@ import ( "github.com/tendermint/go-wire" ) -func RequestEcho(message string) *Request { +func ToRequestEcho(message string) *Request { return &Request{ - Type: MessageType_Echo, - Data: []byte(message), + Value: &Request_Echo{&RequestEcho{message}}, } } -func RequestFlush() *Request { +func ToRequestFlush() *Request { return &Request{ - Type: MessageType_Flush, + Value: &Request_Flush{&RequestFlush{}}, } } -func RequestInfo() *Request { +func ToRequestInfo() *Request { return &Request{ - Type: MessageType_Info, + Value: &Request_Info{&RequestInfo{}}, } } -func RequestSetOption(key string, value string) *Request { +func ToRequestSetOption(key string, value string) *Request { return &Request{ - Type: MessageType_SetOption, - Key: key, - Value: value, + Value: &Request_SetOption{&RequestSetOption{key, value}}, } } -func RequestAppendTx(txBytes []byte) *Request { +func ToRequestAppendTx(txBytes []byte) *Request { return &Request{ - Type: MessageType_AppendTx, - Data: txBytes, + Value: &Request_AppendTx{&RequestAppendTx{txBytes}}, } } -func RequestCheckTx(txBytes []byte) *Request { +func ToRequestCheckTx(txBytes []byte) *Request { return &Request{ - Type: MessageType_CheckTx, - Data: txBytes, + Value: &Request_CheckTx{&RequestCheckTx{txBytes}}, } } -func RequestCommit() *Request { +func ToRequestCommit() *Request { return &Request{ - Type: MessageType_Commit, + Value: &Request_Commit{&RequestCommit{}}, } } -func RequestQuery(queryBytes []byte) *Request { +func ToRequestQuery(queryBytes []byte) *Request { return &Request{ - Type: MessageType_Query, - Data: queryBytes, + Value: &Request_Query{&RequestQuery{queryBytes}}, } } -func RequestInitChain(validators []*Validator) *Request { +func ToRequestInitChain(validators []*Validator) *Request { return &Request{ - Type: MessageType_InitChain, - Validators: validators, + Value: &Request_InitChain{&RequestInitChain{validators}}, } } -func RequestBeginBlock(height uint64) *Request { +func ToRequestBeginBlock(height uint64) *Request { return &Request{ - Type: MessageType_BeginBlock, - Height: height, + Value: &Request_BeginBlock{&RequestBeginBlock{height}}, } } -func RequestEndBlock(height uint64) *Request { +func ToRequestEndBlock(height uint64) *Request { return &Request{ - Type: MessageType_EndBlock, - Height: height, + Value: &Request_EndBlock{&RequestEndBlock{height}}, } } //---------------------------------------- -func ResponseException(errStr string) *Response { +func ToResponseException(errStr string) *Response { return &Response{ - Type: MessageType_Exception, - Error: errStr, + Value: &Response_Exception{&ResponseException{errStr}}, } } -func ResponseEcho(message string) *Response { +func ToResponseEcho(message string) *Response { return &Response{ - Type: MessageType_Echo, - Data: []byte(message), + Value: &Response_Echo{&ResponseEcho{message}}, } } -func ResponseFlush() *Response { +func ToResponseFlush() *Response { return &Response{ - Type: MessageType_Flush, + Value: &Response_Flush{&ResponseFlush{}}, } } -func ResponseInfo(info string) *Response { +func ToResponseInfo(info string) *Response { return &Response{ - Type: MessageType_Info, - Data: []byte(info), + Value: &Response_Info{&ResponseInfo{info}}, } } -func ResponseSetOption(log string) *Response { +func ToResponseSetOption(log string) *Response { return &Response{ - Type: MessageType_SetOption, - Log: log, + Value: &Response_SetOption{&ResponseSetOption{log}}, } } -func ResponseAppendTx(code CodeType, data []byte, log string) *Response { +func ToResponseAppendTx(code CodeType, data []byte, log string) *Response { return &Response{ - Type: MessageType_AppendTx, - Code: code, - Data: data, - Log: log, + Value: &Response_AppendTx{&ResponseAppendTx{code, data, log}}, } } -func ResponseCheckTx(code CodeType, data []byte, log string) *Response { +func ToResponseCheckTx(code CodeType, data []byte, log string) *Response { return &Response{ - Type: MessageType_CheckTx, - Code: code, - Data: data, - Log: log, + Value: &Response_CheckTx{&ResponseCheckTx{code, data, log}}, } } -func ResponseCommit(code CodeType, data []byte, log string) *Response { +func ToResponseCommit(code CodeType, data []byte, log string) *Response { return &Response{ - Type: MessageType_Commit, - Code: code, - Data: data, - Log: log, + Value: &Response_Commit{&ResponseCommit{code, data, log}}, } } -func ResponseQuery(code CodeType, data []byte, log string) *Response { +func ToResponseQuery(code CodeType, data []byte, log string) *Response { return &Response{ - Type: MessageType_Query, - Code: code, - Data: data, - Log: log, + Value: &Response_Query{&ResponseQuery{code, data, log}}, } } -func ResponseInitChain() *Response { +func ToResponseInitChain() *Response { return &Response{ - Type: MessageType_InitChain, + Value: &Response_InitChain{&ResponseInitChain{}}, } } -func ResponseBeginBlock() *Response { +func ToResponseBeginBlock() *Response { return &Response{ - Type: MessageType_BeginBlock, + Value: &Response_BeginBlock{&ResponseBeginBlock{}}, } } -func ResponseEndBlock(validators []*Validator) *Response { +func ToResponseEndBlock(validators []*Validator) *Response { return &Response{ - Type: MessageType_EndBlock, - Validators: validators, + Value: &Response_EndBlock{&ResponseEndBlock{validators}}, } } diff --git a/types/types.pb.go b/types/types.pb.go index 933e62ca0..181a06083 100644 --- a/types/types.pb.go +++ b/types/types.pb.go @@ -1,16 +1,39 @@ // Code generated by protoc-gen-go. -// source: types/types.proto +// source: types.proto // DO NOT EDIT! /* Package types is a generated protocol buffer package. It is generated from these files: - types/types.proto + types.proto It has these top-level messages: Request + RequestEcho + RequestFlush + RequestInfo + RequestSetOption + RequestAppendTx + RequestCheckTx + RequestQuery + RequestCommit + RequestInitChain + RequestBeginBlock + RequestEndBlock Response + ResponseException + ResponseEcho + ResponseFlush + ResponseInfo + ResponseSetOption + ResponseAppendTx + ResponseCheckTx + ResponseQuery + ResponseCommit + ResponseInitChain + ResponseBeginBlock + ResponseEndBlock Validator */ package types @@ -19,11 +42,24 @@ import proto "github.com/golang/protobuf/proto" import fmt "fmt" import math "math" +import ( + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" +) + // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +const _ = proto.ProtoPackageIsVersion1 + +// Not being used +// Could be added to request/response +// so we don't have to type switch +// (would be twice as fast, but we're talking about 15ns) type MessageType int32 const ( @@ -187,12 +223,19 @@ func (x CodeType) String() string { func (CodeType) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } type Request struct { - Type MessageType `protobuf:"varint,1,opt,name=type,enum=types.MessageType" json:"type,omitempty"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - Key string `protobuf:"bytes,3,opt,name=key" json:"key,omitempty"` - Value string `protobuf:"bytes,4,opt,name=value" json:"value,omitempty"` - Validators []*Validator `protobuf:"bytes,5,rep,name=validators" json:"validators,omitempty"` - Height uint64 `protobuf:"varint,6,opt,name=height" json:"height,omitempty"` + // Types that are valid to be assigned to Value: + // *Request_Echo + // *Request_Flush + // *Request_Info + // *Request_SetOption + // *Request_AppendTx + // *Request_CheckTx + // *Request_Commit + // *Request_Query + // *Request_InitChain + // *Request_BeginBlock + // *Request_EndBlock + Value isRequest_Value `protobuf_oneof:"value"` } func (m *Request) Reset() { *m = Request{} } @@ -200,30 +243,1036 @@ func (m *Request) String() string { return proto.CompactTextString(m) func (*Request) ProtoMessage() {} func (*Request) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } -func (m *Request) GetValidators() []*Validator { +type isRequest_Value interface { + isRequest_Value() +} + +type Request_Echo struct { + Echo *RequestEcho `protobuf:"bytes,1,opt,name=echo,oneof"` +} +type Request_Flush struct { + Flush *RequestFlush `protobuf:"bytes,2,opt,name=flush,oneof"` +} +type Request_Info struct { + Info *RequestInfo `protobuf:"bytes,3,opt,name=info,oneof"` +} +type Request_SetOption struct { + SetOption *RequestSetOption `protobuf:"bytes,4,opt,name=set_option,json=setOption,oneof"` +} +type Request_AppendTx struct { + AppendTx *RequestAppendTx `protobuf:"bytes,5,opt,name=append_tx,json=appendTx,oneof"` +} +type Request_CheckTx struct { + CheckTx *RequestCheckTx `protobuf:"bytes,6,opt,name=check_tx,json=checkTx,oneof"` +} +type Request_Commit struct { + Commit *RequestCommit `protobuf:"bytes,7,opt,name=commit,oneof"` +} +type Request_Query struct { + Query *RequestQuery `protobuf:"bytes,8,opt,name=query,oneof"` +} +type Request_InitChain struct { + InitChain *RequestInitChain `protobuf:"bytes,9,opt,name=init_chain,json=initChain,oneof"` +} +type Request_BeginBlock struct { + BeginBlock *RequestBeginBlock `protobuf:"bytes,10,opt,name=begin_block,json=beginBlock,oneof"` +} +type Request_EndBlock struct { + EndBlock *RequestEndBlock `protobuf:"bytes,11,opt,name=end_block,json=endBlock,oneof"` +} + +func (*Request_Echo) isRequest_Value() {} +func (*Request_Flush) isRequest_Value() {} +func (*Request_Info) isRequest_Value() {} +func (*Request_SetOption) isRequest_Value() {} +func (*Request_AppendTx) isRequest_Value() {} +func (*Request_CheckTx) isRequest_Value() {} +func (*Request_Commit) isRequest_Value() {} +func (*Request_Query) isRequest_Value() {} +func (*Request_InitChain) isRequest_Value() {} +func (*Request_BeginBlock) isRequest_Value() {} +func (*Request_EndBlock) isRequest_Value() {} + +func (m *Request) GetValue() isRequest_Value { + if m != nil { + return m.Value + } + return nil +} + +func (m *Request) GetEcho() *RequestEcho { + if x, ok := m.GetValue().(*Request_Echo); ok { + return x.Echo + } + return nil +} + +func (m *Request) GetFlush() *RequestFlush { + if x, ok := m.GetValue().(*Request_Flush); ok { + return x.Flush + } + return nil +} + +func (m *Request) GetInfo() *RequestInfo { + if x, ok := m.GetValue().(*Request_Info); ok { + return x.Info + } + return nil +} + +func (m *Request) GetSetOption() *RequestSetOption { + if x, ok := m.GetValue().(*Request_SetOption); ok { + return x.SetOption + } + return nil +} + +func (m *Request) GetAppendTx() *RequestAppendTx { + if x, ok := m.GetValue().(*Request_AppendTx); ok { + return x.AppendTx + } + return nil +} + +func (m *Request) GetCheckTx() *RequestCheckTx { + if x, ok := m.GetValue().(*Request_CheckTx); ok { + return x.CheckTx + } + return nil +} + +func (m *Request) GetCommit() *RequestCommit { + if x, ok := m.GetValue().(*Request_Commit); ok { + return x.Commit + } + return nil +} + +func (m *Request) GetQuery() *RequestQuery { + if x, ok := m.GetValue().(*Request_Query); ok { + return x.Query + } + return nil +} + +func (m *Request) GetInitChain() *RequestInitChain { + if x, ok := m.GetValue().(*Request_InitChain); ok { + return x.InitChain + } + return nil +} + +func (m *Request) GetBeginBlock() *RequestBeginBlock { + if x, ok := m.GetValue().(*Request_BeginBlock); ok { + return x.BeginBlock + } + return nil +} + +func (m *Request) GetEndBlock() *RequestEndBlock { + if x, ok := m.GetValue().(*Request_EndBlock); ok { + return x.EndBlock + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*Request) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _Request_OneofMarshaler, _Request_OneofUnmarshaler, _Request_OneofSizer, []interface{}{ + (*Request_Echo)(nil), + (*Request_Flush)(nil), + (*Request_Info)(nil), + (*Request_SetOption)(nil), + (*Request_AppendTx)(nil), + (*Request_CheckTx)(nil), + (*Request_Commit)(nil), + (*Request_Query)(nil), + (*Request_InitChain)(nil), + (*Request_BeginBlock)(nil), + (*Request_EndBlock)(nil), + } +} + +func _Request_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*Request) + // value + switch x := m.Value.(type) { + case *Request_Echo: + b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Echo); err != nil { + return err + } + case *Request_Flush: + b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Flush); err != nil { + return err + } + case *Request_Info: + b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Info); err != nil { + return err + } + case *Request_SetOption: + b.EncodeVarint(4<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.SetOption); err != nil { + return err + } + case *Request_AppendTx: + b.EncodeVarint(5<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AppendTx); err != nil { + return err + } + case *Request_CheckTx: + b.EncodeVarint(6<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.CheckTx); err != nil { + return err + } + case *Request_Commit: + b.EncodeVarint(7<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Commit); err != nil { + return err + } + case *Request_Query: + b.EncodeVarint(8<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Query); err != nil { + return err + } + case *Request_InitChain: + b.EncodeVarint(9<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.InitChain); err != nil { + return err + } + case *Request_BeginBlock: + b.EncodeVarint(10<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.BeginBlock); err != nil { + return err + } + case *Request_EndBlock: + b.EncodeVarint(11<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.EndBlock); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("Request.Value has unexpected type %T", x) + } + return nil +} + +func _Request_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*Request) + switch tag { + case 1: // value.echo + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestEcho) + err := b.DecodeMessage(msg) + m.Value = &Request_Echo{msg} + return true, err + case 2: // value.flush + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestFlush) + err := b.DecodeMessage(msg) + m.Value = &Request_Flush{msg} + return true, err + case 3: // value.info + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestInfo) + err := b.DecodeMessage(msg) + m.Value = &Request_Info{msg} + return true, err + case 4: // value.set_option + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestSetOption) + err := b.DecodeMessage(msg) + m.Value = &Request_SetOption{msg} + return true, err + case 5: // value.append_tx + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestAppendTx) + err := b.DecodeMessage(msg) + m.Value = &Request_AppendTx{msg} + return true, err + case 6: // value.check_tx + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestCheckTx) + err := b.DecodeMessage(msg) + m.Value = &Request_CheckTx{msg} + return true, err + case 7: // value.commit + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestCommit) + err := b.DecodeMessage(msg) + m.Value = &Request_Commit{msg} + return true, err + case 8: // value.query + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestQuery) + err := b.DecodeMessage(msg) + m.Value = &Request_Query{msg} + return true, err + case 9: // value.init_chain + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestInitChain) + err := b.DecodeMessage(msg) + m.Value = &Request_InitChain{msg} + return true, err + case 10: // value.begin_block + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestBeginBlock) + err := b.DecodeMessage(msg) + m.Value = &Request_BeginBlock{msg} + return true, err + case 11: // value.end_block + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RequestEndBlock) + err := b.DecodeMessage(msg) + m.Value = &Request_EndBlock{msg} + return true, err + default: + return false, nil + } +} + +func _Request_OneofSizer(msg proto.Message) (n int) { + m := msg.(*Request) + // value + switch x := m.Value.(type) { + case *Request_Echo: + s := proto.Size(x.Echo) + n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_Flush: + s := proto.Size(x.Flush) + n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_Info: + s := proto.Size(x.Info) + n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_SetOption: + s := proto.Size(x.SetOption) + n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_AppendTx: + s := proto.Size(x.AppendTx) + n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_CheckTx: + s := proto.Size(x.CheckTx) + n += proto.SizeVarint(6<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_Commit: + s := proto.Size(x.Commit) + n += proto.SizeVarint(7<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_Query: + s := proto.Size(x.Query) + n += proto.SizeVarint(8<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_InitChain: + s := proto.Size(x.InitChain) + n += proto.SizeVarint(9<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_BeginBlock: + s := proto.Size(x.BeginBlock) + n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Request_EndBlock: + s := proto.Size(x.EndBlock) + n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n +} + +type RequestEcho struct { + Message string `protobuf:"bytes,1,opt,name=message" json:"message,omitempty"` +} + +func (m *RequestEcho) Reset() { *m = RequestEcho{} } +func (m *RequestEcho) String() string { return proto.CompactTextString(m) } +func (*RequestEcho) ProtoMessage() {} +func (*RequestEcho) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +type RequestFlush struct { +} + +func (m *RequestFlush) Reset() { *m = RequestFlush{} } +func (m *RequestFlush) String() string { return proto.CompactTextString(m) } +func (*RequestFlush) ProtoMessage() {} +func (*RequestFlush) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +type RequestInfo struct { +} + +func (m *RequestInfo) Reset() { *m = RequestInfo{} } +func (m *RequestInfo) String() string { return proto.CompactTextString(m) } +func (*RequestInfo) ProtoMessage() {} +func (*RequestInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } + +type RequestSetOption struct { + Key string `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` +} + +func (m *RequestSetOption) Reset() { *m = RequestSetOption{} } +func (m *RequestSetOption) String() string { return proto.CompactTextString(m) } +func (*RequestSetOption) ProtoMessage() {} +func (*RequestSetOption) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } + +type RequestAppendTx struct { + Tx []byte `protobuf:"bytes,1,opt,name=tx,proto3" json:"tx,omitempty"` +} + +func (m *RequestAppendTx) Reset() { *m = RequestAppendTx{} } +func (m *RequestAppendTx) String() string { return proto.CompactTextString(m) } +func (*RequestAppendTx) ProtoMessage() {} +func (*RequestAppendTx) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } + +type RequestCheckTx struct { + Tx []byte `protobuf:"bytes,1,opt,name=tx,proto3" json:"tx,omitempty"` +} + +func (m *RequestCheckTx) Reset() { *m = RequestCheckTx{} } +func (m *RequestCheckTx) String() string { return proto.CompactTextString(m) } +func (*RequestCheckTx) ProtoMessage() {} +func (*RequestCheckTx) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } + +type RequestQuery struct { + Query []byte `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` +} + +func (m *RequestQuery) Reset() { *m = RequestQuery{} } +func (m *RequestQuery) String() string { return proto.CompactTextString(m) } +func (*RequestQuery) ProtoMessage() {} +func (*RequestQuery) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } + +type RequestCommit struct { +} + +func (m *RequestCommit) Reset() { *m = RequestCommit{} } +func (m *RequestCommit) String() string { return proto.CompactTextString(m) } +func (*RequestCommit) ProtoMessage() {} +func (*RequestCommit) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } + +type RequestInitChain struct { + Validators []*Validator `protobuf:"bytes,1,rep,name=validators" json:"validators,omitempty"` +} + +func (m *RequestInitChain) Reset() { *m = RequestInitChain{} } +func (m *RequestInitChain) String() string { return proto.CompactTextString(m) } +func (*RequestInitChain) ProtoMessage() {} +func (*RequestInitChain) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } + +func (m *RequestInitChain) GetValidators() []*Validator { if m != nil { return m.Validators } return nil } +type RequestBeginBlock struct { + Height uint64 `protobuf:"varint,1,opt,name=height" json:"height,omitempty"` +} + +func (m *RequestBeginBlock) Reset() { *m = RequestBeginBlock{} } +func (m *RequestBeginBlock) String() string { return proto.CompactTextString(m) } +func (*RequestBeginBlock) ProtoMessage() {} +func (*RequestBeginBlock) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } + +type RequestEndBlock struct { + Height uint64 `protobuf:"varint,1,opt,name=height" json:"height,omitempty"` +} + +func (m *RequestEndBlock) Reset() { *m = RequestEndBlock{} } +func (m *RequestEndBlock) String() string { return proto.CompactTextString(m) } +func (*RequestEndBlock) ProtoMessage() {} +func (*RequestEndBlock) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } + type Response struct { - Type MessageType `protobuf:"varint,1,opt,name=type,enum=types.MessageType" json:"type,omitempty"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - Code CodeType `protobuf:"varint,3,opt,name=code,enum=types.CodeType" json:"code,omitempty"` - Error string `protobuf:"bytes,4,opt,name=error" json:"error,omitempty"` - Log string `protobuf:"bytes,5,opt,name=log" json:"log,omitempty"` - Validators []*Validator `protobuf:"bytes,6,rep,name=validators" json:"validators,omitempty"` + // Types that are valid to be assigned to Value: + // *Response_Exception + // *Response_Echo + // *Response_Flush + // *Response_Info + // *Response_SetOption + // *Response_AppendTx + // *Response_CheckTx + // *Response_Commit + // *Response_Query + // *Response_InitChain + // *Response_BeginBlock + // *Response_EndBlock + Value isResponse_Value `protobuf_oneof:"value"` } func (m *Response) Reset() { *m = Response{} } func (m *Response) String() string { return proto.CompactTextString(m) } func (*Response) ProtoMessage() {} -func (*Response) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +func (*Response) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } + +type isResponse_Value interface { + isResponse_Value() +} + +type Response_Exception struct { + Exception *ResponseException `protobuf:"bytes,1,opt,name=exception,oneof"` +} +type Response_Echo struct { + Echo *ResponseEcho `protobuf:"bytes,2,opt,name=echo,oneof"` +} +type Response_Flush struct { + Flush *ResponseFlush `protobuf:"bytes,3,opt,name=flush,oneof"` +} +type Response_Info struct { + Info *ResponseInfo `protobuf:"bytes,4,opt,name=info,oneof"` +} +type Response_SetOption struct { + SetOption *ResponseSetOption `protobuf:"bytes,5,opt,name=set_option,json=setOption,oneof"` +} +type Response_AppendTx struct { + AppendTx *ResponseAppendTx `protobuf:"bytes,6,opt,name=append_tx,json=appendTx,oneof"` +} +type Response_CheckTx struct { + CheckTx *ResponseCheckTx `protobuf:"bytes,7,opt,name=check_tx,json=checkTx,oneof"` +} +type Response_Commit struct { + Commit *ResponseCommit `protobuf:"bytes,8,opt,name=commit,oneof"` +} +type Response_Query struct { + Query *ResponseQuery `protobuf:"bytes,9,opt,name=query,oneof"` +} +type Response_InitChain struct { + InitChain *ResponseInitChain `protobuf:"bytes,10,opt,name=init_chain,json=initChain,oneof"` +} +type Response_BeginBlock struct { + BeginBlock *ResponseBeginBlock `protobuf:"bytes,11,opt,name=begin_block,json=beginBlock,oneof"` +} +type Response_EndBlock struct { + EndBlock *ResponseEndBlock `protobuf:"bytes,12,opt,name=end_block,json=endBlock,oneof"` +} + +func (*Response_Exception) isResponse_Value() {} +func (*Response_Echo) isResponse_Value() {} +func (*Response_Flush) isResponse_Value() {} +func (*Response_Info) isResponse_Value() {} +func (*Response_SetOption) isResponse_Value() {} +func (*Response_AppendTx) isResponse_Value() {} +func (*Response_CheckTx) isResponse_Value() {} +func (*Response_Commit) isResponse_Value() {} +func (*Response_Query) isResponse_Value() {} +func (*Response_InitChain) isResponse_Value() {} +func (*Response_BeginBlock) isResponse_Value() {} +func (*Response_EndBlock) isResponse_Value() {} -func (m *Response) GetValidators() []*Validator { +func (m *Response) GetValue() isResponse_Value { if m != nil { - return m.Validators + return m.Value + } + return nil +} + +func (m *Response) GetException() *ResponseException { + if x, ok := m.GetValue().(*Response_Exception); ok { + return x.Exception + } + return nil +} + +func (m *Response) GetEcho() *ResponseEcho { + if x, ok := m.GetValue().(*Response_Echo); ok { + return x.Echo + } + return nil +} + +func (m *Response) GetFlush() *ResponseFlush { + if x, ok := m.GetValue().(*Response_Flush); ok { + return x.Flush + } + return nil +} + +func (m *Response) GetInfo() *ResponseInfo { + if x, ok := m.GetValue().(*Response_Info); ok { + return x.Info + } + return nil +} + +func (m *Response) GetSetOption() *ResponseSetOption { + if x, ok := m.GetValue().(*Response_SetOption); ok { + return x.SetOption + } + return nil +} + +func (m *Response) GetAppendTx() *ResponseAppendTx { + if x, ok := m.GetValue().(*Response_AppendTx); ok { + return x.AppendTx + } + return nil +} + +func (m *Response) GetCheckTx() *ResponseCheckTx { + if x, ok := m.GetValue().(*Response_CheckTx); ok { + return x.CheckTx + } + return nil +} + +func (m *Response) GetCommit() *ResponseCommit { + if x, ok := m.GetValue().(*Response_Commit); ok { + return x.Commit + } + return nil +} + +func (m *Response) GetQuery() *ResponseQuery { + if x, ok := m.GetValue().(*Response_Query); ok { + return x.Query + } + return nil +} + +func (m *Response) GetInitChain() *ResponseInitChain { + if x, ok := m.GetValue().(*Response_InitChain); ok { + return x.InitChain + } + return nil +} + +func (m *Response) GetBeginBlock() *ResponseBeginBlock { + if x, ok := m.GetValue().(*Response_BeginBlock); ok { + return x.BeginBlock + } + return nil +} + +func (m *Response) GetEndBlock() *ResponseEndBlock { + if x, ok := m.GetValue().(*Response_EndBlock); ok { + return x.EndBlock + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*Response) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _Response_OneofMarshaler, _Response_OneofUnmarshaler, _Response_OneofSizer, []interface{}{ + (*Response_Exception)(nil), + (*Response_Echo)(nil), + (*Response_Flush)(nil), + (*Response_Info)(nil), + (*Response_SetOption)(nil), + (*Response_AppendTx)(nil), + (*Response_CheckTx)(nil), + (*Response_Commit)(nil), + (*Response_Query)(nil), + (*Response_InitChain)(nil), + (*Response_BeginBlock)(nil), + (*Response_EndBlock)(nil), + } +} + +func _Response_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*Response) + // value + switch x := m.Value.(type) { + case *Response_Exception: + b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Exception); err != nil { + return err + } + case *Response_Echo: + b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Echo); err != nil { + return err + } + case *Response_Flush: + b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Flush); err != nil { + return err + } + case *Response_Info: + b.EncodeVarint(4<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Info); err != nil { + return err + } + case *Response_SetOption: + b.EncodeVarint(5<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.SetOption); err != nil { + return err + } + case *Response_AppendTx: + b.EncodeVarint(6<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AppendTx); err != nil { + return err + } + case *Response_CheckTx: + b.EncodeVarint(7<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.CheckTx); err != nil { + return err + } + case *Response_Commit: + b.EncodeVarint(8<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Commit); err != nil { + return err + } + case *Response_Query: + b.EncodeVarint(9<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Query); err != nil { + return err + } + case *Response_InitChain: + b.EncodeVarint(10<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.InitChain); err != nil { + return err + } + case *Response_BeginBlock: + b.EncodeVarint(11<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.BeginBlock); err != nil { + return err + } + case *Response_EndBlock: + b.EncodeVarint(12<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.EndBlock); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("Response.Value has unexpected type %T", x) + } + return nil +} + +func _Response_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*Response) + switch tag { + case 1: // value.exception + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseException) + err := b.DecodeMessage(msg) + m.Value = &Response_Exception{msg} + return true, err + case 2: // value.echo + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseEcho) + err := b.DecodeMessage(msg) + m.Value = &Response_Echo{msg} + return true, err + case 3: // value.flush + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseFlush) + err := b.DecodeMessage(msg) + m.Value = &Response_Flush{msg} + return true, err + case 4: // value.info + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseInfo) + err := b.DecodeMessage(msg) + m.Value = &Response_Info{msg} + return true, err + case 5: // value.set_option + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseSetOption) + err := b.DecodeMessage(msg) + m.Value = &Response_SetOption{msg} + return true, err + case 6: // value.append_tx + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseAppendTx) + err := b.DecodeMessage(msg) + m.Value = &Response_AppendTx{msg} + return true, err + case 7: // value.check_tx + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseCheckTx) + err := b.DecodeMessage(msg) + m.Value = &Response_CheckTx{msg} + return true, err + case 8: // value.commit + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseCommit) + err := b.DecodeMessage(msg) + m.Value = &Response_Commit{msg} + return true, err + case 9: // value.query + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseQuery) + err := b.DecodeMessage(msg) + m.Value = &Response_Query{msg} + return true, err + case 10: // value.init_chain + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseInitChain) + err := b.DecodeMessage(msg) + m.Value = &Response_InitChain{msg} + return true, err + case 11: // value.begin_block + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseBeginBlock) + err := b.DecodeMessage(msg) + m.Value = &Response_BeginBlock{msg} + return true, err + case 12: // value.end_block + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ResponseEndBlock) + err := b.DecodeMessage(msg) + m.Value = &Response_EndBlock{msg} + return true, err + default: + return false, nil + } +} + +func _Response_OneofSizer(msg proto.Message) (n int) { + m := msg.(*Response) + // value + switch x := m.Value.(type) { + case *Response_Exception: + s := proto.Size(x.Exception) + n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_Echo: + s := proto.Size(x.Echo) + n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_Flush: + s := proto.Size(x.Flush) + n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_Info: + s := proto.Size(x.Info) + n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_SetOption: + s := proto.Size(x.SetOption) + n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_AppendTx: + s := proto.Size(x.AppendTx) + n += proto.SizeVarint(6<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_CheckTx: + s := proto.Size(x.CheckTx) + n += proto.SizeVarint(7<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_Commit: + s := proto.Size(x.Commit) + n += proto.SizeVarint(8<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_Query: + s := proto.Size(x.Query) + n += proto.SizeVarint(9<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_InitChain: + s := proto.Size(x.InitChain) + n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_BeginBlock: + s := proto.Size(x.BeginBlock) + n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case *Response_EndBlock: + s := proto.Size(x.EndBlock) + n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += proto.SizeVarint(uint64(s)) + n += s + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n +} + +type ResponseException struct { + Error string `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` +} + +func (m *ResponseException) Reset() { *m = ResponseException{} } +func (m *ResponseException) String() string { return proto.CompactTextString(m) } +func (*ResponseException) ProtoMessage() {} +func (*ResponseException) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } + +type ResponseEcho struct { + Message string `protobuf:"bytes,1,opt,name=message" json:"message,omitempty"` +} + +func (m *ResponseEcho) Reset() { *m = ResponseEcho{} } +func (m *ResponseEcho) String() string { return proto.CompactTextString(m) } +func (*ResponseEcho) ProtoMessage() {} +func (*ResponseEcho) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} } + +type ResponseFlush struct { +} + +func (m *ResponseFlush) Reset() { *m = ResponseFlush{} } +func (m *ResponseFlush) String() string { return proto.CompactTextString(m) } +func (*ResponseFlush) ProtoMessage() {} +func (*ResponseFlush) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} } + +type ResponseInfo struct { + Info string `protobuf:"bytes,1,opt,name=info" json:"info,omitempty"` +} + +func (m *ResponseInfo) Reset() { *m = ResponseInfo{} } +func (m *ResponseInfo) String() string { return proto.CompactTextString(m) } +func (*ResponseInfo) ProtoMessage() {} +func (*ResponseInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} } + +type ResponseSetOption struct { + Log string `protobuf:"bytes,1,opt,name=log" json:"log,omitempty"` +} + +func (m *ResponseSetOption) Reset() { *m = ResponseSetOption{} } +func (m *ResponseSetOption) String() string { return proto.CompactTextString(m) } +func (*ResponseSetOption) ProtoMessage() {} +func (*ResponseSetOption) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} } + +type ResponseAppendTx struct { + Code CodeType `protobuf:"varint,1,opt,name=code,enum=types.CodeType" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log" json:"log,omitempty"` +} + +func (m *ResponseAppendTx) Reset() { *m = ResponseAppendTx{} } +func (m *ResponseAppendTx) String() string { return proto.CompactTextString(m) } +func (*ResponseAppendTx) ProtoMessage() {} +func (*ResponseAppendTx) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} } + +type ResponseCheckTx struct { + Code CodeType `protobuf:"varint,1,opt,name=code,enum=types.CodeType" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log" json:"log,omitempty"` +} + +func (m *ResponseCheckTx) Reset() { *m = ResponseCheckTx{} } +func (m *ResponseCheckTx) String() string { return proto.CompactTextString(m) } +func (*ResponseCheckTx) ProtoMessage() {} +func (*ResponseCheckTx) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} } + +type ResponseQuery struct { + Code CodeType `protobuf:"varint,1,opt,name=code,enum=types.CodeType" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log" json:"log,omitempty"` +} + +func (m *ResponseQuery) Reset() { *m = ResponseQuery{} } +func (m *ResponseQuery) String() string { return proto.CompactTextString(m) } +func (*ResponseQuery) ProtoMessage() {} +func (*ResponseQuery) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} } + +type ResponseCommit struct { + Code CodeType `protobuf:"varint,1,opt,name=code,enum=types.CodeType" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log" json:"log,omitempty"` +} + +func (m *ResponseCommit) Reset() { *m = ResponseCommit{} } +func (m *ResponseCommit) String() string { return proto.CompactTextString(m) } +func (*ResponseCommit) ProtoMessage() {} +func (*ResponseCommit) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} } + +type ResponseInitChain struct { +} + +func (m *ResponseInitChain) Reset() { *m = ResponseInitChain{} } +func (m *ResponseInitChain) String() string { return proto.CompactTextString(m) } +func (*ResponseInitChain) ProtoMessage() {} +func (*ResponseInitChain) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} } + +type ResponseBeginBlock struct { +} + +func (m *ResponseBeginBlock) Reset() { *m = ResponseBeginBlock{} } +func (m *ResponseBeginBlock) String() string { return proto.CompactTextString(m) } +func (*ResponseBeginBlock) ProtoMessage() {} +func (*ResponseBeginBlock) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} } + +type ResponseEndBlock struct { + Diffs []*Validator `protobuf:"bytes,4,rep,name=diffs" json:"diffs,omitempty"` +} + +func (m *ResponseEndBlock) Reset() { *m = ResponseEndBlock{} } +func (m *ResponseEndBlock) String() string { return proto.CompactTextString(m) } +func (*ResponseEndBlock) ProtoMessage() {} +func (*ResponseEndBlock) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} } + +func (m *ResponseEndBlock) GetDiffs() []*Validator { + if m != nil { + return m.Diffs } return nil } @@ -236,62 +1285,527 @@ type Validator struct { func (m *Validator) Reset() { *m = Validator{} } func (m *Validator) String() string { return proto.CompactTextString(m) } func (*Validator) ProtoMessage() {} -func (*Validator) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +func (*Validator) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} } func init() { proto.RegisterType((*Request)(nil), "types.Request") + proto.RegisterType((*RequestEcho)(nil), "types.RequestEcho") + proto.RegisterType((*RequestFlush)(nil), "types.RequestFlush") + proto.RegisterType((*RequestInfo)(nil), "types.RequestInfo") + proto.RegisterType((*RequestSetOption)(nil), "types.RequestSetOption") + proto.RegisterType((*RequestAppendTx)(nil), "types.RequestAppendTx") + proto.RegisterType((*RequestCheckTx)(nil), "types.RequestCheckTx") + proto.RegisterType((*RequestQuery)(nil), "types.RequestQuery") + proto.RegisterType((*RequestCommit)(nil), "types.RequestCommit") + proto.RegisterType((*RequestInitChain)(nil), "types.RequestInitChain") + proto.RegisterType((*RequestBeginBlock)(nil), "types.RequestBeginBlock") + proto.RegisterType((*RequestEndBlock)(nil), "types.RequestEndBlock") proto.RegisterType((*Response)(nil), "types.Response") + proto.RegisterType((*ResponseException)(nil), "types.ResponseException") + proto.RegisterType((*ResponseEcho)(nil), "types.ResponseEcho") + proto.RegisterType((*ResponseFlush)(nil), "types.ResponseFlush") + proto.RegisterType((*ResponseInfo)(nil), "types.ResponseInfo") + proto.RegisterType((*ResponseSetOption)(nil), "types.ResponseSetOption") + proto.RegisterType((*ResponseAppendTx)(nil), "types.ResponseAppendTx") + proto.RegisterType((*ResponseCheckTx)(nil), "types.ResponseCheckTx") + proto.RegisterType((*ResponseQuery)(nil), "types.ResponseQuery") + proto.RegisterType((*ResponseCommit)(nil), "types.ResponseCommit") + proto.RegisterType((*ResponseInitChain)(nil), "types.ResponseInitChain") + proto.RegisterType((*ResponseBeginBlock)(nil), "types.ResponseBeginBlock") + proto.RegisterType((*ResponseEndBlock)(nil), "types.ResponseEndBlock") proto.RegisterType((*Validator)(nil), "types.Validator") proto.RegisterEnum("types.MessageType", MessageType_name, MessageType_value) proto.RegisterEnum("types.CodeType", CodeType_name, CodeType_value) } +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion2 + +// Client API for TMSPApplication service + +type TMSPApplicationClient interface { + Echo(ctx context.Context, in *RequestEcho, opts ...grpc.CallOption) (*ResponseEcho, error) + Flush(ctx context.Context, in *RequestFlush, opts ...grpc.CallOption) (*ResponseFlush, error) + Info(ctx context.Context, in *RequestInfo, opts ...grpc.CallOption) (*ResponseInfo, error) + SetOption(ctx context.Context, in *RequestSetOption, opts ...grpc.CallOption) (*ResponseSetOption, error) + AppendTx(ctx context.Context, in *RequestAppendTx, opts ...grpc.CallOption) (*ResponseAppendTx, error) + CheckTx(ctx context.Context, in *RequestCheckTx, opts ...grpc.CallOption) (*ResponseCheckTx, error) + Query(ctx context.Context, in *RequestQuery, opts ...grpc.CallOption) (*ResponseQuery, error) + Commit(ctx context.Context, in *RequestCommit, opts ...grpc.CallOption) (*ResponseCommit, error) + InitChain(ctx context.Context, in *RequestInitChain, opts ...grpc.CallOption) (*ResponseInitChain, error) + BeginBlock(ctx context.Context, in *RequestBeginBlock, opts ...grpc.CallOption) (*ResponseBeginBlock, error) + EndBlock(ctx context.Context, in *RequestEndBlock, opts ...grpc.CallOption) (*ResponseEndBlock, error) +} + +type tMSPApplicationClient struct { + cc *grpc.ClientConn +} + +func NewTMSPApplicationClient(cc *grpc.ClientConn) TMSPApplicationClient { + return &tMSPApplicationClient{cc} +} + +func (c *tMSPApplicationClient) Echo(ctx context.Context, in *RequestEcho, opts ...grpc.CallOption) (*ResponseEcho, error) { + out := new(ResponseEcho) + err := grpc.Invoke(ctx, "/types.TMSPApplication/Echo", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) Flush(ctx context.Context, in *RequestFlush, opts ...grpc.CallOption) (*ResponseFlush, error) { + out := new(ResponseFlush) + err := grpc.Invoke(ctx, "/types.TMSPApplication/Flush", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) Info(ctx context.Context, in *RequestInfo, opts ...grpc.CallOption) (*ResponseInfo, error) { + out := new(ResponseInfo) + err := grpc.Invoke(ctx, "/types.TMSPApplication/Info", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) SetOption(ctx context.Context, in *RequestSetOption, opts ...grpc.CallOption) (*ResponseSetOption, error) { + out := new(ResponseSetOption) + err := grpc.Invoke(ctx, "/types.TMSPApplication/SetOption", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) AppendTx(ctx context.Context, in *RequestAppendTx, opts ...grpc.CallOption) (*ResponseAppendTx, error) { + out := new(ResponseAppendTx) + err := grpc.Invoke(ctx, "/types.TMSPApplication/AppendTx", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) CheckTx(ctx context.Context, in *RequestCheckTx, opts ...grpc.CallOption) (*ResponseCheckTx, error) { + out := new(ResponseCheckTx) + err := grpc.Invoke(ctx, "/types.TMSPApplication/CheckTx", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) Query(ctx context.Context, in *RequestQuery, opts ...grpc.CallOption) (*ResponseQuery, error) { + out := new(ResponseQuery) + err := grpc.Invoke(ctx, "/types.TMSPApplication/Query", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) Commit(ctx context.Context, in *RequestCommit, opts ...grpc.CallOption) (*ResponseCommit, error) { + out := new(ResponseCommit) + err := grpc.Invoke(ctx, "/types.TMSPApplication/Commit", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) InitChain(ctx context.Context, in *RequestInitChain, opts ...grpc.CallOption) (*ResponseInitChain, error) { + out := new(ResponseInitChain) + err := grpc.Invoke(ctx, "/types.TMSPApplication/InitChain", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) BeginBlock(ctx context.Context, in *RequestBeginBlock, opts ...grpc.CallOption) (*ResponseBeginBlock, error) { + out := new(ResponseBeginBlock) + err := grpc.Invoke(ctx, "/types.TMSPApplication/BeginBlock", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tMSPApplicationClient) EndBlock(ctx context.Context, in *RequestEndBlock, opts ...grpc.CallOption) (*ResponseEndBlock, error) { + out := new(ResponseEndBlock) + err := grpc.Invoke(ctx, "/types.TMSPApplication/EndBlock", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// Server API for TMSPApplication service + +type TMSPApplicationServer interface { + Echo(context.Context, *RequestEcho) (*ResponseEcho, error) + Flush(context.Context, *RequestFlush) (*ResponseFlush, error) + Info(context.Context, *RequestInfo) (*ResponseInfo, error) + SetOption(context.Context, *RequestSetOption) (*ResponseSetOption, error) + AppendTx(context.Context, *RequestAppendTx) (*ResponseAppendTx, error) + CheckTx(context.Context, *RequestCheckTx) (*ResponseCheckTx, error) + Query(context.Context, *RequestQuery) (*ResponseQuery, error) + Commit(context.Context, *RequestCommit) (*ResponseCommit, error) + InitChain(context.Context, *RequestInitChain) (*ResponseInitChain, error) + BeginBlock(context.Context, *RequestBeginBlock) (*ResponseBeginBlock, error) + EndBlock(context.Context, *RequestEndBlock) (*ResponseEndBlock, error) +} + +func RegisterTMSPApplicationServer(s *grpc.Server, srv TMSPApplicationServer) { + s.RegisterService(&_TMSPApplication_serviceDesc, srv) +} + +func _TMSPApplication_Echo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestEcho) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).Echo(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/Echo", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).Echo(ctx, req.(*RequestEcho)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestFlush) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).Flush(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/Flush", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).Flush(ctx, req.(*RequestFlush)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_Info_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestInfo) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).Info(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/Info", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).Info(ctx, req.(*RequestInfo)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_SetOption_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestSetOption) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).SetOption(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/SetOption", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).SetOption(ctx, req.(*RequestSetOption)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_AppendTx_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestAppendTx) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).AppendTx(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/AppendTx", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).AppendTx(ctx, req.(*RequestAppendTx)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_CheckTx_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestCheckTx) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).CheckTx(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/CheckTx", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).CheckTx(ctx, req.(*RequestCheckTx)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_Query_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestQuery) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).Query(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/Query", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).Query(ctx, req.(*RequestQuery)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_Commit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestCommit) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).Commit(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/Commit", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).Commit(ctx, req.(*RequestCommit)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_InitChain_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestInitChain) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).InitChain(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/InitChain", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).InitChain(ctx, req.(*RequestInitChain)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_BeginBlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestBeginBlock) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).BeginBlock(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/BeginBlock", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).BeginBlock(ctx, req.(*RequestBeginBlock)) + } + return interceptor(ctx, in, info, handler) +} + +func _TMSPApplication_EndBlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RequestEndBlock) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TMSPApplicationServer).EndBlock(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/types.TMSPApplication/EndBlock", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TMSPApplicationServer).EndBlock(ctx, req.(*RequestEndBlock)) + } + return interceptor(ctx, in, info, handler) +} + +var _TMSPApplication_serviceDesc = grpc.ServiceDesc{ + ServiceName: "types.TMSPApplication", + HandlerType: (*TMSPApplicationServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "Echo", + Handler: _TMSPApplication_Echo_Handler, + }, + { + MethodName: "Flush", + Handler: _TMSPApplication_Flush_Handler, + }, + { + MethodName: "Info", + Handler: _TMSPApplication_Info_Handler, + }, + { + MethodName: "SetOption", + Handler: _TMSPApplication_SetOption_Handler, + }, + { + MethodName: "AppendTx", + Handler: _TMSPApplication_AppendTx_Handler, + }, + { + MethodName: "CheckTx", + Handler: _TMSPApplication_CheckTx_Handler, + }, + { + MethodName: "Query", + Handler: _TMSPApplication_Query_Handler, + }, + { + MethodName: "Commit", + Handler: _TMSPApplication_Commit_Handler, + }, + { + MethodName: "InitChain", + Handler: _TMSPApplication_InitChain_Handler, + }, + { + MethodName: "BeginBlock", + Handler: _TMSPApplication_BeginBlock_Handler, + }, + { + MethodName: "EndBlock", + Handler: _TMSPApplication_EndBlock_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, +} + var fileDescriptor0 = []byte{ - // 729 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xa4, 0x94, 0x4b, 0x4f, 0xeb, 0x46, - 0x14, 0xc7, 0xeb, 0xc4, 0x09, 0xc9, 0x09, 0x84, 0xc9, 0x90, 0x80, 0x5b, 0x75, 0x81, 0xa8, 0x54, - 0x21, 0x16, 0xb4, 0xa2, 0xab, 0x2e, 0x49, 0x1a, 0x50, 0x84, 0x80, 0xd4, 0x3c, 0xf6, 0xc6, 0x3e, - 0x89, 0xa7, 0x71, 0x66, 0x8c, 0x3d, 0x06, 0xd2, 0xef, 0x55, 0x55, 0xba, 0xfb, 0xbb, 0xb8, 0xef, - 0xc7, 0x27, 0xba, 0x33, 0x7e, 0x24, 0x21, 0x77, 0x71, 0x17, 0x77, 0x13, 0xcd, 0xf9, 0xfd, 0x67, - 0xce, 0x39, 0xff, 0x33, 0x13, 0x43, 0x4b, 0xce, 0x42, 0x8c, 0x7f, 0x4b, 0x7f, 0x0f, 0xc3, 0x48, - 0x48, 0x41, 0x2b, 0x69, 0xb0, 0xf7, 0xbf, 0x01, 0x6b, 0x36, 0xde, 0x27, 0x18, 0x4b, 0xfa, 0x2b, - 0x98, 0x1a, 0x5a, 0xc6, 0xae, 0xb1, 0xdf, 0x3c, 0xa2, 0x87, 0xd9, 0xf6, 0x73, 0x8c, 0x63, 0x67, - 0x8c, 0xd7, 0x2a, 0xb0, 0x53, 0x9d, 0x52, 0x30, 0x3d, 0x47, 0x3a, 0x56, 0x49, 0xed, 0x5b, 0xb7, - 0xd3, 0x35, 0x25, 0x50, 0x9e, 0xe0, 0xcc, 0x2a, 0x2b, 0x54, 0xb7, 0xf5, 0x92, 0xb6, 0xa1, 0xf2, - 0xe0, 0x04, 0x09, 0x5a, 0x66, 0xca, 0xb2, 0x80, 0xfe, 0x0e, 0xa0, 0x16, 0x4c, 0x9d, 0x11, 0x51, - 0x6c, 0x55, 0x76, 0xcb, 0xfb, 0x8d, 0x23, 0x92, 0x57, 0xba, 0x2d, 0x04, 0x7b, 0x69, 0x0f, 0xdd, - 0x86, 0xaa, 0x8f, 0x6c, 0xec, 0x4b, 0xab, 0xaa, 0x12, 0x99, 0x76, 0x1e, 0xed, 0xbd, 0x34, 0xa0, - 0x66, 0x63, 0x1c, 0x0a, 0x1e, 0xe3, 0x77, 0xb5, 0xfe, 0x0b, 0x98, 0xae, 0xf0, 0x30, 0xed, 0xbd, - 0x79, 0xb4, 0x99, 0x9f, 0xed, 0x29, 0x94, 0x1d, 0xd4, 0xa2, 0x76, 0x83, 0x51, 0x24, 0xa2, 0xc2, - 0x4d, 0x1a, 0x68, 0xd7, 0x81, 0x18, 0x2b, 0x1b, 0xa9, 0x6b, 0xb5, 0x5c, 0xf1, 0x57, 0xfd, 0xb6, - 0xbf, 0xbd, 0x3f, 0xa1, 0x3e, 0x17, 0xb4, 0xd9, 0x30, 0xb9, 0x3b, 0x53, 0x93, 0x34, 0xd2, 0x0e, - 0xf3, 0x48, 0x97, 0x0f, 0xc5, 0x23, 0x46, 0x69, 0xe3, 0xa6, 0x9d, 0x05, 0x07, 0x2f, 0x0c, 0x68, - 0x2c, 0x79, 0xa4, 0x9b, 0xd0, 0xb8, 0x48, 0x82, 0x20, 0x47, 0xe4, 0x07, 0x5a, 0x03, 0xb3, 0xef, - 0xfa, 0x82, 0x18, 0xb4, 0x0e, 0x95, 0x93, 0x20, 0x89, 0x7d, 0x52, 0xd2, 0x70, 0xc0, 0x47, 0x82, - 0x94, 0xe9, 0x06, 0xd4, 0xaf, 0x50, 0x5e, 0x86, 0x92, 0x09, 0x4e, 0x4c, 0x1d, 0xf6, 0x9f, 0x5c, - 0xcc, 0xc2, 0x0a, 0x5d, 0x87, 0xda, 0x71, 0x18, 0x22, 0xf7, 0xae, 0x9f, 0x48, 0x8b, 0x36, 0x60, - 0xad, 0xe7, 0xa3, 0x3b, 0x51, 0x81, 0x9a, 0x22, 0x54, 0x7b, 0x62, 0x3a, 0x65, 0x92, 0x6c, 0xe9, - 0xcc, 0x7f, 0x27, 0x18, 0xcd, 0x48, 0x5b, 0x27, 0x18, 0x70, 0x26, 0x7b, 0xbe, 0xc3, 0x38, 0xe9, - 0xd0, 0x26, 0x40, 0x17, 0xc7, 0x8c, 0x77, 0x03, 0xe1, 0x4e, 0xc8, 0xb6, 0x4e, 0xd8, 0xe7, 0x5e, - 0x16, 0xed, 0x1c, 0xfc, 0x57, 0x81, 0x5a, 0x31, 0x64, 0x5a, 0x85, 0xd2, 0xe5, 0x99, 0x6a, 0xb8, - 0x05, 0x1b, 0x03, 0x2e, 0x31, 0xe2, 0x4e, 0xd0, 0xd7, 0x13, 0x56, 0x9d, 0x2b, 0xd4, 0xe7, 0xea, - 0x0e, 0x18, 0x1f, 0x67, 0xa8, 0xa4, 0x13, 0x75, 0x1d, 0xef, 0x42, 0x70, 0x17, 0x95, 0x0b, 0x02, - 0xeb, 0x37, 0xdc, 0x49, 0xa4, 0x2f, 0x22, 0xf6, 0x2f, 0x7a, 0xca, 0x48, 0x07, 0x5a, 0x03, 0x1e, - 0x27, 0xa3, 0x11, 0x73, 0x19, 0x72, 0x79, 0x92, 0x70, 0x2f, 0x56, 0x86, 0x28, 0x34, 0x6f, 0xf8, - 0x84, 0x8b, 0x47, 0x9e, 0xbf, 0x78, 0x52, 0xa5, 0x16, 0xb4, 0xbb, 0x4e, 0x8c, 0x7f, 0x25, 0x61, - 0xc0, 0x5c, 0x47, 0xe2, 0xb1, 0xe7, 0x45, 0x6a, 0x7c, 0x04, 0x75, 0x12, 0xad, 0x3c, 0xaf, 0x3d, - 0x2a, 0x0e, 0x3c, 0xcb, 0x8f, 0x18, 0x93, 0x31, 0xfd, 0x11, 0x3a, 0x5f, 0x29, 0x69, 0x65, 0x9f, - 0xfe, 0x0c, 0xd6, 0xaa, 0x74, 0xea, 0xc4, 0xc3, 0x88, 0x29, 0x03, 0x4c, 0x5d, 0x2e, 0xc9, 0xd4, - 0xf4, 0x55, 0x0c, 0x78, 0x98, 0x48, 0xf2, 0x4f, 0x51, 0x3f, 0xa7, 0x97, 0x89, 0xd4, 0x78, 0xb2, - 0x82, 0x87, 0xe9, 0xf3, 0x20, 0x01, 0xdd, 0x81, 0xad, 0x25, 0x7c, 0xa5, 0xfd, 0xe9, 0xe9, 0x4c, - 0x17, 0xfd, 0x66, 0x02, 0x1b, 0x73, 0x47, 0x26, 0x11, 0x12, 0xae, 0xde, 0x1a, 0xd5, 0x4a, 0x3e, - 0x92, 0xc2, 0xb8, 0x28, 0x2a, 0xe4, 0x3c, 0xaf, 0x10, 0xae, 0xe2, 0x20, 0x51, 0x37, 0x4b, 0xee, - 0x15, 0x26, 0xa7, 0xe2, 0x21, 0xa7, 0x7d, 0x2e, 0x99, 0x9c, 0x91, 0x57, 0x86, 0xf2, 0xb4, 0xb9, - 0xc0, 0xa7, 0x91, 0x48, 0x42, 0xf2, 0xda, 0x50, 0x5d, 0xd2, 0x05, 0x1d, 0x46, 0x22, 0x14, 0xb1, - 0x13, 0x90, 0x37, 0x86, 0xea, 0xa5, 0xa5, 0x84, 0xf9, 0x2d, 0x64, 0x07, 0xde, 0x16, 0x07, 0xe6, - 0xfc, 0x1c, 0xa7, 0x77, 0x18, 0x91, 0x77, 0x86, 0x1a, 0x76, 0x7b, 0x59, 0x98, 0xe7, 0x7a, 0x6f, - 0xe4, 0x1d, 0xcd, 0xa5, 0x5b, 0x21, 0x91, 0x7c, 0x28, 0x70, 0x3e, 0x87, 0x3c, 0xd1, 0x47, 0x83, - 0x6e, 0x41, 0x73, 0x81, 0xd3, 0xbd, 0x9f, 0x0c, 0xfa, 0x13, 0x74, 0x9e, 0x41, 0x75, 0xff, 0x43, - 0xfd, 0x8f, 0x23, 0x9f, 0x8d, 0xbb, 0x6a, 0xfa, 0xfd, 0xfc, 0xe3, 0x4b, 0x00, 0x00, 0x00, 0xff, - 0xff, 0xc9, 0xcf, 0x96, 0x2d, 0x54, 0x05, 0x00, 0x00, + // 1379 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xac, 0x57, 0xc9, 0x72, 0x14, 0x47, + 0x13, 0xa6, 0xa5, 0x59, 0x73, 0xb4, 0x94, 0x4a, 0x12, 0x0c, 0x13, 0xff, 0x81, 0xbf, 0xbd, 0x21, + 0x20, 0xc0, 0x21, 0xc2, 0x84, 0xb1, 0x1d, 0x8e, 0x40, 0x20, 0xb0, 0x82, 0x00, 0xe4, 0x61, 0x39, + 0x78, 0x09, 0x45, 0x6b, 0xa6, 0x66, 0xa6, 0xad, 0x51, 0x75, 0xd3, 0x0b, 0x20, 0xbf, 0x81, 0x1f, + 0xc8, 0x17, 0x5f, 0x7d, 0xf2, 0xbe, 0x3c, 0x91, 0x33, 0xab, 0xaa, 0x57, 0x75, 0x73, 0xd2, 0xad, + 0x2b, 0xb7, 0xaa, 0xca, 0xfc, 0xf2, 0xeb, 0x2c, 0xe8, 0x45, 0x27, 0xbe, 0x08, 0xaf, 0xfb, 0x81, + 0x17, 0x79, 0xbc, 0xa9, 0x16, 0xf6, 0xcf, 0x0d, 0x68, 0x0f, 0xc5, 0xcb, 0x58, 0x84, 0x11, 0xbf, + 0x0c, 0x0d, 0x31, 0x9a, 0x79, 0x7d, 0xeb, 0x92, 0x75, 0xb9, 0xb7, 0xcd, 0xaf, 0x6b, 0x73, 0xa3, + 0xdd, 0x45, 0xcd, 0x17, 0xe7, 0x86, 0xca, 0x82, 0x5f, 0x85, 0xe6, 0x64, 0x1e, 0x87, 0xb3, 0xfe, + 0x82, 0x32, 0x5d, 0x2f, 0x9a, 0xde, 0x27, 0x15, 0xda, 0x6a, 0x1b, 0x0a, 0xeb, 0xca, 0x89, 0xd7, + 0x5f, 0xac, 0x0a, 0xbb, 0x87, 0x1a, 0x0a, 0x4b, 0x16, 0xfc, 0x63, 0x80, 0x50, 0x44, 0x07, 0x9e, + 0x1f, 0xb9, 0x9e, 0xec, 0x37, 0x94, 0xfd, 0x85, 0xa2, 0xfd, 0x53, 0x11, 0x3d, 0x51, 0x6a, 0x74, + 0xea, 0x86, 0xc9, 0x82, 0x7f, 0x04, 0x5d, 0xc7, 0xf7, 0x85, 0x1c, 0x1f, 0x44, 0x6f, 0xfa, 0x4d, + 0xe5, 0x78, 0xbe, 0xe8, 0x78, 0x47, 0xa9, 0x9f, 0xbd, 0x41, 0xbf, 0x8e, 0x63, 0xbe, 0xf9, 0x36, + 0x74, 0x46, 0x33, 0x31, 0x3a, 0x22, 0xaf, 0x96, 0xf2, 0xda, 0x2c, 0x7a, 0xdd, 0x25, 0xad, 0x72, + 0x6a, 0x8f, 0xf4, 0x27, 0xbf, 0x0e, 0xad, 0x91, 0x77, 0x7c, 0xec, 0x46, 0xfd, 0xb6, 0xf2, 0xd8, + 0x28, 0x79, 0x28, 0x1d, 0x3a, 0x18, 0x2b, 0xca, 0x15, 0x2a, 0x82, 0x93, 0x7e, 0xa7, 0x2a, 0x57, + 0x5f, 0x92, 0x8a, 0x72, 0xa5, 0x6c, 0x28, 0x03, 0xae, 0x74, 0xa3, 0x83, 0xd1, 0xcc, 0x71, 0x65, + 0xbf, 0x5b, 0x95, 0x81, 0x3d, 0xd4, 0xdf, 0x25, 0x35, 0x65, 0xc0, 0x4d, 0x16, 0xfc, 0x53, 0xe8, + 0x1d, 0x8a, 0xa9, 0x2b, 0x0f, 0x0e, 0xe7, 0xde, 0xe8, 0xa8, 0x0f, 0xca, 0xb5, 0x5f, 0x74, 0xdd, + 0x21, 0x83, 0x1d, 0xd2, 0xa3, 0x2f, 0x1c, 0xa6, 0x2b, 0x4a, 0x1f, 0xe5, 0x4e, 0xbb, 0xf6, 0xaa, + 0xd2, 0xb7, 0x2b, 0xc7, 0x89, 0x63, 0x47, 0x98, 0xef, 0x9d, 0x36, 0x34, 0x5f, 0x39, 0xf3, 0x58, + 0xd8, 0x1f, 0x40, 0x2f, 0x07, 0x13, 0xde, 0x87, 0xf6, 0xb1, 0x08, 0x43, 0x67, 0x2a, 0x14, 0x96, + 0xba, 0xc3, 0x64, 0x69, 0xaf, 0xc0, 0x52, 0x1e, 0x24, 0xf6, 0x72, 0xea, 0x48, 0x40, 0xb0, 0x3f, + 0x01, 0x56, 0xae, 0x33, 0x67, 0xb0, 0x78, 0x24, 0x4e, 0x4c, 0x20, 0xfa, 0xe4, 0x1b, 0x66, 0x5b, + 0x85, 0xbe, 0xee, 0xd0, 0x9c, 0xe1, 0xff, 0xb0, 0x5a, 0x2a, 0x35, 0x5f, 0x81, 0x05, 0x2c, 0x2c, + 0x79, 0x2e, 0x0d, 0xf1, 0xcb, 0xbe, 0x04, 0x2b, 0xc5, 0xba, 0x9e, 0xb2, 0x78, 0x37, 0x3d, 0x9f, + 0x2a, 0x0c, 0x6d, 0xa5, 0x8b, 0xa7, 0x4d, 0xf4, 0xc2, 0x5e, 0x85, 0xe5, 0x42, 0xb5, 0xed, 0x7b, + 0xe9, 0xb9, 0xd3, 0xea, 0xf0, 0x0f, 0x01, 0xf0, 0x60, 0xee, 0xd8, 0x89, 0xbc, 0x20, 0x44, 0xff, + 0x45, 0x4c, 0x2a, 0x33, 0x49, 0x7d, 0x91, 0x28, 0x86, 0x39, 0x1b, 0xfb, 0x2a, 0xac, 0x9d, 0x2a, + 0x14, 0x3f, 0x0f, 0xad, 0x99, 0x70, 0xa7, 0xb3, 0x48, 0x1d, 0xa1, 0x31, 0x34, 0x2b, 0x7b, 0x2b, + 0xbd, 0x6e, 0x52, 0x9a, 0x5a, 0xd3, 0x1f, 0x9a, 0xd0, 0x19, 0x8a, 0xd0, 0xf7, 0x64, 0x28, 0x10, + 0x61, 0x5d, 0xf1, 0x66, 0x24, 0x74, 0x8b, 0x59, 0x25, 0x94, 0x68, 0x9b, 0xdd, 0x44, 0x4f, 0x08, + 0x4b, 0x8d, 0xf9, 0x96, 0xa1, 0x87, 0x72, 0xcf, 0x1b, 0xa7, 0x3c, 0x3f, 0x5c, 0x4b, 0xf8, 0x61, + 0xb1, 0xd4, 0x22, 0xda, 0xb6, 0x44, 0x10, 0x5b, 0x86, 0x20, 0x1a, 0x95, 0x81, 0x0b, 0x0c, 0x71, + 0xbb, 0xc0, 0x10, 0xcd, 0xca, 0xe3, 0xd7, 0x50, 0xc4, 0xad, 0x3c, 0x45, 0xb4, 0x4a, 0x9d, 0xa5, + 0x3d, 0x2b, 0x39, 0xe2, 0x66, 0x8e, 0x23, 0xda, 0xa5, 0xd6, 0xd0, 0x6e, 0x15, 0x24, 0x71, 0x23, + 0x25, 0x89, 0x4e, 0x89, 0x56, 0x8c, 0x4b, 0x99, 0x25, 0xae, 0x25, 0x40, 0xeb, 0x56, 0x66, 0xac, + 0x44, 0x13, 0xb7, 0x0b, 0x34, 0x01, 0x95, 0x69, 0xa8, 0xe1, 0x89, 0xcf, 0x8a, 0x3c, 0xa1, 0x9b, + 0xfd, 0x62, 0xc9, 0xb7, 0x96, 0x28, 0x6e, 0xe5, 0x89, 0x62, 0xa9, 0x32, 0x89, 0x6f, 0x67, 0x8a, + 0x2d, 0xc2, 0x78, 0x09, 0x66, 0xd4, 0x65, 0x22, 0x08, 0xbc, 0xc0, 0x34, 0xb9, 0x5e, 0xd8, 0x97, + 0xa9, 0x17, 0x33, 0x70, 0xbd, 0x85, 0x55, 0x54, 0x3f, 0xe6, 0xa0, 0x65, 0xdb, 0x99, 0x2b, 0xc1, + 0x87, 0x73, 0x83, 0x30, 0xed, 0xa7, 0xbe, 0xed, 0xf7, 0xb2, 0x93, 0x14, 0xc8, 0x66, 0xee, 0x4d, + 0x13, 0xb2, 0xc1, 0x4f, 0xfb, 0x5b, 0x6a, 0xed, 0x22, 0x3c, 0xf8, 0x3b, 0xd0, 0x18, 0x79, 0x63, + 0x7d, 0x8c, 0x95, 0xed, 0x55, 0x93, 0x80, 0xbb, 0x28, 0x7a, 0x86, 0x5f, 0x43, 0xa5, 0xa4, 0x3d, + 0xb1, 0xaf, 0x1d, 0xd5, 0x2e, 0x4b, 0x43, 0xf5, 0x9d, 0x84, 0x5f, 0xcc, 0xc2, 0x7f, 0x43, 0x6d, + 0x5c, 0x80, 0xd1, 0x59, 0x46, 0xff, 0x2a, 0x4b, 0x8c, 0xe6, 0xb3, 0x33, 0x8c, 0xfd, 0x35, 0x91, + 0x69, 0x1e, 0xcd, 0x67, 0x19, 0x7c, 0x3d, 0x2b, 0x4e, 0x8a, 0x63, 0x7b, 0x03, 0xf8, 0x69, 0x80, + 0xea, 0x7f, 0x46, 0x11, 0x7a, 0xfc, 0x7d, 0x68, 0x8e, 0xdd, 0xc9, 0x24, 0x44, 0x4a, 0xa9, 0xa6, + 0x5d, 0xad, 0xb6, 0x6f, 0x43, 0x37, 0x95, 0x11, 0x7d, 0xfa, 0xf1, 0xe1, 0x43, 0x91, 0x90, 0xbd, + 0x59, 0x11, 0x3a, 0x7d, 0xef, 0xb5, 0x08, 0xd4, 0x91, 0x1b, 0x43, 0xbd, 0xb8, 0xf2, 0x93, 0x05, + 0xbd, 0x47, 0x1a, 0x7f, 0x74, 0x3b, 0xbe, 0x0a, 0xbd, 0xc7, 0xf1, 0x7c, 0x6e, 0x44, 0xec, 0x1c, + 0xef, 0x40, 0x83, 0x60, 0xcb, 0x2c, 0xde, 0x85, 0xa6, 0x82, 0x25, 0x5b, 0x20, 0x21, 0x01, 0x92, + 0x2d, 0xf2, 0x65, 0xe8, 0xa6, 0xb0, 0x63, 0x0d, 0x5a, 0xa6, 0xfd, 0xc0, 0x9a, 0x7c, 0x09, 0x3a, + 0x09, 0xda, 0xd8, 0x1a, 0xef, 0x41, 0xdb, 0x80, 0x83, 0x61, 0xfe, 0xa0, 0xa5, 0xf3, 0xcd, 0xd6, + 0x29, 0xb2, 0xaa, 0x2b, 0xdb, 0xa0, 0x00, 0x69, 0xa6, 0xd8, 0x26, 0xfe, 0xd8, 0x20, 0xcb, 0x11, + 0x3b, 0x4f, 0x01, 0x93, 0xec, 0xb0, 0x0b, 0x57, 0x7e, 0xc4, 0x3f, 0x42, 0x52, 0x17, 0xde, 0x82, + 0x85, 0x27, 0x0f, 0xf1, 0xc0, 0x6b, 0xb0, 0xbc, 0x27, 0x23, 0x11, 0x48, 0x67, 0xbe, 0x4b, 0x0d, + 0x88, 0x27, 0x47, 0xd1, 0xae, 0xc4, 0xb2, 0xb9, 0x72, 0xaa, 0x45, 0x0b, 0x14, 0x68, 0xc7, 0x19, + 0x3f, 0xf6, 0xe4, 0x48, 0xe0, 0x2d, 0x18, 0x2c, 0x3d, 0x97, 0x4e, 0x1c, 0xcd, 0xbc, 0xc0, 0xfd, + 0x5e, 0x8c, 0xf1, 0x22, 0x9b, 0xb0, 0xb6, 0x27, 0xc3, 0x78, 0x32, 0x71, 0x47, 0xae, 0x90, 0xd1, + 0xfd, 0x58, 0x8e, 0x43, 0xbc, 0x10, 0x87, 0x95, 0xe7, 0xf2, 0x48, 0x7a, 0xaf, 0xa5, 0xf9, 0x6b, + 0xb1, 0x16, 0x36, 0xf4, 0xc6, 0x8e, 0x13, 0x8a, 0x7b, 0xb1, 0x3f, 0x77, 0x47, 0x4e, 0x24, 0xee, + 0x8c, 0xc7, 0x01, 0xa6, 0x8f, 0x09, 0x0a, 0x42, 0x9a, 0xe2, 0xde, 0x93, 0xc4, 0xa1, 0x10, 0x5f, + 0x88, 0x90, 0x4d, 0xf9, 0x45, 0xd8, 0x3c, 0xa5, 0x51, 0x3b, 0xcf, 0xf8, 0xff, 0xa0, 0x5f, 0x56, + 0x3d, 0x70, 0xc2, 0xfd, 0xc0, 0xc5, 0x0b, 0xb8, 0x58, 0x5c, 0xa6, 0xb5, 0xea, 0x3f, 0xbc, 0x27, + 0xfd, 0x38, 0x62, 0xdf, 0x25, 0xfb, 0x1b, 0xe9, 0x93, 0x38, 0x22, 0xf1, 0x51, 0x49, 0xbc, 0xaf, + 0xe0, 0xc1, 0xe6, 0xfc, 0x02, 0xac, 0xe7, 0xc4, 0x4f, 0xe9, 0x7e, 0x94, 0x9d, 0xe3, 0xec, 0xbc, + 0x5a, 0xe1, 0x4e, 0xa5, 0x13, 0xc5, 0x81, 0x60, 0x12, 0xb1, 0xc6, 0x49, 0x63, 0x52, 0x92, 0x5c, + 0xdc, 0x4b, 0x76, 0x30, 0x72, 0xb3, 0x83, 0x5f, 0x16, 0xcf, 0x63, 0xac, 0x2c, 0x7b, 0x89, 0x62, + 0xf6, 0xc0, 0x7b, 0x65, 0xa4, 0xbb, 0x32, 0x72, 0xa3, 0x13, 0xf6, 0x8b, 0x85, 0x77, 0x5a, 0xcd, + 0xc4, 0x0f, 0x02, 0x2f, 0xf6, 0xd9, 0xaf, 0x16, 0x9e, 0x92, 0x67, 0xd2, 0xfd, 0xc0, 0xf3, 0xbd, + 0xd0, 0x99, 0xb3, 0xdf, 0x2c, 0x3c, 0xcb, 0x1a, 0x2a, 0xd2, 0x2a, 0x68, 0x87, 0xdf, 0x13, 0x87, + 0x54, 0xfe, 0x48, 0x1c, 0x1f, 0x8a, 0x80, 0xfd, 0x61, 0x61, 0xb2, 0x37, 0xf2, 0x8a, 0x34, 0xd6, + 0x9f, 0x96, 0x39, 0x51, 0xaa, 0x7a, 0xe1, 0x45, 0x82, 0xfd, 0x95, 0x88, 0x4d, 0x1e, 0x4c, 0xa0, + 0xbf, 0x2d, 0xbe, 0x0e, 0x2b, 0x99, 0x58, 0xd9, 0xfe, 0x63, 0xf1, 0x01, 0x6c, 0x16, 0x84, 0x58, + 0xff, 0x7d, 0xea, 0x38, 0xf6, 0xaf, 0xb5, 0x8d, 0x93, 0xcc, 0xea, 0xb3, 0x47, 0x4f, 0xf7, 0xb1, + 0x37, 0xd4, 0x06, 0x44, 0xd9, 0x37, 0x74, 0x9f, 0xf1, 0x8a, 0xf7, 0xca, 0xa0, 0x6a, 0x48, 0xc1, + 0xa1, 0x5f, 0xb7, 0x23, 0xaf, 0x7a, 0xb6, 0x0c, 0x2a, 0x67, 0x15, 0xda, 0x44, 0xff, 0x48, 0x4e, + 0xbf, 0x5e, 0x06, 0x55, 0x03, 0x0b, 0xff, 0x3c, 0xd7, 0xde, 0xbc, 0xee, 0x0d, 0x33, 0xa8, 0x1d, + 0x5d, 0x70, 0x9c, 0x4f, 0x09, 0x80, 0xd7, 0xbc, 0x64, 0x06, 0x75, 0xe3, 0x0b, 0xce, 0x78, 0x09, + 0x5f, 0xf0, 0xea, 0xf7, 0xcc, 0xa0, 0x66, 0x84, 0xa1, 0xdc, 0xe8, 0x1f, 0x45, 0xd5, 0x33, 0x65, + 0x50, 0x39, 0x95, 0xe0, 0xe3, 0xc1, 0x10, 0x12, 0xaf, 0x7c, 0x0a, 0x0d, 0xaa, 0x67, 0x1f, 0xca, + 0x50, 0x36, 0x2c, 0xd7, 0xbd, 0x71, 0x06, 0xb5, 0x53, 0x0d, 0xbf, 0x93, 0x67, 0x38, 0x5e, 0xfb, + 0xd2, 0x19, 0xd4, 0xcf, 0x36, 0x94, 0xe4, 0x6c, 0x78, 0xae, 0x7e, 0xef, 0x0c, 0xea, 0xc6, 0x9b, + 0xc3, 0x96, 0x7a, 0x47, 0xdf, 0xfc, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xd8, 0xe3, 0x58, 0x48, 0x56, + 0x0f, 0x00, 0x00, } diff --git a/types/types.proto b/types/types.proto index 37a4fa6a0..cb3d82c1a 100644 --- a/types/types.proto +++ b/types/types.proto @@ -6,6 +6,10 @@ package types; //---------------------------------------- // Message types +// Not being used +// Could be added to request/response +// so we don't have to type switch +// (would be twice as fast, but we're talking about 15ns) enum MessageType { NullMessage = 0x00; @@ -70,24 +74,136 @@ enum CodeType { // Request types message Request { - MessageType type = 1; - bytes data = 2; - string key = 3; - string value = 4; - repeated Validator validators = 5; - uint64 height = 6; + oneof value{ + RequestEcho echo = 1; + RequestFlush flush = 2; + RequestInfo info = 3; + RequestSetOption set_option = 4; + RequestAppendTx append_tx = 5; + RequestCheckTx check_tx = 6; + RequestCommit commit = 7; + RequestQuery query = 8; + RequestInitChain init_chain = 9; + RequestBeginBlock begin_block = 10; + RequestEndBlock end_block = 11; + } +} + +message RequestEcho { + string message = 1; +} + +message RequestFlush { +} + +message RequestInfo { +} + +message RequestSetOption{ + string key = 1; + string value = 2; +} + +message RequestAppendTx{ + bytes tx = 1; +} + +message RequestCheckTx{ + bytes tx = 1; +} + +message RequestQuery{ + bytes query = 1; +} + +message RequestCommit{ +} + +message RequestInitChain{ + repeated Validator validators = 1; +} + +message RequestBeginBlock{ + uint64 height = 1; +} + +message RequestEndBlock{ + uint64 height = 1; } //---------------------------------------- // Response types + message Response { - MessageType type = 1; - bytes data = 2; - CodeType code = 3; - string error = 4; - string log = 5; - repeated Validator validators = 6; + oneof value{ + ResponseException exception = 1; + ResponseEcho echo = 2; + ResponseFlush flush = 3; + ResponseInfo info = 4; + ResponseSetOption set_option = 5; + ResponseAppendTx append_tx = 6; + ResponseCheckTx check_tx = 7; + ResponseCommit commit = 8; + ResponseQuery query = 9; + ResponseInitChain init_chain = 10; + ResponseBeginBlock begin_block = 11; + ResponseEndBlock end_block = 12; + } +} + +message ResponseException{ + string error = 1; +} + +message ResponseEcho { + string message = 1; +} + +message ResponseFlush{ +} + +message ResponseInfo { + string info = 1; +} + +message ResponseSetOption{ + string log = 1; +} + +message ResponseAppendTx{ + CodeType code = 1; + bytes data = 2; + string log = 3; +} + +message ResponseCheckTx{ + CodeType code = 1; + bytes data = 2; + string log = 3; +} + +message ResponseQuery{ + CodeType code = 1; + bytes data = 2; + string log = 3; +} + +message ResponseCommit{ + CodeType code = 1; + bytes data = 2; + string log = 3; +} + + +message ResponseInitChain{ +} + +message ResponseBeginBlock{ +} + +message ResponseEndBlock{ + repeated Validator diffs = 4; } //---------------------------------------- @@ -97,3 +213,20 @@ message Validator { bytes pubKey = 1; uint64 power = 2; } + +//---------------------------------------- +// Service Definition + +service TMSPApplication { + rpc Echo(RequestEcho) returns (ResponseEcho) ; + rpc Flush(RequestFlush) returns (ResponseFlush); + rpc Info(RequestInfo) returns (ResponseInfo); + rpc SetOption(RequestSetOption) returns (ResponseSetOption); + rpc AppendTx(RequestAppendTx) returns (ResponseAppendTx); + rpc CheckTx(RequestCheckTx) returns (ResponseCheckTx); + rpc Query(RequestQuery) returns (ResponseQuery); + rpc Commit(RequestCommit) returns (ResponseCommit); + rpc InitChain(RequestInitChain) returns (ResponseInitChain); + rpc BeginBlock(RequestBeginBlock) returns (ResponseBeginBlock); + rpc EndBlock(RequestEndBlock) returns (ResponseEndBlock); +}