diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index a800b8d33..45762880f 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -82,6 +82,7 @@ Friendly reminder: We have a [bug bounty program](https://hackerone.com/tendermi - Tooling - [tools] \#6498 Set OS home dir to instead of the hardcoded PATH. (@JayT106) + - [cli/indexer] \#6676 Reindex events command line tooling. (@JayT106) ### FEATURES diff --git a/cmd/tendermint/commands/reindex_event.go b/cmd/tendermint/commands/reindex_event.go new file mode 100644 index 000000000..ddc585c1f --- /dev/null +++ b/cmd/tendermint/commands/reindex_event.go @@ -0,0 +1,251 @@ +package commands + +import ( + "errors" + "fmt" + "strings" + + "github.com/spf13/cobra" + tmdb "github.com/tendermint/tm-db" + + abcitypes "github.com/tendermint/tendermint/abci/types" + tmcfg "github.com/tendermint/tendermint/config" + "github.com/tendermint/tendermint/internal/libs/progressbar" + ctypes "github.com/tendermint/tendermint/rpc/core/types" + "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/state/indexer" + "github.com/tendermint/tendermint/state/indexer/sink/kv" + "github.com/tendermint/tendermint/state/indexer/sink/psql" + "github.com/tendermint/tendermint/store" + "github.com/tendermint/tendermint/types" +) + +const ( + reindexFailed = "event re-index failed: " +) + +// ReIndexEventCmd allows re-index the event by given block height interval +var ReIndexEventCmd = &cobra.Command{ + Use: "reindex-event", + Short: "reindex events to the event store backends", + Long: ` + reindex-event is an offline tooling to re-index block and tx events to the eventsinks, + you can run this command when the event store backend dropped/disconnected or you want to replace the backend. + The default start-height is 0, meaning the tooling will start reindex from the base block height(inclusive); and the + default end-height is 0, meaning the tooling will reindex until the latest block height(inclusive). User can omits + either or both arguments. + `, + Example: ` + tendermint reindex-event + tendermint reindex-event --start-height 2 + tendermint reindex-event --end-height 10 + tendermint reindex-event --start-height 2 --end-height 10 + `, + Run: func(cmd *cobra.Command, args []string) { + bs, ss, err := loadStateAndBlockStore(config) + if err != nil { + fmt.Println(reindexFailed, err) + return + } + + if err := checkValidHeight(bs); err != nil { + fmt.Println(reindexFailed, err) + return + } + + es, err := loadEventSinks(config) + if err != nil { + fmt.Println(reindexFailed, err) + return + } + + if err = eventReIndex(cmd, es, bs, ss); err != nil { + fmt.Println(reindexFailed, err) + return + } + + fmt.Println("event re-index finished") + }, +} + +var ( + startHeight int64 + endHeight int64 +) + +func init() { + ReIndexEventCmd.Flags().Int64Var(&startHeight, "start-height", 0, "the block height would like to start for re-index") + ReIndexEventCmd.Flags().Int64Var(&endHeight, "end-height", 0, "the block height would like to finish for re-index") +} + +func loadEventSinks(cfg *tmcfg.Config) ([]indexer.EventSink, error) { + // Check duplicated sinks. + sinks := map[string]bool{} + for _, s := range cfg.TxIndex.Indexer { + sl := strings.ToLower(s) + if sinks[sl] { + return nil, errors.New("found duplicated sinks, please check the tx-index section in the config.toml") + } + sinks[sl] = true + } + + eventSinks := []indexer.EventSink{} + + for k := range sinks { + switch k { + case string(indexer.NULL): + return nil, errors.New("found null event sink, please check the tx-index section in the config.toml") + case string(indexer.KV): + store, err := tmcfg.DefaultDBProvider(&tmcfg.DBContext{ID: "tx_index", Config: cfg}) + if err != nil { + return nil, err + } + eventSinks = append(eventSinks, kv.NewEventSink(store)) + case string(indexer.PSQL): + conn := cfg.TxIndex.PsqlConn + if conn == "" { + return nil, errors.New("the psql connection settings cannot be empty") + } + es, _, err := psql.NewEventSink(conn, chainID) + if err != nil { + return nil, err + } + eventSinks = append(eventSinks, es) + default: + return nil, errors.New("unsupported event sink type") + } + } + + if len(eventSinks) == 0 { + return nil, errors.New("no proper event sink can do event re-indexing," + + " please check the tx-index section in the config.toml") + } + + if !indexer.IndexingEnabled(eventSinks) { + return nil, fmt.Errorf("no event sink has been enabled") + } + + return eventSinks, nil +} + +func loadStateAndBlockStore(cfg *tmcfg.Config) (*store.BlockStore, state.Store, error) { + dbType := tmdb.BackendType(cfg.DBBackend) + + // Get BlockStore + blockStoreDB, err := tmdb.NewDB("blockstore", dbType, cfg.DBDir()) + if err != nil { + return nil, nil, err + } + blockStore := store.NewBlockStore(blockStoreDB) + + // Get StateStore + stateDB, err := tmdb.NewDB("state", dbType, cfg.DBDir()) + if err != nil { + return nil, nil, err + } + stateStore := state.NewStore(stateDB) + + return blockStore, stateStore, nil +} + +func eventReIndex(cmd *cobra.Command, es []indexer.EventSink, bs state.BlockStore, ss state.Store) error { + + var bar progressbar.Bar + bar.NewOption(startHeight-1, endHeight) + + fmt.Println("start re-indexing events:") + defer bar.Finish() + for i := startHeight; i <= endHeight; i++ { + select { + case <-cmd.Context().Done(): + return fmt.Errorf("event re-index terminated at height %d: %w", i, cmd.Context().Err()) + default: + b := bs.LoadBlock(i) + if b == nil { + return fmt.Errorf("not able to load block at height %d from the blockstore", i) + } + + r, err := ss.LoadABCIResponses(i) + if err != nil { + return fmt.Errorf("not able to load ABCI Response at height %d from the statestore", i) + } + + e := types.EventDataNewBlockHeader{ + Header: b.Header, + NumTxs: int64(len(b.Txs)), + ResultBeginBlock: *r.BeginBlock, + ResultEndBlock: *r.EndBlock, + } + + var batch *indexer.Batch + if e.NumTxs > 0 { + batch = indexer.NewBatch(e.NumTxs) + + for i, tx := range b.Data.Txs { + tr := abcitypes.TxResult{ + Height: b.Height, + Index: uint32(i), + Tx: tx, + Result: *(r.DeliverTxs[i]), + } + + _ = batch.Add(&tr) + } + } + + for _, sink := range es { + if err := sink.IndexBlockEvents(e); err != nil { + return fmt.Errorf("block event re-index at height %d failed: %w", i, err) + } + + if batch != nil { + if err := sink.IndexTxEvents(batch.Ops); err != nil { + return fmt.Errorf("tx event re-index at height %d failed: %w", i, err) + } + } + } + } + + bar.Play(i) + } + + return nil +} + +func checkValidHeight(bs state.BlockStore) error { + base := bs.Base() + + if startHeight == 0 { + startHeight = base + fmt.Printf("set the start block height to the base height of the blockstore %d \n", base) + } + + if startHeight < base { + return fmt.Errorf("%s (requested start height: %d, base height: %d)", ctypes.ErrHeightNotAvailable, startHeight, base) + } + + height := bs.Height() + + if startHeight > height { + return fmt.Errorf( + "%s (requested start height: %d, store height: %d)", ctypes.ErrHeightNotAvailable, startHeight, height) + } + + if endHeight == 0 || endHeight > height { + endHeight = height + fmt.Printf("set the end block height to the latest height of the blockstore %d \n", height) + } + + if endHeight < base { + return fmt.Errorf( + "%s (requested end height: %d, base height: %d)", ctypes.ErrHeightNotAvailable, endHeight, base) + } + + if endHeight < startHeight { + return fmt.Errorf( + "%s (requested the end height: %d is less than the start height: %d)", + ctypes.ErrInvalidRequest, startHeight, endHeight) + } + + return nil +} diff --git a/cmd/tendermint/commands/reindex_event_test.go b/cmd/tendermint/commands/reindex_event_test.go new file mode 100644 index 000000000..5d9459f5a --- /dev/null +++ b/cmd/tendermint/commands/reindex_event_test.go @@ -0,0 +1,171 @@ +package commands + +import ( + "context" + "errors" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + abcitypes "github.com/tendermint/tendermint/abci/types" + tmcfg "github.com/tendermint/tendermint/config" + prototmstate "github.com/tendermint/tendermint/proto/tendermint/state" + "github.com/tendermint/tendermint/state/indexer" + "github.com/tendermint/tendermint/state/mocks" + "github.com/tendermint/tendermint/types" +) + +const ( + height int64 = 10 + base int64 = 2 +) + +func setupReIndexEventCmd() *cobra.Command { + reIndexEventCmd := &cobra.Command{ + Use: ReIndexEventCmd.Use, + Run: func(cmd *cobra.Command, args []string) {}, + } + + _ = reIndexEventCmd.ExecuteContext(context.Background()) + + return reIndexEventCmd +} + +func TestReIndexEventCheckHeight(t *testing.T) { + mockBlockStore := &mocks.BlockStore{} + mockBlockStore. + On("Base").Return(base). + On("Height").Return(height) + + testCases := []struct { + startHeight int64 + endHeight int64 + validHeight bool + }{ + {0, 0, true}, + {0, base, true}, + {0, base - 1, false}, + {0, height, true}, + {0, height + 1, true}, + {0, 0, true}, + {base - 1, 0, false}, + {base, 0, true}, + {base, base, true}, + {base, base - 1, false}, + {base, height, true}, + {base, height + 1, true}, + {height, 0, true}, + {height, base, false}, + {height, height - 1, false}, + {height, height, true}, + {height, height + 1, true}, + {height + 1, 0, false}, + } + + for _, tc := range testCases { + startHeight = tc.startHeight + endHeight = tc.endHeight + + err := checkValidHeight(mockBlockStore) + if tc.validHeight { + require.NoError(t, err) + } else { + require.Error(t, err) + } + } +} + +func TestLoadEventSink(t *testing.T) { + testCases := []struct { + sinks []string + connURL string + loadErr bool + }{ + {[]string{}, "", true}, + {[]string{"NULL"}, "", true}, + {[]string{"KV"}, "", false}, + {[]string{"KV", "KV"}, "", true}, + {[]string{"PSQL"}, "", true}, // true because empty connect url + {[]string{"PSQL"}, "wrongUrl", true}, // true because wrong connect url + // skip to test PSQL connect with correct url + {[]string{"UnsupportedSinkType"}, "wrongUrl", true}, + } + + for _, tc := range testCases { + cfg := tmcfg.TestConfig() + cfg.TxIndex.Indexer = tc.sinks + cfg.TxIndex.PsqlConn = tc.connURL + _, err := loadEventSinks(cfg) + if tc.loadErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + } +} + +func TestLoadBlockStore(t *testing.T) { + bs, ss, err := loadStateAndBlockStore(tmcfg.TestConfig()) + require.NoError(t, err) + require.NotNil(t, bs) + require.NotNil(t, ss) + +} +func TestReIndexEvent(t *testing.T) { + mockBlockStore := &mocks.BlockStore{} + mockStateStore := &mocks.Store{} + mockEventSink := &mocks.EventSink{} + + mockBlockStore. + On("Base").Return(base). + On("Height").Return(height). + On("LoadBlock", base).Return(nil).Once(). + On("LoadBlock", base).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}}). + On("LoadBlock", height).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}}) + + mockEventSink. + On("Type").Return(indexer.KV). + On("IndexBlockEvents", mock.AnythingOfType("types.EventDataNewBlockHeader")).Return(errors.New("")).Once(). + On("IndexBlockEvents", mock.AnythingOfType("types.EventDataNewBlockHeader")).Return(nil). + On("IndexTxEvents", mock.AnythingOfType("[]*types.TxResult")).Return(errors.New("")).Once(). + On("IndexTxEvents", mock.AnythingOfType("[]*types.TxResult")).Return(nil) + + dtx := abcitypes.ResponseDeliverTx{} + abciResp := &prototmstate.ABCIResponses{ + DeliverTxs: []*abcitypes.ResponseDeliverTx{&dtx}, + EndBlock: &abcitypes.ResponseEndBlock{}, + BeginBlock: &abcitypes.ResponseBeginBlock{}, + } + + mockStateStore. + On("LoadABCIResponses", base).Return(nil, errors.New("")).Once(). + On("LoadABCIResponses", base).Return(abciResp, nil). + On("LoadABCIResponses", height).Return(abciResp, nil) + + testCases := []struct { + startHeight int64 + endHeight int64 + reIndexErr bool + }{ + {base, height, true}, // LoadBlock error + {base, height, true}, // LoadABCIResponses error + {base, height, true}, // index block event error + {base, height, true}, // index tx event error + {base, base, false}, + {height, height, false}, + } + + for _, tc := range testCases { + startHeight = tc.startHeight + endHeight = tc.endHeight + + err := eventReIndex(setupReIndexEventCmd(), []indexer.EventSink{mockEventSink}, mockBlockStore, mockStateStore) + if tc.reIndexErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + } +} diff --git a/cmd/tendermint/main.go b/cmd/tendermint/main.go index 52a26b4f3..b40624cc3 100644 --- a/cmd/tendermint/main.go +++ b/cmd/tendermint/main.go @@ -15,6 +15,7 @@ func main() { rootCmd := cmd.RootCmd rootCmd.AddCommand( cmd.GenValidatorCmd, + cmd.ReIndexEventCmd, cmd.InitFilesCmd, cmd.ProbeUpnpCmd, cmd.LightCmd, diff --git a/internal/libs/progressbar/progressbar.go b/internal/libs/progressbar/progressbar.go new file mode 100644 index 000000000..072804c76 --- /dev/null +++ b/internal/libs/progressbar/progressbar.go @@ -0,0 +1,41 @@ +package progressbar + +import "fmt" + +// the progressbar indicates the current status and progress would be desired. +// ref: https://www.pixelstech.net/article/1596946473-A-simple-example-on-implementing-progress-bar-in-GoLang + +type Bar struct { + percent int64 // progress percentage + cur int64 // current progress + start int64 // the init starting value for progress + total int64 // total value for progress + rate string // the actual progress bar to be printed + graph string // the fill value for progress bar +} + +func (bar *Bar) NewOption(start, total int64) { + bar.cur = start + bar.start = start + bar.total = total + bar.graph = "█" + bar.percent = bar.getPercent() +} + +func (bar *Bar) getPercent() int64 { + return int64(float32(bar.cur-bar.start) / float32(bar.total-bar.start) * 100) +} + +func (bar *Bar) Play(cur int64) { + bar.cur = cur + last := bar.percent + bar.percent = bar.getPercent() + if bar.percent != last && bar.percent%2 == 0 { + bar.rate += bar.graph + } + fmt.Printf("\r[%-50s]%3d%% %8d/%d", bar.rate, bar.percent, bar.cur, bar.total) +} + +func (bar *Bar) Finish() { + fmt.Println() +} diff --git a/internal/libs/progressbar/progressbar_test.go b/internal/libs/progressbar/progressbar_test.go new file mode 100644 index 000000000..d135748f6 --- /dev/null +++ b/internal/libs/progressbar/progressbar_test.go @@ -0,0 +1,41 @@ +package progressbar + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestProgressBar(t *testing.T) { + zero := int64(0) + hundred := int64(100) + + var bar Bar + bar.NewOption(zero, hundred) + + require.Equal(t, zero, bar.start) + require.Equal(t, zero, bar.cur) + require.Equal(t, hundred, bar.total) + require.Equal(t, zero, bar.percent) + require.Equal(t, "█", bar.graph) + require.Equal(t, "", bar.rate) + + defer bar.Finish() + for i := zero; i <= hundred; i++ { + time.Sleep(1 * time.Millisecond) + bar.Play(i) + } + + require.Equal(t, zero, bar.start) + require.Equal(t, hundred, bar.cur) + require.Equal(t, hundred, bar.total) + require.Equal(t, hundred, bar.percent) + + var rate string + for i := zero; i < hundred/2; i++ { + rate += "█" + } + + require.Equal(t, rate, bar.rate) +} diff --git a/state/indexer/eventsink.go b/state/indexer/eventsink.go index 8c2529103..19d93e4e5 100644 --- a/state/indexer/eventsink.go +++ b/state/indexer/eventsink.go @@ -16,6 +16,8 @@ const ( PSQL EventSinkType = "psql" ) +//go:generate mockery --case underscore --name EventSink + // EventSink interface is defined the APIs for the IndexerService to interact with the data store, // including the block/transaction indexing and the search functions. // diff --git a/state/mocks/block_store.go b/state/mocks/block_store.go new file mode 100644 index 000000000..99cadd081 --- /dev/null +++ b/state/mocks/block_store.go @@ -0,0 +1,194 @@ +// Code generated by mockery 2.7.5. DO NOT EDIT. + +package mocks + +import ( + mock "github.com/stretchr/testify/mock" + + types "github.com/tendermint/tendermint/types" +) + +// BlockStore is an autogenerated mock type for the BlockStore type +type BlockStore struct { + mock.Mock +} + +// Base provides a mock function with given fields: +func (_m *BlockStore) Base() int64 { + ret := _m.Called() + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// Height provides a mock function with given fields: +func (_m *BlockStore) Height() int64 { + ret := _m.Called() + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// LoadBaseMeta provides a mock function with given fields: +func (_m *BlockStore) LoadBaseMeta() *types.BlockMeta { + ret := _m.Called() + + var r0 *types.BlockMeta + if rf, ok := ret.Get(0).(func() *types.BlockMeta); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.BlockMeta) + } + } + + return r0 +} + +// LoadBlock provides a mock function with given fields: height +func (_m *BlockStore) LoadBlock(height int64) *types.Block { + ret := _m.Called(height) + + var r0 *types.Block + if rf, ok := ret.Get(0).(func(int64) *types.Block); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Block) + } + } + + return r0 +} + +// LoadBlockByHash provides a mock function with given fields: hash +func (_m *BlockStore) LoadBlockByHash(hash []byte) *types.Block { + ret := _m.Called(hash) + + var r0 *types.Block + if rf, ok := ret.Get(0).(func([]byte) *types.Block); ok { + r0 = rf(hash) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Block) + } + } + + return r0 +} + +// LoadBlockCommit provides a mock function with given fields: height +func (_m *BlockStore) LoadBlockCommit(height int64) *types.Commit { + ret := _m.Called(height) + + var r0 *types.Commit + if rf, ok := ret.Get(0).(func(int64) *types.Commit); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Commit) + } + } + + return r0 +} + +// LoadBlockMeta provides a mock function with given fields: height +func (_m *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta { + ret := _m.Called(height) + + var r0 *types.BlockMeta + if rf, ok := ret.Get(0).(func(int64) *types.BlockMeta); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.BlockMeta) + } + } + + return r0 +} + +// LoadBlockPart provides a mock function with given fields: height, index +func (_m *BlockStore) LoadBlockPart(height int64, index int) *types.Part { + ret := _m.Called(height, index) + + var r0 *types.Part + if rf, ok := ret.Get(0).(func(int64, int) *types.Part); ok { + r0 = rf(height, index) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Part) + } + } + + return r0 +} + +// LoadSeenCommit provides a mock function with given fields: height +func (_m *BlockStore) LoadSeenCommit(height int64) *types.Commit { + ret := _m.Called(height) + + var r0 *types.Commit + if rf, ok := ret.Get(0).(func(int64) *types.Commit); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Commit) + } + } + + return r0 +} + +// PruneBlocks provides a mock function with given fields: height +func (_m *BlockStore) PruneBlocks(height int64) (uint64, error) { + ret := _m.Called(height) + + var r0 uint64 + if rf, ok := ret.Get(0).(func(int64) uint64); ok { + r0 = rf(height) + } else { + r0 = ret.Get(0).(uint64) + } + + var r1 error + if rf, ok := ret.Get(1).(func(int64) error); ok { + r1 = rf(height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SaveBlock provides a mock function with given fields: block, blockParts, seenCommit +func (_m *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { + _m.Called(block, blockParts, seenCommit) +} + +// Size provides a mock function with given fields: +func (_m *BlockStore) Size() int64 { + ret := _m.Called() + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} \ No newline at end of file diff --git a/state/mocks/event_sink.go b/state/mocks/event_sink.go new file mode 100644 index 000000000..749515ccf --- /dev/null +++ b/state/mocks/event_sink.go @@ -0,0 +1,167 @@ +// Code generated by mockery 2.7.5. DO NOT EDIT. + +package mocks + +import ( + context "context" + + mock "github.com/stretchr/testify/mock" + indexer "github.com/tendermint/tendermint/state/indexer" + + query "github.com/tendermint/tendermint/libs/pubsub/query" + + tenderminttypes "github.com/tendermint/tendermint/types" + + types "github.com/tendermint/tendermint/abci/types" +) + +// EventSink is an autogenerated mock type for the EventSink type +type EventSink struct { + mock.Mock +} + +// GetTxByHash provides a mock function with given fields: _a0 +func (_m *EventSink) GetTxByHash(_a0 []byte) (*types.TxResult, error) { + ret := _m.Called(_a0) + + var r0 *types.TxResult + if rf, ok := ret.Get(0).(func([]byte) *types.TxResult); ok { + r0 = rf(_a0) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.TxResult) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func([]byte) error); ok { + r1 = rf(_a0) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// HasBlock provides a mock function with given fields: _a0 +func (_m *EventSink) HasBlock(_a0 int64) (bool, error) { + ret := _m.Called(_a0) + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(_a0) + } else { + r0 = ret.Get(0).(bool) + } + + var r1 error + if rf, ok := ret.Get(1).(func(int64) error); ok { + r1 = rf(_a0) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// IndexBlockEvents provides a mock function with given fields: _a0 +func (_m *EventSink) IndexBlockEvents(_a0 tenderminttypes.EventDataNewBlockHeader) error { + ret := _m.Called(_a0) + + var r0 error + if rf, ok := ret.Get(0).(func(tenderminttypes.EventDataNewBlockHeader) error); ok { + r0 = rf(_a0) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// IndexTxEvents provides a mock function with given fields: _a0 +func (_m *EventSink) IndexTxEvents(_a0 []*types.TxResult) error { + ret := _m.Called(_a0) + + var r0 error + if rf, ok := ret.Get(0).(func([]*types.TxResult) error); ok { + r0 = rf(_a0) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// SearchBlockEvents provides a mock function with given fields: _a0, _a1 +func (_m *EventSink) SearchBlockEvents(_a0 context.Context, _a1 *query.Query) ([]int64, error) { + ret := _m.Called(_a0, _a1) + + var r0 []int64 + if rf, ok := ret.Get(0).(func(context.Context, *query.Query) []int64); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *query.Query) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SearchTxEvents provides a mock function with given fields: _a0, _a1 +func (_m *EventSink) SearchTxEvents(_a0 context.Context, _a1 *query.Query) ([]*types.TxResult, error) { + ret := _m.Called(_a0, _a1) + + var r0 []*types.TxResult + if rf, ok := ret.Get(0).(func(context.Context, *query.Query) []*types.TxResult); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*types.TxResult) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, *query.Query) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Stop provides a mock function with given fields: +func (_m *EventSink) Stop() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Type provides a mock function with given fields: +func (_m *EventSink) Type() indexer.EventSinkType { + ret := _m.Called() + + var r0 indexer.EventSinkType + if rf, ok := ret.Get(0).(func() indexer.EventSinkType); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(indexer.EventSinkType) + } + + return r0 +} diff --git a/state/services.go b/state/services.go index a46863904..0ab6b78d0 100644 --- a/state/services.go +++ b/state/services.go @@ -9,6 +9,8 @@ import ( // NOTE: Interfaces used by RPC must be thread safe! //------------------------------------------------------ +//go:generate mockery --case underscore --name BlockStore + //------------------------------------------------------ // blockstore