diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index 838afa4db..d891128da 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -71,6 +71,8 @@ program](https://hackerone.com/tendermint). - [libs/common] \#3862 Remove `errors.go` from `libs/common` - [libs/common] \#4230 Move `KV` out of common to its own pkg - [libs/common] \#4230 Rename `cmn.KVPair(s)` to `kv.Pair(s)`s + - [libs/common] \#4232 Move `Service` & `BaseService` from `libs/common` to `libs/service` + - [libs/common] \#4232 Move `common/nil.go` to `types/utils.go` & make the functions private - [libs/common] \#4231 Move random functions from `libs/common` into pkg `rand` diff --git a/abci/client/client.go b/abci/client/client.go index b7f941e7b..4f7c7b69a 100644 --- a/abci/client/client.go +++ b/abci/client/client.go @@ -5,7 +5,7 @@ import ( "sync" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) const ( @@ -19,7 +19,7 @@ const ( // Note these are client errors, eg. ABCI socket connectivity issues. // Application-related errors are reflected in response via ABCI error codes and logs. type Client interface { - cmn.Service + service.Service SetResponseCallback(Callback) Error() error diff --git a/abci/client/grpc_client.go b/abci/client/grpc_client.go index e326055fb..514e27533 100644 --- a/abci/client/grpc_client.go +++ b/abci/client/grpc_client.go @@ -11,6 +11,7 @@ import ( "github.com/tendermint/tendermint/abci/types" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) var _ Client = (*grpcClient)(nil) @@ -18,7 +19,7 @@ var _ Client = (*grpcClient)(nil) // A stripped copy of the remoteClient that makes // synchronous calls using grpc type grpcClient struct { - cmn.BaseService + service.BaseService mustConnect bool client types.ABCIApplicationClient @@ -35,7 +36,7 @@ func NewGRPCClient(addr string, mustConnect bool) *grpcClient { addr: addr, mustConnect: mustConnect, } - cli.BaseService = *cmn.NewBaseService(nil, "grpcClient", cli) + cli.BaseService = *service.NewBaseService(nil, "grpcClient", cli) return cli } diff --git a/abci/client/local_client.go b/abci/client/local_client.go index bb009173a..f50d4eae1 100644 --- a/abci/client/local_client.go +++ b/abci/client/local_client.go @@ -4,7 +4,7 @@ import ( "sync" types "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) var _ Client = (*localClient)(nil) @@ -14,7 +14,7 @@ var _ Client = (*localClient)(nil) // methods like CheckTx (/broadcast_tx_* RPC endpoint) or Query (/abci_query // RPC endpoint), but defers are used everywhere for the sake of consistency. type localClient struct { - cmn.BaseService + service.BaseService mtx *sync.Mutex types.Application @@ -29,7 +29,7 @@ func NewLocalClient(mtx *sync.Mutex, app types.Application) *localClient { mtx: mtx, Application: app, } - cli.BaseService = *cmn.NewBaseService(nil, "localClient", cli) + cli.BaseService = *service.NewBaseService(nil, "localClient", cli) return cli } diff --git a/abci/client/socket_client.go b/abci/client/socket_client.go index 3d4a03c9a..98ddbb610 100644 --- a/abci/client/socket_client.go +++ b/abci/client/socket_client.go @@ -13,6 +13,7 @@ import ( "github.com/tendermint/tendermint/abci/types" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) const reqQueueSize = 256 // TODO make configurable @@ -25,7 +26,7 @@ var _ Client = (*socketClient)(nil) // the application in general is not meant to be interfaced // with concurrent callers. type socketClient struct { - cmn.BaseService + service.BaseService addr string mustConnect bool @@ -51,7 +52,7 @@ func NewSocketClient(addr string, mustConnect bool) *socketClient { reqSent: list.New(), resCb: nil, } - cli.BaseService = *cmn.NewBaseService(nil, "socketClient", cli) + cli.BaseService = *service.NewBaseService(nil, "socketClient", cli) return cli } diff --git a/abci/client/socket_client_test.go b/abci/client/socket_client_test.go index 2a00142b0..e96fec1c6 100644 --- a/abci/client/socket_client_test.go +++ b/abci/client/socket_client_test.go @@ -12,8 +12,8 @@ import ( abcicli "github.com/tendermint/tendermint/abci/client" "github.com/tendermint/tendermint/abci/server" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" ) func TestSocketClientStopForErrorDeadlock(t *testing.T) { @@ -95,7 +95,7 @@ func TestHangingSyncCalls(t *testing.T) { } func setupClientServer(t *testing.T, app types.Application) ( - cmn.Service, abcicli.Client) { + service.Service, abcicli.Client) { // some port between 20k and 30k port := 20000 + rand.RandInt32()%10000 addr := fmt.Sprintf("localhost:%d", port) diff --git a/abci/example/kvstore/kvstore_test.go b/abci/example/kvstore/kvstore_test.go index 0a9e836ac..e8dee67d9 100644 --- a/abci/example/kvstore/kvstore_test.go +++ b/abci/example/kvstore/kvstore_test.go @@ -9,8 +9,8 @@ import ( "github.com/stretchr/testify/require" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" abcicli "github.com/tendermint/tendermint/abci/client" "github.com/tendermint/tendermint/abci/example/code" @@ -217,7 +217,7 @@ func valsEqual(t *testing.T, vals1, vals2 []types.ValidatorUpdate) { } } -func makeSocketClientServer(app types.Application, name string) (abcicli.Client, cmn.Service, error) { +func makeSocketClientServer(app types.Application, name string) (abcicli.Client, service.Service, error) { // Start the listener socket := fmt.Sprintf("unix://%s.sock", name) logger := log.TestingLogger() @@ -239,7 +239,7 @@ func makeSocketClientServer(app types.Application, name string) (abcicli.Client, return client, server, nil } -func makeGRPCClientServer(app types.Application, name string) (abcicli.Client, cmn.Service, error) { +func makeGRPCClientServer(app types.Application, name string) (abcicli.Client, service.Service, error) { // Start the listener socket := fmt.Sprintf("unix://%s.sock", name) logger := log.TestingLogger() diff --git a/abci/server/grpc_server.go b/abci/server/grpc_server.go index ccbe609cc..0f8e42d7f 100644 --- a/abci/server/grpc_server.go +++ b/abci/server/grpc_server.go @@ -7,10 +7,11 @@ import ( "github.com/tendermint/tendermint/abci/types" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) type GRPCServer struct { - cmn.BaseService + service.BaseService proto string addr string @@ -21,7 +22,7 @@ type GRPCServer struct { } // NewGRPCServer returns a new gRPC ABCI server -func NewGRPCServer(protoAddr string, app types.ABCIApplicationServer) cmn.Service { +func NewGRPCServer(protoAddr string, app types.ABCIApplicationServer) service.Service { proto, addr := cmn.ProtocolAndAddress(protoAddr) s := &GRPCServer{ proto: proto, @@ -29,7 +30,7 @@ func NewGRPCServer(protoAddr string, app types.ABCIApplicationServer) cmn.Servic listener: nil, app: app, } - s.BaseService = *cmn.NewBaseService(nil, "ABCIServer", s) + s.BaseService = *service.NewBaseService(nil, "ABCIServer", s) return s } diff --git a/abci/server/server.go b/abci/server/server.go index 65d79529e..6dd13ad02 100644 --- a/abci/server/server.go +++ b/abci/server/server.go @@ -12,11 +12,11 @@ import ( "fmt" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) -func NewServer(protoAddr, transport string, app types.Application) (cmn.Service, error) { - var s cmn.Service +func NewServer(protoAddr, transport string, app types.Application) (service.Service, error) { + var s service.Service var err error switch transport { case "socket": diff --git a/abci/server/socket_server.go b/abci/server/socket_server.go index 3c89de2bf..c69be3813 100644 --- a/abci/server/socket_server.go +++ b/abci/server/socket_server.go @@ -9,12 +9,13 @@ import ( "github.com/tendermint/tendermint/abci/types" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) // var maxNumberConnections = 2 type SocketServer struct { - cmn.BaseService + service.BaseService proto string addr string @@ -28,7 +29,7 @@ type SocketServer struct { app types.Application } -func NewSocketServer(protoAddr string, app types.Application) cmn.Service { +func NewSocketServer(protoAddr string, app types.Application) service.Service { proto, addr := cmn.ProtocolAndAddress(protoAddr) s := &SocketServer{ proto: proto, @@ -37,7 +38,7 @@ func NewSocketServer(protoAddr string, app types.Application) cmn.Service { app: app, conns: make(map[int]net.Conn), } - s.BaseService = *cmn.NewBaseService(nil, "ABCIServer", s) + s.BaseService = *service.NewBaseService(nil, "ABCIServer", s) return s } diff --git a/blockchain/v0/pool.go b/blockchain/v0/pool.go index a1901842e..1931d7960 100644 --- a/blockchain/v0/pool.go +++ b/blockchain/v0/pool.go @@ -8,9 +8,9 @@ import ( "sync/atomic" "time" - cmn "github.com/tendermint/tendermint/libs/common" flow "github.com/tendermint/tendermint/libs/flowrate" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/types" @@ -61,7 +61,7 @@ var peerTimeout = 15 * time.Second // not const so we can override with tests // BlockPool keeps track of the fast sync peers, block requests and block responses. type BlockPool struct { - cmn.BaseService + service.BaseService startTime time.Time mtx sync.Mutex @@ -92,11 +92,11 @@ func NewBlockPool(start int64, requestsCh chan<- BlockRequest, errorsCh chan<- p requestsCh: requestsCh, errorsCh: errorsCh, } - bp.BaseService = *cmn.NewBaseService(nil, "BlockPool", bp) + bp.BaseService = *service.NewBaseService(nil, "BlockPool", bp) return bp } -// OnStart implements cmn.Service by spawning requesters routine and recording +// OnStart implements service.Service by spawning requesters routine and recording // pool's start time. func (pool *BlockPool) OnStart() error { go pool.makeRequestersRoutine() @@ -501,7 +501,7 @@ func (peer *bpPeer) onTimeout() { //------------------------------------- type bpRequester struct { - cmn.BaseService + service.BaseService pool *BlockPool height int64 gotBlockCh chan struct{} @@ -522,7 +522,7 @@ func newBPRequester(pool *BlockPool, height int64) *bpRequester { peerID: "", block: nil, } - bpr.BaseService = *cmn.NewBaseService(nil, "bpRequester", bpr) + bpr.BaseService = *service.NewBaseService(nil, "bpRequester", bpr) return bpr } diff --git a/blockchain/v0/reactor.go b/blockchain/v0/reactor.go index 6e7bea660..c35a3e6a9 100644 --- a/blockchain/v0/reactor.go +++ b/blockchain/v0/reactor.go @@ -102,13 +102,13 @@ func NewBlockchainReactor(state sm.State, blockExec *sm.BlockExecutor, store *st return bcR } -// SetLogger implements cmn.Service by setting the logger on reactor and pool. +// SetLogger implements service.Service by setting the logger on reactor and pool. func (bcR *BlockchainReactor) SetLogger(l log.Logger) { bcR.BaseService.Logger = l bcR.pool.Logger = l } -// OnStart implements cmn.Service. +// OnStart implements service.Service. func (bcR *BlockchainReactor) OnStart() error { if bcR.fastSync { err := bcR.pool.Start() @@ -120,7 +120,7 @@ func (bcR *BlockchainReactor) OnStart() error { return nil } -// OnStop implements cmn.Service. +// OnStop implements service.Service. func (bcR *BlockchainReactor) OnStop() { bcR.pool.Stop() } diff --git a/blockchain/v1/reactor.go b/blockchain/v1/reactor.go index 480b87f34..c1932d406 100644 --- a/blockchain/v1/reactor.go +++ b/blockchain/v1/reactor.go @@ -133,13 +133,13 @@ type bcFsmMessage struct { data bFsmEventData } -// SetLogger implements cmn.Service by setting the logger on reactor and pool. +// SetLogger implements service.Service by setting the logger on reactor and pool. func (bcR *BlockchainReactor) SetLogger(l log.Logger) { bcR.BaseService.Logger = l bcR.fsm.SetLogger(l) } -// OnStart implements cmn.Service. +// OnStart implements service.Service. func (bcR *BlockchainReactor) OnStart() error { bcR.swReporter = behaviour.NewSwitcReporter(bcR.BaseReactor.Switch) if bcR.fastSync { @@ -148,7 +148,7 @@ func (bcR *BlockchainReactor) OnStart() error { return nil } -// OnStop implements cmn.Service. +// OnStop implements service.Service. func (bcR *BlockchainReactor) OnStop() { _ = bcR.Stop() } diff --git a/consensus/byzantine_test.go b/consensus/byzantine_test.go index 85f771bcc..65bd88f67 100644 --- a/consensus/byzantine_test.go +++ b/consensus/byzantine_test.go @@ -8,7 +8,7 @@ import ( "time" "github.com/stretchr/testify/require" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -244,7 +244,7 @@ func sendProposalAndParts( // byzantine consensus reactor type ByzantineReactor struct { - cmn.Service + service.Service reactor *Reactor } diff --git a/consensus/state.go b/consensus/state.go index ee40f9215..f7395cba7 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -13,6 +13,7 @@ import ( cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/fail" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" tmtime "github.com/tendermint/tendermint/types/time" cfg "github.com/tendermint/tendermint/config" @@ -72,7 +73,7 @@ type evidencePool interface { // commits blocks to the chain and executes them against the application. // The internal state machine receives input from peers, the internal validator, and from a timer. type State struct { - cmn.BaseService + service.BaseService // config details config *cfg.ConsensusConfig @@ -174,7 +175,7 @@ func NewState( // Don't call scheduleRound0 yet. // We do that upon Start(). cs.reconstructLastCommit(state) - cs.BaseService = *cmn.NewBaseService(nil, "State", cs) + cs.BaseService = *service.NewBaseService(nil, "State", cs) for _, option := range options { option(cs) } @@ -275,7 +276,7 @@ func (cs *State) LoadCommit(height int64) *types.Commit { return cs.blockStore.LoadBlockCommit(height) } -// OnStart implements cmn.Service. +// OnStart implements service.Service. // It loads the latest state via the WAL, and starts the timeout and receive routines. func (cs *State) OnStart() error { if err := cs.evsw.Start(); err != nil { @@ -351,7 +352,7 @@ func (cs *State) startRoutines(maxSteps int) { go cs.receiveRoutine(maxSteps) } -// OnStop implements cmn.Service. +// OnStop implements service.Service. func (cs *State) OnStop() { cs.evsw.Stop() cs.timeoutTicker.Stop() diff --git a/consensus/ticker.go b/consensus/ticker.go index a1e2174c3..fb3571ac8 100644 --- a/consensus/ticker.go +++ b/consensus/ticker.go @@ -3,8 +3,8 @@ package consensus import ( "time" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" ) var ( @@ -29,7 +29,7 @@ type TimeoutTicker interface { // Timeouts are scheduled along the tickChan, // and fired on the tockChan. type timeoutTicker struct { - cmn.BaseService + service.BaseService timer *time.Timer tickChan chan timeoutInfo // for scheduling timeouts @@ -43,12 +43,12 @@ func NewTimeoutTicker() TimeoutTicker { tickChan: make(chan timeoutInfo, tickTockBufferSize), tockChan: make(chan timeoutInfo, tickTockBufferSize), } - tt.BaseService = *cmn.NewBaseService(nil, "TimeoutTicker", tt) + tt.BaseService = *service.NewBaseService(nil, "TimeoutTicker", tt) tt.stopTimer() // don't want to fire until the first scheduled timeout return tt } -// OnStart implements cmn.Service. It starts the timeout routine. +// OnStart implements service.Service. It starts the timeout routine. func (t *timeoutTicker) OnStart() error { go t.timeoutRoutine() @@ -56,7 +56,7 @@ func (t *timeoutTicker) OnStart() error { return nil } -// OnStop implements cmn.Service. It stops the timeout routine. +// OnStop implements service.Service. It stops the timeout routine. func (t *timeoutTicker) OnStop() { t.BaseService.OnStop() t.stopTimer() diff --git a/consensus/wal.go b/consensus/wal.go index 1e979fc2a..c00df13e7 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -14,6 +14,7 @@ import ( auto "github.com/tendermint/tendermint/libs/autofile" cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/types" tmtime "github.com/tendermint/tendermint/types/time" ) @@ -78,7 +79,7 @@ type WAL interface { // so it's either reading or appending - must read to end to start appending // again. type baseWAL struct { - cmn.BaseService + service.BaseService group *auto.Group @@ -107,7 +108,7 @@ func NewWAL(walFile string, groupOptions ...func(*auto.Group)) (*baseWAL, error) enc: NewWALEncoder(group), flushInterval: walDefaultFlushInterval, } - wal.BaseService = *cmn.NewBaseService(nil, "baseWAL", wal) + wal.BaseService = *service.NewBaseService(nil, "baseWAL", wal) return wal, nil } diff --git a/libs/autofile/group.go b/libs/autofile/group.go index 7cc345478..e859149b9 100644 --- a/libs/autofile/group.go +++ b/libs/autofile/group.go @@ -14,7 +14,7 @@ import ( "sync" "time" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) const ( @@ -53,7 +53,7 @@ The Group can also be used to binary-search for some line, assuming that marker lines are written occasionally. */ type Group struct { - cmn.BaseService + service.BaseService ID string Head *AutoFile // The head AutoFile to write to @@ -102,7 +102,7 @@ func OpenGroup(headPath string, groupOptions ...func(*Group)) (g *Group, err err option(g) } - g.BaseService = *cmn.NewBaseService(nil, "Group", g) + g.BaseService = *service.NewBaseService(nil, "Group", g) gInfo := g.readGroupInfo() g.minIndex = gInfo.MinIndex @@ -131,7 +131,7 @@ func GroupTotalSizeLimit(limit int64) func(*Group) { } } -// OnStart implements cmn.Service by starting the goroutine that checks file +// OnStart implements service.Service by starting the goroutine that checks file // and group limits. func (g *Group) OnStart() error { g.ticker = time.NewTicker(g.groupCheckDuration) @@ -139,7 +139,7 @@ func (g *Group) OnStart() error { return nil } -// OnStop implements cmn.Service by stopping the goroutine described above. +// OnStop implements service.Service by stopping the goroutine described above. // NOTE: g.Head must be closed separately using Close. func (g *Group) OnStop() { g.ticker.Stop() diff --git a/libs/events/README.md b/libs/events/README.md index 14aa498ff..9f6908a7f 100644 --- a/libs/events/README.md +++ b/libs/events/README.md @@ -110,7 +110,7 @@ via concrete implementation of this interface ## type [EventSwitch](/src/target/events.go?s=560:771#L29) ``` go type EventSwitch interface { - cmn.Service + service.Service Fireable AddListenerForEvent(listenerID, event string, cb EventCallback) diff --git a/libs/events/events.go b/libs/events/events.go index 34333a068..2468e4838 100644 --- a/libs/events/events.go +++ b/libs/events/events.go @@ -5,7 +5,7 @@ import ( "fmt" "sync" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) // ErrListenerWasRemoved is returned by AddEvent if the listener was removed. @@ -43,7 +43,7 @@ type Fireable interface { // They can be removed by calling either RemoveListenerForEvent or // RemoveListener (for all events). type EventSwitch interface { - cmn.Service + service.Service Fireable AddListenerForEvent(listenerID, event string, cb EventCallback) error @@ -52,7 +52,7 @@ type EventSwitch interface { } type eventSwitch struct { - cmn.BaseService + service.BaseService mtx sync.RWMutex eventCells map[string]*eventCell @@ -64,7 +64,7 @@ func NewEventSwitch() EventSwitch { eventCells: make(map[string]*eventCell), listeners: make(map[string]*eventListener), } - evsw.BaseService = *cmn.NewBaseService(nil, "EventSwitch", evsw) + evsw.BaseService = *service.NewBaseService(nil, "EventSwitch", evsw) return evsw } diff --git a/libs/pubsub/pubsub.go b/libs/pubsub/pubsub.go index bead2aae0..325403cd8 100644 --- a/libs/pubsub/pubsub.go +++ b/libs/pubsub/pubsub.go @@ -39,7 +39,7 @@ import ( "sync" "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) type operation int @@ -88,7 +88,7 @@ type cmd struct { // Server allows clients to subscribe/unsubscribe for messages, publishing // messages with or without events, and manages internal state. type Server struct { - cmn.BaseService + service.BaseService cmds chan cmd cmdsCap int @@ -109,7 +109,7 @@ func NewServer(options ...Option) *Server { s := &Server{ subscriptions: make(map[string]map[string]struct{}), } - s.BaseService = *cmn.NewBaseService(nil, "PubSub", s) + s.BaseService = *service.NewBaseService(nil, "PubSub", s) for _, option := range options { option(s) diff --git a/libs/common/service.go b/libs/service/service.go similarity index 99% rename from libs/common/service.go rename to libs/service/service.go index 8eee48138..9b3f36fff 100644 --- a/libs/common/service.go +++ b/libs/service/service.go @@ -1,4 +1,4 @@ -package common +package service import ( "errors" diff --git a/libs/common/service_test.go b/libs/service/service_test.go similarity index 98% rename from libs/common/service_test.go rename to libs/service/service_test.go index ef360a648..0f9962530 100644 --- a/libs/common/service_test.go +++ b/libs/service/service_test.go @@ -1,4 +1,4 @@ -package common +package service import ( "testing" diff --git a/lite2/rpc/client.go b/lite2/rpc/client.go index 8ef7f6ed5..06aa57eea 100644 --- a/lite2/rpc/client.go +++ b/lite2/rpc/client.go @@ -11,6 +11,7 @@ import ( "github.com/tendermint/tendermint/crypto/merkle" cmn "github.com/tendermint/tendermint/libs/common" + service "github.com/tendermint/tendermint/libs/service" lite "github.com/tendermint/tendermint/lite2" rpcclient "github.com/tendermint/tendermint/rpc/client" ctypes "github.com/tendermint/tendermint/rpc/core/types" @@ -21,7 +22,7 @@ import ( // Client is an RPC client, which uses lite#Client to verify data (if it can be // proved!). type Client struct { - cmn.BaseService + service.BaseService next rpcclient.Client lc *lite.Client @@ -37,7 +38,7 @@ func NewClient(next rpcclient.Client, lc *lite.Client) *Client { lc: lc, prt: defaultProofRuntime(), } - c.BaseService = *cmn.NewBaseService(nil, "Client", c) + c.BaseService = *service.NewBaseService(nil, "Client", c) return c } diff --git a/mempool/clist_mempool_test.go b/mempool/clist_mempool_test.go index 5efc4ea72..ae73e9cc5 100644 --- a/mempool/clist_mempool_test.go +++ b/mempool/clist_mempool_test.go @@ -22,9 +22,9 @@ import ( abciserver "github.com/tendermint/tendermint/abci/server" abci "github.com/tendermint/tendermint/abci/types" cfg "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" tmrand "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/types" ) @@ -581,7 +581,7 @@ func newRemoteApp( app abci.Application, ) ( clientCreator proxy.ClientCreator, - server cmn.Service, + server service.Service, ) { clientCreator = proxy.NewRemoteClientCreator(addr, "socket", true) diff --git a/node/node.go b/node/node.go index 174267b58..00754002a 100644 --- a/node/node.go +++ b/node/node.go @@ -25,9 +25,9 @@ import ( cs "github.com/tendermint/tendermint/consensus" "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/evidence" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" tmpubsub "github.com/tendermint/tendermint/libs/pubsub" + "github.com/tendermint/tendermint/libs/service" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p/pex" @@ -169,7 +169,7 @@ func CustomReactors(reactors map[string]p2p.Reactor) Option { // Node is the highest level interface to a full Tendermint node. // It includes all configuration information and running services. type Node struct { - cmn.BaseService + service.BaseService // config config *cfg.Config @@ -736,7 +736,7 @@ func NewNode(config *cfg.Config, indexerService: indexerService, eventBus: eventBus, } - node.BaseService = *cmn.NewBaseService(logger, "Node", node) + node.BaseService = *service.NewBaseService(logger, "Node", node) for _, option := range options { option(node) @@ -745,7 +745,7 @@ func NewNode(config *cfg.Config, return node, nil } -// OnStart starts the Node. It implements cmn.Service. +// OnStart starts the Node. It implements service.Service. func (n *Node) OnStart() error { now := tmtime.Now() genTime := n.genesisDoc.GenesisTime @@ -802,7 +802,7 @@ func (n *Node) OnStart() error { return nil } -// OnStop stops the Node. It implements cmn.Service. +// OnStop stops the Node. It implements service.Service. func (n *Node) OnStop() { n.BaseService.OnStop() @@ -834,7 +834,7 @@ func (n *Node) OnStop() { } } - if pvsc, ok := n.privValidator.(cmn.Service); ok { + if pvsc, ok := n.privValidator.(service.Service); ok { pvsc.Stop() } diff --git a/p2p/base_reactor.go b/p2p/base_reactor.go index 3bccabd64..86b0d980a 100644 --- a/p2p/base_reactor.go +++ b/p2p/base_reactor.go @@ -1,7 +1,7 @@ package p2p import ( - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p/conn" ) @@ -13,7 +13,7 @@ import ( // // Peer#Send or Peer#TrySend should be used to send the message to a peer. type Reactor interface { - cmn.Service // Start, Stop + service.Service // Start, Stop // SetSwitch allows setting a switch. SetSwitch(*Switch) @@ -50,13 +50,13 @@ type Reactor interface { //-------------------------------------- type BaseReactor struct { - cmn.BaseService // Provides Start, Stop, .Quit - Switch *Switch + service.BaseService // Provides Start, Stop, .Quit + Switch *Switch } func NewBaseReactor(name string, impl Reactor) *BaseReactor { return &BaseReactor{ - BaseService: *cmn.NewBaseService(nil, name, impl), + BaseService: *service.NewBaseService(nil, name, impl), Switch: nil, } } diff --git a/p2p/conn/connection.go b/p2p/conn/connection.go index 77fca6648..6a5e9d110 100644 --- a/p2p/conn/connection.go +++ b/p2p/conn/connection.go @@ -19,6 +19,7 @@ import ( cmn "github.com/tendermint/tendermint/libs/common" flow "github.com/tendermint/tendermint/libs/flowrate" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" ) const ( @@ -72,7 +73,7 @@ channel's queue is full. Inbound message bytes are handled with an onReceive callback function. */ type MConnection struct { - cmn.BaseService + service.BaseService conn net.Conn bufConnReader *bufio.Reader @@ -197,7 +198,7 @@ func NewMConnectionWithConfig( mconn.channels = channels mconn.channelsIdx = channelsIdx - mconn.BaseService = *cmn.NewBaseService(nil, "MConnection", mconn) + mconn.BaseService = *service.NewBaseService(nil, "MConnection", mconn) // maxPacketMsgSize() is a bit heavy, so call just once mconn._maxPacketMsgSize = mconn.maxPacketMsgSize() diff --git a/p2p/mock/peer.go b/p2p/mock/peer.go index 7b8bc6869..b2db913b8 100644 --- a/p2p/mock/peer.go +++ b/p2p/mock/peer.go @@ -4,13 +4,13 @@ import ( "net" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p/conn" ) type Peer struct { - *cmn.BaseService + *service.BaseService ip net.IP id p2p.ID addr *p2p.NetAddress @@ -35,7 +35,7 @@ func NewPeer(ip net.IP) *Peer { addr: netAddr, kv: make(map[string]interface{}), } - mp.BaseService = cmn.NewBaseService(nil, "MockPeer", mp) + mp.BaseService = service.NewBaseService(nil, "MockPeer", mp) mp.Start() return mp } diff --git a/p2p/peer.go b/p2p/peer.go index 80be0db53..1454b1cc1 100644 --- a/p2p/peer.go +++ b/p2p/peer.go @@ -7,6 +7,7 @@ import ( cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" tmconn "github.com/tendermint/tendermint/p2p/conn" ) @@ -15,7 +16,7 @@ const metricsTickerDuration = 10 * time.Second // Peer is an interface representing a peer connected on a reactor. type Peer interface { - cmn.Service + service.Service FlushStop() ID() ID // peer's cryptographic ID @@ -97,7 +98,7 @@ func (pc peerConn) RemoteIP() net.IP { // // Before using a peer, you will need to perform a handshake on connection. type peer struct { - cmn.BaseService + service.BaseService // raw peerConn and the multiplex connection peerConn @@ -144,7 +145,7 @@ func newPeer( onPeerError, mConfig, ) - p.BaseService = *cmn.NewBaseService(nil, "Peer", p) + p.BaseService = *service.NewBaseService(nil, "Peer", p) for _, option := range options { option(p) } @@ -162,7 +163,7 @@ func (p *peer) String() string { } //--------------------------------------------------- -// Implements cmn.Service +// Implements service.Service // SetLogger implements BaseService. func (p *peer) SetLogger(l log.Logger) { diff --git a/p2p/peer_set_test.go b/p2p/peer_set_test.go index c6f3fa5aa..3273a59aa 100644 --- a/p2p/peer_set_test.go +++ b/p2p/peer_set_test.go @@ -8,12 +8,12 @@ import ( "github.com/stretchr/testify/assert" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) // mockPeer for testing the PeerSet type mockPeer struct { - cmn.BaseService + service.BaseService ip net.IP id ID } diff --git a/p2p/pex/addrbook.go b/p2p/pex/addrbook.go index 47adca31d..94576084c 100644 --- a/p2p/pex/addrbook.go +++ b/p2p/pex/addrbook.go @@ -17,6 +17,7 @@ import ( "github.com/tendermint/tendermint/crypto" cmn "github.com/tendermint/tendermint/libs/common" tmrand "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" ) @@ -30,7 +31,7 @@ const ( // peers to dial. // TODO: break this up? type AddrBook interface { - cmn.Service + service.Service // Add our own addresses so we don't later add ourselves AddOurAddress(*p2p.NetAddress) @@ -78,7 +79,7 @@ var _ AddrBook = (*addrBook)(nil) // addrBook - concurrency safe peer address manager. // Implements AddrBook. type addrBook struct { - cmn.BaseService + service.BaseService // accessed concurrently mtx sync.Mutex @@ -111,7 +112,7 @@ func NewAddrBook(filePath string, routabilityStrict bool) *addrBook { routabilityStrict: routabilityStrict, } am.init() - am.BaseService = *cmn.NewBaseService(nil, "AddrBook", am) + am.BaseService = *service.NewBaseService(nil, "AddrBook", am) return am } diff --git a/p2p/pex/pex_reactor.go b/p2p/pex/pex_reactor.go index d623100b3..77235741a 100644 --- a/p2p/pex/pex_reactor.go +++ b/p2p/pex/pex_reactor.go @@ -11,6 +11,7 @@ import ( amino "github.com/tendermint/go-amino" cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p/conn" ) @@ -141,7 +142,7 @@ func NewReactor(b AddrBook, config *ReactorConfig) *Reactor { // OnStart implements BaseService func (r *Reactor) OnStart() error { err := r.book.Start() - if err != nil && err != cmn.ErrAlreadyStarted { + if err != nil && err != service.ErrAlreadyStarted { return err } diff --git a/p2p/switch.go b/p2p/switch.go index 4d50e438c..587de39aa 100644 --- a/p2p/switch.go +++ b/p2p/switch.go @@ -11,6 +11,7 @@ import ( "github.com/tendermint/tendermint/config" cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p/conn" ) @@ -66,7 +67,7 @@ type PeerFilterFunc func(IPeerSet, Peer) error // or more `Channels`. So while sending outgoing messages is typically performed on the peer, // incoming messages are received on the reactor. type Switch struct { - cmn.BaseService + service.BaseService config *config.P2PConfig reactors map[string]Reactor @@ -125,7 +126,7 @@ func NewSwitch( // Ensure we have a completely undeterministic PRNG. sw.rng = rand.NewRand() - sw.BaseService = *cmn.NewBaseService(nil, "P2P Switch", sw) + sw.BaseService = *service.NewBaseService(nil, "P2P Switch", sw) for _, option := range options { option(sw) diff --git a/p2p/trust/metric.go b/p2p/trust/metric.go index 7a1e12421..b54415ed8 100644 --- a/p2p/trust/metric.go +++ b/p2p/trust/metric.go @@ -8,7 +8,7 @@ import ( "sync" "time" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) //--------------------------------------------------------------------------------------- @@ -33,7 +33,7 @@ type MetricHistoryJSON struct { // Metric - keeps track of peer reliability // See tendermint/docs/architecture/adr-006-trust-metric.md for details type Metric struct { - cmn.BaseService + service.BaseService // Mutex that protects the metric from concurrent access mtx sync.Mutex @@ -104,7 +104,7 @@ func NewMetricWithConfig(tmc MetricConfig) *Metric { // This metric has a perfect history so far tm.historyValue = 1.0 - tm.BaseService = *cmn.NewBaseService(nil, "Metric", tm) + tm.BaseService = *service.NewBaseService(nil, "Metric", tm) return tm } diff --git a/p2p/trust/store.go b/p2p/trust/store.go index f3ebeba88..c40f4ff2f 100644 --- a/p2p/trust/store.go +++ b/p2p/trust/store.go @@ -9,7 +9,7 @@ import ( "sync" "time" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" dbm "github.com/tendermint/tm-db" ) @@ -19,7 +19,7 @@ var trustMetricKey = []byte("trustMetricStore") // MetricStore - Manages all trust metrics for peers type MetricStore struct { - cmn.BaseService + service.BaseService // Maps a Peer.Key to that peer's TrustMetric peerMetrics map[string]*Metric @@ -44,7 +44,7 @@ func NewTrustMetricStore(db dbm.DB, tmc MetricConfig) *MetricStore { config: tmc, } - tms.BaseService = *cmn.NewBaseService(nil, "MetricStore", tms) + tms.BaseService = *service.NewBaseService(nil, "MetricStore", tms) return tms } diff --git a/privval/signer_dialer_endpoint.go b/privval/signer_dialer_endpoint.go index 359fcf15b..7336f64be 100644 --- a/privval/signer_dialer_endpoint.go +++ b/privval/signer_dialer_endpoint.go @@ -3,8 +3,8 @@ package privval import ( "time" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" ) const ( @@ -56,7 +56,7 @@ func NewSignerDialerEndpoint( maxConnRetries: defaultMaxDialRetries, } - sd.BaseService = *cmn.NewBaseService(logger, "SignerDialerEndpoint", sd) + sd.BaseService = *service.NewBaseService(logger, "SignerDialerEndpoint", sd) sd.signerEndpoint.timeoutReadWrite = defaultTimeoutReadWriteSeconds * time.Second return sd diff --git a/privval/signer_endpoint.go b/privval/signer_endpoint.go index d29e7a4b4..98c64fb89 100644 --- a/privval/signer_endpoint.go +++ b/privval/signer_endpoint.go @@ -8,7 +8,7 @@ import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) const ( @@ -16,7 +16,7 @@ const ( ) type signerEndpoint struct { - cmn.BaseService + service.BaseService connMtx sync.Mutex conn net.Conn diff --git a/privval/signer_listener_endpoint.go b/privval/signer_listener_endpoint.go index e25f18756..70a23181d 100644 --- a/privval/signer_listener_endpoint.go +++ b/privval/signer_listener_endpoint.go @@ -6,8 +6,8 @@ import ( "sync" "time" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" ) // SignerValidatorEndpointOption sets an optional parameter on the SocketVal. @@ -38,12 +38,12 @@ func NewSignerListenerEndpoint( timeoutAccept: defaultTimeoutAcceptSeconds * time.Second, } - sc.BaseService = *cmn.NewBaseService(logger, "SignerListenerEndpoint", sc) + sc.BaseService = *service.NewBaseService(logger, "SignerListenerEndpoint", sc) sc.signerEndpoint.timeoutReadWrite = defaultTimeoutReadWriteSeconds * time.Second return sc } -// OnStart implements cmn.Service. +// OnStart implements service.Service. func (sl *SignerListenerEndpoint) OnStart() error { sl.connectRequestCh = make(chan struct{}) sl.connectionAvailableCh = make(chan net.Conn) @@ -58,7 +58,7 @@ func (sl *SignerListenerEndpoint) OnStart() error { return nil } -// OnStop implements cmn.Service +// OnStop implements service.Service func (sl *SignerListenerEndpoint) OnStop() { sl.instanceMtx.Lock() defer sl.instanceMtx.Unlock() diff --git a/privval/signer_server.go b/privval/signer_server.go index 62dcc461c..242423b24 100644 --- a/privval/signer_server.go +++ b/privval/signer_server.go @@ -4,7 +4,7 @@ import ( "io" "sync" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/types" ) @@ -15,7 +15,7 @@ type ValidationRequestHandlerFunc func( chainID string) (SignerMessage, error) type SignerServer struct { - cmn.BaseService + service.BaseService endpoint *SignerDialerEndpoint chainID string @@ -33,18 +33,18 @@ func NewSignerServer(endpoint *SignerDialerEndpoint, chainID string, privVal typ validationRequestHandler: DefaultValidationRequestHandler, } - ss.BaseService = *cmn.NewBaseService(endpoint.Logger, "SignerServer", ss) + ss.BaseService = *service.NewBaseService(endpoint.Logger, "SignerServer", ss) return ss } -// OnStart implements cmn.Service. +// OnStart implements service.Service. func (ss *SignerServer) OnStart() error { go ss.serviceLoop() return nil } -// OnStop implements cmn.Service. +// OnStop implements service.Service. func (ss *SignerServer) OnStop() { ss.endpoint.Logger.Debug("SignerServer: OnStop calling Close") _ = ss.endpoint.Close() diff --git a/proxy/multi_app_conn.go b/proxy/multi_app_conn.go index b5897d8a5..44ff242b1 100644 --- a/proxy/multi_app_conn.go +++ b/proxy/multi_app_conn.go @@ -3,14 +3,14 @@ package proxy import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ) //----------------------------- // Tendermint's interface to the application consists of multiple connections type AppConns interface { - cmn.Service + service.Service Mempool() AppConnMempool Consensus() AppConnConsensus @@ -28,7 +28,7 @@ func NewAppConns(clientCreator ClientCreator) AppConns { // and manages their underlying abci clients // TODO: on app restart, clients must reboot together type multiAppConn struct { - cmn.BaseService + service.BaseService mempoolConn *appConnMempool consensusConn *appConnConsensus @@ -42,7 +42,7 @@ func NewMultiAppConn(clientCreator ClientCreator) *multiAppConn { multiAppConn := &multiAppConn{ clientCreator: clientCreator, } - multiAppConn.BaseService = *cmn.NewBaseService(nil, "multiAppConn", multiAppConn) + multiAppConn.BaseService = *service.NewBaseService(nil, "multiAppConn", multiAppConn) return multiAppConn } diff --git a/rpc/client/httpclient.go b/rpc/client/httpclient.go index 928c88a06..528c90446 100644 --- a/rpc/client/httpclient.go +++ b/rpc/client/httpclient.go @@ -14,6 +14,7 @@ import ( cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" tmpubsub "github.com/tendermint/tendermint/libs/pubsub" + "github.com/tendermint/tendermint/libs/service" ctypes "github.com/tendermint/tendermint/rpc/core/types" rpcclient "github.com/tendermint/tendermint/rpc/lib/client" "github.com/tendermint/tendermint/types" @@ -379,7 +380,7 @@ func (c *baseRPCClient) BroadcastEvidence(ev types.Evidence) (*ctypes.ResultBroa // WSEvents type WSEvents struct { - cmn.BaseService + service.BaseService cdc *amino.Codec remote string endpoint string @@ -398,11 +399,11 @@ func newWSEvents(cdc *amino.Codec, remote, endpoint string) *WSEvents { subscriptions: make(map[string]chan ctypes.ResultEvent), } - wsEvents.BaseService = *cmn.NewBaseService(nil, "WSEvents", wsEvents) + wsEvents.BaseService = *service.NewBaseService(nil, "WSEvents", wsEvents) return wsEvents } -// OnStart implements cmn.Service by starting WSClient and event loop. +// OnStart implements service.Service by starting WSClient and event loop. func (w *WSEvents) OnStart() error { w.ws = rpcclient.NewWSClient(w.remote, w.endpoint, rpcclient.OnReconnect(func() { // resubscribe immediately @@ -420,7 +421,7 @@ func (w *WSEvents) OnStart() error { return nil } -// OnStop implements cmn.Service by stopping WSClient. +// OnStop implements service.Service by stopping WSClient. func (w *WSEvents) OnStop() { _ = w.ws.Stop() } diff --git a/rpc/client/interface.go b/rpc/client/interface.go index b4752c03d..3d9cdc9f6 100644 --- a/rpc/client/interface.go +++ b/rpc/client/interface.go @@ -24,6 +24,7 @@ import ( "context" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" ctypes "github.com/tendermint/tendermint/rpc/core/types" "github.com/tendermint/tendermint/types" ) @@ -31,7 +32,7 @@ import ( // Client wraps most important rpc calls a client would make if you want to // listen for events, test if it also implements events.EventSwitch. type Client interface { - cmn.Service + service.Service ABCIClient EventsClient HistoryClient diff --git a/rpc/client/mock/client.go b/rpc/client/mock/client.go index 313820eb7..05379061e 100644 --- a/rpc/client/mock/client.go +++ b/rpc/client/mock/client.go @@ -18,6 +18,7 @@ import ( "reflect" cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/rpc/client" "github.com/tendermint/tendermint/rpc/core" ctypes "github.com/tendermint/tendermint/rpc/core/types" @@ -38,7 +39,7 @@ type Client struct { client.EventsClient client.EvidenceClient client.MempoolClient - cmn.Service + service.Service } var _ client.Client = Client{} diff --git a/rpc/lib/client/ws_client.go b/rpc/lib/client/ws_client.go index bde6e7168..e937ce37b 100644 --- a/rpc/lib/client/ws_client.go +++ b/rpc/lib/client/ws_client.go @@ -14,8 +14,8 @@ import ( metrics "github.com/rcrowley/go-metrics" amino "github.com/tendermint/go-amino" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/rand" + "github.com/tendermint/tendermint/libs/service" types "github.com/tendermint/tendermint/rpc/lib/types" ) @@ -74,7 +74,7 @@ type WSClient struct { // nolint: maligned // Send pings to server with this period. Must be less than readWait. If 0, no pings will be sent. pingPeriod time.Duration - cmn.BaseService + service.BaseService // Time between sending a ping and receiving a pong. See // https://godoc.org/github.com/rcrowley/go-metrics#Timer. @@ -110,7 +110,7 @@ func NewWSClient(remoteAddr, endpoint string, options ...func(*WSClient)) *WSCli // sentIDs: make(map[types.JSONRPCIntID]bool), } - c.BaseService = *cmn.NewBaseService(nil, "WSClient", c) + c.BaseService = *service.NewBaseService(nil, "WSClient", c) for _, option := range options { option(c) } @@ -162,7 +162,7 @@ func (c *WSClient) String() string { return fmt.Sprintf("WSClient{%s (%s)}", c.Address, c.Endpoint) } -// OnStart implements cmn.Service by dialing a server and creating read and +// OnStart implements service.Service by dialing a server and creating read and // write routines. func (c *WSClient) OnStart() error { err := c.dial() @@ -186,7 +186,7 @@ func (c *WSClient) OnStart() error { return nil } -// Stop overrides cmn.Service#Stop. There is no other way to wait until Quit +// Stop overrides service.Service#Stop. There is no other way to wait until Quit // channel is closed. func (c *WSClient) Stop() error { if err := c.BaseService.Stop(); err != nil { diff --git a/rpc/lib/server/ws_handler.go b/rpc/lib/server/ws_handler.go index 80bda39f3..548a244cb 100644 --- a/rpc/lib/server/ws_handler.go +++ b/rpc/lib/server/ws_handler.go @@ -14,8 +14,8 @@ import ( amino "github.com/tendermint/go-amino" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" types "github.com/tendermint/tendermint/rpc/lib/types" ) @@ -112,7 +112,7 @@ func (wm *WebsocketManager) WebsocketHandler(w http.ResponseWriter, r *http.Requ // // In case of an error, the connection is stopped. type wsConnection struct { - cmn.BaseService + service.BaseService remoteAddr string baseConn *websocket.Conn @@ -175,7 +175,7 @@ func NewWSConnection( option(wsc) } wsc.baseConn.SetReadLimit(wsc.readLimit) - wsc.BaseService = *cmn.NewBaseService(nil, "wsConnection", wsc) + wsc.BaseService = *service.NewBaseService(nil, "wsConnection", wsc) return wsc } @@ -227,7 +227,7 @@ func ReadLimit(readLimit int64) func(*wsConnection) { } } -// OnStart implements cmn.Service by starting the read and write routines. It +// OnStart implements service.Service by starting the read and write routines. It // blocks until there's some error. func (wsc *wsConnection) OnStart() error { wsc.writeChan = make(chan types.RPCResponse, wsc.writeChanCapacity) @@ -240,7 +240,7 @@ func (wsc *wsConnection) OnStart() error { return nil } -// OnStop implements cmn.Service by unsubscribing remoteAddr from all +// OnStop implements service.Service by unsubscribing remoteAddr from all // subscriptions. func (wsc *wsConnection) OnStop() { if wsc.onDisconnect != nil { diff --git a/state/txindex/indexer_service.go b/state/txindex/indexer_service.go index e1c424734..961269d23 100644 --- a/state/txindex/indexer_service.go +++ b/state/txindex/indexer_service.go @@ -3,7 +3,7 @@ package txindex import ( "context" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/types" ) @@ -15,7 +15,7 @@ const ( // IndexerService connects event bus and transaction indexer together in order // to index transactions coming from event bus. type IndexerService struct { - cmn.BaseService + service.BaseService idr TxIndexer eventBus *types.EventBus @@ -24,11 +24,11 @@ type IndexerService struct { // NewIndexerService returns a new service instance. func NewIndexerService(idr TxIndexer, eventBus *types.EventBus) *IndexerService { is := &IndexerService{idr: idr, eventBus: eventBus} - is.BaseService = *cmn.NewBaseService(nil, "IndexerService", is) + is.BaseService = *service.NewBaseService(nil, "IndexerService", is) return is } -// OnStart implements cmn.Service by subscribing for all transactions +// OnStart implements service.Service by subscribing for all transactions // and indexing them by events. func (is *IndexerService) OnStart() error { // Use SubscribeUnbuffered here to ensure both subscriptions does not get @@ -74,7 +74,7 @@ func (is *IndexerService) OnStart() error { return nil } -// OnStop implements cmn.Service by unsubscribing from all transactions. +// OnStop implements service.Service by unsubscribing from all transactions. func (is *IndexerService) OnStop() { if is.eventBus.IsRunning() { _ = is.eventBus.UnsubscribeAll(context.Background(), subscriber) diff --git a/types/encoding_helper.go b/types/encoding_helper.go index f825de8a6..a5c278938 100644 --- a/types/encoding_helper.go +++ b/types/encoding_helper.go @@ -1,13 +1,9 @@ package types -import ( - cmn "github.com/tendermint/tendermint/libs/common" -) - // cdcEncode returns nil if the input is nil, otherwise returns // cdc.MustMarshalBinaryBare(item) func cdcEncode(item interface{}) []byte { - if item != nil && !cmn.IsTypedNil(item) && !cmn.IsEmpty(item) { + if item != nil && !isTypedNil(item) && !isEmpty(item) { return cdc.MustMarshalBinaryBare(item) } return nil diff --git a/types/event_bus.go b/types/event_bus.go index 7200a9a70..1c838d13b 100644 --- a/types/event_bus.go +++ b/types/event_bus.go @@ -5,9 +5,9 @@ import ( "fmt" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" tmpubsub "github.com/tendermint/tendermint/libs/pubsub" + "github.com/tendermint/tendermint/libs/service" ) const defaultCapacity = 0 @@ -31,7 +31,7 @@ type Subscription interface { // are proxied to underlying pubsub server. All events must be published using // EventBus to ensure correct data types. type EventBus struct { - cmn.BaseService + service.BaseService pubsub *tmpubsub.Server } @@ -45,7 +45,7 @@ func NewEventBusWithBufferCapacity(cap int) *EventBus { // capacity could be exposed later if needed pubsub := tmpubsub.NewServer(tmpubsub.BufferCapacity(cap)) b := &EventBus{pubsub: pubsub} - b.BaseService = *cmn.NewBaseService(nil, "EventBus", b) + b.BaseService = *service.NewBaseService(nil, "EventBus", b) return b } diff --git a/libs/common/nil.go b/types/utils.go similarity index 88% rename from libs/common/nil.go rename to types/utils.go index 31f75f008..cec47e202 100644 --- a/libs/common/nil.go +++ b/types/utils.go @@ -1,4 +1,4 @@ -package common +package types import "reflect" @@ -7,7 +7,7 @@ import "reflect" // - https://dave.cheney.net/2017/08/09/typed-nils-in-go-2 // - https://groups.google.com/forum/#!topic/golang-nuts/wnH302gBa4I/discussion // - https://github.com/golang/go/issues/21538 -func IsTypedNil(o interface{}) bool { +func isTypedNil(o interface{}) bool { rv := reflect.ValueOf(o) switch rv.Kind() { case reflect.Chan, reflect.Func, reflect.Map, reflect.Ptr, reflect.Slice: @@ -18,7 +18,7 @@ func IsTypedNil(o interface{}) bool { } // Returns true if it has zero length. -func IsEmpty(o interface{}) bool { +func isEmpty(o interface{}) bool { rv := reflect.ValueOf(o) switch rv.Kind() { case reflect.Array, reflect.Chan, reflect.Map, reflect.Slice, reflect.String: