diff --git a/config/templates.go b/config/templates.go index f38cbecabc027839baabe26406108e0744741f12..36da48589f4a6b6e16763a04f9e4adec97c8d7fe 100644 --- a/config/templates.go +++ b/config/templates.go @@ -86,7 +86,7 @@ const separatorChainConsensus = ` const sectionChainConsensus = ` [chain.consensus] # consensus defines the module to use for consensus and # this will define the peer-to-peer consensus network; - # accepted values are ("noops", "tmsp",) "tendermint" + # accepted values are ("noops", "abci",) "tendermint" name = "{{.Name}}" # version is the major and minor semantic version; # the version will be asserted on @@ -184,17 +184,17 @@ const separatorModules = ` ` // TODO: [ben] make configurable -const sectionTmsp = ` +const sectionabci = ` ################################################################################ ## -## Tendermint Socket Protocol (TMSP) +## Tendermint Socket Protocol (abci) ## version 0.6.0 ## -## TMSP expects a tendermint consensus process to run and connect to Eris-DB +## abci expects a tendermint consensus process to run and connect to Eris-DB ## ################################################################################ -[tmsp] +[abci] # listener address for accepting tendermint socket protocol connections listener = "tcp://0.0.0.0:46658" @@ -244,7 +244,7 @@ private_validator_file = "priv_validator.json" # NOTE: value is ignored when run in-process as RPC is # handled by [servers.tendermint] rpc_laddr = "0.0.0.0:46657" - # proxy application address - used for tmsp connections, + # proxy application address - used for abci connections, # and this port should not be exposed for in-process Tendermint proxy_app = "tcp://127.0.0.1:46658" diff --git a/consensus/config.go b/consensus/config.go index d676d8c214f8c794aaea43a845260f9407d2d311..951b4cf3e426a2b72ce480622b0f6fd8d8a3fce2 100644 --- a/consensus/config.go +++ b/consensus/config.go @@ -22,7 +22,6 @@ package consensus import ( // noops "github.com/eris-ltd/eris-db/consensus/noops" tendermint "github.com/eris-ltd/eris-db/consensus/tendermint" - tmsp "github.com/eris-ltd/eris-db/consensus/tmsp" ) //------------------------------------------------------------------------------ @@ -34,8 +33,6 @@ func AssertValidConsensusModule(name, minorVersionString string) bool { // noops should not have any external interfaces that can change // over iterations return true - case "tmsp": - return minorVersionString == tmsp.GetTmspVersion().GetMinorVersionString() case "tendermint": return minorVersionString == tendermint.GetTendermintVersion().GetMinorVersionString() case "bigchaindb": diff --git a/consensus/tendermint/local_client.go b/consensus/tendermint/local_client.go deleted file mode 100644 index 97b732cac6ab66237ce6250478a6a886855a3b6f..0000000000000000000000000000000000000000 --- a/consensus/tendermint/local_client.go +++ /dev/null @@ -1,270 +0,0 @@ -// Copyright 2015, 2016 Eris Industries (UK) Ltd. -// This file is part of Eris-RT - -// Eris-RT is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// Eris-RT is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with Eris-RT. If not, see <http://www.gnu.org/licenses/>. - -// version provides the current Eris-DB version and a VersionIdentifier -// for the modules to identify their version with. - -// This file is originally based on github.com/tendermint/tmsp/client/... -// .../local_client.go - -package tendermint - -import ( - "sync" - - tmsp_client "github.com/tendermint/tmsp/client" - tmsp_types "github.com/tendermint/tmsp/types" - - manager_types "github.com/eris-ltd/eris-db/manager/types" -) - -// NOTE [ben] Compiler check to ensure localClient successfully implements -// tendermint/tmsp/client -var _ tmsp_client.Client = (*localClient)(nil) - -type localClient struct { - mtx *sync.Mutex - Application manager_types.Application - Callback tmsp_client.Callback -} - -func NewLocalClient(mtx *sync.Mutex, app manager_types.Application) *localClient { - if mtx == nil { - mtx = new(sync.Mutex) - } - return &localClient{ - mtx: mtx, - Application: app, - } -} - -func (app *localClient) SetResponseCallback(cb tmsp_client.Callback) { - app.mtx.Lock() - defer app.mtx.Unlock() - app.Callback = cb -} - -// TODO: change manager_types.Application to include Error()? -func (app *localClient) Error() error { - return nil -} - -func (app *localClient) Stop() bool { - return true -} - -func (app *localClient) FlushAsync() *tmsp_client.ReqRes { - // Do nothing - return newLocalReqRes(tmsp_types.ToRequestFlush(), nil) -} - -func (app *localClient) EchoAsync(msg string) *tmsp_client.ReqRes { - return app.callback( - tmsp_types.ToRequestEcho(msg), - tmsp_types.ToResponseEcho(msg), - ) -} - -func (app *localClient) InfoAsync() *tmsp_client.ReqRes { - app.mtx.Lock() - info := app.Application.Info() - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestInfo(), - tmsp_types.ToResponseInfo(info), - ) -} - -func (app *localClient) SetOptionAsync(key string, value string) *tmsp_client.ReqRes { - app.mtx.Lock() - log := app.Application.SetOption(key, value) - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestSetOption(key, value), - tmsp_types.ToResponseSetOption(log), - ) -} - -func (app *localClient) AppendTxAsync(tx []byte) *tmsp_client.ReqRes { - app.mtx.Lock() - res := app.Application.AppendTx(tx) - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestAppendTx(tx), - tmsp_types.ToResponseAppendTx(res.Code, res.Data, res.Log), - ) -} - -func (app *localClient) CheckTxAsync(tx []byte) *tmsp_client.ReqRes { - app.mtx.Lock() - res := app.Application.CheckTx(tx) - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestCheckTx(tx), - tmsp_types.ToResponseCheckTx(res.Code, res.Data, res.Log), - ) -} - -func (app *localClient) QueryAsync(tx []byte) *tmsp_client.ReqRes { - app.mtx.Lock() - res := app.Application.Query(tx) - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestQuery(tx), - tmsp_types.ToResponseQuery(res.Code, res.Data, res.Log), - ) -} - -func (app *localClient) CommitAsync() *tmsp_client.ReqRes { - app.mtx.Lock() - res := app.Application.Commit() - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestCommit(), - tmsp_types.ToResponseCommit(res.Code, res.Data, res.Log), - ) -} - -func (app *localClient) InitChainAsync(validators []*tmsp_types.Validator) *tmsp_client.ReqRes { - app.mtx.Lock() - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - bcApp.InitChain(validators) - } - reqRes := app.callback( - tmsp_types.ToRequestInitChain(validators), - tmsp_types.ToResponseInitChain(), - ) - app.mtx.Unlock() - return reqRes -} - -func (app *localClient) BeginBlockAsync(height uint64) *tmsp_client.ReqRes { - app.mtx.Lock() - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - bcApp.BeginBlock(height) - } - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestBeginBlock(height), - tmsp_types.ToResponseBeginBlock(), - ) -} - -func (app *localClient) EndBlockAsync(height uint64) *tmsp_client.ReqRes { - app.mtx.Lock() - var validators []*tmsp_types.Validator - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - validators = bcApp.EndBlock(height) - } - app.mtx.Unlock() - return app.callback( - tmsp_types.ToRequestEndBlock(height), - tmsp_types.ToResponseEndBlock(validators), - ) -} - -//------------------------------------------------------- - -func (app *localClient) FlushSync() error { - return nil -} - -func (app *localClient) EchoSync(msg string) (res tmsp_types.Result) { - return tmsp_types.OK.SetData([]byte(msg)) -} - -func (app *localClient) InfoSync() (res tmsp_types.Result) { - app.mtx.Lock() - info := app.Application.Info() - app.mtx.Unlock() - return tmsp_types.OK.SetData([]byte(info)) -} - -func (app *localClient) SetOptionSync(key string, value string) (res tmsp_types.Result) { - app.mtx.Lock() - log := app.Application.SetOption(key, value) - app.mtx.Unlock() - return tmsp_types.OK.SetLog(log) -} - -func (app *localClient) AppendTxSync(tx []byte) (res tmsp_types.Result) { - app.mtx.Lock() - res = app.Application.AppendTx(tx) - app.mtx.Unlock() - return res -} - -func (app *localClient) CheckTxSync(tx []byte) (res tmsp_types.Result) { - app.mtx.Lock() - res = app.Application.CheckTx(tx) - app.mtx.Unlock() - return res -} - -func (app *localClient) QuerySync(query []byte) (res tmsp_types.Result) { - app.mtx.Lock() - res = app.Application.Query(query) - app.mtx.Unlock() - return res -} - -func (app *localClient) CommitSync() (res tmsp_types.Result) { - app.mtx.Lock() - res = app.Application.Commit() - app.mtx.Unlock() - return res -} - -func (app *localClient) InitChainSync(validators []*tmsp_types.Validator) (err error) { - app.mtx.Lock() - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - bcApp.InitChain(validators) - } - app.mtx.Unlock() - return nil -} - -func (app *localClient) BeginBlockSync(height uint64) (err error) { - app.mtx.Lock() - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - bcApp.BeginBlock(height) - } - app.mtx.Unlock() - return nil -} - -func (app *localClient) EndBlockSync(height uint64) (changedValidators []*tmsp_types.Validator, err error) { - app.mtx.Lock() - if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok { - changedValidators = bcApp.EndBlock(height) - } - app.mtx.Unlock() - return changedValidators, nil -} - -//------------------------------------------------------- - -func (app *localClient) callback(req *tmsp_types.Request, res *tmsp_types.Response) *tmsp_client.ReqRes { - app.Callback(req, res) - return newLocalReqRes(req, res) -} - -func newLocalReqRes(req *tmsp_types.Request, res *tmsp_types.Response) *tmsp_client.ReqRes { - reqRes := tmsp_client.NewReqRes(req) - reqRes.Response = res - reqRes.SetDone() - return reqRes -} diff --git a/consensus/tendermint/tendermint.go b/consensus/tendermint/tendermint.go index 5adc08e1a2619723123088901bea595f75a300eb..c0c3b3a7b4d1bb606d93ed5d9a91542ccdcc6848 100644 --- a/consensus/tendermint/tendermint.go +++ b/consensus/tendermint/tendermint.go @@ -23,15 +23,14 @@ import ( "fmt" "path" "strings" - "sync" + abci_types "github.com/tendermint/abci/types" crypto "github.com/tendermint/go-crypto" p2p "github.com/tendermint/go-p2p" tendermint_consensus "github.com/tendermint/tendermint/consensus" node "github.com/tendermint/tendermint/node" proxy "github.com/tendermint/tendermint/proxy" tendermint_types "github.com/tendermint/tendermint/types" - tmsp_types "github.com/tendermint/tmsp/types" edb_event "github.com/eris-ltd/eris-db/event" @@ -125,10 +124,8 @@ func NewTendermint(moduleConfig *config.ModuleConfig, tmintConfig.Set("rpc_laddr", "") } - newNode := node.NewNode(tmintConfig, privateValidator, func(_, _ string, - hash []byte) proxy.AppConn { - return NewLocalClient(new(sync.Mutex), application) - }) + newNode := node.NewNode(tmintConfig, privateValidator, + proxy.NewLocalClientCreator(application)) listener := p2p.NewDefaultListener("tcp", tmintConfig.GetString("node_laddr"), tmintConfig.GetBool("skip_upnp")) @@ -235,7 +232,7 @@ func (tendermint *Tendermint) Events() edb_event.EventEmitter { } func (tendermint *Tendermint) BroadcastTransaction(transaction []byte, - callback func(*tmsp_types.Response)) error { + callback func(*abci_types.Response)) error { return tendermint.tmintNode.MempoolReactor().BroadcastTx(transaction, callback) } diff --git a/consensus/tendermint/version.go b/consensus/tendermint/version.go index 870d1313bf43977a8976f0415d07f1aa5d1f2b2d..cc7f1807584c759399b760b27fbd61b8a25e3965 100644 --- a/consensus/tendermint/version.go +++ b/consensus/tendermint/version.go @@ -30,7 +30,7 @@ const ( // Major version component of the current release tendermintVersionMajorConst uint8 = 0 // Minor version component of the current release - tendermintVersionMinorConst uint8 = 6 + tendermintVersionMinorConst uint8 = 8 // Patch version component of the current release tendermintVersionPatchConst uint8 = 0 ) diff --git a/consensus/tmsp/server.go b/consensus/tmsp/server.go deleted file mode 100644 index 62abb9d641d747b748972e02e1b1d9b77ebacf47..0000000000000000000000000000000000000000 --- a/consensus/tmsp/server.go +++ /dev/null @@ -1,211 +0,0 @@ -// Copyright 2015, 2016 Eris Industries (UK) Ltd. -// This file is part of Eris-RT - -// Eris-RT is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// Eris-RT is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with Eris-RT. If not, see <http://www.gnu.org/licenses/>. - -// Taken originally from github.com/tendermint/tmsp/server.go - -package tmsp - -import ( - "bufio" - "fmt" - "io" - "net" - "strings" - "sync" - - . "github.com/tendermint/go-common" - tmsp_types "github.com/tendermint/tmsp/types" - - manager_types "github.com/eris-ltd/eris-db/manager/types" -) - -// var maxNumberConnections = 2 - -type Server struct { - QuitService - - proto string - addr string - listener net.Listener - - appMtx sync.Mutex - app manager_types.Application -} - -func NewServer(protoAddr string, app manager_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 *tmsp_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<- *tmsp_types.Response) { - var count int - var bufReader = bufio.NewReader(conn) - for { - - var req = &tmsp_types.Request{} - err := tmsp_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 *tmsp_types.Request, responses chan<- *tmsp_types.Response) { - switch r := req.Value.(type) { - case *tmsp_types.Request_Echo: - responses <- tmsp_types.ToResponseEcho(r.Echo.Message) - case *tmsp_types.Request_Flush: - responses <- tmsp_types.ToResponseFlush() - case *tmsp_types.Request_Info: - data := s.app.Info() - responses <- tmsp_types.ToResponseInfo(data) - case *tmsp_types.Request_SetOption: - so := r.SetOption - logStr := s.app.SetOption(so.Key, so.Value) - responses <- tmsp_types.ToResponseSetOption(logStr) - case *tmsp_types.Request_AppendTx: - res := s.app.AppendTx(r.AppendTx.Tx) - responses <- tmsp_types.ToResponseAppendTx(res.Code, res.Data, res.Log) - case *tmsp_types.Request_CheckTx: - res := s.app.CheckTx(r.CheckTx.Tx) - responses <- tmsp_types.ToResponseCheckTx(res.Code, res.Data, res.Log) - case *tmsp_types.Request_Commit: - res := s.app.Commit() - responses <- tmsp_types.ToResponseCommit(res.Code, res.Data, res.Log) - case *tmsp_types.Request_Query: - res := s.app.Query(r.Query.Query) - responses <- tmsp_types.ToResponseQuery(res.Code, res.Data, res.Log) - case *tmsp_types.Request_InitChain: - if app, ok := s.app.(tmsp_types.BlockchainAware); ok { - app.InitChain(r.InitChain.Validators) - responses <- tmsp_types.ToResponseInitChain() - } else { - responses <- tmsp_types.ToResponseInitChain() - } - case *tmsp_types.Request_EndBlock: - if app, ok := s.app.(tmsp_types.BlockchainAware); ok { - validators := app.EndBlock(r.EndBlock.Height) - responses <- tmsp_types.ToResponseEndBlock(validators) - } else { - responses <- tmsp_types.ToResponseEndBlock(nil) - } - default: - responses <- tmsp_types.ToResponseException("Unknown request") - } -} - -// Pull responses from 'responses' and write them to conn. -func (s *Server) handleResponses(closeConn chan error, responses <-chan *tmsp_types.Response, conn net.Conn) { - var count int - var bufWriter = bufio.NewWriter(conn) - for { - var res = <-responses - err := tmsp_types.WriteMessage(res, bufWriter) - if err != nil { - closeConn <- fmt.Errorf("Error in handleResponses: %v", err.Error()) - return - } - if _, ok := res.Value.(*tmsp_types.Response_Flush); ok { - err = bufWriter.Flush() - if err != nil { - closeConn <- fmt.Errorf("Error in handleValue: %v", err.Error()) - return - } - } - count++ - } -} diff --git a/consensus/tmsp/version.go b/consensus/tmsp/version.go deleted file mode 100644 index 266b37d792c4b606cdd547469f05d050edcc0b3b..0000000000000000000000000000000000000000 --- a/consensus/tmsp/version.go +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright 2015, 2016 Eris Industries (UK) Ltd. -// This file is part of Eris-RT - -// Eris-RT is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// Eris-RT is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with Eris-RT. If not, see <http://www.gnu.org/licenses/>. - -package tmsp - -import ( - version "github.com/eris-ltd/eris-db/version" -) - -const ( - // Client identifier to advertise over the network - tmspClientIdentifier = "tmsp" - // Major version component of the current release - tmspVersionMajor = 0 - // Minor version component of the current release - tmspVersionMinor = 6 - // Patch version component of the current release - tmspVersionPatch = 0 -) - -func GetTmspVersion() *version.VersionIdentifier { - return version.New(tmspClientIdentifier, tmspVersionMajor, tmspVersionMinor, - tmspVersionPatch) -} diff --git a/consensus/types/consensus_engine.go b/consensus/types/consensus_engine.go index ea02d3b2ec36459765f755cfd2f11930a622b38c..52eb9d0b9b01197620c8abcb4874672a4f21c27c 100644 --- a/consensus/types/consensus_engine.go +++ b/consensus/types/consensus_engine.go @@ -5,7 +5,7 @@ import ( "github.com/eris-ltd/eris-db/txs" "github.com/tendermint/go-crypto" "github.com/tendermint/go-p2p" - tmsp_types "github.com/tendermint/tmsp/types" + abci_types "github.com/tendermint/abci/types" ) type ConsensusEngine interface { @@ -20,7 +20,7 @@ type ConsensusEngine interface { // Memory pool BroadcastTransaction(transaction []byte, - callback func(*tmsp_types.Response)) error + callback func(*abci_types.Response)) error // Events // For consensus events like NewBlock diff --git a/core/core.go b/core/core.go index 6c58d1a3312eba6c17cf06c121be1bbb201d48eb..9ce189aae12a2a71bc3b9445d1d44cdd15433581 100644 --- a/core/core.go +++ b/core/core.go @@ -40,7 +40,7 @@ import ( // Core is the high-level structure type Core struct { chainId string - evsw *events.EventSwitch + evsw events.EventSwitch pipe definitions.Pipe tendermintPipe definitions.TendermintPipe } diff --git a/event/events.go b/event/events.go index ec790a9ed8b1d120a46ecf1642bec840756fcbdd..65e439e5a7b284a7717ab13ec819cc22c9bedc43 100644 --- a/event/events.go +++ b/event/events.go @@ -44,7 +44,7 @@ type EventEmitter interface { Unsubscribe(subId string) error } -func NewEvents(eventSwitch *go_events.EventSwitch, logger loggers.InfoTraceLogger) *events { +func NewEvents(eventSwitch go_events.EventSwitch, logger loggers.InfoTraceLogger) *events { return &events{eventSwitch: eventSwitch, logger: logging.WithScope(logger, "Events")} } @@ -57,7 +57,7 @@ func Multiplex(events ...EventEmitter) *multiplexedEvents { // The events struct has methods for working with events. type events struct { - eventSwitch *go_events.EventSwitch + eventSwitch go_events.EventSwitch logger loggers.InfoTraceLogger } diff --git a/glide.lock b/glide.lock index c226621ad80436ea994d07152b9b19871bb89fc0..6887d752d90fc9ab64b4b6d7181b92262ed992d3 100644 --- a/glide.lock +++ b/glide.lock @@ -195,8 +195,6 @@ imports: - state - types - version -- name: github.com/tendermint/tmsp - version: f41bc5f11969e22b357d94b4247403fd62d40445 - name: github.com/tommy351/gin-cors version: dc91dec6313ae4db53481bf3b29cf6b94bf80357 - name: golang.org/x/crypto diff --git a/manager/eris-mint/eris-mint.go b/manager/eris-mint/eris-mint.go index 29050723ae9e89850b0ebbb91d432591ffdb4ef1..899bbec89333a4f77ec2a7b0ea245f8139f29d45 100644 --- a/manager/eris-mint/eris-mint.go +++ b/manager/eris-mint/eris-mint.go @@ -24,7 +24,7 @@ import ( tendermint_events "github.com/tendermint/go-events" wire "github.com/tendermint/go-wire" - tmsp "github.com/tendermint/tmsp/types" + abci "github.com/tendermint/abci/types" "github.com/eris-ltd/eris-db/logging" "github.com/eris-ltd/eris-db/logging/loggers" @@ -47,7 +47,7 @@ type ErisMint struct { checkCache *sm.BlockCache // for CheckTx (eg. so we get nonces right) evc *tendermint_events.EventCache - evsw *tendermint_events.EventSwitch + evsw tendermint_events.EventSwitch nTxs int // count txs in a block logger loggers.InfoTraceLogger @@ -57,9 +57,9 @@ type ErisMint struct { // eris-db/manager/types.Application var _ manager_types.Application = (*ErisMint)(nil) -// NOTE: [ben] also automatically implements tmsp.Application, +// NOTE: [ben] also automatically implements abci.Application, // undesired but unharmful -// var _ tmsp.Application = (*ErisMint)(nil) +// var _ abci.Application = (*ErisMint)(nil) func (app *ErisMint) GetState() *sm.State { app.mtx.Lock() @@ -74,7 +74,7 @@ func (app *ErisMint) GetCheckCache() *sm.BlockCache { return app.checkCache } -func NewErisMint(s *sm.State, evsw *tendermint_events.EventSwitch, logger loggers.InfoTraceLogger) *ErisMint { +func NewErisMint(s *sm.State, evsw tendermint_events.EventSwitch, logger loggers.InfoTraceLogger) *ErisMint { return &ErisMint{ state: s, cache: sm.NewBlockCache(s), @@ -86,8 +86,8 @@ func NewErisMint(s *sm.State, evsw *tendermint_events.EventSwitch, logger logger } // Implements manager/types.Application -func (app *ErisMint) Info() (info string) { - return "ErisDB" +func (app *ErisMint) Info() (info abci.ResponseInfo) { + return abci.ResponseInfo{} } // Implements manager/types.Application @@ -96,7 +96,7 @@ func (app *ErisMint) SetOption(key string, value string) (log string) { } // Implements manager/types.Application -func (app *ErisMint) AppendTx(txBytes []byte) tmsp.Result { +func (app *ErisMint) DeliverTx(txBytes []byte) abci.Result { app.nTxs += 1 // XXX: if we had tx ids we could cache the decoded txs on CheckTx @@ -106,45 +106,45 @@ func (app *ErisMint) AppendTx(txBytes []byte) tmsp.Result { buf := bytes.NewBuffer(txBytes) wire.ReadBinaryPtr(tx, buf, len(txBytes), &n, &err) if err != nil { - return tmsp.NewError(tmsp.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err)) + return abci.NewError(abci.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err)) } err = sm.ExecTx(app.cache, *tx, true, app.evc) if err != nil { - return tmsp.NewError(tmsp.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err)) + return abci.NewError(abci.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err)) } receipt := txs.GenerateReceipt(app.state.ChainID, *tx) receiptBytes := wire.BinaryBytes(receipt) - return tmsp.NewResultOK(receiptBytes, "Success") + return abci.NewResultOK(receiptBytes, "Success") } // Implements manager/types.Application -func (app *ErisMint) CheckTx(txBytes []byte) tmsp.Result { +func (app *ErisMint) CheckTx(txBytes []byte) abci.Result { var n int var err error tx := new(txs.Tx) buf := bytes.NewBuffer(txBytes) wire.ReadBinaryPtr(tx, buf, len(txBytes), &n, &err) if err != nil { - return tmsp.NewError(tmsp.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err)) + return abci.NewError(abci.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err)) } - // TODO: map ExecTx errors to sensible TMSP error codes + // TODO: map ExecTx errors to sensible abci error codes err = sm.ExecTx(app.checkCache, *tx, false, nil) if err != nil { - return tmsp.NewError(tmsp.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err)) + return abci.NewError(abci.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err)) } receipt := txs.GenerateReceipt(app.state.ChainID, *tx) receiptBytes := wire.BinaryBytes(receipt) - return tmsp.NewResultOK(receiptBytes, "Success") + return abci.NewResultOK(receiptBytes, "Success") } // Implements manager/types.Application // Commit the state (called at end of block) // NOTE: CheckTx/AppendTx must not run concurrently with Commit - // the mempool should run during AppendTxs, but lock for Commit and Update -func (app *ErisMint) Commit() (res tmsp.Result) { +func (app *ErisMint) Commit() (res abci.Result) { app.mtx.Lock() // the lock protects app.state defer app.mtx.Unlock() @@ -175,10 +175,10 @@ func (app *ErisMint) Commit() (res tmsp.Result) { // MARMOT: appHash := app.state.Hash() fmt.Printf("\n\nMARMOT COMMIT: %X\n\n", appHash) - // return tmsp.NewResultOK(app.state.Hash(), "Success") - return tmsp.NewResultOK(appHash, "Success") + // return abci.NewResultOK(app.state.Hash(), "Success") + return abci.NewResultOK(appHash, "Success") } -func (app *ErisMint) Query(query []byte) (res tmsp.Result) { - return tmsp.NewResultOK(nil, "Success") +func (app *ErisMint) Query(query []byte) (res abci.Result) { + return abci.NewResultOK(nil, "Success") } diff --git a/manager/eris-mint/pipe.go b/manager/eris-mint/pipe.go index 014de66207a1c2b0f27de336016d15291f9c8de6..9fda3eb6cb74974962034d8a1a943cdd15713a14 100644 --- a/manager/eris-mint/pipe.go +++ b/manager/eris-mint/pipe.go @@ -20,13 +20,13 @@ import ( "bytes" "fmt" + abci_types "github.com/tendermint/abci/types" tm_common "github.com/tendermint/go-common" crypto "github.com/tendermint/go-crypto" db "github.com/tendermint/go-db" go_events "github.com/tendermint/go-events" wire "github.com/tendermint/go-wire" tm_types "github.com/tendermint/tendermint/types" - tmsp_types "github.com/tendermint/tmsp/types" "github.com/eris-ltd/eris-db/account" blockchain_types "github.com/eris-ltd/eris-db/blockchain/types" @@ -71,7 +71,7 @@ var _ definitions.Pipe = (*erisMintPipe)(nil) var _ definitions.TendermintPipe = (*erisMintPipe)(nil) func NewErisMintPipe(moduleConfig *config.ModuleConfig, - eventSwitch *go_events.EventSwitch, + eventSwitch go_events.EventSwitch, logger loggers.InfoTraceLogger) (*erisMintPipe, error) { startedState, genesisDoc, err := startState(moduleConfig.DataDir, @@ -144,8 +144,8 @@ func NewErisMintPipe(moduleConfig *config.ModuleConfig, func startState(dataDir, backend, genesisFile, chainId string) (*state.State, *genesis.GenesisDoc, error) { // avoid Tendermints PanicSanity and return a clean error - if backend != db.DBBackendMemDB && - backend != db.DBBackendLevelDB { + if backend != db.MemDBBackendStr && + backend != db.LevelDBBackendStr { return nil, nil, fmt.Errorf("Database backend %s is not supported by %s", backend, GetErisMintVersion) } @@ -534,7 +534,7 @@ func (pipe *erisMintPipe) ListNames() (*rpc_tm_types.ResultListNames, error) { } func (pipe *erisMintPipe) broadcastTx(tx txs.Tx, - callback func(res *tmsp_types.Response)) (*rpc_tm_types.ResultBroadcastTx, error) { + callback func(res *abci_types.Response)) (*rpc_tm_types.ResultBroadcastTx, error) { txBytes, err := txs.EncodeTx(tx) if err != nil { @@ -554,9 +554,9 @@ func (pipe *erisMintPipe) BroadcastTxAsync(tx txs.Tx) (*rpc_tm_types.ResultBroad } func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadcastTx, error) { - responseChannel := make(chan *tmsp_types.Response, 1) + responseChannel := make(chan *abci_types.Response, 1) _, err := pipe.broadcastTx(tx, - func(res *tmsp_types.Response) { + func(res *abci_types.Response) { responseChannel <- res }) if err != nil { @@ -564,7 +564,7 @@ func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadc } // NOTE: [ben] This Response is set in /consensus/tendermint/local_client.go // a call to Application, here implemented by ErisMint, over local callback, - // or TMSP RPC call. Hence the result is determined by ErisMint/erismint.go + // or abci RPC call. Hence the result is determined by ErisMint/erismint.go // CheckTx() Result (Result converted to ReqRes into Response returned here) // NOTE: [ben] BroadcastTx just calls CheckTx in Tendermint (oddly... [Silas]) response := <-responseChannel @@ -578,17 +578,17 @@ func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadc Log: responseCheckTx.Log, } switch responseCheckTx.Code { - case tmsp_types.CodeType_OK: + case abci_types.CodeType_OK: return resultBroadCastTx, nil - case tmsp_types.CodeType_EncodingError: + case abci_types.CodeType_EncodingError: return resultBroadCastTx, fmt.Errorf(resultBroadCastTx.Log) - case tmsp_types.CodeType_InternalError: + case abci_types.CodeType_InternalError: return resultBroadCastTx, fmt.Errorf(resultBroadCastTx.Log) default: logging.InfoMsg(pipe.logger, "Unknown error returned from Tendermint CheckTx on BroadcastTxSync", "application", GetErisMintVersion().GetVersionString(), - "TMSP_code_type", responseCheckTx.Code, - "TMSP_log", responseCheckTx.Log, + "abci_code_type", responseCheckTx.Code, + "abci_log", responseCheckTx.Log, ) return resultBroadCastTx, fmt.Errorf("Unknown error returned: " + responseCheckTx.Log) } diff --git a/manager/eris-mint/version.go b/manager/eris-mint/version.go index ad15d4a34d7cf0b2eb78a32d4831194e6db9aeb6..e6ca9a4c2a6b1c7d60eafb4c653e51a5f6951181 100644 --- a/manager/eris-mint/version.go +++ b/manager/eris-mint/version.go @@ -36,8 +36,7 @@ const ( // Define the compatible consensus engines this application manager // is compatible and has been tested with. var compatibleConsensus = [...]string{ - "tendermint-0.6", - // "tmsp-0.6", + "tendermint-0.8", } func GetErisMintVersion() *version.VersionIdentifier { diff --git a/manager/manager.go b/manager/manager.go index 6f71aefa92dfe857cc4bf8d65b9c3cb22fae6eac..9251aeed5bb2e10cd65d83a663fb90b4a36e028c 100644 --- a/manager/manager.go +++ b/manager/manager.go @@ -36,7 +36,7 @@ import ( // of an application. It is feasible this will be insufficient to support // different types of applications later down the line. func NewApplicationPipe(moduleConfig *config.ModuleConfig, - evsw *events.EventSwitch, logger loggers.InfoTraceLogger, + evsw events.EventSwitch, logger loggers.InfoTraceLogger, consensusMinorVersion string) (definitions.Pipe, error) { switch moduleConfig.Name { diff --git a/manager/types/application.go b/manager/types/application.go index cfcf2b82cc35cfc01c6b9da487f6908143183534..3ab29f6d43c60e72e8bb5db6b8c7f3f1e32471da 100644 --- a/manager/types/application.go +++ b/manager/types/application.go @@ -17,10 +17,10 @@ package types import ( - // TODO: [ben] this is currently only used for tmsp result type; but should - // be removed as tmsp dependencies shouldn't feature in the application + // TODO: [ben] this is currently only used for abci result type; but should + // be removed as abci dependencies shouldn't feature in the application // manager - tmsp_types "github.com/tendermint/tmsp/types" + abci_types "github.com/tendermint/abci/types" ) // NOTE: [ben] this interface is likely to be changed. Currently it is taken @@ -32,16 +32,16 @@ type Application interface { // Info returns application information as a string // NOTE: [ben] likely to move - Info() (info string) + Info() (info abci_types.ResponseInfo) // Set application option (e.g. mode=mempool, mode=consensus) // NOTE: [ben] taken from tendermint, but it is unclear what the use is, - // specifically, when will tendermint call this over tmsp ? + // specifically, when will tendermint call this over abci ? SetOption(key string, value string) (log string) // Append transaction applies a transaction to the state regardless of // whether the transaction is valid or not. - // Currently AppendTx is taken from tmsp, and returns a result. + // Currently AppendTx is taken from abci, and returns a result. // This will be altered, as AppendTransaction needs to more strongly reflect // the theoretical logic: // Append(StateN, Transaction) = StateN+1 @@ -51,18 +51,18 @@ type Application interface { // TODO: implementation notes: // 1. at this point the transaction should already be strongly typed // 2. - AppendTx(tx []byte) tmsp_types.Result + DeliverTx(tx []byte) abci_types.Result // Check Transaction validates a transaction before being allowed into the // consensus' engine memory pool. This is the original defintion and - // intention as taken from tmsp, but should be remapped to the more + // intention as taken from abci, but should be remapped to the more // general concept of basic, cheap verification; // Check Transaction does not alter the state, but does require an immutable // copy of the state. In particular there is no consensus on ordering yet. // TODO: implementation notes: // 1. at this point the transaction should already be strongly typed // 2. - CheckTx(tx []byte) tmsp_types.Result + CheckTx(tx []byte) abci_types.Result // Commit returns the root hash of the current application state // NOTE: [ben] Because the concept of the block has been erased here @@ -70,14 +70,14 @@ type Application interface { // the opposit the principle of explicit stateless functions. // This will be amended when we introduce the concept of (streaming) // blocks in the pipe. - Commit() tmsp_types.Result + Commit() abci_types.Result // Query for state. This query request is not passed over the p2p network // and is called from Tendermint rpc directly up to the application. // NOTE: [ben] Eris-DB will give preference to queries from the local client // directly over the Eris-DB rpc. // We will support this for Tendermint compatibility. - Query(query []byte) tmsp_types.Result + Query(query []byte) abci_types.Result } // Tendermint has a separate interface for reintroduction of blocks @@ -85,7 +85,7 @@ type BlockchainAware interface { // Initialise the blockchain // validators: genesis validators from tendermint core - InitChain(validators []*tmsp_types.Validator) + InitChain(validators []*abci_types.Validator) // Signals the beginning of a block; // NOTE: [ben] currently not supported by tendermint @@ -95,5 +95,5 @@ type BlockchainAware interface { // validators: changed validators from app to Tendermint // NOTE: [ben] currently not supported by tendermint // not yet well defined what the change set contains. - EndBlock(height uint64) (validators []*tmsp_types.Validator) + EndBlock(height uint64) (validators []*abci_types.Validator) } diff --git a/rpc/tendermint/core/types/responses.go b/rpc/tendermint/core/types/responses.go index 98b72fbbd200d120c46c834a02d5cc237ff6b5a4..87d9eeafc67252fd4a4228f0388d1c9eeb9cebd7 100644 --- a/rpc/tendermint/core/types/responses.go +++ b/rpc/tendermint/core/types/responses.go @@ -12,7 +12,7 @@ import ( "github.com/tendermint/go-p2p" "github.com/tendermint/go-rpc/types" "github.com/tendermint/go-wire" - tmsptypes "github.com/tendermint/tmsp/types" + abcitypes "github.com/tendermint/abci/types" ) type ResultGetStorage struct { @@ -111,7 +111,7 @@ type ResultGetAccount struct { } type ResultBroadcastTx struct { - Code tmsptypes.CodeType `json:"code"` + Code abcitypes.CodeType `json:"code"` Data []byte `json:"data"` Log string `json:"log"` } diff --git a/rpc/tendermint/core/websocket.go b/rpc/tendermint/core/websocket.go index 6824b78e052becb787ccf8092870cb4a73b37c2a..846ddfdc29fcd72aeb1064074ed9bbef016e1a86 100644 --- a/rpc/tendermint/core/websocket.go +++ b/rpc/tendermint/core/websocket.go @@ -35,7 +35,7 @@ type TendermintWebsocketServer struct { } func NewTendermintWebsocketServer(config *server.ServerConfig, - tendermintPipe definitions.TendermintPipe, evsw *events.EventSwitch) ( + tendermintPipe definitions.TendermintPipe, evsw events.EventSwitch) ( *TendermintWebsocketServer, error) { if tendermintPipe == nil { diff --git a/rpc/tendermint/test/config.go b/rpc/tendermint/test/config.go index fcf964bbfc7c0b53183c8838dd521b822eb84a31..ee5bd5a20e94bbf941acc0b75081fc4be2a3ade9 100644 --- a/rpc/tendermint/test/config.go +++ b/rpc/tendermint/test/config.go @@ -40,12 +40,12 @@ genesis_file = "genesis.json" [chain.consensus] # consensus defines the module to use for consensus and # this will define the peer-to-peer consensus network; - # accepted values are "noops", "tmsp", "tendermint" + # accepted values are "noops", "abci", "tendermint" name = "tendermint" # version is the major and minor semantic version; # the version will be asserted on major_version = 0 - minor_version = 6 + minor_version = 8 # relative path to consensus' module root folder relative_root = "tendermint" @@ -123,21 +123,19 @@ genesis_file = "genesis.json" ################################################################################ ## -## Tendermint Socket Protocol (TMSP) -## version 0.6.0 +## Tendermint Socket Protocol (abci) ## -## TMSP expects a tendermint consensus process to run and connect to Eris-DB +## abci expects a tendermint consensus process to run and connect to Eris-DB ## ################################################################################ -[tmsp] +[abci] # listener address for accepting tendermint socket protocol connections listener = "tcp://0.0.0.0:46658" ################################################################################ -## +##yeah we had partial support for that with TMSP ## Tendermint -## version 0.6.0 ## ## in-process execution of Tendermint consensus engine ## @@ -174,7 +172,7 @@ private_validator_file = "priv_validator.json" # NOTE: value is ignored when run in-process as RPC is # handled by [servers.tendermint] rpc_laddr = "" - # proxy application address - used for tmsp connections, + # proxy application address - used for abci connections, # and this port should not be exposed for in-process Tendermint proxy_app = "tcp://127.0.0.1:46658" diff --git a/rpc/tendermint/test/rpc_client_test.go b/rpc/tendermint/test/rpc_client_test.go index 7183f2df8f1effbafe76705e344fd1658c01852a..ba5f472a0a098794456e4c7466253ef20ed092c6 100644 --- a/rpc/tendermint/test/rpc_client_test.go +++ b/rpc/tendermint/test/rpc_client_test.go @@ -295,7 +295,7 @@ func TestBlockchainInfo(t *testing.T) { lastBlockHash := resp.BlockMetas[nBlocks-1].Hash for i := nBlocks - 2; i >= 0; i-- { - assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastBlockHash, + assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastCommitHash, "Blockchain should be a hash tree!") lastBlockHash = resp.BlockMetas[i].Hash } diff --git a/test/mock/pipe.go b/test/mock/pipe.go index ecd245ec87e2c9934e360ae72869210e134ee820..cfa192c9cf817e71bca77ce2fa1510efe4dbbe1b 100644 --- a/test/mock/pipe.go +++ b/test/mock/pipe.go @@ -18,7 +18,7 @@ import ( "github.com/tendermint/go-crypto" "github.com/tendermint/go-p2p" mintTypes "github.com/tendermint/tendermint/types" - tmsp_types "github.com/tendermint/tmsp/types" + abci_types "github.com/tendermint/abci/types" ) // Base struct. @@ -165,7 +165,7 @@ type consensusEngine struct { } func (cons *consensusEngine) BroadcastTransaction(transaction []byte, - callback func(*tmsp_types.Response)) error { + callback func(*abci_types.Response)) error { return nil }