Browse Source

service: change stop interface (#7816)

pull/7844/head
Sam Kleinman 2 years ago
committed by GitHub
parent
commit
28d34d635c
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 126 additions and 223 deletions
  1. +1
    -3
      abci/client/grpc_client.go
  2. +1
    -3
      abci/client/socket_client.go
  3. +2
    -6
      internal/blocksync/pool.go
  4. +2
    -6
      internal/blocksync/reactor.go
  5. +3
    -11
      internal/consensus/common_test.go
  6. +1
    -5
      internal/consensus/reactor.go
  7. +1
    -3
      internal/consensus/replay_file.go
  8. +3
    -5
      internal/consensus/replay_test.go
  9. +5
    -20
      internal/consensus/state.go
  10. +1
    -1
      internal/consensus/ticker.go
  11. +4
    -10
      internal/consensus/wal.go
  12. +3
    -7
      internal/consensus/wal_generator.go
  13. +1
    -7
      internal/consensus/wal_test.go
  14. +2
    -1
      internal/evidence/reactor_test.go
  15. +2
    -3
      internal/mempool/reactor_test.go
  16. +1
    -3
      internal/p2p/conn/connection.go
  17. +4
    -2
      internal/p2p/p2ptest/network.go
  18. +9
    -9
      internal/p2p/router_test.go
  19. +3
    -2
      internal/p2p/transport_mconn.go
  20. +40
    -67
      internal/proxy/multi_app_conn.go
  21. +1
    -1
      internal/proxy/multi_app_conn_test.go
  22. +1
    -1
      internal/state/execution_test.go
  23. +1
    -1
      libs/events/events.go
  24. +20
    -24
      libs/service/service.go
  25. +4
    -4
      libs/service/service_test.go
  26. +2
    -8
      node/node.go
  27. +2
    -2
      node/node_test.go
  28. +3
    -3
      privval/signer_client.go
  29. +1
    -1
      privval/signer_client_test.go
  30. +2
    -4
      privval/signer_listener_endpoint_test.go

+ 1
- 3
abci/client/grpc_client.go View File

@ -162,9 +162,7 @@ func (cli *grpcClient) StopForError(err error) {
cli.mtx.Unlock() cli.mtx.Unlock()
cli.logger.Error("Stopping abci.grpcClient for error", "err", err) cli.logger.Error("Stopping abci.grpcClient for error", "err", err)
if err := cli.Stop(); err != nil {
cli.logger.Error("error stopping abci.grpcClient", "err", err)
}
cli.Stop()
} }
func (cli *grpcClient) Error() error { func (cli *grpcClient) Error() error {


+ 1
- 3
abci/client/socket_client.go View File

@ -555,7 +555,5 @@ func (cli *socketClient) stopForError(err error) {
cli.mtx.Unlock() cli.mtx.Unlock()
cli.logger.Info("Stopping abci.socketClient", "reason", err) cli.logger.Info("Stopping abci.socketClient", "reason", err)
if err := cli.Stop(); err != nil {
cli.logger.Error("error stopping abci.socketClient", "err", err)
}
cli.Stop()
} }

+ 2
- 6
internal/blocksync/pool.go View File

@ -235,9 +235,7 @@ func (pool *BlockPool) PopRequest() {
defer pool.mtx.Unlock() defer pool.mtx.Unlock()
if r := pool.requesters[pool.height]; r != nil { if r := pool.requesters[pool.height]; r != nil {
if err := r.Stop(); err != nil {
pool.logger.Error("error stopping requester", "err", err)
}
r.Stop()
delete(pool.requesters, pool.height) delete(pool.requesters, pool.height)
pool.height++ pool.height++
pool.lastAdvance = time.Now() pool.lastAdvance = time.Now()
@ -676,9 +674,7 @@ OUTER_LOOP:
case <-ctx.Done(): case <-ctx.Done():
return return
case <-bpr.pool.exitedCh: case <-bpr.pool.exitedCh:
if err := bpr.Stop(); err != nil {
bpr.logger.Error("error stopped requester", "err", err)
}
bpr.Stop()
return return
case peerID := <-bpr.redoCh: case peerID := <-bpr.redoCh:
if peerID == bpr.peerID { if peerID == bpr.peerID {


+ 2
- 6
internal/blocksync/reactor.go View File

@ -187,9 +187,7 @@ func (r *Reactor) OnStart(ctx context.Context) error {
// blocking until they all exit. // blocking until they all exit.
func (r *Reactor) OnStop() { func (r *Reactor) OnStop() {
if r.blockSync.IsSet() { if r.blockSync.IsSet() {
if err := r.pool.Stop(); err != nil {
r.logger.Error("failed to stop pool", "err", err)
}
r.pool.Stop()
} }
// wait for the poolRoutine and requestRoutine goroutines to gracefully exit // wait for the poolRoutine and requestRoutine goroutines to gracefully exit
@ -485,9 +483,7 @@ FOR_LOOP:
continue continue
} }
if err := r.pool.Stop(); err != nil {
r.logger.Error("failed to stop pool", "err", err)
}
r.pool.Stop()
r.blockSync.UnSet() r.blockSync.UnSet()


+ 3
- 11
internal/consensus/common_test.go View File

@ -915,15 +915,9 @@ type mockTicker struct {
fired bool fired bool
} }
func (m *mockTicker) Start(context.Context) error {
return nil
}
func (m *mockTicker) Stop() error {
return nil
}
func (m *mockTicker) IsRunning() bool { return false }
func (m *mockTicker) Start(context.Context) error { return nil }
func (m *mockTicker) Stop() {}
func (m *mockTicker) IsRunning() bool { return false }
func (m *mockTicker) ScheduleTimeout(ti timeoutInfo) { func (m *mockTicker) ScheduleTimeout(ti timeoutInfo) {
m.mtx.Lock() m.mtx.Lock()
@ -941,8 +935,6 @@ func (m *mockTicker) Chan() <-chan timeoutInfo {
return m.c return m.c
} }
func (*mockTicker) SetLogger(log.Logger) {}
func newPersistentKVStore(t *testing.T, logger log.Logger) abci.Application { func newPersistentKVStore(t *testing.T, logger log.Logger) abci.Application {
t.Helper() t.Helper()


+ 1
- 5
internal/consensus/reactor.go View File

@ -219,11 +219,7 @@ func (r *Reactor) OnStart(ctx context.Context) error {
func (r *Reactor) OnStop() { func (r *Reactor) OnStop() {
r.unsubscribeFromBroadcastEvents() r.unsubscribeFromBroadcastEvents()
if err := r.state.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
r.logger.Error("failed to stop consensus state", "err", err)
}
}
r.state.Stop()
if !r.WaitSync() { if !r.WaitSync() {
r.state.Wait() r.state.Wait()


+ 1
- 3
internal/consensus/replay_file.go View File

@ -142,9 +142,7 @@ func newPlayback(fileName string, fp *os.File, cs *State, genState sm.State) *pl
// go back count steps by resetting the state and running (pb.count - count) steps // go back count steps by resetting the state and running (pb.count - count) steps
func (pb *playback) replayReset(ctx context.Context, count int, newStepSub eventbus.Subscription) error { func (pb *playback) replayReset(ctx context.Context, count int, newStepSub eventbus.Subscription) error {
if err := pb.cs.Stop(); err != nil {
return err
}
pb.cs.Stop()
pb.cs.Wait() pb.cs.Wait()
newCS := NewState(ctx, pb.cs.logger, pb.cs.config, pb.genesisState.Copy(), pb.cs.blockExec, newCS := NewState(ctx, pb.cs.logger, pb.cs.config, pb.genesisState.Copy(), pb.cs.blockExec,


+ 3
- 5
internal/consensus/replay_test.go View File

@ -79,9 +79,7 @@ func startNewStateAndWaitForBlock(ctx context.Context, t *testing.T, consensusRe
require.NoError(t, cs.Start(ctx)) require.NoError(t, cs.Start(ctx))
defer func() { defer func() {
if err := cs.Stop(); err != nil {
t.Error(err)
}
cs.Stop()
}() }()
t.Cleanup(cs.Wait) t.Cleanup(cs.Wait)
// This is just a signal that we haven't halted; its not something contained // This is just a signal that we haven't halted; its not something contained
@ -208,7 +206,7 @@ LOOP:
startNewStateAndWaitForBlock(ctx, t, consensusReplayConfig, cs.Height, blockDB, stateStore) startNewStateAndWaitForBlock(ctx, t, consensusReplayConfig, cs.Height, blockDB, stateStore)
// stop consensus state and transactions sender (initFn) // stop consensus state and transactions sender (initFn)
cs.Stop() //nolint:errcheck // Logging this error causes failure
cs.Stop()
cancel() cancel()
// if we reached the required height, exit // if we reached the required height, exit
@ -292,7 +290,7 @@ func (w *crashingWAL) SearchForEndHeight(
} }
func (w *crashingWAL) Start(ctx context.Context) error { return w.next.Start(ctx) } func (w *crashingWAL) Start(ctx context.Context) error { return w.next.Start(ctx) }
func (w *crashingWAL) Stop() error { return w.next.Stop() }
func (w *crashingWAL) Stop() { w.next.Stop() }
func (w *crashingWAL) Wait() { w.next.Wait() } func (w *crashingWAL) Wait() { w.next.Wait() }
//------------------------------------------------------------------------------------------ //------------------------------------------------------------------------------------------


+ 5
- 20
internal/consensus/state.go View File

@ -396,10 +396,7 @@ func (cs *State) OnStart(ctx context.Context) error {
cs.logger.Error("the WAL file is corrupted; attempting repair", "err", err) cs.logger.Error("the WAL file is corrupted; attempting repair", "err", err)
// 1) prep work // 1) prep work
if err := cs.wal.Stop(); err != nil {
return err
}
cs.wal.Stop()
repairAttempted = true repairAttempted = true
@ -494,19 +491,11 @@ func (cs *State) OnStop() {
close(cs.onStopCh) close(cs.onStopCh)
if cs.evsw.IsRunning() { if cs.evsw.IsRunning() {
if err := cs.evsw.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
cs.logger.Error("failed trying to stop eventSwitch", "error", err)
}
}
cs.evsw.Stop()
} }
if cs.timeoutTicker.IsRunning() { if cs.timeoutTicker.IsRunning() {
if err := cs.timeoutTicker.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
cs.logger.Error("failed trying to stop timeoutTicket", "error", err)
}
}
cs.timeoutTicker.Stop()
} }
// WAL is stopped in receiveRoutine. // WAL is stopped in receiveRoutine.
} }
@ -515,6 +504,7 @@ func (cs *State) OnStop() {
// NOTE: be sure to Stop() the event switch and drain // NOTE: be sure to Stop() the event switch and drain
// any event channels or this may deadlock // any event channels or this may deadlock
func (cs *State) Wait() { func (cs *State) Wait() {
cs.evsw.Wait()
<-cs.done <-cs.done
} }
@ -840,12 +830,7 @@ func (cs *State) receiveRoutine(ctx context.Context, maxSteps int) {
// priv_val tracks LastSig // priv_val tracks LastSig
// close wal now that we're done writing to it // close wal now that we're done writing to it
if err := cs.wal.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
cs.logger.Error("failed trying to stop WAL", "error", err)
}
}
cs.wal.Stop()
cs.wal.Wait() cs.wal.Wait()
close(cs.done) close(cs.done)
} }


+ 1
- 1
internal/consensus/ticker.go View File

@ -17,7 +17,7 @@ var (
// The timeoutInfo.Duration may be non-positive. // The timeoutInfo.Duration may be non-positive.
type TimeoutTicker interface { type TimeoutTicker interface {
Start(context.Context) error Start(context.Context) error
Stop() error
Stop()
IsRunning() bool IsRunning() bool
Chan() <-chan timeoutInfo // on which to receive a timeout Chan() <-chan timeoutInfo // on which to receive a timeout
ScheduleTimeout(ti timeoutInfo) // reset the timer ScheduleTimeout(ti timeoutInfo) // reset the timer


+ 4
- 10
internal/consensus/wal.go View File

@ -67,7 +67,7 @@ type WAL interface {
// service methods // service methods
Start(context.Context) error Start(context.Context) error
Stop() error
Stop()
Wait() Wait()
} }
@ -164,15 +164,9 @@ func (wal *BaseWAL) FlushAndSync() error {
func (wal *BaseWAL) OnStop() { func (wal *BaseWAL) OnStop() {
wal.flushTicker.Stop() wal.flushTicker.Stop()
if err := wal.FlushAndSync(); err != nil { if err := wal.FlushAndSync(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
wal.logger.Error("error on flush data to disk", "error", err)
}
}
if err := wal.group.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
wal.logger.Error("error trying to stop wal", "error", err)
}
wal.logger.Error("error on flush data to disk", "error", err)
} }
wal.group.Stop()
wal.group.Close() wal.group.Close()
} }
@ -438,5 +432,5 @@ func (nilWAL) SearchForEndHeight(height int64, options *WALSearchOptions) (rd io
return nil, false, nil return nil, false, nil
} }
func (nilWAL) Start(context.Context) error { return nil } func (nilWAL) Start(context.Context) error { return nil }
func (nilWAL) Stop() error { return nil }
func (nilWAL) Stop() {}
func (nilWAL) Wait() {} func (nilWAL) Wait() {}

+ 3
- 7
internal/consensus/wal_generator.go View File

@ -102,14 +102,10 @@ func WALGenerateNBlocks(ctx context.Context, t *testing.T, logger log.Logger, wr
select { select {
case <-numBlocksWritten: case <-numBlocksWritten:
if err := consensusState.Stop(); err != nil {
t.Error(err)
}
consensusState.Stop()
return nil return nil
case <-time.After(1 * time.Minute): case <-time.After(1 * time.Minute):
if err := consensusState.Stop(); err != nil {
t.Error(err)
}
consensusState.Stop()
return fmt.Errorf("waited too long for tendermint to produce %d blocks (grep logs for `wal_generator`)", numBlocks) return fmt.Errorf("waited too long for tendermint to produce %d blocks (grep logs for `wal_generator`)", numBlocks)
} }
} }
@ -219,5 +215,5 @@ func (w *byteBufferWAL) SearchForEndHeight(
} }
func (w *byteBufferWAL) Start(context.Context) error { return nil } func (w *byteBufferWAL) Start(context.Context) error { return nil }
func (w *byteBufferWAL) Stop() error { return nil }
func (w *byteBufferWAL) Stop() {}
func (w *byteBufferWAL) Wait() {} func (w *byteBufferWAL) Wait() {}

+ 1
- 7
internal/consensus/wal_test.go View File

@ -3,7 +3,6 @@ package consensus
import ( import (
"bytes" "bytes"
"context" "context"
"errors"
"path/filepath" "path/filepath"
"testing" "testing"
@ -17,7 +16,6 @@ import (
"github.com/tendermint/tendermint/internal/consensus/types" "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/libs/autofile" "github.com/tendermint/tendermint/internal/libs/autofile"
"github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
tmtime "github.com/tendermint/tendermint/libs/time" tmtime "github.com/tendermint/tendermint/libs/time"
tmtypes "github.com/tendermint/tendermint/types" tmtypes "github.com/tendermint/tendermint/types"
) )
@ -191,11 +189,7 @@ func TestWALPeriodicSync(t *testing.T) {
require.NoError(t, wal.Start(ctx)) require.NoError(t, wal.Start(ctx))
t.Cleanup(func() { t.Cleanup(func() {
if err := wal.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
t.Error(err)
}
}
wal.Stop()
wal.Wait() wal.Wait()
}) })


+ 2
- 1
internal/evidence/reactor_test.go View File

@ -110,7 +110,8 @@ func setup(ctx context.Context, t *testing.T, stateStores []sm.Store, chBuf uint
t.Cleanup(func() { t.Cleanup(func() {
for _, r := range rts.reactors { for _, r := range rts.reactors {
if r.IsRunning() { if r.IsRunning() {
require.NoError(t, r.Stop())
r.Stop()
r.Wait()
require.False(t, r.IsRunning()) require.False(t, r.IsRunning())
} }
} }


+ 2
- 3
internal/mempool/reactor_test.go View File

@ -96,7 +96,7 @@ func setupReactors(ctx context.Context, t *testing.T, numNodes int, chBuf uint)
t.Cleanup(func() { t.Cleanup(func() {
for nodeID := range rts.reactors { for nodeID := range rts.reactors {
if rts.reactors[nodeID].IsRunning() { if rts.reactors[nodeID].IsRunning() {
require.NoError(t, rts.reactors[nodeID].Stop())
rts.reactors[nodeID].Stop()
rts.reactors[nodeID].Wait() rts.reactors[nodeID].Wait()
require.False(t, rts.reactors[nodeID].IsRunning()) require.False(t, rts.reactors[nodeID].IsRunning())
} }
@ -184,8 +184,7 @@ func TestReactorBroadcastDoesNotPanic(t *testing.T) {
}() }()
} }
err := primaryReactor.Stop()
require.NoError(t, err)
primaryReactor.Stop()
wg.Wait() wg.Wait()
} }


+ 1
- 3
internal/p2p/conn/connection.go View File

@ -294,9 +294,7 @@ func (c *MConnection) _recover(ctx context.Context) {
} }
func (c *MConnection) stopForError(ctx context.Context, r interface{}) { func (c *MConnection) stopForError(ctx context.Context, r interface{}) {
if err := c.Stop(); err != nil {
c.logger.Error("error stopping connection", "err", err)
}
c.Stop()
if atomic.CompareAndSwapUint32(&c.errored, 0, 1) { if atomic.CompareAndSwapUint32(&c.errored, 0, 1) {
if c.onError != nil { if c.onError != nil {


+ 4
- 2
internal/p2p/p2ptest/network.go View File

@ -208,7 +208,8 @@ func (n *Network) Remove(ctx context.Context, t *testing.T, id types.NodeID) {
require.NoError(t, node.Transport.Close()) require.NoError(t, node.Transport.Close())
node.cancel() node.cancel()
if node.Router.IsRunning() { if node.Router.IsRunning() {
require.NoError(t, node.Router.Stop())
node.Router.Stop()
node.Router.Wait()
} }
for _, sub := range subs { for _, sub := range subs {
@ -275,7 +276,8 @@ func (n *Network) MakeNode(ctx context.Context, t *testing.T, opts NodeOptions)
t.Cleanup(func() { t.Cleanup(func() {
if router.IsRunning() { if router.IsRunning() {
require.NoError(t, router.Stop())
router.Stop()
router.Wait()
} }
require.NoError(t, transport.Close()) require.NoError(t, transport.Close())
cancel() cancel()


+ 9
- 9
internal/p2p/router_test.go View File

@ -438,7 +438,7 @@ func TestRouter_AcceptPeers(t *testing.T) {
} }
} }
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
}) })
@ -478,7 +478,7 @@ func TestRouter_AcceptPeers_Error(t *testing.T) {
require.NoError(t, router.Start(ctx)) require.NoError(t, router.Start(ctx))
time.Sleep(time.Second) time.Sleep(time.Second)
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
} }
@ -516,7 +516,7 @@ func TestRouter_AcceptPeers_ErrorEOF(t *testing.T) {
require.NoError(t, router.Start(ctx)) require.NoError(t, router.Start(ctx))
time.Sleep(time.Second) time.Sleep(time.Second)
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
} }
@ -573,7 +573,7 @@ func TestRouter_AcceptPeers_HeadOfLineBlocking(t *testing.T) {
close(closeCh) close(closeCh)
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
} }
@ -687,7 +687,7 @@ func TestRouter_DialPeers(t *testing.T) {
} }
} }
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
}) })
@ -778,7 +778,7 @@ func TestRouter_DialPeers_Parallel(t *testing.T) {
close(closeCh) close(closeCh)
time.Sleep(500 * time.Millisecond) time.Sleep(500 * time.Millisecond)
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
} }
@ -845,7 +845,7 @@ func TestRouter_EvictPeers(t *testing.T) {
Status: p2p.PeerStatusDown, Status: p2p.PeerStatusDown,
}) })
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
} }
@ -895,7 +895,7 @@ func TestRouter_ChannelCompatability(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.NoError(t, router.Start(ctx)) require.NoError(t, router.Start(ctx))
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
require.NoError(t, router.Stop())
router.Stop()
require.Empty(t, peerManager.Peers()) require.Empty(t, peerManager.Peers())
mockConnection.AssertExpectations(t) mockConnection.AssertExpectations(t)
@ -964,6 +964,6 @@ func TestRouter_DontSendOnInvalidChannel(t *testing.T) {
Message: &p2ptest.Message{Value: "Hi"}, Message: &p2ptest.Message{Value: "Hi"},
})) }))
require.NoError(t, router.Stop())
router.Stop()
mockTransport.AssertExpectations(t) mockTransport.AssertExpectations(t)
} }

