Browse Source

pubsub: Move indexing out of the primary subscription path (#7231)

This is part of the work described by #7156.

Remove "unbuffered subscriptions" from the pubsub service.
Replace them with a dedicated blocking "observer" mechanism.
Use the observer mechanism for indexing.

Add a SubscribeWithArgs method and deprecate the old Subscribe
method. Remove SubscribeUnbuffered entirely (breaking).

Rework the Subscription interface to eliminate exposed channels.
Subscriptions now use a context to manage lifecycle notifications.

Internalize the eventbus package.
pull/7251/head
M. J. Fromberger 3 years ago
committed by GitHub
parent
commit
54d7030510
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 1404 additions and 1398 deletions
  1. +2
    -0
      CHANGELOG_PENDING.md
  2. +15
    -13
      internal/consensus/byzantine_test.go
  3. +15
    -15
      internal/consensus/common_test.go
  4. +10
    -3
      internal/consensus/invalid_test.go
  5. +6
    -6
      internal/consensus/mempool_test.go
  6. +3
    -2
      internal/consensus/reactor.go
  7. +111
    -51
      internal/consensus/reactor_test.go
  8. +15
    -13
      internal/consensus/replay.go
  9. +11
    -4
      internal/consensus/replay_file.go
  10. +14
    -8
      internal/consensus/replay_test.go
  11. +3
    -2
      internal/consensus/state.go
  12. +123
    -107
      internal/consensus/state_test.go
  13. +2
    -1
      internal/consensus/wal_generator.go
  14. +242
    -0
      internal/eventbus/event_bus.go
  15. +161
    -146
      internal/eventbus/event_bus_test.go
  16. +3
    -2
      internal/inspect/inspect.go
  17. +13
    -0
      internal/inspect/inspect_test.go
  18. +5
    -1
      internal/inspect/rpc/rpc.go
  19. +2
    -1
      internal/rpc/core/env.go
  20. +30
    -28
      internal/rpc/core/events.go
  21. +2
    -1
      internal/state/execution.go
  22. +16
    -18
      internal/state/execution_test.go
  23. +8
    -8
      internal/state/indexer/indexer.go
  24. +81
    -73
      internal/state/indexer/indexer_service.go
  25. +2
    -1
      internal/state/indexer/indexer_service_test.go
  26. +7
    -18
      libs/pubsub/example_test.go
  27. +117
    -54
      libs/pubsub/pubsub.go
  28. +283
    -326
      libs/pubsub/pubsub_test.go
  29. +40
    -109
      libs/pubsub/subscription.go
  30. +3
    -2
      node/node.go
  31. +9
    -7
      node/node_test.go
  32. +5
    -4
      node/setup.go
  33. +45
    -48
      rpc/client/local/local.go
  34. +0
    -326
      types/event_bus.go

+ 2
- 0
CHANGELOG_PENDING.md View File

@ -26,6 +26,8 @@ Special thanks to external contributors on this release:
- Go API
- [pubsub] \#7231 Remove unbuffered subscriptions and rework the Subscription interface. (@creachadair)
- [eventbus] \#7231 Move the EventBus type to the internal/eventbus package. (@creachadair)
- [blocksync] \#7046 Remove v2 implementation of the blocksync service and recactor, which was disabled in the previous release. (@tychoish)
- [p2p] \#7064 Remove WDRR queue implementation. (@tychoish)
- [config] \#7169 `WriteConfigFile` now returns an error. (@tychoish)


+ 15
- 13
internal/consensus/byzantine_test.go View File

@ -14,6 +14,7 @@ import (
abciclient "github.com/tendermint/tendermint/abci/client"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/evidence"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
@ -92,7 +93,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
pv := privVals[i]
cs.SetPrivValidator(pv)
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
err = eventBus.Start()
require.NoError(t, err)
@ -238,24 +239,25 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
// we will check the first six just in case
evidenceFromEachValidator := make([]types.Evidence, nValidators)
wg := new(sync.WaitGroup)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
i := 0
for _, sub := range rts.subs {
wg.Add(1)
go func(j int, s types.Subscription) {
go func(j int, s eventbus.Subscription) {
defer wg.Done()
for {
select {
case msg := <-s.Out():
require.NotNil(t, msg)
block := msg.Data().(types.EventDataNewBlock).Block
if len(block.Evidence.Evidence) != 0 {
evidenceFromEachValidator[j] = block.Evidence.Evidence[0]
return
}
case <-s.Canceled():
require.Fail(t, "subscription failed for %d", j)
msg, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
require.NotNil(t, msg)
block := msg.Data().(types.EventDataNewBlock).Block
if len(block.Evidence.Evidence) != 0 {
evidenceFromEachValidator[j] = block.Evidence.Evidence[0]
return
}
}


+ 15
- 15
internal/consensus/common_test.go View File

@ -20,6 +20,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/config"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
sm "github.com/tendermint/tendermint/internal/state"
@ -368,21 +369,20 @@ func validatePrevoteAndPrecommit(
cs.mtx.Unlock()
}
func subscribeToVoter(cs *State, addr []byte) <-chan tmpubsub.Message {
votesSub, err := cs.eventBus.SubscribeUnbuffered(context.Background(), testSubscriber, types.EventQueryVote)
if err != nil {
panic(fmt.Sprintf("failed to subscribe %s to %v", testSubscriber, types.EventQueryVote))
}
ch := make(chan tmpubsub.Message)
go func() {
for msg := range votesSub.Out() {
vote := msg.Data().(types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
ch <- msg
}
func subscribeToVoter(t *testing.T, cs *State, addr []byte) <-chan tmpubsub.Message {
t.Helper()
ch := make(chan tmpubsub.Message, 1)
if err := cs.eventBus.Observe(context.Background(), func(msg tmpubsub.Message) error {
vote := msg.Data().(types.EventDataVote)
// we only fire for our own votes
if bytes.Equal(addr, vote.Vote.ValidatorAddress) {
ch <- msg
}
}()
return nil
}, types.EventQueryVote); err != nil {
t.Fatalf("Failed to observe query %v: %v", types.EventQueryVote, err)
}
return ch
}
@ -446,7 +446,7 @@ func newStateWithConfigAndBlockStore(
cs.SetLogger(log.TestingLogger().With("module", "consensus"))
cs.SetPrivValidator(pv)
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
err := eventBus.Start()
if err != nil {


+ 10
- 3
internal/consensus/invalid_test.go View File

@ -5,7 +5,9 @@ import (
"sync"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/libs/bytes"
tmrand "github.com/tendermint/tendermint/libs/rand"
@ -55,13 +57,18 @@ func TestReactorInvalidPrecommit(t *testing.T) {
//
// TODO: Make this tighter by ensuring the halt happens by block 2.
var wg sync.WaitGroup
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
for i := 0; i < 10; i++ {
for _, sub := range rts.subs {
wg.Add(1)
go func(s types.Subscription) {
<-s.Out()
wg.Done()
go func(s eventbus.Subscription) {
defer wg.Done()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel() // cancel other subscribers on failure
}
}(sub)
}
}


+ 6
- 6
internal/consensus/mempool_test.go View File

@ -37,7 +37,7 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
cs := newStateWithConfig(config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
newBlockCh := subscribe(t, cs.eventBus, types.EventQueryNewBlock)
startTestRound(cs, height, round)
ensureNewEventOnChannel(newBlockCh) // first block gets committed
@ -61,7 +61,7 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
assertMempool(cs.txNotifier).EnableTxsAvailable()
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
newBlockCh := subscribe(t, cs.eventBus, types.EventQueryNewBlock)
startTestRound(cs, cs.Height, cs.Round)
ensureNewEventOnChannel(newBlockCh) // first block gets committed
@ -81,9 +81,9 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
cs := newStateWithConfig(config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
newRoundCh := subscribe(cs.eventBus, types.EventQueryNewRound)
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
newBlockCh := subscribe(t, cs.eventBus, types.EventQueryNewBlock)
newRoundCh := subscribe(t, cs.eventBus, types.EventQueryNewRound)
timeoutCh := subscribe(t, cs.eventBus, types.EventQueryTimeoutPropose)
cs.setProposal = func(proposal *types.Proposal) error {
if cs.Height == 2 && cs.Round == 0 {
// dont set the proposal in round 0 so we timeout and
@ -131,7 +131,7 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
cs := newStateWithConfigAndBlockStore(config, state, privVals[0], NewCounterApplication(), blockStore)
err := stateStore.Save(state)
require.NoError(t, err)
newBlockHeaderCh := subscribe(cs.eventBus, types.EventQueryNewBlockHeader)
newBlockHeaderCh := subscribe(t, cs.eventBus, types.EventQueryNewBlockHeader)
const numTxs int64 = 3000
go deliverTxsRange(cs, 0, int(numTxs))


+ 3
- 2
internal/consensus/reactor.go View File

@ -6,6 +6,7 @@ import (
"time"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/p2p"
sm "github.com/tendermint/tendermint/internal/state"
@ -110,7 +111,7 @@ type Reactor struct {
service.BaseService
state *State
eventBus *types.EventBus
eventBus *eventbus.EventBus
Metrics *Metrics
mtx tmsync.RWMutex
@ -243,7 +244,7 @@ func (r *Reactor) OnStop() {
}
// SetEventBus sets the reactor's event bus.
func (r *Reactor) SetEventBus(b *types.EventBus) {
func (r *Reactor) SetEventBus(b *eventbus.EventBus) {
r.eventBus = b
r.state.SetEventBus(b)
}


+ 111
- 51
internal/consensus/reactor_test.go View File

@ -10,6 +10,7 @@ import (
"time"
"github.com/fortytw2/leaktest"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
@ -19,6 +20,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
@ -41,8 +43,8 @@ type reactorTestSuite struct {
network *p2ptest.Network
states map[types.NodeID]*State
reactors map[types.NodeID]*Reactor
subs map[types.NodeID]types.Subscription
blocksyncSubs map[types.NodeID]types.Subscription
subs map[types.NodeID]eventbus.Subscription
blocksyncSubs map[types.NodeID]eventbus.Subscription
stateChannels map[types.NodeID]*p2p.Channel
dataChannels map[types.NodeID]*p2p.Channel
voteChannels map[types.NodeID]*p2p.Channel
@ -64,8 +66,8 @@ func setup(t *testing.T, numNodes int, states []*State, size int) *reactorTestSu
network: p2ptest.MakeNetwork(t, p2ptest.NetworkOptions{NumNodes: numNodes}),
states: make(map[types.NodeID]*State),
reactors: make(map[types.NodeID]*Reactor, numNodes),
subs: make(map[types.NodeID]types.Subscription, numNodes),
blocksyncSubs: make(map[types.NodeID]types.Subscription, numNodes),
subs: make(map[types.NodeID]eventbus.Subscription, numNodes),
blocksyncSubs: make(map[types.NodeID]eventbus.Subscription, numNodes),
}
rts.stateChannels = rts.network.MakeChannelsNoCleanup(t, chDesc(StateChannel, size))
@ -73,7 +75,8 @@ func setup(t *testing.T, numNodes int, states []*State, size int) *reactorTestSu
rts.voteChannels = rts.network.MakeChannelsNoCleanup(t, chDesc(VoteChannel, size))
rts.voteSetBitsChannels = rts.network.MakeChannelsNoCleanup(t, chDesc(VoteSetBitsChannel, size))
_, cancel := context.WithCancel(context.Background())
ctx, cancel := context.WithCancel(context.Background())
// Canceled during cleanup (see below).
i := 0
for nodeID, node := range rts.network.Nodes {
@ -92,10 +95,18 @@ func setup(t *testing.T, numNodes int, states []*State, size int) *reactorTestSu
reactor.SetEventBus(state.eventBus)
blocksSub, err := state.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, size)
blocksSub, err := state.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{
ClientID: testSubscriber,
Query: types.EventQueryNewBlock,
Limit: size,
})
require.NoError(t, err)
fsSub, err := state.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryBlockSyncStatus, size)
fsSub, err := state.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{
ClientID: testSubscriber,
Query: types.EventQueryBlockSyncStatus,
Limit: size,
})
require.NoError(t, err)
rts.states[nodeID] = state
@ -154,15 +165,21 @@ func waitForAndValidateBlock(
t *testing.T,
n int,
activeVals map[string]struct{},
blocksSubs []types.Subscription,
blocksSubs []eventbus.Subscription,
states []*State,
txs ...[]byte,
) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
fn := func(j int) {
msg := <-blocksSubs[j].Out()
newBlock := msg.Data().(types.EventDataNewBlock).Block
msg, err := blocksSubs[j].Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
newBlock := msg.Data().(types.EventDataNewBlock).Block
require.NoError(t, validateBlock(newBlock, activeVals))
for _, tx := range txs {
@ -171,12 +188,11 @@ func waitForAndValidateBlock(
}
var wg sync.WaitGroup
wg.Add(n)
for i := 0; i < n; i++ {
wg.Add(1)
go func(j int) {
defer wg.Done()
fn(j)
wg.Done()
}(i)
}
@ -187,18 +203,23 @@ func waitForAndValidateBlockWithTx(
t *testing.T,
n int,
activeVals map[string]struct{},
blocksSubs []types.Subscription,
blocksSubs []eventbus.Subscription,
states []*State,
txs ...[]byte,
) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
fn := func(j int) {
ntxs := 0
BLOCK_TX_LOOP:
for {
msg := <-blocksSubs[j].Out()
newBlock := msg.Data().(types.EventDataNewBlock).Block
msg, err := blocksSubs[j].Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
newBlock := msg.Data().(types.EventDataNewBlock).Block
require.NoError(t, validateBlock(newBlock, activeVals))
// check that txs match the txs we're waiting for.
@ -210,18 +231,17 @@ func waitForAndValidateBlockWithTx(
}
if ntxs == len(txs) {
break BLOCK_TX_LOOP
break
}
}
}
var wg sync.WaitGroup
wg.Add(n)
for i := 0; i < n; i++ {
wg.Add(1)
go func(j int) {
defer wg.Done()
fn(j)
wg.Done()
}(i)
}
@ -232,19 +252,25 @@ func waitForBlockWithUpdatedValsAndValidateIt(
t *testing.T,
n int,
updatedVals map[string]struct{},
blocksSubs []types.Subscription,
blocksSubs []eventbus.Subscription,
css []*State,
) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
fn := func(j int) {
var newBlock *types.Block
LOOP:
for {
msg := <-blocksSubs[j].Out()
msg, err := blocksSubs[j].Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
newBlock = msg.Data().(types.EventDataNewBlock).Block
if newBlock.LastCommit.Size() == len(updatedVals) {
break LOOP
break
}
}
@ -252,12 +278,11 @@ func waitForBlockWithUpdatedValsAndValidateIt(
}
var wg sync.WaitGroup
wg.Add(n)
for i := 0; i < n; i++ {
wg.Add(1)
go func(j int) {
defer wg.Done()
fn(j)
wg.Done()
}(i)
}
@ -289,14 +314,19 @@ func TestReactorBasic(t *testing.T) {
reactor.SwitchToConsensus(state, false)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// wait till everyone makes the first new block
go func(s types.Subscription) {
go func(s eventbus.Subscription) {
defer wg.Done()
<-s.Out()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
}
}(sub)
}
@ -306,9 +336,13 @@ func TestReactorBasic(t *testing.T) {
wg.Add(1)
// wait till everyone makes the consensus switch
go func(s types.Subscription) {
go func(s eventbus.Subscription) {
defer wg.Done()
msg := <-s.Out()
msg, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
ensureBlockSyncStatus(t, msg, true, 0)
}(sub)
}
@ -381,7 +415,7 @@ func TestReactorWithEvidence(t *testing.T) {
cs.SetLogger(log.TestingLogger().With("module", "consensus"))
cs.SetPrivValidator(pv)
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
err = eventBus.Start()
require.NoError(t, err)
@ -400,18 +434,24 @@ func TestReactorWithEvidence(t *testing.T) {
reactor.SwitchToConsensus(state, false)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// We expect for each validator that is the proposer to propose one piece of
// evidence.
go func(s types.Subscription) {
msg := <-s.Out()
block := msg.Data().(types.EventDataNewBlock).Block
go func(s eventbus.Subscription) {
defer wg.Done()
msg, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
return
}
block := msg.Data().(types.EventDataNewBlock).Block
require.Len(t, block.Evidence.Evidence, 1)
wg.Done()
}(sub)
}
@ -454,14 +494,19 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) {
),
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// wait till everyone makes the first new block
go func(s types.Subscription) {
<-s.Out()
wg.Done()
go func(s eventbus.Subscription) {
defer wg.Done()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
}
}(sub)
}
@ -484,14 +529,19 @@ func TestReactorRecordsVotesAndBlockParts(t *testing.T) {
reactor.SwitchToConsensus(state, false)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// wait till everyone makes the first new block
go func(s types.Subscription) {
<-s.Out()
wg.Done()
go func(s eventbus.Subscription) {
defer wg.Done()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
}
}(sub)
}
@ -559,20 +609,25 @@ func TestReactorVotingPowerChange(t *testing.T) {
activeVals[string(addr)] = struct{}{}
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// wait till everyone makes the first new block
go func(s types.Subscription) {
<-s.Out()
wg.Done()
go func(s eventbus.Subscription) {
defer wg.Done()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
}
}(sub)
}
wg.Wait()
blocksSubs := []types.Subscription{}
blocksSubs := []eventbus.Subscription{}
for _, sub := range rts.subs {
blocksSubs = append(blocksSubs, sub)
}
@ -659,14 +714,19 @@ func TestReactorValidatorSetChanges(t *testing.T) {
activeVals[string(pubKey.Address())] = struct{}{}
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for _, sub := range rts.subs {
wg.Add(1)
// wait till everyone makes the first new block
go func(s types.Subscription) {
<-s.Out()
wg.Done()
go func(s eventbus.Subscription) {
defer wg.Done()
_, err := s.Next(ctx)
if !assert.NoError(t, err) {
cancel()
}
}(sub)
}
@ -680,7 +740,7 @@ func TestReactorValidatorSetChanges(t *testing.T) {
newValidatorTx1 := kvstore.MakeValSetChangeTx(valPubKey1ABCI, testMinPower)
blocksSubs := []types.Subscription{}
blocksSubs := []eventbus.Subscription{}
for _, sub := range rts.subs {
blocksSubs = append(blocksSubs, sub)
}


+ 15
- 13
internal/consensus/replay.go View File

@ -3,6 +3,7 @@ package consensus
import (
"bytes"
"context"
"errors"
"fmt"
"hash/crc32"
"io"
@ -11,6 +12,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/libs/log"
@ -36,7 +38,7 @@ var crc32c = crc32.MakeTable(crc32.Castagnoli)
// Unmarshal and apply a single message to the consensus state as if it were
// received in receiveRoutine. Lines that start with "#" are ignored.
// NOTE: receiveRoutine should not be running.
func (cs *State) readReplayMessage(msg *TimedWALMessage, newStepSub types.Subscription) error {
func (cs *State) readReplayMessage(msg *TimedWALMessage, newStepSub eventbus.Subscription) error {
// Skip meta messages which exist for demarcating boundaries.
if _, ok := msg.Msg.(EndHeightMessage); ok {
return nil
@ -47,18 +49,18 @@ func (cs *State) readReplayMessage(msg *TimedWALMessage, newStepSub types.Subscr
case types.EventDataRoundState:
cs.Logger.Info("Replay: New Step", "height", m.Height, "round", m.Round, "step", m.Step)
// these are playback checks
ticker := time.After(time.Second * 2)
if newStepSub != nil {
select {
case stepMsg := <-newStepSub.Out():
m2 := stepMsg.Data().(types.EventDataRoundState)
if m.Height != m2.Height || m.Round != m2.Round || m.Step != m2.Step {
return fmt.Errorf("roundState mismatch. Got %v; Expected %v", m2, m)
}
case <-newStepSub.Canceled():
return fmt.Errorf("failed to read off newStepSub.Out(). newStepSub was canceled")
case <-ticker:
return fmt.Errorf("failed to read off newStepSub.Out()")
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
stepMsg, err := newStepSub.Next(ctx)
if errors.Is(err, context.DeadlineExceeded) {
return fmt.Errorf("subscription timed out: %w", err)
} else if err != nil {
return fmt.Errorf("subscription canceled: %w", err)
}
m2 := stepMsg.Data().(types.EventDataRoundState)
if m.Height != m2.Height || m.Round != m2.Round || m.Step != m2.Step {
return fmt.Errorf("roundState mismatch. Got %v; Expected %v", m2, m)
}
}
case msgInfo:
@ -216,7 +218,7 @@ func NewHandshaker(stateStore sm.Store, state sm.State,
stateStore: stateStore,
initialState: state,
store: store,
eventBus: types.NopEventBus{},
eventBus: eventbus.NopEventBus{},
genDoc: genDoc,
logger: log.NewNopLogger(),
nBlocks: 0,


+ 11
- 4
internal/consensus/replay_file.go View File

@ -13,6 +13,7 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/internal/store"
@ -54,7 +55,10 @@ func (cs *State) ReplayFile(file string, console bool) error {
// ensure all new step events are regenerated as expected
ctx := context.Background()
newStepSub, err := cs.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep)
newStepSub, err := cs.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{
ClientID: subscriber,
Query: types.EventQueryNewRoundStep,
})
if err != nil {
return fmt.Errorf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep)
}
@ -125,7 +129,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
func (pb *playback) replayReset(count int, newStepSub types.Subscription) error {
func (pb *playback) replayReset(count int, newStepSub eventbus.Subscription) error {
if err := pb.cs.Stop(); err != nil {
return err
}
@ -222,7 +226,10 @@ func (pb *playback) replayConsoleLoop() int {
ctx := context.Background()
// ensure all new step events are regenerated as expected
newStepSub, err := pb.cs.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep)
newStepSub, err := pb.cs.eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{
ClientID: subscriber,
Query: types.EventQueryNewRoundStep,
})
if err != nil {
tmos.Exit(fmt.Sprintf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep))
}
@ -318,7 +325,7 @@ func newConsensusStateForReplay(cfg config.BaseConfig, csConfig *config.Consensu
tmos.Exit(fmt.Sprintf("Error starting proxy app conns: %v", err))
}
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
if err := eventBus.Start(); err != nil {
tmos.Exit(fmt.Sprintf("Failed to start event bus: %v", err))
}


+ 14
- 8
internal/consensus/replay_test.go View File

@ -3,6 +3,7 @@ package consensus
import (
"bytes"
"context"
"errors"
"fmt"
"io"
"math/rand"
@ -31,6 +32,7 @@ import (
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/pubsub"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/privval"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
@ -84,14 +86,18 @@ func startNewStateAndWaitForBlock(t *testing.T, consensusReplayConfig *config.Co
// in the WAL itself. Assuming the consensus state is running, replay of any
// WAL, including the empty one, should eventually be followed by a new
// block, or else something is wrong.
newBlockSub, err := cs.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock)
newBlockSub, err := cs.eventBus.SubscribeWithArgs(context.Background(), pubsub.SubscribeArgs{
ClientID: testSubscriber,
Query: types.EventQueryNewBlock,
})
require.NoError(t, err)
select {
case <-newBlockSub.Out():
case <-newBlockSub.Canceled():
t.Fatal("newBlockSub was canceled")
case <-time.After(120 * time.Second):
ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second)
defer cancel()
_, err = newBlockSub.Next(ctx)
if errors.Is(err, context.DeadlineExceeded) {
t.Fatal("Timed out waiting for new block (see trace above)")
} else if err != nil {
t.Fatal("newBlockSub was canceled")
}
}
@ -334,8 +340,8 @@ func setupSimulator(t *testing.T) *simulatorTestSuite {
partSize := types.BlockPartSizeBytes
newRoundCh := subscribe(css[0].eventBus, types.EventQueryNewRound)
proposalCh := subscribe(css[0].eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(t, css[0].eventBus, types.EventQueryNewRound)
proposalCh := subscribe(t, css[0].eventBus, types.EventQueryCompleteProposal)
vss := make([]*validatorStub, nPeers)
for i := 0; i < nPeers; i++ {


+ 3
- 2
internal/consensus/state.go View File

@ -15,6 +15,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/libs/fail"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
sm "github.com/tendermint/tendermint/internal/state"
@ -117,7 +118,7 @@ type State struct {
// we use eventBus to trigger msg broadcasts in the reactor,
// and to notify external subscribers, eg. through a websocket
eventBus *types.EventBus
eventBus *eventbus.EventBus
// a Write-Ahead Log ensures we can recover from any kind of crash
// and helps us avoid signing conflicting votes
@ -207,7 +208,7 @@ func (cs *State) SetLogger(l log.Logger) {
}
// SetEventBus sets event bus.
func (cs *State) SetEventBus(b *types.EventBus) {
func (cs *State) SetEventBus(b *eventbus.EventBus) {
cs.eventBus = b
cs.blockExec.SetEventBus(b)
}


+ 123
- 107
internal/consensus/state_test.go View File

@ -13,6 +13,7 @@ import (
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/crypto/tmhash"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
tmrand "github.com/tendermint/tendermint/libs/rand"
@ -62,8 +63,8 @@ func TestStateProposerSelection0(t *testing.T) {
height, round := cs1.Height, cs1.Round
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
startTestRound(cs1, height, round)
@ -105,7 +106,7 @@ func TestStateProposerSelection2(t *testing.T) {
require.NoError(t, err)
height := cs1.Height
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
// this time we jump in at round 2
incrementRound(vss[1:]...)
@ -148,7 +149,7 @@ func TestStateEnterProposeNoPrivValidator(t *testing.T) {
height, round := cs.Height, cs.Round
// Listen for propose timeout event
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
timeoutCh := subscribe(t, cs.eventBus, types.EventQueryTimeoutPropose)
startTestRound(cs, height, round)
@ -170,8 +171,8 @@ func TestStateEnterProposeYesPrivValidator(t *testing.T) {
// Listen for propose timeout event
timeoutCh := subscribe(cs.eventBus, types.EventQueryTimeoutPropose)
proposalCh := subscribe(cs.eventBus, types.EventQueryCompleteProposal)
timeoutCh := subscribe(t, cs.eventBus, types.EventQueryTimeoutPropose)
proposalCh := subscribe(t, cs.eventBus, types.EventQueryCompleteProposal)
cs.enterNewRound(height, round)
cs.startRoutines(3)
@ -204,8 +205,8 @@ func TestStateBadProposal(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribe(t, cs1.eventBus, types.EventQueryVote)
propBlock, _ := cs1.createProposalBlock() // changeProposer(t, cs1, vs2)
@ -266,8 +267,8 @@ func TestStateOversizedBlock(t *testing.T) {
partSize := types.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
voteCh := subscribe(t, cs1.eventBus, types.EventQueryVote)
propBlock, _ := cs1.createProposalBlock()
propBlock.Data.Txs = []types.Tx{tmrand.Bytes(2001)}
@ -332,16 +333,16 @@ func TestStateFullRound1(t *testing.T) {
if err := cs.eventBus.Stop(); err != nil {
t.Error(err)
}
eventBus := types.NewEventBusWithBufferCapacity(0)
eventBus := eventbus.NewDefault()
eventBus.SetLogger(log.TestingLogger().With("module", "events"))
cs.SetEventBus(eventBus)
if err := eventBus.Start(); err != nil {
t.Error(err)
}
voteCh := subscribe(cs.eventBus, types.EventQueryVote)
propCh := subscribe(cs.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(cs.eventBus, types.EventQueryNewRound)
voteCh := subscribe(t, cs.eventBus, types.EventQueryVote)
propCh := subscribe(t, cs.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(t, cs.eventBus, types.EventQueryNewRound)
// Maybe it would be better to call explicitly startRoutines(4)
startTestRound(cs, height, round)
@ -370,7 +371,7 @@ func TestStateFullRoundNil(t *testing.T) {
require.NoError(t, err)
height, round := cs.Height, cs.Round
voteCh := subscribe(cs.eventBus, types.EventQueryVote)
voteCh := subscribe(t, cs.eventBus, types.EventQueryVote)
cs.enterPrevote(height, round)
cs.startRoutines(4)
@ -392,8 +393,8 @@ func TestStateFullRound2(t *testing.T) {
vs2 := vss[1]
height, round := cs1.Height, cs1.Round
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlock)
voteCh := subscribe(t, cs1.eventBus, types.EventQueryVote)
newBlockCh := subscribe(t, cs1.eventBus, types.EventQueryNewBlock)
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
@ -437,11 +438,11 @@ func TestStateLockNoPOL(t *testing.T) {
partSize := types.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
voteCh := subscribe(t, cs1.eventBus, types.EventQueryVote)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
/*
Round1 (cs1, B) // B B // B B2
@ -629,14 +630,14 @@ func TestStateLockPOLRelock(t *testing.T) {
partSize := types.BlockPartSizeBytes
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader)
voteCh := subscribeToVoter(t, cs1, addr)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(t, cs1.eventBus, types.EventQueryNewBlockHeader)
// everything done from perspective of cs1
@ -731,14 +732,14 @@ func TestStateLockPOLUnlock(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(t, cs1.eventBus, types.EventQueryUnlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// everything done from perspective of cs1
@ -826,13 +827,13 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) {
partSize := types.BlockPartSizeBytes
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(t, cs1, addr)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
// everything done from perspective of cs1
/*
@ -957,14 +958,14 @@ func TestStateLockPOLSafety1(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -1053,7 +1054,7 @@ func TestStateLockPOLSafety1(t *testing.T) {
// we should prevote what we're locked on
validatePrevote(t, cs1, round, vss[0], propBlockHash)
newStepCh := subscribe(cs1.eventBus, types.EventQueryNewRoundStep)
newStepCh := subscribe(t, cs1.eventBus, types.EventQueryNewRoundStep)
// before prevotes from the previous round are added
// add prevotes from the earlier round
@ -1081,14 +1082,14 @@ func TestStateLockPOLSafety2(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(t, cs1.eventBus, types.EventQueryUnlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// the block for R0: gets polkad but we miss it
// (even though we signed it, shhh)
@ -1181,15 +1182,15 @@ func TestProposeValidBlock(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(cs1.eventBus, types.EventQueryUnlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
unlockCh := subscribe(t, cs1.eventBus, types.EventQueryUnlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -1274,14 +1275,14 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(t, cs1.eventBus, types.EventQueryValidBlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, round)
@ -1339,15 +1340,15 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
partSize := types.BlockPartSizeBytes
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(t, cs1.eventBus, types.EventQueryValidBlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribeToVoter(t, cs1, addr)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
round++ // move to round in which P0 is not proposer
incrementRound(vs2, vs3, vs4)
@ -1396,8 +1397,8 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) {
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
// start round
startTestRound(cs1, height, round)
@ -1420,12 +1421,12 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round
startTestRound(cs1, height, round)
@ -1459,12 +1460,12 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) {
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round
startTestRound(cs1, height, round)
@ -1498,12 +1499,12 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, int32(1)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round in which PO is not proposer
startTestRound(cs1, height, round)
@ -1532,8 +1533,8 @@ func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) {
partSize := types.BlockPartSizeBytes
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(t, cs1.eventBus, types.EventQueryValidBlock)
_, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round)
propBlockHash := propBlock.Hash()
@ -1567,9 +1568,9 @@ func TestCommitFromPreviousRound(t *testing.T) {
partSize := types.BlockPartSizeBytes
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(cs1.eventBus, types.EventQueryValidBlock)
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
validBlockCh := subscribe(t, cs1.eventBus, types.EventQueryValidBlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
prop, propBlock := decideProposal(cs1, vs2, vs2.Height, vs2.Round)
propBlockHash := propBlock.Hash()
@ -1624,16 +1625,16 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) {
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
precommitTimeoutCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutPropose)
precommitTimeoutCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockHeader := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
newBlockHeader := subscribe(t, cs1.eventBus, types.EventQueryNewBlockHeader)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
@ -1689,14 +1690,14 @@ func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) {
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockHeader := subscribe(cs1.eventBus, types.EventQueryNewBlockHeader)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
newBlockHeader := subscribe(t, cs1.eventBus, types.EventQueryNewBlockHeader)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
@ -1747,10 +1748,10 @@ func TestStateSlashingPrevotes(t *testing.T) {
vs2 := vss[1]
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(t, cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -1782,10 +1783,10 @@ func TestStateSlashingPrecommits(t *testing.T) {
vs2 := vss[1]
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(cs1, cs1.privValidator.GetAddress())
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
voteCh := subscribeToVoter(t, cs1, cs1.privValidator.GetAddress())
// start round and wait for propose and prevote
startTestRound(cs1, cs1.Height, 0)
@ -1831,14 +1832,14 @@ func TestStateHalt1(t *testing.T) {
height, round := cs1.Height, cs1.Round
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(cs1.eventBus, types.EventQueryNewBlock)
proposalCh := subscribe(t, cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(t, cs1.eventBus, types.EventQueryTimeoutWait)
newRoundCh := subscribe(t, cs1.eventBus, types.EventQueryNewRound)
newBlockCh := subscribe(t, cs1.eventBus, types.EventQueryNewBlock)
pv1, err := cs1.privValidator.GetPubKey(context.Background())
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(cs1, addr)
voteCh := subscribeToVoter(t, cs1, addr)
// start round and wait for propose and prevote
startTestRound(cs1, height, round)
@ -2002,10 +2003,25 @@ func TestSignSameVoteTwice(t *testing.T) {
}
// subscribe subscribes test client to the given query and returns a channel with cap = 1.
func subscribe(eventBus *types.EventBus, q tmpubsub.Query) <-chan tmpubsub.Message {
sub, err := eventBus.Subscribe(context.Background(), testSubscriber, q)
func subscribe(t *testing.T, eventBus *eventbus.EventBus, q tmpubsub.Query) <-chan tmpubsub.Message {
t.Helper()
sub, err := eventBus.SubscribeWithArgs(context.Background(), tmpubsub.SubscribeArgs{
ClientID: testSubscriber,
Query: q,
})
if err != nil {
panic(fmt.Sprintf("failed to subscribe %s to %v", testSubscriber, q))
t.Fatalf("Failed to subscribe %q to %v: %v", testSubscriber, q, err)
}
return sub.Out()
ch := make(chan tmpubsub.Message)
go func() {
for {
next, err := sub.Next(context.Background())
if err != nil {
t.Errorf("Subscription for %v unexpectedly terminated: %v", q, err)
return
}
ch <- next
}
}()
return ch
}

+ 2
- 1
internal/consensus/wal_generator.go View File

@ -16,6 +16,7 @@ import (
abciclient "github.com/tendermint/tendermint/abci/client"
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/internal/store"
@ -76,7 +77,7 @@ func WALGenerateNBlocks(t *testing.T, wr io.Writer, numBlocks int) (err error) {
}
})
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
eventBus.SetLogger(logger.With("module", "events"))
if err := eventBus.Start(); err != nil {
return fmt.Errorf("failed to start event bus: %w", err)


+ 242
- 0
internal/eventbus/event_bus.go View File

@ -0,0 +1,242 @@
package eventbus
import (
"context"
"fmt"
"strings"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/types"
)
// Subscription is a proxy interface for a pubsub Subscription.
type Subscription interface {
ID() string
Next(context.Context) (tmpubsub.Message, error)
}
// EventBus is a common bus for all events going through the system.
// It is a type-aware wrapper around an underlying pubsub server.
// All events should be published via the bus.
type EventBus struct {
service.BaseService
pubsub *tmpubsub.Server
}
// NewDefault returns a new event bus with default options.
func NewDefault() *EventBus {
pubsub := tmpubsub.NewServer(tmpubsub.BufferCapacity(0))
b := &EventBus{pubsub: pubsub}
b.BaseService = *service.NewBaseService(nil, "EventBus", b)
return b
}
func (b *EventBus) SetLogger(l log.Logger) {
b.BaseService.SetLogger(l)
b.pubsub.SetLogger(l.With("module", "pubsub"))
}
func (b *EventBus) OnStart() error {
return b.pubsub.Start()
}
func (b *EventBus) OnStop() {
if err := b.pubsub.Stop(); err != nil {
b.pubsub.Logger.Error("error trying to stop eventBus", "error", err)
}
}
func (b *EventBus) NumClients() int {
return b.pubsub.NumClients()
}
func (b *EventBus) NumClientSubscriptions(clientID string) int {
return b.pubsub.NumClientSubscriptions(clientID)
}
// Deprecated: Use SubscribeWithArgs instead.
func (b *EventBus) Subscribe(ctx context.Context,
clientID string, query tmpubsub.Query, capacities ...int) (Subscription, error) {
return b.pubsub.Subscribe(ctx, clientID, query, capacities...)
}
func (b *EventBus) SubscribeWithArgs(ctx context.Context, args tmpubsub.SubscribeArgs) (Subscription, error) {
return b.pubsub.SubscribeWithArgs(ctx, args)
}
func (b *EventBus) Unsubscribe(ctx context.Context, args tmpubsub.UnsubscribeArgs) error {
return b.pubsub.Unsubscribe(ctx, args)
}
func (b *EventBus) UnsubscribeAll(ctx context.Context, subscriber string) error {
return b.pubsub.UnsubscribeAll(ctx, subscriber)
}
func (b *EventBus) Observe(ctx context.Context, observe func(tmpubsub.Message) error, queries ...tmpubsub.Query) error {
return b.pubsub.Observe(ctx, observe, queries...)
}
func (b *EventBus) Publish(eventValue string, eventData types.TMEventData) error {
// no explicit deadline for publishing events
ctx := context.Background()
tokens := strings.Split(types.EventTypeKey, ".")
event := abci.Event{
Type: tokens[0],
Attributes: []abci.EventAttribute{
{
Key: tokens[1],
Value: eventValue,
},
},
}
return b.pubsub.PublishWithEvents(ctx, eventData, []abci.Event{event})
}
func (b *EventBus) PublishEventNewBlock(data types.EventDataNewBlock) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := append(data.ResultBeginBlock.Events, data.ResultEndBlock.Events...)
// add Tendermint-reserved new block event
events = append(events, types.EventNewBlock)
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewBlockHeader(data types.EventDataNewBlockHeader) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := append(data.ResultBeginBlock.Events, data.ResultEndBlock.Events...)
// add Tendermint-reserved new block header event
events = append(events, types.EventNewBlockHeader)
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewEvidence(evidence types.EventDataNewEvidence) error {
return b.Publish(types.EventNewEvidenceValue, evidence)
}
func (b *EventBus) PublishEventVote(data types.EventDataVote) error {
return b.Publish(types.EventVoteValue, data)
}
func (b *EventBus) PublishEventValidBlock(data types.EventDataRoundState) error {
return b.Publish(types.EventValidBlockValue, data)
}
func (b *EventBus) PublishEventBlockSyncStatus(data types.EventDataBlockSyncStatus) error {
return b.Publish(types.EventBlockSyncStatusValue, data)
}
func (b *EventBus) PublishEventStateSyncStatus(data types.EventDataStateSyncStatus) error {
return b.Publish(types.EventStateSyncStatusValue, data)
}
// PublishEventTx publishes tx event with events from Result. Note it will add
// predefined keys (EventTypeKey, TxHashKey). Existing events with the same keys
// will be overwritten.
func (b *EventBus) PublishEventTx(data types.EventDataTx) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := data.Result.Events
// add Tendermint-reserved events
events = append(events, types.EventTx)
tokens := strings.Split(types.TxHashKey, ".")
events = append(events, abci.Event{
Type: tokens[0],
Attributes: []abci.EventAttribute{
{
Key: tokens[1],
Value: fmt.Sprintf("%X", types.Tx(data.Tx).Hash()),
},
},
})
tokens = strings.Split(types.TxHeightKey, ".")
events = append(events, abci.Event{
Type: tokens[0],
Attributes: []abci.EventAttribute{
{
Key: tokens[1],
Value: fmt.Sprintf("%d", data.Height),
},
},
})
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewRoundStep(data types.EventDataRoundState) error {
return b.Publish(types.EventNewRoundStepValue, data)
}
func (b *EventBus) PublishEventTimeoutPropose(data types.EventDataRoundState) error {
return b.Publish(types.EventTimeoutProposeValue, data)
}
func (b *EventBus) PublishEventTimeoutWait(data types.EventDataRoundState) error {
return b.Publish(types.EventTimeoutWaitValue, data)
}
func (b *EventBus) PublishEventNewRound(data types.EventDataNewRound) error {
return b.Publish(types.EventNewRoundValue, data)
}
func (b *EventBus) PublishEventCompleteProposal(data types.EventDataCompleteProposal) error {
return b.Publish(types.EventCompleteProposalValue, data)
}
func (b *EventBus) PublishEventPolka(data types.EventDataRoundState) error {
return b.Publish(types.EventPolkaValue, data)
}
func (b *EventBus) PublishEventUnlock(data types.EventDataRoundState) error {
return b.Publish(types.EventUnlockValue, data)
}
func (b *EventBus) PublishEventRelock(data types.EventDataRoundState) error {
return b.Publish(types.EventRelockValue, data)
}
func (b *EventBus) PublishEventLock(data types.EventDataRoundState) error {
return b.Publish(types.EventLockValue, data)
}
func (b *EventBus) PublishEventValidatorSetUpdates(data types.EventDataValidatorSetUpdates) error {
return b.Publish(types.EventValidatorSetUpdatesValue, data)
}
//-----------------------------------------------------------------------------
// NopEventBus implements a types.BlockEventPublisher that discards all events.
type NopEventBus struct{}
func (NopEventBus) PublishEventNewBlock(types.EventDataNewBlock) error {
return nil
}
func (NopEventBus) PublishEventNewBlockHeader(types.EventDataNewBlockHeader) error {
return nil
}
func (NopEventBus) PublishEventNewEvidence(types.EventDataNewEvidence) error {
return nil
}
func (NopEventBus) PublishEventTx(types.EventDataTx) error {
return nil
}
func (NopEventBus) PublishEventValidatorSetUpdates(types.EventDataValidatorSetUpdates) error {
return nil
}

types/event_bus_test.go → internal/eventbus/event_bus_test.go View File


+ 3
- 2
internal/inspect/inspect.go View File

@ -7,6 +7,7 @@ import (
"net"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/inspect/rpc"
rpccore "github.com/tendermint/tendermint/internal/rpc/core"
"github.com/tendermint/tendermint/internal/state"
@ -32,7 +33,7 @@ type Inspector struct {
config *config.RPCConfig
indexerService *indexer.Service
eventBus *types.EventBus
eventBus *eventbus.EventBus
logger log.Logger
}
@ -44,7 +45,7 @@ type Inspector struct {
//nolint:lll
func New(cfg *config.RPCConfig, bs state.BlockStore, ss state.Store, es []indexer.EventSink, logger log.Logger) *Inspector {
routes := rpc.Routes(*cfg, ss, bs, es, logger)
eb := types.NewEventBus()
eb := eventbus.NewDefault()
eb.SetLogger(logger.With("module", "events"))
is := indexer.NewIndexerService(es, eb)
is.SetLogger(logger.With("module", "txindex"))


+ 13
- 0
internal/inspect/inspect_test.go View File

@ -79,6 +79,7 @@ func TestBlock(t *testing.T) {
blockStoreMock.On("LoadBlock", testHeight).Return(testBlock)
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
@ -223,6 +224,8 @@ func TestConsensusParams(t *testing.T) {
}, nil)
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)
@ -274,6 +277,8 @@ func TestBlockResults(t *testing.T) {
blockStoreMock.On("Height").Return(testHeight)
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)
@ -320,6 +325,8 @@ func TestCommit(t *testing.T) {
}, nil)
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)
@ -372,6 +379,8 @@ func TestBlockByHash(t *testing.T) {
blockStoreMock.On("LoadBlockByHash", testHash).Return(testBlock, nil)
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)
@ -423,6 +432,8 @@ func TestBlockchain(t *testing.T) {
})
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)
@ -474,6 +485,8 @@ func TestValidators(t *testing.T) {
blockStoreMock.On("Base").Return(int64(0))
eventSinkMock := &indexermocks.EventSink{}
eventSinkMock.On("Stop").Return(nil)
eventSinkMock.On("Type").Return(indexer.EventSinkType("Mock"))
rpcConfig := config.TestRPCConfig()
l := log.TestingLogger()
d := inspect.New(rpcConfig, blockStoreMock, stateStoreMock, []indexer.EventSink{eventSinkMock}, l)


+ 5
- 1
internal/inspect/rpc/rpc.go View File

@ -26,6 +26,10 @@ type Server struct {
Config *config.RPCConfig
}
type eventBusUnsubscriber interface {
UnsubscribeAll(ctx context.Context, subscriber string) error
}
// Routes returns the set of routes used by the Inspector server.
//
//nolint: lll
@ -59,7 +63,7 @@ func Handler(rpcConfig *config.RPCConfig, routes core.RoutesMap, logger log.Logg
mux := http.NewServeMux()
wmLogger := logger.With("protocol", "websocket")
var eventBus types.EventBusSubscriber
var eventBus eventBusUnsubscriber
websocketDisconnectFn := func(remoteAddr string) {
err := eventBus.UnsubscribeAll(context.Background(), remoteAddr)


+ 2
- 1
internal/rpc/core/env.go View File

@ -8,6 +8,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/internal/consensus"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/proxy"
@ -86,7 +87,7 @@ type Environment struct {
PubKey crypto.PubKey
GenDoc *types.GenesisDoc // cache the genesis structure
EventSinks []indexer.EventSink
EventBus *types.EventBus // thread safe
EventBus *eventbus.EventBus // thread safe
Mempool mempool.Mempool
BlockSyncReactor consensus.BlockSyncReactor
StateSyncMetricer statesync.Metricer


+ 30
- 28
internal/rpc/core/events.go View File

@ -2,6 +2,7 @@ package core
import (
"context"
"errors"
"fmt"
"time"
@ -37,7 +38,11 @@ func (env *Environment) Subscribe(ctx *rpctypes.Context, query string) (*coretyp
subCtx, cancel := context.WithTimeout(ctx.Context(), SubscribeTimeout)
defer cancel()
sub, err := env.EventBus.Subscribe(subCtx, addr, q, subBufferSize)
sub, err := env.EventBus.SubscribeWithArgs(subCtx, tmpubsub.SubscribeArgs{
ClientID: addr,
Query: q,
Limit: subBufferSize,
})
if err != nil {
return nil, err
}
@ -46,37 +51,34 @@ func (env *Environment) Subscribe(ctx *rpctypes.Context, query string) (*coretyp
subscriptionID := ctx.JSONReq.ID
go func() {
for {
select {
case msg := <-sub.Out():
var (
resultEvent = &coretypes.ResultEvent{Query: query, Data: msg.Data(), Events: msg.Events()}
resp = rpctypes.NewRPCSuccessResponse(subscriptionID, resultEvent)
)
writeCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
if err := ctx.WSConn.WriteRPCResponse(writeCtx, resp); err != nil {
env.Logger.Info("Can't write response (slow client)",
msg, err := sub.Next(context.Background())
if errors.Is(err, tmpubsub.ErrUnsubscribed) {
// The subscription was removed by the client.
return
} else if errors.Is(err, tmpubsub.ErrTerminated) {
// The subscription was terminated by the publisher.
resp := rpctypes.RPCServerError(subscriptionID, err)
ok := ctx.WSConn.TryWriteRPCResponse(resp)
if !ok {
env.Logger.Info("Unable to write response (slow client)",
"to", addr, "subscriptionID", subscriptionID, "err", err)
}
case <-sub.Canceled():
if sub.Err() != tmpubsub.ErrUnsubscribed {
var reason string
if sub.Err() == nil {
reason = "Tendermint exited"
} else {
reason = sub.Err().Error()
}
var (
err = fmt.Errorf("subscription was canceled (reason: %s)", reason)
resp = rpctypes.RPCServerError(subscriptionID, err)
)
if ok := ctx.WSConn.TryWriteRPCResponse(resp); !ok {
env.Logger.Info("Can't write response (slow client)",
"to", addr, "subscriptionID", subscriptionID, "err", err)
}
}
return
}
// We have a message to deliver to the client.
resp := rpctypes.NewRPCSuccessResponse(subscriptionID, &coretypes.ResultEvent{
Query: query,
Data: msg.Data(),
Events: msg.Events(),
})
wctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
err = ctx.WSConn.WriteRPCResponse(wctx, resp)
cancel()
if err != nil {
env.Logger.Info("Unable to write response (slow client)",
"to", addr, "subscriptionID", subscriptionID, "err", err)
}
}
}()


+ 2
- 1
internal/state/execution.go View File

@ -8,6 +8,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/libs/fail"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/proxy"
@ -69,7 +70,7 @@ func NewBlockExecutor(
res := &BlockExecutor{
store: stateStore,
proxyApp: proxyApp,
eventBus: types.NopEventBus{},
eventBus: eventbus.NopEventBus{},
mempool: pool,
evpool: evpool,
logger: logger,


+ 16
- 18
internal/state/execution_test.go View File

@ -16,6 +16,7 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/internal/eventbus"
mmock "github.com/tendermint/tendermint/internal/mempool/mock"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
@ -23,6 +24,7 @@ import (
sf "github.com/tendermint/tendermint/internal/state/test/factory"
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/pubsub"
tmtime "github.com/tendermint/tendermint/libs/time"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
@ -367,18 +369,17 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
blockStore,
)
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
err = eventBus.Start()
require.NoError(t, err)
defer eventBus.Stop() //nolint:errcheck // ignore for tests
blockExec.SetEventBus(eventBus)
updatesSub, err := eventBus.Subscribe(
context.Background(),
"TestEndBlockValidatorUpdates",
types.EventQueryValidatorSetUpdates,
)
updatesSub, err := eventBus.SubscribeWithArgs(context.Background(), pubsub.SubscribeArgs{
ClientID: "TestEndBlockValidatorUpdates",
Query: types.EventQueryValidatorSetUpdates,
})
require.NoError(t, err)
block := sf.MakeBlock(state, 1, new(types.Commit))
@ -402,18 +403,15 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
}
// test we threw an event
select {
case msg := <-updatesSub.Out():
event, ok := msg.Data().(types.EventDataValidatorSetUpdates)
require.True(t, ok, "Expected event of type EventDataValidatorSetUpdates, got %T", msg.Data())
if assert.NotEmpty(t, event.ValidatorUpdates) {
assert.Equal(t, pubkey, event.ValidatorUpdates[0].PubKey)
assert.EqualValues(t, 10, event.ValidatorUpdates[0].VotingPower)
}
case <-updatesSub.Canceled():
t.Fatalf("updatesSub was canceled (reason: %v)", updatesSub.Err())
case <-time.After(1 * time.Second):
t.Fatal("Did not receive EventValidatorSetUpdates within 1 sec.")
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
defer cancel()
msg, err := updatesSub.Next(ctx)
require.NoError(t, err)
event, ok := msg.Data().(types.EventDataValidatorSetUpdates)
require.True(t, ok, "Expected event of type EventDataValidatorSetUpdates, got %T", msg.Data())
if assert.NotEmpty(t, event.ValidatorUpdates) {
assert.Equal(t, pubkey, event.ValidatorUpdates[0].PubKey)
assert.EqualValues(t, 10, event.ValidatorUpdates[0].VotingPower)
}
}


+ 8
- 8
internal/state/indexer/indexer.go View File

@ -41,26 +41,26 @@ type BlockIndexer interface {
// Batch groups together multiple Index operations to be performed at the same time.
// NOTE: Batch is NOT thread-safe and must not be modified after starting its execution.
type Batch struct {
Ops []*abci.TxResult
Ops []*abci.TxResult
Pending int64
}
// NewBatch creates a new Batch.
func NewBatch(n int64) *Batch {
return &Batch{
Ops: make([]*abci.TxResult, n),
}
return &Batch{Ops: make([]*abci.TxResult, n), Pending: n}
}
// Add or update an entry for the given result.Index.
func (b *Batch) Add(result *abci.TxResult) error {
b.Ops[result.Index] = result
if b.Ops[result.Index] == nil {
b.Pending--
b.Ops[result.Index] = result
}
return nil
}
// Size returns the total number of operations inside the batch.
func (b *Batch) Size() int {
return len(b.Ops)
}
func (b *Batch) Size() int { return len(b.Ops) }
// ErrorEmptyHash indicates empty hash
var ErrorEmptyHash = errors.New("transaction hash cannot be empty")

+ 81
- 73
internal/state/indexer/indexer_service.go View File

@ -3,110 +3,118 @@ package indexer
import (
"context"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/types"
)
// XXX/TODO: These types should be moved to the indexer package.
const (
subscriber = "IndexerService"
)
// Service connects event bus, transaction and block indexers together in
// order to index transactions and blocks coming from the event bus.
type Service struct {
service.BaseService
eventSinks []EventSink
eventBus *types.EventBus
eventBus *eventbus.EventBus
currentBlock struct {
header types.EventDataNewBlockHeader
height int64
batch *Batch
}
}
// NewIndexerService returns a new service instance.
func NewIndexerService(es []EventSink, eventBus *types.EventBus) *Service {
func NewIndexerService(es []EventSink, eventBus *eventbus.EventBus) *Service {
is := &Service{eventSinks: es, eventBus: eventBus}
is.BaseService = *service.NewBaseService(nil, "IndexerService", is)
return is
}
// OnStart implements service.Service by subscribing for all transactions
// and indexing them by events.
func (is *Service) OnStart() error {
// Use SubscribeUnbuffered here to ensure both subscriptions does not get
// canceled due to not pulling messages fast enough. Cause this might
// sometimes happen when there are no other subscribers.
blockHeadersSub, err := is.eventBus.SubscribeUnbuffered(
context.Background(),
subscriber,
types.EventQueryNewBlockHeader)
if err != nil {
return err
// publish publishes a pubsub message to the service. The service blocks until
// the message has been fully processed.
func (is *Service) publish(msg pubsub.Message) error {
// Indexing has three states. Initially, no block is in progress (WAIT) and
// we expect a block header. Upon seeing a header, we are waiting for zero
// or more transactions (GATHER). Once all the expected transactions have
// been delivered (in some order), we are ready to index. After indexing a
// block, we revert to the WAIT state for the next block.
if is.currentBlock.batch == nil {
// WAIT: Start a new block.
hdr := msg.Data().(types.EventDataNewBlockHeader)
is.currentBlock.header = hdr
is.currentBlock.height = hdr.Header.Height
is.currentBlock.batch = NewBatch(hdr.NumTxs)
if hdr.NumTxs != 0 {
return nil
}
// If the block does not expect any transactions, fall through and index
// it immediately. This shouldn't happen, but this check ensures we do
// not get stuck if it does.
}
txsSub, err := is.eventBus.SubscribeUnbuffered(context.Background(), subscriber, types.EventQueryTx)
if err != nil {
return err
}
curr := is.currentBlock.batch
if curr.Pending != 0 {
// GATHER: Accumulate a transaction into the current block's batch.
txResult := msg.Data().(types.EventDataTx).TxResult
if err := curr.Add(&txResult); err != nil {
is.Logger.Error("failed to add tx to batch",
"height", is.currentBlock.height, "index", txResult.Index, "err", err)
}
go func() {
for {
select {
case <-blockHeadersSub.Canceled():
return
case msg := <-blockHeadersSub.Out():
eventDataHeader := msg.Data().(types.EventDataNewBlockHeader)
height := eventDataHeader.Header.Height
batch := NewBatch(eventDataHeader.NumTxs)
for i := int64(0); i < eventDataHeader.NumTxs; i++ {
msg2 := <-txsSub.Out()
txResult := msg2.Data().(types.EventDataTx).TxResult
if err = batch.Add(&txResult); err != nil {
is.Logger.Error(
"failed to add tx to batch",
"height", height,
"index", txResult.Index,
"err", err,
)
}
}
// This may have been the last transaction in the batch, so fall through
// to check whether it is time to index.
}
if !IndexingEnabled(is.eventSinks) {
continue
}
if curr.Pending == 0 {
// INDEX: We have all the transactions we expect for the current block.
for _, sink := range is.eventSinks {
if err := sink.IndexBlockEvents(is.currentBlock.header); err != nil {
is.Logger.Error("failed to index block header",
"height", is.currentBlock.height, "err", err)
} else {
is.Logger.Debug("indexed block",
"height", is.currentBlock.height, "sink", sink.Type())
}
for _, sink := range is.eventSinks {
if err := sink.IndexBlockEvents(eventDataHeader); err != nil {
is.Logger.Error("failed to index block", "height", height, "err", err)
} else {
is.Logger.Debug("indexed block", "height", height, "sink", sink.Type())
}
if len(batch.Ops) > 0 {
err := sink.IndexTxEvents(batch.Ops)
if err != nil {
is.Logger.Error("failed to index block txs", "height", height, "err", err)
} else {
is.Logger.Debug("indexed txs", "height", height, "sink", sink.Type())
}
}
if curr.Size() != 0 {
err := sink.IndexTxEvents(curr.Ops)
if err != nil {
is.Logger.Error("failed to index block txs",
"height", is.currentBlock.height, "err", err)
} else {
is.Logger.Debug("indexed txs",
"height", is.currentBlock.height, "sink", sink.Type())
}
}
}
}()
is.currentBlock.batch = nil // return to the WAIT state for the next block
}
return nil
}
// OnStop implements service.Service by unsubscribing from all transactions and
// close the eventsink.
func (is *Service) OnStop() {
if is.eventBus.IsRunning() {
_ = is.eventBus.UnsubscribeAll(context.Background(), subscriber)
// OnStart implements part of service.Service. It registers an observer for the
// indexer if the underlying event sinks support indexing.
//
// TODO(creachadair): Can we get rid of the "enabled" check?
func (is *Service) OnStart() error {
// If the event sinks support indexing, register an observer to capture
// block header data for the indexer.
if IndexingEnabled(is.eventSinks) {
err := is.eventBus.Observe(context.TODO(), is.publish,
types.EventQueryNewBlockHeader, types.EventQueryTx)
if err != nil {
return err
}
}
return nil
}
// OnStop implements service.Service by closing the event sinks.
func (is *Service) OnStop() {
for _, sink := range is.eventSinks {
if err := sink.Stop(); err != nil {
is.Logger.Error("failed to close eventsink", "eventsink", sink.Type(), "err", err)


+ 2
- 1
internal/state/indexer/indexer_service_test.go View File

@ -15,6 +15,7 @@ import (
dbm "github.com/tendermint/tm-db"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/state/indexer"
"github.com/tendermint/tendermint/internal/state/indexer/sink/kv"
"github.com/tendermint/tendermint/internal/state/indexer/sink/psql"
@ -39,7 +40,7 @@ var (
func TestIndexerServiceIndexesBlocks(t *testing.T) {
// event bus
eventBus := types.NewEventBus()
eventBus := eventbus.NewDefault()
eventBus.SetLogger(tmlog.TestingLogger())
err := eventBus.Start()
require.NoError(t, err)


+ 7
- 18
libs/pubsub/example_test.go View File

@ -7,27 +7,18 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/pubsub/query"
)
func TestExample(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
require.NoError(t, s.Start())
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
subscription, err := s.Subscribe(ctx, "example-client", query.MustParse("abci.account.name='John'"))
require.NoError(t, err)
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "example-client",
Query: query.MustParse("abci.account.name='John'"),
}))
events := []abci.Event{
{
@ -35,8 +26,6 @@ func TestExample(t *testing.T) {
Attributes: []abci.EventAttribute{{Key: "name", Value: "John"}},
},
}
err = s.PublishWithEvents(ctx, "Tombstone", events)
require.NoError(t, err)
assertReceive(t, "Tombstone", subscription.Out())
require.NoError(t, s.PublishWithEvents(ctx, "Tombstone", events))
sub.mustReceive(ctx, "Tombstone")
}

+ 117
- 54
libs/pubsub/pubsub.go View File

@ -10,24 +10,26 @@
//
// Example:
//
// q, err := query.New("account.name='John'")
// q, err := query.New(`account.name='John'`)
// if err != nil {
// return err
// }
// ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
// defer cancel()
// subscription, err := pubsub.Subscribe(ctx, "johns-transactions", q)
// sub, err := pubsub.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
// ClientID: "johns-transactions",
// Query: q,
// })
// if err != nil {
// return err
// }
//
// for {
// select {
// case msg <- subscription.Out():
// // handle msg.Data() and msg.Events()
// case <-subscription.Canceled():
// return subscription.Err()
// next, err := sub.Next(ctx)
// if err == pubsub.ErrTerminated {
// return err // terminated by publisher
// } else if err != nil {
// return err // timed out, client unsubscribed, etc.
// }
// process(next)
// }
//
package pubsub
@ -67,6 +69,14 @@ type Query interface {
String() string
}
// SubscribeArgs are the parameters to create a new subscription.
type SubscribeArgs struct {
ClientID string // Client ID
Query Query // filter query for events (required)
Limit int // subscription queue capacity limit (0 means 1)
Quota int // subscription queue soft quota (0 uses Limit)
}
// UnsubscribeArgs are the parameters to remove a subscription.
// The subscriber ID must be populated, and at least one of the client ID or
// the registered query.
@ -106,6 +116,11 @@ type Server struct {
subs struct {
sync.RWMutex
index *subIndex
// This function is called synchronously with each message published
// before it is delivered to any other subscriber. This allows an index
// to be persisted before any subscribers see the messages.
observe func(Message) error
}
// TODO(creachadair): Rework the options so that this does not need to live
@ -149,54 +164,93 @@ func BufferCapacity(cap int) Option {
// BufferCapacity returns capacity of the publication queue.
func (s *Server) BufferCapacity() int { return cap(s.queue) }
// Subscribe creates a subscription for the given client.
// Subscribe creates a subscription for the given client ID and query.
// If len(capacities) > 0, its first value is used as the queue capacity.
//
// An error will be returned to the caller if the context is canceled or if
// subscription already exist for pair clientID and query.
//
// outCapacity can be used to set a capacity for Subscription#Out channel (1 by
// default). Panics if outCapacity is less than or equal to zero. If you want
// an unbuffered channel, use SubscribeUnbuffered.
func (s *Server) Subscribe(
ctx context.Context,
clientID string,
query Query,
outCapacity ...int) (*Subscription, error) {
outCap := 1
if len(outCapacity) > 0 {
if outCapacity[0] <= 0 {
panic("Negative or zero capacity. Use SubscribeUnbuffered if you want an unbuffered channel")
// Deprecated: Use SubscribeWithArgs. This method will be removed in v0.36.
func (s *Server) Subscribe(ctx context.Context,
clientID string, query Query, capacities ...int) (*Subscription, error) {
args := SubscribeArgs{
ClientID: clientID,
Query: query,
Limit: 1,
}
if len(capacities) > 0 {
args.Limit = capacities[0]
if len(capacities) > 1 {
args.Quota = capacities[1]
}
outCap = outCapacity[0]
// bounds are checked below
}
return s.subscribe(ctx, clientID, query, outCap)
return s.SubscribeWithArgs(ctx, args)
}
// SubscribeUnbuffered does the same as Subscribe, except it returns a
// subscription with unbuffered channel. Use with caution as it can freeze the
// server.
func (s *Server) SubscribeUnbuffered(ctx context.Context, clientID string, query Query) (*Subscription, error) {
return s.subscribe(ctx, clientID, query, 0)
// Observe registers an observer function that will be called synchronously
// with each published message matching any of the given queries, prior to it
// being forwarded to any subscriber. If no queries are specified, all
// messages will be observed. An error is reported if an observer is already
// registered.
func (s *Server) Observe(ctx context.Context, observe func(Message) error, queries ...Query) error {
s.subs.Lock()
defer s.subs.Unlock()
if observe == nil {
return errors.New("observe callback is nil")
} else if s.subs.observe != nil {
return errors.New("an observer is already registered")
}
// Compile the message filter.
var matches func(Message) bool
if len(queries) == 0 {
matches = func(Message) bool { return true }
} else {
matches = func(msg Message) bool {
for _, q := range queries {
match, err := q.Matches(msg.events)
if err == nil && match {
return true
}
}
return false
}
}
s.subs.observe = func(msg Message) error {
if matches(msg) {
return observe(msg)
}
return nil // nothing to do for this message
}
return nil
}
func (s *Server) subscribe(ctx context.Context, clientID string, query Query, outCapacity int) (*Subscription, error) {
// SubscribeWithArgs creates a subscription for the given arguments. It is an
// error if the query is nil, a subscription already exists for the specified
// client ID and query, or if the capacity arguments are invalid.
func (s *Server) SubscribeWithArgs(ctx context.Context, args SubscribeArgs) (*Subscription, error) {
if args.Query == nil {
return nil, errors.New("query is nil")
}
s.subs.Lock()
defer s.subs.Unlock()
if s.subs.index == nil {
return nil, ErrServerStopped
} else if s.subs.index.contains(clientID, query.String()) {
} else if s.subs.index.contains(args.ClientID, args.Query.String()) {
return nil, ErrAlreadySubscribed
}
sub, err := newSubscription(outCapacity)
if args.Limit == 0 {
args.Limit = 1
}
sub, err := newSubscription(args.Quota, args.Limit)
if err != nil {
return nil, err
}
s.subs.index.add(&subInfo{
clientID: clientID,
query: query,
clientID: args.ClientID,
query: args.Query,
subID: sub.id,
sub: sub,
})
@ -334,11 +388,11 @@ func (s *Server) run() {
s.Logger.Error("Error sending event", "err", err)
}
}
// Terminate all subscribers without error before exit.
// Terminate all subscribers before exit.
s.subs.Lock()
defer s.subs.Unlock()
for si := range s.subs.index.all {
si.sub.cancel(nil)
si.sub.stop(ErrTerminated)
}
s.subs.index = nil
}()
@ -348,7 +402,7 @@ func (s *Server) run() {
// error. The caller must hold the s.subs lock.
func (s *Server) removeSubs(evict subInfoSet, reason error) {
for si := range evict {
si.sub.cancel(reason)
si.sub.stop(reason)
}
s.subs.index.removeAll(evict)
}
@ -362,7 +416,7 @@ func (s *Server) send(data interface{}, events []types.Event) error {
if len(evict) != 0 {
s.subs.Lock()
defer s.subs.Unlock()
s.removeSubs(evict, ErrOutOfCapacity)
s.removeSubs(evict, ErrTerminated)
}
}()
@ -372,6 +426,19 @@ func (s *Server) send(data interface{}, events []types.Event) error {
s.subs.RLock()
defer s.subs.RUnlock()
// If an observer is defined, give it control of the message before
// attempting to deliver it to any matching subscribers. If the observer
// fails, the message will not be forwarded.
if s.subs.observe != nil {
err := s.subs.observe(Message{
data: data,
events: events,
})
if err != nil {
return fmt.Errorf("observer failed on message: %w", err)
}
}
for si := range s.subs.index.all {
match, err := si.query.Matches(events)
if err != nil {
@ -381,18 +448,14 @@ func (s *Server) send(data interface{}, events []types.Event) error {
continue
}
// Subscriptions may be buffered or unbuffered. Unbuffered subscriptions
// are intended for internal use such as indexing, where we don't want to
// penalize a slow reader. Buffered subscribers must keep up with their
// queue, or they will be terminated.
//
// TODO(creachadair): Unbuffered subscriptions used by the event indexer
// to avoid losing events if it happens to be slow. Rework this so that
// use case doesn't require this affordance, and then remove unbuffered
// subscriptions.
msg := NewMessage(si.sub.id, data, events)
if err := si.sub.putMessage(msg); err != nil {
// The subscriber was too slow, cancel them.
// Publish the events to the subscriber's queue. If this fails, e.g.,
// because the queue is over capacity or out of quota, evict the
// subscription from the index.
if err := si.sub.publish(Message{
subID: si.sub.id,
data: data,
events: events,
}); err != nil {
evict.add(si)
}
}


+ 283
- 326
libs/pubsub/pubsub_test.go View File

@ -2,17 +2,15 @@ package pubsub_test
import (
"context"
"errors"
"fmt"
"runtime/debug"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/pubsub/query"
)
@ -21,172 +19,144 @@ const (
clientID = "test-client"
)
func TestSubscribe(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
func TestSubscribeWithArgs(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
subscription, err := s.Subscribe(ctx, clientID, query.Empty{})
require.NoError(t, err)
require.Equal(t, 1, s.NumClients())
require.Equal(t, 1, s.NumClientSubscriptions(clientID))
err = s.Publish(ctx, "Ka-Zar")
require.NoError(t, err)
assertReceive(t, "Ka-Zar", subscription.Out())
published := make(chan struct{})
go func() {
defer close(published)
t.Run("DefaultLimit", func(t *testing.T) {
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.Empty{},
}))
err := s.Publish(ctx, "Quicksilver")
require.NoError(t, err)
require.Equal(t, 1, s.NumClients())
require.Equal(t, 1, s.NumClientSubscriptions(clientID))
err = s.Publish(ctx, "Asylum")
require.NoError(t, err)
err = s.Publish(ctx, "Ivan")
require.NoError(t, err)
require.NoError(t, s.Publish(ctx, "Ka-Zar"))
sub.mustReceive(ctx, "Ka-Zar")
})
t.Run("PositiveLimit", func(t *testing.T) {
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID + "-2",
Query: query.Empty{},
Limit: 10,
}))
require.NoError(t, s.Publish(ctx, "Aggamon"))
sub.mustReceive(ctx, "Aggamon")
})
}
err = s.Publish(ctx, "Natasha")
require.NoError(t, err)
}()
func TestObserver(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
select {
case <-published:
assertReceive(t, "Quicksilver", subscription.Out())
assertCanceled(t, subscription, pubsub.ErrOutOfCapacity)
case <-time.After(3 * time.Second):
t.Fatal("Expected Publish(Asylum) not to block")
}
done := make(chan struct{})
var got interface{}
require.NoError(t, s.Observe(ctx, func(msg pubsub.Message) error {
defer close(done)
got = msg.Data()
return nil
}))
const input = "Lions and tigers and bears, oh my!"
require.NoError(t, s.Publish(ctx, input))
<-done
require.Equal(t, got, input)
}
func TestSubscribeWithCapacity(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
func TestObserverErrors(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
require.Panics(t, func() {
_, err = s.Subscribe(ctx, clientID, query.Empty{}, -1)
require.NoError(t, err)
})
require.Panics(t, func() {
_, err = s.Subscribe(ctx, clientID, query.Empty{}, 0)
require.NoError(t, err)
})
subscription, err := s.Subscribe(ctx, clientID, query.Empty{}, 1)
require.NoError(t, err)
err = s.Publish(ctx, "Aggamon")
require.NoError(t, err)
assertReceive(t, "Aggamon", subscription.Out())
}
func TestSubscribeUnbuffered(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
require.Error(t, s.Observe(ctx, nil, query.Empty{}))
require.NoError(t, s.Observe(ctx, func(pubsub.Message) error { return nil }))
require.Error(t, s.Observe(ctx, func(pubsub.Message) error { return nil }, query.Empty{}))
}
func TestPublishDoesNotBlock(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
subscription, err := s.SubscribeUnbuffered(ctx, clientID, query.Empty{})
require.NoError(t, err)
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.Empty{},
}))
published := make(chan struct{})
go func() {
defer close(published)
err := s.Publish(ctx, "Ultron")
require.NoError(t, err)
err = s.Publish(ctx, "Darkhawk")
require.NoError(t, err)
require.NoError(t, s.Publish(ctx, "Quicksilver"))
require.NoError(t, s.Publish(ctx, "Asylum"))
require.NoError(t, s.Publish(ctx, "Ivan"))
}()
select {
case <-published:
t.Fatal("Expected Publish(Darkhawk) to block")
sub.mustReceive(ctx, "Quicksilver")
sub.mustFail(ctx, pubsub.ErrTerminated)
case <-time.After(3 * time.Second):
assertReceive(t, "Ultron", subscription.Out())
assertReceive(t, "Darkhawk", subscription.Out())
t.Fatal("Publishing should not have blocked")
}
}
func TestSlowClientIsRemovedWithErrOutOfCapacity(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
func TestSubscribeErrors(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
t.Run("EmptyQueryErr", func(t *testing.T) {
_, err := s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{ClientID: clientID})
require.Error(t, err)
})
t.Run("NegativeLimitErr", func(t *testing.T) {
_, err := s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.Empty{},
Limit: -5,
})
require.Error(t, err)
})
}
func TestSlowSubscriber(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
subscription, err := s.Subscribe(ctx, clientID, query.Empty{})
require.NoError(t, err)
err = s.Publish(ctx, "Fat Cobra")
require.NoError(t, err)
err = s.Publish(ctx, "Viper")
require.NoError(t, err)
err = s.Publish(ctx, "Black Panther")
require.NoError(t, err)
assertCanceled(t, subscription, pubsub.ErrOutOfCapacity)
}
func TestDifferentClients(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.Empty{},
}))
require.NoError(t, s.Start())
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
require.NoError(t, s.Publish(ctx, "Fat Cobra"))
require.NoError(t, s.Publish(ctx, "Viper"))
require.NoError(t, s.Publish(ctx, "Black Panther"))
// We had capacity for one item, so we should get that item, but after that
// the subscription should have been terminated by the publisher.
sub.mustReceive(ctx, "Fat Cobra")
sub.mustFail(ctx, pubsub.ErrTerminated)
}
func TestDifferentClients(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
subscription1, err := s.Subscribe(ctx, "client-1", query.MustParse("tm.events.type='NewBlock'"))
require.NoError(t, err)
sub1 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "client-1",
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
events := []abci.Event{
{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewBlock"}},
},
}
events := []abci.Event{{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewBlock"}},
}}
require.NoError(t, s.PublishWithEvents(ctx, "Iceman", events))
assertReceive(t, "Iceman", subscription1.Out())
sub1.mustReceive(ctx, "Iceman")
subscription2, err := s.Subscribe(
ctx,
"client-2",
query.MustParse("tm.events.type='NewBlock' AND abci.account.name='Igor'"),
)
require.NoError(t, err)
sub2 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "client-2",
Query: query.MustParse("tm.events.type='NewBlock' AND abci.account.name='Igor'"),
}))
events = []abci.Event{
{
@ -200,234 +170,190 @@ func TestDifferentClients(t *testing.T) {
}
require.NoError(t, s.PublishWithEvents(ctx, "Ultimo", events))
assertReceive(t, "Ultimo", subscription1.Out())
assertReceive(t, "Ultimo", subscription2.Out())
sub1.mustReceive(ctx, "Ultimo")
sub2.mustReceive(ctx, "Ultimo")
subscription3, err := s.Subscribe(
ctx,
"client-3",
query.MustParse("tm.events.type='NewRoundStep' AND abci.account.name='Igor' AND abci.invoice.number = 10"),
)
require.NoError(t, err)
sub3 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "client-3",
Query: query.MustParse("tm.events.type='NewRoundStep' AND abci.account.name='Igor' AND abci.invoice.number = 10"),
}))
events = []abci.Event{
{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewRoundStep"}},
},
}
events = []abci.Event{{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewRoundStep"}},
}}
require.NoError(t, s.PublishWithEvents(ctx, "Valeria Richards", events))
require.Zero(t, len(subscription3.Out()))
sub3.mustTimeOut(ctx, 100*time.Millisecond)
}
func TestSubscribeDuplicateKeys(t *testing.T) {
s := newTestServer(t)
ctx := context.Background()
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
require.NoError(t, s.Start())
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
testCases := []struct {
query string
expected interface{}
}{
{
"withdraw.rewards='17'",
"Iceman",
},
{
"withdraw.rewards='22'",
"Iceman",
},
{
"withdraw.rewards='1' AND withdraw.rewards='22'",
"Iceman",
},
{
"withdraw.rewards='100'",
nil,
},
{`withdraw.rewards='17'`, "Iceman"},
{`withdraw.rewards='22'`, "Iceman"},
{`withdraw.rewards='1' AND withdraw.rewards='22'`, "Iceman"},
{`withdraw.rewards='100'`, nil},
}
for i, tc := range testCases {
sub, err := s.Subscribe(ctx, fmt.Sprintf("client-%d", i), query.MustParse(tc.query))
require.NoError(t, err)
events := []abci.Event{
{
Type: "transfer",
Attributes: []abci.EventAttribute{
{Key: "sender", Value: "foo"},
{Key: "sender", Value: "bar"},
{Key: "sender", Value: "baz"},
id := fmt.Sprintf("client-%d", i)
q := query.MustParse(tc.query)
t.Run(id, func(t *testing.T) {
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: id,
Query: q,
}))
events := []abci.Event{
{
Type: "transfer",
Attributes: []abci.EventAttribute{
{Key: "sender", Value: "foo"},
{Key: "sender", Value: "bar"},
{Key: "sender", Value: "baz"},
},
},
},
{
Type: "withdraw",
Attributes: []abci.EventAttribute{
{Key: "rewards", Value: "1"},
{Key: "rewards", Value: "17"},
{Key: "rewards", Value: "22"},
{
Type: "withdraw",
Attributes: []abci.EventAttribute{
{Key: "rewards", Value: "1"},
{Key: "rewards", Value: "17"},
{Key: "rewards", Value: "22"},
},
},
},
}
}
require.NoError(t, s.PublishWithEvents(ctx, "Iceman", events))
require.NoError(t, s.PublishWithEvents(ctx, "Iceman", events))
if tc.expected != nil {
assertReceive(t, tc.expected, sub.Out())
} else {
require.Zero(t, len(sub.Out()))
}
if tc.expected != nil {
sub.mustReceive(ctx, tc.expected)
} else {
sub.mustTimeOut(ctx, 100*time.Millisecond)
}
})
}
}
func TestClientSubscribesTwice(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
q := query.MustParse("tm.events.type='NewBlock'")
subscription1, err := s.Subscribe(ctx, clientID, q)
require.NoError(t, err)
q := query.MustParse("tm.events.type='NewBlock'")
events := []abci.Event{{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewBlock"}},
}}
events := []abci.Event{
{
Type: "tm.events",
Attributes: []abci.EventAttribute{{Key: "type", Value: "NewBlock"}},
},
}
sub1 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: q,
}))
require.NoError(t, s.PublishWithEvents(ctx, "Goblin Queen", events))
assertReceive(t, "Goblin Queen", subscription1.Out())
subscription2, err := s.Subscribe(ctx, clientID, q)
require.Error(t, err)
require.Nil(t, subscription2)
sub1.mustReceive(ctx, "Goblin Queen")
// Subscribing a second time with the same client ID and query fails.
{
sub2, err := s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: q,
})
require.Error(t, err)
require.Nil(t, sub2)
}
// The attempt to re-subscribe does not disrupt the existing sub.
require.NoError(t, s.PublishWithEvents(ctx, "Spider-Man", events))
assertReceive(t, "Spider-Man", subscription1.Out())
sub1.mustReceive(ctx, "Spider-Man")
}
func TestUnsubscribe(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
subscription, err := s.Subscribe(ctx, clientID, query.MustParse("tm.events.type='NewBlock'"))
require.NoError(t, err)
err = s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
// Removing the subscription we just made should succeed.
require.NoError(t, s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
Subscriber: clientID,
Query: query.MustParse("tm.events.type='NewBlock'")})
require.NoError(t, err)
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
err = s.Publish(ctx, "Nick Fury")
require.NoError(t, err)
require.Zero(t, len(subscription.Out()), "Should not receive anything after Unsubscribe")
// Publishing should still work.
require.NoError(t, s.Publish(ctx, "Nick Fury"))
assertCanceled(t, subscription, pubsub.ErrUnsubscribed)
// The unsubscribed subscriber should report as such.
sub.mustFail(ctx, pubsub.ErrUnsubscribed)
}
func TestClientUnsubscribesTwice(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
_, err = s.Subscribe(ctx, clientID, query.MustParse("tm.events.type='NewBlock'"))
require.NoError(t, err)
err = s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
Subscriber: clientID,
Query: query.MustParse("tm.events.type='NewBlock'")})
require.NoError(t, err)
err = s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
require.NoError(t, s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
Subscriber: clientID,
Query: query.MustParse("tm.events.type='NewBlock'")})
require.Equal(t, pubsub.ErrSubscriptionNotFound, err)
err = s.UnsubscribeAll(ctx, clientID)
require.Equal(t, pubsub.ErrSubscriptionNotFound, err)
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
require.ErrorIs(t, s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
Subscriber: clientID,
Query: query.MustParse("tm.events.type='NewBlock'"),
}), pubsub.ErrSubscriptionNotFound)
require.ErrorIs(t, s.UnsubscribeAll(ctx, clientID), pubsub.ErrSubscriptionNotFound)
}
func TestResubscribe(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
_, err = s.Subscribe(ctx, clientID, query.Empty{})
require.NoError(t, err)
err = s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{Subscriber: clientID, Query: query.Empty{}})
require.NoError(t, err)
subscription, err := s.Subscribe(ctx, clientID, query.Empty{})
require.NoError(t, err)
err = s.Publish(ctx, "Cable")
require.NoError(t, err)
assertReceive(t, "Cable", subscription.Out())
args := pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.Empty{},
}
newTestSub(t).must(s.SubscribeWithArgs(ctx, args))
require.NoError(t, s.Unsubscribe(ctx, pubsub.UnsubscribeArgs{
Subscriber: clientID,
Query: query.Empty{},
}))
sub := newTestSub(t).must(s.SubscribeWithArgs(ctx, args))
require.NoError(t, s.Publish(ctx, "Cable"))
sub.mustReceive(ctx, "Cable")
}
func TestUnsubscribeAll(t *testing.T) {
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
err := s.Start()
require.NoError(t, err)
t.Cleanup(func() {
if err := s.Stop(); err != nil {
t.Error(err)
}
})
s := newTestServer(t)
ctx := context.Background()
subscription1, err := s.Subscribe(ctx, clientID, query.MustParse("tm.events.type='NewBlock'"))
require.NoError(t, err)
subscription2, err := s.Subscribe(ctx, clientID, query.MustParse("tm.events.type='NewBlockHeader'"))
require.NoError(t, err)
err = s.UnsubscribeAll(ctx, clientID)
require.NoError(t, err)
sub1 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.MustParse("tm.events.type='NewBlock'"),
}))
sub2 := newTestSub(t).must(s.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: clientID,
Query: query.MustParse("tm.events.type='NewBlockHeader'"),
}))
require.NoError(t, s.UnsubscribeAll(ctx, clientID))
require.NoError(t, s.Publish(ctx, "Nick Fury"))
err = s.Publish(ctx, "Nick Fury")
require.NoError(t, err)
require.Zero(t, len(subscription1.Out()), "Should not receive anything after UnsubscribeAll")
require.Zero(t, len(subscription2.Out()), "Should not receive anything after UnsubscribeAll")
sub1.mustFail(ctx, pubsub.ErrUnsubscribed)
sub2.mustFail(ctx, pubsub.ErrUnsubscribed)
assertCanceled(t, subscription1, pubsub.ErrUnsubscribed)
assertCanceled(t, subscription2, pubsub.ErrUnsubscribed)
}
func TestBufferCapacity(t *testing.T) {
@ -437,34 +363,65 @@ func TestBufferCapacity(t *testing.T) {
require.Equal(t, 2, s.BufferCapacity())
ctx := context.Background()
err := s.Publish(ctx, "Nighthawk")
require.NoError(t, err)
err = s.Publish(ctx, "Sage")
require.NoError(t, err)
ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond)
require.NoError(t, s.Publish(ctx, "Nighthawk"))
require.NoError(t, s.Publish(ctx, "Sage"))
ctx, cancel := context.WithTimeout(ctx, 100*time.Millisecond)
defer cancel()
err = s.Publish(ctx, "Ironclad")
if assert.Error(t, err) {
require.Equal(t, context.DeadlineExceeded, err)
}
require.ErrorIs(t, s.Publish(ctx, "Ironclad"), context.DeadlineExceeded)
}
// HELPERS
func newTestServer(t testing.TB) *pubsub.Server {
t.Helper()
func assertReceive(t *testing.T, expected interface{}, ch <-chan pubsub.Message, msgAndArgs ...interface{}) {
select {
case actual := <-ch:
require.Equal(t, expected, actual.Data(), msgAndArgs...)
case <-time.After(1 * time.Second):
t.Errorf("expected to receive %v from the channel, got nothing after 1s", expected)
debug.PrintStack()
s := pubsub.NewServer()
s.SetLogger(log.TestingLogger())
require.NoError(t, s.Start())
t.Cleanup(func() {
assert.NoError(t, s.Stop())
})
return s
}
type testSub struct {
t testing.TB
*pubsub.Subscription
}
func newTestSub(t testing.TB) *testSub { return &testSub{t: t} }
func (s *testSub) must(sub *pubsub.Subscription, err error) *testSub {
s.t.Helper()
require.NoError(s.t, err)
require.NotNil(s.t, sub)
s.Subscription = sub
return s
}
func (s *testSub) mustReceive(ctx context.Context, want interface{}) {
s.t.Helper()
got, err := s.Next(ctx)
require.NoError(s.t, err)
require.Equal(s.t, want, got.Data())
}
func (s *testSub) mustTimeOut(ctx context.Context, dur time.Duration) {
s.t.Helper()
tctx, cancel := context.WithTimeout(ctx, dur)
defer cancel()
got, err := s.Next(tctx)
if !errors.Is(err, context.DeadlineExceeded) {
s.t.Errorf("Next: got (%+v, %v), want %v", got, err, context.DeadlineExceeded)
}
}
func assertCanceled(t *testing.T, subscription *pubsub.Subscription, err error) {
_, ok := <-subscription.Canceled()
require.False(t, ok)
require.Equal(t, err, subscription.Err())
func (s *testSub) mustFail(ctx context.Context, want error) {
s.t.Helper()
got, err := s.Next(ctx)
if err == nil && want != nil {
s.t.Fatalf("Next: got (%+v, %v), want error %v", got, err, want)
}
require.ErrorIs(s.t, err, want)
}

+ 40
- 109
libs/pubsub/subscription.go View File

@ -3,132 +3,71 @@ package pubsub
import (
"context"
"errors"
"fmt"
"github.com/google/uuid"
"github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/libs/queue"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
var (
// ErrUnsubscribed is returned by Err when a client unsubscribes.
ErrUnsubscribed = errors.New("client unsubscribed")
// ErrUnsubscribed is returned by Next when the client has unsubscribed.
ErrUnsubscribed = errors.New("subscription removed by client")
// ErrOutOfCapacity is returned by Err when a client is not pulling messages
// fast enough. Note the client's subscription will be terminated.
ErrOutOfCapacity = errors.New("client is not pulling messages fast enough")
// ErrTerminated is returned by Next when the subscription was terminated by
// the publisher.
ErrTerminated = errors.New("subscription terminated by publisher")
)
// A Subscription represents a client subscription for a particular query and
// consists of three things:
// 1) channel onto which messages and events are published
// 2) channel which is closed if a client is too slow or choose to unsubscribe
// 3) err indicating the reason for (2)
// A Subscription represents a client subscription for a particular query.
type Subscription struct {
id string
out chan Message
queue *queue.Queue
canceled chan struct{}
stop func()
mtx tmsync.RWMutex
err error
id string
queue *queue.Queue // open until the subscription ends
stopErr error // after queue is closed, the reason why
}
// newSubscription returns a new subscription with the given outCapacity.
func newSubscription(outCapacity int) (*Subscription, error) {
sub := &Subscription{
id: uuid.NewString(),
out: make(chan Message),
canceled: make(chan struct{}),
// N.B. The output channel is always unbuffered. For an unbuffered
// subscription that was already the case, and for a buffered one the
// queue now serves as the buffer.
}
if outCapacity == 0 {
sub.stop = func() { close(sub.canceled) }
return sub, nil
}
q, err := queue.New(queue.Options{
SoftQuota: outCapacity,
HardLimit: outCapacity,
// newSubscription returns a new subscription with the given queue capacity.
func newSubscription(quota, limit int) (*Subscription, error) {
queue, err := queue.New(queue.Options{
SoftQuota: quota,
HardLimit: limit,
})
if err != nil {
return nil, fmt.Errorf("creating queue: %w", err)
return nil, err
}
sub.queue = q
sub.stop = func() { q.Close(); close(sub.canceled) }
// Start a goroutine to bridge messages from the queue to the channel.
// TODO(creachadair): This is a temporary hack until we can change the
// interface not to expose the channel directly.
go func() {
for {
next, err := q.Wait(context.Background())
if err != nil {
return // the subscription was terminated
}
sub.out <- next.(Message)
}
}()
return sub, nil
return &Subscription{
id: uuid.NewString(),
queue: queue,
}, nil
}
// putMessage transmits msg to the subscriber. If s is unbuffered, this blocks
// until msg is delivered and returns nil; otherwise it reports an error if the
// queue cannot accept any further messages.
func (s *Subscription) putMessage(msg Message) error {
if s.queue != nil {
return s.queue.Add(msg)
// Next blocks until a message is available, ctx ends, or the subscription
// ends. Next returns ErrUnsubscribed if s was unsubscribed, ErrTerminated if
// s was terminated by the publisher, or a context error if ctx ended without a
// message being available.
func (s *Subscription) Next(ctx context.Context) (Message, error) {
next, err := s.queue.Wait(ctx)
if errors.Is(err, queue.ErrQueueClosed) {
return Message{}, s.stopErr
} else if err != nil {
return Message{}, err
}
s.out <- msg
return nil
return next.(Message), nil
}
// Out returns a channel onto which messages and events are published.
// Unsubscribe/UnsubscribeAll does not close the channel to avoid clients from
// receiving a nil message.
func (s *Subscription) Out() <-chan Message { return s.out }
// ID returns the unique subscription identifier for s.
func (s *Subscription) ID() string { return s.id }
// Canceled returns a channel that's closed when the subscription is
// terminated and supposed to be used in a select statement.
func (s *Subscription) Canceled() <-chan struct{} {
return s.canceled
}
// Err returns nil if the channel returned by Canceled is not yet closed.
// If the channel is closed, Err returns a non-nil error explaining why:
// - ErrUnsubscribed if the subscriber choose to unsubscribe,
// - ErrOutOfCapacity if the subscriber is not pulling messages fast enough
// and the channel returned by Out became full,
// After Err returns a non-nil error, successive calls to Err return the same
// error.
func (s *Subscription) Err() error {
s.mtx.RLock()
defer s.mtx.RUnlock()
return s.err
}
func (s *Subscription) cancel(err error) {
s.mtx.Lock()
defer s.mtx.Unlock()
defer func() {
perr := recover()
if err == nil && perr != nil {
err = fmt.Errorf("problem closing subscription: %v", perr)
}
}()
// publish transmits msg to the subscriber. It reports a queue error if the
// queue cannot accept any further messages.
func (s *Subscription) publish(msg Message) error { return s.queue.Add(msg) }
if s.err == nil && err != nil {
s.err = err
// stop terminates the subscription with the given error reason.
func (s *Subscription) stop(err error) {
if err == nil {
panic("nil stop error")
}
s.stop()
s.stopErr = err
s.queue.Close()
}
// Message glues data and events together.
@ -138,14 +77,6 @@ type Message struct {
events []types.Event
}
func NewMessage(subID string, data interface{}, events []types.Event) Message {
return Message{
subID: subID,
data: data,
events: events,
}
}
// SubscriptionID returns the unique identifier for the subscription
// that produced this message.
func (msg Message) SubscriptionID() string { return msg.subID }


+ 3
- 2
node/node.go View File

@ -17,6 +17,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/internal/consensus"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/proxy"
@ -59,7 +60,7 @@ type nodeImpl struct {
isListening bool
// services
eventBus *types.EventBus // pub/sub for services
eventBus *eventbus.EventBus // pub/sub for services
eventSinks []indexer.EventSink
stateStore sm.Store
blockStore *store.BlockStore // store the blockchain to disk
@ -847,7 +848,7 @@ func (n *nodeImpl) Mempool() mempool.Mempool {
}
// EventBus returns the Node's EventBus.
func (n *nodeImpl) EventBus() *types.EventBus {
func (n *nodeImpl) EventBus() *eventbus.EventBus {
return n.eventBus
}


+ 9
- 7
node/node_test.go View File

@ -29,6 +29,7 @@ import (
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/pubsub"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/libs/service"
tmtime "github.com/tendermint/tendermint/libs/time"
@ -61,14 +62,15 @@ func TestNodeStartStop(t *testing.T) {
defer cancel()
// wait for the node to produce a block
blocksSub, err := n.EventBus().Subscribe(ctx, "node_test", types.EventQueryNewBlock)
blocksSub, err := n.EventBus().SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "node_test",
Query: types.EventQueryNewBlock,
})
require.NoError(t, err)
select {
case <-blocksSub.Out():
case <-blocksSub.Canceled():
t.Fatal("blocksSub was canceled")
case <-time.After(10 * time.Second):
t.Fatal("timed out waiting for the node to produce a block")
tctx, cancel := context.WithTimeout(ctx, 10*time.Second)
defer cancel()
if _, err := blocksSub.Next(tctx); err != nil {
t.Fatalf("Waiting for event: %v", err)
}
// stop the node


+ 5
- 4
node/setup.go View File

@ -14,6 +14,7 @@ import (
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/internal/blocksync"
"github.com/tendermint/tendermint/internal/consensus"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/evidence"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
@ -97,8 +98,8 @@ func createAndStartProxyAppConns(clientCreator abciclient.Creator, logger log.Lo
return proxyApp, nil
}
func createAndStartEventBus(logger log.Logger) (*types.EventBus, error) {
eventBus := types.NewEventBus()
func createAndStartEventBus(logger log.Logger) (*eventbus.EventBus, error) {
eventBus := eventbus.NewDefault()
eventBus.SetLogger(logger.With("module", "events"))
if err := eventBus.Start(); err != nil {
return nil, err
@ -109,7 +110,7 @@ func createAndStartEventBus(logger log.Logger) (*types.EventBus, error) {
func createAndStartIndexerService(
cfg *config.Config,
dbProvider config.DBProvider,
eventBus *types.EventBus,
eventBus *eventbus.EventBus,
logger log.Logger,
chainID string,
) (*indexer.Service, []indexer.EventSink, error) {
@ -315,7 +316,7 @@ func createConsensusReactor(
privValidator types.PrivValidator,
csMetrics *consensus.Metrics,
waitSync bool,
eventBus *types.EventBus,
eventBus *eventbus.EventBus,
peerManager *p2p.PeerManager,
router *p2p.Router,
logger log.Logger,


+ 45
- 48
rpc/client/local/local.go View File

@ -6,6 +6,7 @@ import (
"fmt"
"time"
"github.com/tendermint/tendermint/internal/eventbus"
rpccore "github.com/tendermint/tendermint/internal/rpc/core"
"github.com/tendermint/tendermint/libs/bytes"
"github.com/tendermint/tendermint/libs/log"
@ -38,7 +39,7 @@ don't need to do anything). It will keep trying indefinitely with exponential
backoff (10ms -> 20ms -> 40ms) until successful.
*/
type Local struct {
*types.EventBus
*eventbus.EventBus
Logger log.Logger
ctx *rpctypes.Context
env *rpccore.Environment
@ -48,7 +49,7 @@ type Local struct {
// local RPC client constructor needs to build a local client.
type NodeService interface {
RPCEnvironment() *rpccore.Environment
EventBus() *types.EventBus
EventBus() *eventbus.EventBus
}
// New configures a client that calls the Node directly.
@ -204,82 +205,78 @@ func (c *Local) Subscribe(
ctx context.Context,
subscriber,
queryString string,
outCapacity ...int) (out <-chan coretypes.ResultEvent, err error) {
capacity ...int) (out <-chan coretypes.ResultEvent, err error) {
q, err := query.New(queryString)
if err != nil {
return nil, fmt.Errorf("failed to parse query: %w", err)
}
outCap := 1
if len(outCapacity) > 0 {
outCap = outCapacity[0]
limit, quota := 1, 0
if len(capacity) > 0 {
limit = capacity[0]
if len(capacity) > 1 {
quota = capacity[1]
}
}
var sub types.Subscription
if outCap > 0 {
sub, err = c.EventBus.Subscribe(ctx, subscriber, q, outCap)
} else {
sub, err = c.EventBus.SubscribeUnbuffered(ctx, subscriber, q)
ctx, cancel := context.WithCancel(ctx)
go func() { <-c.Quit(); cancel() }()
subArgs := pubsub.SubscribeArgs{
ClientID: subscriber,
Query: q,
Quota: quota,
Limit: limit,
}
sub, err := c.EventBus.SubscribeWithArgs(ctx, subArgs)
if err != nil {
return nil, fmt.Errorf("failed to subscribe: %w", err)
}
outc := make(chan coretypes.ResultEvent, outCap)
go c.eventsRoutine(sub, subscriber, q, outc)
outc := make(chan coretypes.ResultEvent, 1)
go c.eventsRoutine(ctx, sub, subArgs, outc)
return outc, nil
}
func (c *Local) eventsRoutine(
sub types.Subscription,
subscriber string,
q pubsub.Query,
outc chan<- coretypes.ResultEvent) {
ctx context.Context,
sub eventbus.Subscription,
subArgs pubsub.SubscribeArgs,
outc chan<- coretypes.ResultEvent,
) {
qstr := subArgs.Query.String()
for {
select {
case msg := <-sub.Out():
result := coretypes.ResultEvent{
SubscriptionID: msg.SubscriptionID(),
Query: q.String(),
Data: msg.Data(),
Events: msg.Events(),
}
if cap(outc) == 0 {
outc <- result
} else {
select {
case outc <- result:
default:
c.Logger.Error("wanted to publish ResultEvent, but out channel is full", "result", result, "query", result.Query)
}
msg, err := sub.Next(ctx)
if errors.Is(err, pubsub.ErrUnsubscribed) {
return // client unsubscribed
} else if err != nil {
c.Logger.Error("subscription was canceled, resubscribing",
"err", err, "query", subArgs.Query.String())
sub = c.resubscribe(ctx, subArgs)
if sub == nil {
return // client terminated
}
case <-sub.Canceled():
if sub.Err() == pubsub.ErrUnsubscribed {
return
}
c.Logger.Error("subscription was canceled, resubscribing...", "err", sub.Err(), "query", q.String())
sub = c.resubscribe(subscriber, q)
if sub == nil { // client was stopped
return
}
case <-c.Quit():
return
continue
}
outc <- coretypes.ResultEvent{
SubscriptionID: msg.SubscriptionID(),
Query: qstr,
Data: msg.Data(),
Events: msg.Events(),
}
}
}
// Try to resubscribe with exponential backoff.
func (c *Local) resubscribe(subscriber string, q pubsub.Query) types.Subscription {
func (c *Local) resubscribe(ctx context.Context, subArgs pubsub.SubscribeArgs) eventbus.Subscription {
attempts := 0
for {
if !c.IsRunning() {
return nil
}
sub, err := c.EventBus.Subscribe(context.Background(), subscriber, q)
sub, err := c.EventBus.SubscribeWithArgs(ctx, subArgs)
if err == nil {
return sub
}


+ 0
- 326
types/event_bus.go View File

@ -1,326 +0,0 @@
package types
import (
"context"
"fmt"
"strings"
"github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/service"
)
const defaultCapacity = 0
type EventBusSubscriber interface {
Subscribe(ctx context.Context, subscriber string, query tmpubsub.Query, outCapacity ...int) (Subscription, error)
Unsubscribe(ctx context.Context, args tmpubsub.UnsubscribeArgs) error
UnsubscribeAll(ctx context.Context, subscriber string) error
NumClients() int
NumClientSubscriptions(clientID string) int
}
type Subscription interface {
ID() string
Out() <-chan tmpubsub.Message
Canceled() <-chan struct{}
Err() error
}
// EventBus is a common bus for all events going through the system. All calls
// are proxied to underlying pubsub server. All events must be published using
// EventBus to ensure correct data types.
type EventBus struct {
service.BaseService
pubsub *tmpubsub.Server
}
// NewEventBus returns a new event bus.
func NewEventBus() *EventBus {
return NewEventBusWithBufferCapacity(defaultCapacity)
}
// NewEventBusWithBufferCapacity returns a new event bus with the given buffer capacity.
func NewEventBusWithBufferCapacity(cap int) *EventBus {
// capacity could be exposed later if needed
pubsub := tmpubsub.NewServer(tmpubsub.BufferCapacity(cap))
b := &EventBus{pubsub: pubsub}
b.BaseService = *service.NewBaseService(nil, "EventBus", b)
return b
}
func (b *EventBus) SetLogger(l log.Logger) {
b.BaseService.SetLogger(l)
b.pubsub.SetLogger(l.With("module", "pubsub"))
}
func (b *EventBus) OnStart() error {
return b.pubsub.Start()
}
func (b *EventBus) OnStop() {
if err := b.pubsub.Stop(); err != nil {
b.pubsub.Logger.Error("error trying to stop eventBus", "error", err)
}
}
func (b *EventBus) NumClients() int {
return b.pubsub.NumClients()
}
func (b *EventBus) NumClientSubscriptions(clientID string) int {
return b.pubsub.NumClientSubscriptions(clientID)
}
func (b *EventBus) Subscribe(
ctx context.Context,
subscriber string,
query tmpubsub.Query,
outCapacity ...int,
) (Subscription, error) {
return b.pubsub.Subscribe(ctx, subscriber, query, outCapacity...)
}
// This method can be used for a local consensus explorer and synchronous
// testing. Do not use for for public facing / untrusted subscriptions!
func (b *EventBus) SubscribeUnbuffered(
ctx context.Context,
subscriber string,
query tmpubsub.Query,
) (Subscription, error) {
return b.pubsub.SubscribeUnbuffered(ctx, subscriber, query)
}
func (b *EventBus) Unsubscribe(ctx context.Context, args tmpubsub.UnsubscribeArgs) error {
return b.pubsub.Unsubscribe(ctx, args)
}
func (b *EventBus) UnsubscribeAll(ctx context.Context, subscriber string) error {
return b.pubsub.UnsubscribeAll(ctx, subscriber)
}
func (b *EventBus) Publish(eventValue string, eventData TMEventData) error {
// no explicit deadline for publishing events
ctx := context.Background()
tokens := strings.Split(EventTypeKey, ".")
event := types.Event{
Type: tokens[0],
Attributes: []types.EventAttribute{
{
Key: tokens[1],
Value: eventValue,
},
},
}
return b.pubsub.PublishWithEvents(ctx, eventData, []types.Event{event})
}
func (b *EventBus) PublishEventNewBlock(data EventDataNewBlock) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := append(data.ResultBeginBlock.Events, data.ResultEndBlock.Events...)
// add Tendermint-reserved new block event
events = append(events, EventNewBlock)
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewBlockHeader(data EventDataNewBlockHeader) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := append(data.ResultBeginBlock.Events, data.ResultEndBlock.Events...)
// add Tendermint-reserved new block header event
events = append(events, EventNewBlockHeader)
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewEvidence(evidence EventDataNewEvidence) error {
return b.Publish(EventNewEvidenceValue, evidence)
}
func (b *EventBus) PublishEventVote(data EventDataVote) error {
return b.Publish(EventVoteValue, data)
}
func (b *EventBus) PublishEventValidBlock(data EventDataRoundState) error {
return b.Publish(EventValidBlockValue, data)
}
func (b *EventBus) PublishEventBlockSyncStatus(data EventDataBlockSyncStatus) error {
return b.Publish(EventBlockSyncStatusValue, data)
}
func (b *EventBus) PublishEventStateSyncStatus(data EventDataStateSyncStatus) error {
return b.Publish(EventStateSyncStatusValue, data)
}
// PublishEventTx publishes tx event with events from Result. Note it will add
// predefined keys (EventTypeKey, TxHashKey). Existing events with the same keys
// will be overwritten.
func (b *EventBus) PublishEventTx(data EventDataTx) error {
// no explicit deadline for publishing events
ctx := context.Background()
events := data.Result.Events
// add Tendermint-reserved events
events = append(events, EventTx)
tokens := strings.Split(TxHashKey, ".")
events = append(events, types.Event{
Type: tokens[0],
Attributes: []types.EventAttribute{
{
Key: tokens[1],
Value: fmt.Sprintf("%X", Tx(data.Tx).Hash()),
},
},
})
tokens = strings.Split(TxHeightKey, ".")
events = append(events, types.Event{
Type: tokens[0],
Attributes: []types.EventAttribute{
{
Key: tokens[1],
Value: fmt.Sprintf("%d", data.Height),
},
},
})
return b.pubsub.PublishWithEvents(ctx, data, events)
}
func (b *EventBus) PublishEventNewRoundStep(data EventDataRoundState) error {
return b.Publish(EventNewRoundStepValue, data)
}
func (b *EventBus) PublishEventTimeoutPropose(data EventDataRoundState) error {
return b.Publish(EventTimeoutProposeValue, data)
}
func (b *EventBus) PublishEventTimeoutWait(data EventDataRoundState) error {
return b.Publish(EventTimeoutWaitValue, data)
}
func (b *EventBus) PublishEventNewRound(data EventDataNewRound) error {
return b.Publish(EventNewRoundValue, data)
}
func (b *EventBus) PublishEventCompleteProposal(data EventDataCompleteProposal) error {
return b.Publish(EventCompleteProposalValue, data)
}
func (b *EventBus) PublishEventPolka(data EventDataRoundState) error {
return b.Publish(EventPolkaValue, data)
}
func (b *EventBus) PublishEventUnlock(data EventDataRoundState) error {
return b.Publish(EventUnlockValue, data)
}
func (b *EventBus) PublishEventRelock(data EventDataRoundState) error {
return b.Publish(EventRelockValue, data)
}
func (b *EventBus) PublishEventLock(data EventDataRoundState) error {
return b.Publish(EventLockValue, data)
}
func (b *EventBus) PublishEventValidatorSetUpdates(data EventDataValidatorSetUpdates) error {
return b.Publish(EventValidatorSetUpdatesValue, data)
}
//-----------------------------------------------------------------------------
type NopEventBus struct{}
func (NopEventBus) Subscribe(
ctx context.Context,
subscriber string,
query tmpubsub.Query,
out chan<- interface{},
) error {
return nil
}
func (NopEventBus) Unsubscribe(ctx context.Context, args tmpubsub.UnsubscribeArgs) error {
return nil
}
func (NopEventBus) UnsubscribeAll(ctx context.Context, subscriber string) error {
return nil
}
func (NopEventBus) PublishEventNewBlock(data EventDataNewBlock) error {
return nil
}
func (NopEventBus) PublishEventNewBlockHeader(data EventDataNewBlockHeader) error {
return nil
}
func (NopEventBus) PublishEventNewEvidence(evidence EventDataNewEvidence) error {
return nil
}
func (NopEventBus) PublishEventVote(data EventDataVote) error {
return nil
}
func (NopEventBus) PublishEventTx(data EventDataTx) error {
return nil
}
func (NopEventBus) PublishEventNewRoundStep(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventTimeoutPropose(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventTimeoutWait(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventNewRound(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventCompleteProposal(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventPolka(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventUnlock(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventRelock(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventLock(data EventDataRoundState) error {
return nil
}
func (NopEventBus) PublishEventValidatorSetUpdates(data EventDataValidatorSetUpdates) error {
return nil
}
func (NopEventBus) PublishEventBlockSyncStatus(data EventDataBlockSyncStatus) error {
return nil
}
func (NopEventBus) PublishEventStateSyncStatus(data EventDataStateSyncStatus) error {
return nil
}

Loading…
Cancel
Save