+ 3
- 2
internal/p2p/transport_mconn.go View File

@ -478,12 +478,13 @@ func (c *mConnConnection) RemoteEndpoint() Endpoint {
func (c *mConnConnection) Close() error { func (c *mConnConnection) Close() error {
var err error var err error
c.closeOnce.Do(func() { c.closeOnce.Do(func() {
defer close(c.doneCh)
if c.mconn != nil && c.mconn.IsRunning() { if c.mconn != nil && c.mconn.IsRunning() {
err = c.mconn.Stop()
c.mconn.Stop()
} else { } else {
err = c.conn.Close() err = c.conn.Close()
} }
close(c.doneCh)
}) })
return err return err
} }

+ 40
- 67
internal/proxy/multi_app_conn.go View File

@ -2,7 +2,6 @@ package proxy
import ( import (
"context" "context"
"errors"
"fmt" "fmt"
"os" "os"
"syscall" "syscall"
@ -67,7 +66,7 @@ type multiAppConn struct {
// of reasonable lifecycle witout needing an explicit stop method. // of reasonable lifecycle witout needing an explicit stop method.
type stoppableClient interface { type stoppableClient interface {
abciclient.Client abciclient.Client
Stop() error
Stop()
} }
// NewMultiAppConn makes all necessary abci connections to the application. // NewMultiAppConn makes all necessary abci connections to the application.
@ -81,53 +80,42 @@ func NewMultiAppConn(clientCreator abciclient.Creator, logger log.Logger, metric
return multiAppConn return multiAppConn
} }
func (app *multiAppConn) Mempool() AppConnMempool {
return app.mempoolConn
}
func (app *multiAppConn) Consensus() AppConnConsensus {
return app.consensusConn
}
func (app *multiAppConn) Query() AppConnQuery {
return app.queryConn
}
func (app *multiAppConn) Snapshot() AppConnSnapshot {
return app.snapshotConn
}
func (app *multiAppConn) Mempool() AppConnMempool { return app.mempoolConn }
func (app *multiAppConn) Consensus() AppConnConsensus { return app.consensusConn }
func (app *multiAppConn) Query() AppConnQuery { return app.queryConn }
func (app *multiAppConn) Snapshot() AppConnSnapshot { return app.snapshotConn }
func (app *multiAppConn) OnStart(ctx context.Context) error { func (app *multiAppConn) OnStart(ctx context.Context) error {
c, err := app.abciClientFor(ctx, connQuery)
var err error
defer func() {
if err != nil {
app.stopAllClients()
}
}()
app.queryConnClient, err = app.abciClientFor(ctx, connQuery)
if err != nil { if err != nil {
return err return err
} }
app.queryConnClient = c.(stoppableClient)
app.queryConn = NewAppConnQuery(c, app.metrics)
app.queryConn = NewAppConnQuery(app.queryConnClient, app.metrics)
c, err = app.abciClientFor(ctx, connSnapshot)
app.snapshotConnClient, err = app.abciClientFor(ctx, connSnapshot)
if err != nil { if err != nil {
app.stopAllClients()
return err return err
} }
app.snapshotConnClient = c.(stoppableClient)
app.snapshotConn = NewAppConnSnapshot(c, app.metrics)
app.snapshotConn = NewAppConnSnapshot(app.snapshotConnClient, app.metrics)
c, err = app.abciClientFor(ctx, connMempool)
app.mempoolConnClient, err = app.abciClientFor(ctx, connMempool)
if err != nil { if err != nil {
app.stopAllClients()
return err return err
} }
app.mempoolConnClient = c.(stoppableClient)
app.mempoolConn = NewAppConnMempool(c, app.metrics)
app.mempoolConn = NewAppConnMempool(app.mempoolConnClient, app.metrics)
c, err = app.abciClientFor(ctx, connConsensus)
app.consensusConnClient, err = app.abciClientFor(ctx, connConsensus)
if err != nil { if err != nil {
app.stopAllClients()
return err return err
} }
app.consensusConnClient = c.(stoppableClient)
app.consensusConn = NewAppConnConsensus(c, app.metrics)
app.consensusConn = NewAppConnConsensus(app.consensusConnClient, app.metrics)
// Kill Tendermint if the ABCI application crashes. // Kill Tendermint if the ABCI application crashes.
app.startWatchersForClientErrorToKillTendermint(ctx) app.startWatchersForClientErrorToKillTendermint(ctx)
@ -135,9 +123,7 @@ func (app *multiAppConn) OnStart(ctx context.Context) error {
return nil return nil
} }
func (app *multiAppConn) OnStop() {
app.stopAllClients()
}
func (app *multiAppConn) OnStop() { app.stopAllClients() }
func (app *multiAppConn) startWatchersForClientErrorToKillTendermint(ctx context.Context) { func (app *multiAppConn) startWatchersForClientErrorToKillTendermint(ctx context.Context) {
// this function starts a number of threads (per abci client) // this function starts a number of threads (per abci client)
@ -154,12 +140,10 @@ func (app *multiAppConn) startWatchersForClientErrorToKillTendermint(ctx context
} }
} }
type op struct {
for _, client := range []struct {
connClient stoppableClient connClient stoppableClient
name string name string
}
for _, client := range []op{
}{
{ {
connClient: app.consensusConnClient, connClient: app.consensusConnClient,
name: connConsensus, name: connConsensus,
@ -190,47 +174,36 @@ func (app *multiAppConn) startWatchersForClientErrorToKillTendermint(ctx context
} }
func (app *multiAppConn) stopAllClients() { func (app *multiAppConn) stopAllClients() {
if app.consensusConnClient != nil {
if err := app.consensusConnClient.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
app.logger.Error("error while stopping consensus client", "error", err)
}
}
}
if app.mempoolConnClient != nil {
if err := app.mempoolConnClient.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
app.logger.Error("error while stopping mempool client", "error", err)
}
}
}
if app.queryConnClient != nil {
if err := app.queryConnClient.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
app.logger.Error("error while stopping query client", "error", err)
}
}
}
if app.snapshotConnClient != nil {
if err := app.snapshotConnClient.Stop(); err != nil {
if !errors.Is(err, service.ErrAlreadyStopped) {
app.logger.Error("error while stopping snapshot client", "error", err)
}
for _, client := range []stoppableClient{
app.consensusConnClient,
app.mempoolConnClient,
app.queryConnClient,
app.snapshotConnClient,
} {
if client != nil {
client.Stop()
} }
} }
} }
func (app *multiAppConn) abciClientFor(ctx context.Context, conn string) (abciclient.Client, error) {
func (app *multiAppConn) abciClientFor(ctx context.Context, conn string) (stoppableClient, error) {
c, err := app.clientCreator(app.logger.With( c, err := app.clientCreator(app.logger.With(
"module", "abci-client", "module", "abci-client",
"connection", conn)) "connection", conn))
if err != nil { if err != nil {
return nil, fmt.Errorf("error creating ABCI client (%s connection): %w", conn, err) return nil, fmt.Errorf("error creating ABCI client (%s connection): %w", conn, err)
} }
if err := c.Start(ctx); err != nil { if err := c.Start(ctx); err != nil {
return nil, fmt.Errorf("error starting ABCI client (%s connection): %w", conn, err) return nil, fmt.Errorf("error starting ABCI client (%s connection): %w", conn, err)
} }
return c, nil
client, ok := c.(stoppableClient)
if !ok {
return nil, fmt.Errorf("%T is not a stoppable client", c)
}
return client, nil
} }
func kill() error { func kill() error {


+ 1
- 1
internal/proxy/multi_app_conn_test.go View File

@ -23,7 +23,7 @@ type noopStoppableClientImpl struct {
count int count int
} }
func (c *noopStoppableClientImpl) Stop() error { c.count++; return nil }
func (c *noopStoppableClientImpl) Stop() { c.count++ }
func TestAppConns_Start_Stop(t *testing.T) { func TestAppConns_Start_Stop(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())


+ 1
- 1
internal/state/execution_test.go View File

@ -439,7 +439,7 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
eventBus := eventbus.NewDefault(logger) eventBus := eventbus.NewDefault(logger)
err = eventBus.Start(ctx) err = eventBus.Start(ctx)
require.NoError(t, err) require.NoError(t, err)
defer eventBus.Stop() //nolint:errcheck // ignore for tests
defer eventBus.Stop()
blockExec.SetEventBus(eventBus) blockExec.SetEventBus(eventBus)


+ 1
- 1
libs/events/events.go View File

@ -47,7 +47,7 @@ type Fireable interface {
type EventSwitch interface { type EventSwitch interface {
service.Service service.Service
Fireable Fireable
Stop() error
Stop()
AddListenerForEvent(listenerID, eventValue string, cb EventCallback) error AddListenerForEvent(listenerID, eventValue string, cb EventCallback) error
RemoveListenerForEvent(event string, listenerID string) RemoveListenerForEvent(event string, listenerID string)


+ 20
- 24
libs/service/service.go View File

@ -9,15 +9,9 @@ import (
) )
var ( var (
// ErrAlreadyStarted is returned when somebody tries to start an already
// running service.
ErrAlreadyStarted = errors.New("already started")
// ErrAlreadyStopped is returned when somebody tries to stop an already
// errAlreadyStopped is returned when somebody tries to stop an already
// stopped service (without resetting it). // stopped service (without resetting it).
ErrAlreadyStopped = errors.New("already stopped")
// ErrNotStarted is returned when somebody tries to stop a not running
// service.
ErrNotStarted = errors.New("not started")
errAlreadyStopped = errors.New("already stopped")
) )
// Service defines a service that can be started, stopped, and reset. // Service defines a service that can be started, stopped, and reset.
@ -46,13 +40,14 @@ type Implementation interface {
/* /*
Classical-inheritance-style service declarations. Services can be started, then Classical-inheritance-style service declarations. Services can be started, then
stopped, then optionally restarted.
stopped, but cannot be restarted.
Users can override the OnStart/OnStop methods. In the absence of errors, these
Users must implement OnStart/OnStop methods. In the absence of errors, these
methods are guaranteed to be called at most once. If OnStart returns an error, methods are guaranteed to be called at most once. If OnStart returns an error,
service won't be marked as started, so the user can call Start again. service won't be marked as started, so the user can call Start again.
It is safe, but an error, to call Stop without calling Start first.
The BaseService implementation ensures that the OnStop method is
called after the context passed to Start is canceled.
Typical usage: Typical usage:
@ -74,7 +69,7 @@ Typical usage:
// start subroutines, etc. // start subroutines, etc.
} }
func (fs *FooService) OnStop() error {
func (fs *FooService) OnStop() {
// close/destroy private fields // close/destroy private fields
// stop subroutines, etc. // stop subroutines, etc.
} }
@ -99,20 +94,20 @@ func NewBaseService(logger log.Logger, name string, impl Implementation) *BaseSe
} }
} }
// Start starts the Service and calls its OnStart method. An error will be
// returned if the service is already running or stopped. To restart a
// stopped service, call Reset.
// Start starts the Service and calls its OnStart method. An error
// will be returned if the service is stopped, but not if it is
// already running.
func (bs *BaseService) Start(ctx context.Context) error { func (bs *BaseService) Start(ctx context.Context) error {
bs.mtx.Lock() bs.mtx.Lock()
defer bs.mtx.Unlock() defer bs.mtx.Unlock()
if bs.quit != nil { if bs.quit != nil {
return ErrAlreadyStarted
return nil
} }
select { select {
case <-bs.quit: case <-bs.quit:
return ErrAlreadyStopped
return errAlreadyStopped
default: default:
bs.logger.Info("starting service", "service", bs.name, "impl", bs.name) bs.logger.Info("starting service", "service", bs.name, "impl", bs.name)
if err := bs.impl.OnStart(ctx); err != nil { if err := bs.impl.OnStart(ctx); err != nil {
@ -132,7 +127,7 @@ func (bs *BaseService) Start(ctx context.Context) error {
// this means stop was called manually // this means stop was called manually
return return
case <-ctx.Done(): case <-ctx.Done():
_ = bs.Stop()
bs.Stop()
} }
bs.logger.Info("stopped service", bs.logger.Info("stopped service",
@ -143,25 +138,26 @@ func (bs *BaseService) Start(ctx context.Context) error {
} }
} }
// Stop implements Service by calling OnStop (if defined) and closing quit
// channel. An error will be returned if the service is already stopped.
func (bs *BaseService) Stop() error {
// Stop manually terminates the service by calling OnStop method from
// the implementation and releases all resources related to the
// service.
func (bs *BaseService) Stop() {
bs.mtx.Lock() bs.mtx.Lock()
defer bs.mtx.Unlock() defer bs.mtx.Unlock()
if bs.quit == nil { if bs.quit == nil {
return ErrNotStarted
return
} }
select { select {
case <-bs.quit: case <-bs.quit:
return ErrAlreadyStopped
return
default: default:
bs.logger.Info("stopping service", "service", bs.name) bs.logger.Info("stopping service", "service", bs.name)
bs.impl.OnStop() bs.impl.OnStop()
bs.cancel() bs.cancel()
return nil
return
} }
} }


+ 4
- 4
libs/service/service_test.go View File

@ -96,7 +96,7 @@ func TestBaseService(t *testing.T) {
require.True(t, ts.isStarted()) require.True(t, ts.isStarted())
require.NoError(t, ts.Stop())
ts.Stop()
require.True(t, ts.isStopped()) require.True(t, ts.isStopped())
require.False(t, ts.IsRunning()) require.False(t, ts.IsRunning())
}) })
@ -107,10 +107,10 @@ func TestBaseService(t *testing.T) {
require.NoError(t, ts.Start(ctx)) require.NoError(t, ts.Start(ctx))
require.True(t, ts.isStarted()) require.True(t, ts.isStarted())
require.NoError(t, ts.Stop())
ts.Stop()
require.True(t, ts.isStopped()) require.True(t, ts.isStopped())
require.False(t, ts.isMultiStopped()) require.False(t, ts.isMultiStopped())
require.Error(t, ts.Stop())
ts.Stop()
require.False(t, ts.isMultiStopped()) require.False(t, ts.isMultiStopped())
}) })
t.Run("MultiThreaded", func(t *testing.T) { t.Run("MultiThreaded", func(t *testing.T) {
@ -123,7 +123,7 @@ func TestBaseService(t *testing.T) {
require.NoError(t, ts.Start(ctx)) require.NoError(t, ts.Start(ctx))
require.True(t, ts.isStarted()) require.True(t, ts.isStarted())
go func() { _ = ts.Stop() }()
go ts.Stop()
go cancel() go cancel()
ts.Wait() ts.Wait()


+ 2
- 8
node/node.go View File

@ -177,6 +177,7 @@ func makeNode(
if err != nil { if err != nil {
return nil, combineCloseError(err, makeCloser(closers)) return nil, combineCloseError(err, makeCloser(closers))
} }
closers = append(closers, func() error { indexerService.Stop(); return nil })
privValidator, err := createPrivval(ctx, logger, cfg, genDoc, filePrivval) privValidator, err := createPrivval(ctx, logger, cfg, genDoc, filePrivval)
if err != nil { if err != nil {
@ -363,7 +364,6 @@ func makeNode(
services: []service.Service{ services: []service.Service{
eventBus, eventBus,
indexerService,
evReactor, evReactor,
mpReactor, mpReactor,
csReactor, csReactor,
@ -473,10 +473,6 @@ func (n *nodeImpl) OnStart(ctx context.Context) error {
for _, reactor := range n.services { for _, reactor := range n.services {
if err := reactor.Start(ctx); err != nil { if err := reactor.Start(ctx); err != nil {
if errors.Is(err, service.ErrAlreadyStarted) {
continue
}
return fmt.Errorf("problem starting service '%T': %w ", reactor, err) return fmt.Errorf("problem starting service '%T': %w ", reactor, err)
} }
} }
@ -515,9 +511,7 @@ func (n *nodeImpl) OnStart(ctx context.Context) error {
if err != nil { if err != nil {
n.logger.Error("state sync failed; shutting down this node", "err", err) n.logger.Error("state sync failed; shutting down this node", "err", err)
// stop the node // stop the node
if err := n.Stop(); err != nil {
n.logger.Error("failed to shut down node", "err", err)
}
n.Stop()
return err return err
} }


+ 2
- 2
node/node_test.go View File

@ -178,7 +178,7 @@ func TestNodeSetPrivValTCP(t *testing.T) {
err := signerServer.Start(ctx) err := signerServer.Start(ctx)
require.NoError(t, err) require.NoError(t, err)
}() }()
defer signerServer.Stop() //nolint:errcheck // ignore for tests
defer signerServer.Stop()
genDoc, err := defaultGenesisDocProviderFunc(cfg)() genDoc, err := defaultGenesisDocProviderFunc(cfg)()
require.NoError(t, err) require.NoError(t, err)
@ -242,7 +242,7 @@ func TestNodeSetPrivValIPC(t *testing.T) {
err := pvsc.Start(ctx) err := pvsc.Start(ctx)
require.NoError(t, err) require.NoError(t, err)
}() }()
defer pvsc.Stop() //nolint:errcheck // ignore for tests
defer pvsc.Stop()
genDoc, err := defaultGenesisDocProviderFunc(cfg)() genDoc, err := defaultGenesisDocProviderFunc(cfg)()
require.NoError(t, err) require.NoError(t, err)


+ 3
- 3
privval/signer_client.go View File

@ -41,12 +41,12 @@ func NewSignerClient(ctx context.Context, endpoint *SignerListenerEndpoint, chai
// Close closes the underlying connection // Close closes the underlying connection
func (sc *SignerClient) Close() error { func (sc *SignerClient) Close() error {
err := sc.endpoint.Stop()
cerr := sc.endpoint.Close()
sc.endpoint.Stop()
err := sc.endpoint.Close()
if err != nil { if err != nil {
return err return err
} }
return cerr
return nil
} }
// IsConnected indicates with the signer is connected to a remote signing service // IsConnected indicates with the signer is connected to a remote signing service


+ 1
- 1
privval/signer_client_test.go View File

@ -81,7 +81,7 @@ func TestSignerClose(t *testing.T) {
}() }()
assert.NoError(t, tc.signerClient.Close()) assert.NoError(t, tc.signerClient.Close())
assert.NoError(t, tc.signerServer.Stop())
tc.signerServer.Stop()
}) })
} }
} }


+ 2
- 4
privval/signer_listener_endpoint_test.go View File

@ -125,9 +125,7 @@ func TestRetryConnToRemoteSigner(t *testing.T) {
t.Cleanup(signerServer.Wait) t.Cleanup(signerServer.Wait)
<-endpointIsOpenCh <-endpointIsOpenCh
if err := signerServer.Stop(); err != nil {
t.Error(err)
}
signerServer.Stop()
dialerEndpoint2 := NewSignerDialerEndpoint( dialerEndpoint2 := NewSignerDialerEndpoint(
logger, logger,
@ -138,8 +136,8 @@ func TestRetryConnToRemoteSigner(t *testing.T) {
// let some pings pass // let some pings pass
require.NoError(t, signerServer2.Start(ctx)) require.NoError(t, signerServer2.Start(ctx))
assert.True(t, signerServer2.IsRunning()) assert.True(t, signerServer2.IsRunning())
t.Cleanup(signerServer2.Stop)
t.Cleanup(signerServer2.Wait) t.Cleanup(signerServer2.Wait)
t.Cleanup(func() { _ = signerServer2.Stop() })
// give the client some time to re-establish the conn to the remote signer // give the client some time to re-establish the conn to the remote signer
// should see sth like this in the logs: // should see sth like this in the logs:


Loading…
Cancel
Save