eth/filters: remove use of event.TypeMux for pending logs (#20312)

This commit is contained in:
Felix Lange 2019-12-10 12:39:14 +01:00 committed by GitHub
parent b8bc9b3d8e
commit d90d1db609
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 240 additions and 247 deletions

View File

@ -76,7 +76,7 @@ func NewSimulatedBackendWithDatabase(database ethdb.Database, alloc core.Genesis
database: database, database: database,
blockchain: blockchain, blockchain: blockchain,
config: genesis.Config, config: genesis.Config,
events: filters.NewEventSystem(new(event.TypeMux), &filterBackend{database, blockchain}, false), events: filters.NewEventSystem(&filterBackend{database, blockchain}, false),
} }
backend.rollback() backend.rollback()
return backend return backend
@ -502,22 +502,34 @@ func (fb *filterBackend) GetLogs(ctx context.Context, hash common.Hash) ([][]*ty
} }
func (fb *filterBackend) SubscribeNewTxsEvent(ch chan<- core.NewTxsEvent) event.Subscription { func (fb *filterBackend) SubscribeNewTxsEvent(ch chan<- core.NewTxsEvent) event.Subscription {
return nullSubscription()
}
func (fb *filterBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
return fb.bc.SubscribeChainEvent(ch)
}
func (fb *filterBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription {
return fb.bc.SubscribeRemovedLogsEvent(ch)
}
func (fb *filterBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription {
return fb.bc.SubscribeLogsEvent(ch)
}
func (fb *filterBackend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
return nullSubscription()
}
func (fb *filterBackend) BloomStatus() (uint64, uint64) { return 4096, 0 }
func (fb *filterBackend) ServiceFilter(ctx context.Context, ms *bloombits.MatcherSession) {
panic("not supported")
}
func nullSubscription() event.Subscription {
return event.NewSubscription(func(quit <-chan struct{}) error { return event.NewSubscription(func(quit <-chan struct{}) error {
<-quit <-quit
return nil return nil
}) })
} }
func (fb *filterBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
return fb.bc.SubscribeChainEvent(ch)
}
func (fb *filterBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription {
return fb.bc.SubscribeRemovedLogsEvent(ch)
}
func (fb *filterBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription {
return fb.bc.SubscribeLogsEvent(ch)
}
func (fb *filterBackend) BloomStatus() (uint64, uint64) { return 4096, 0 }
func (fb *filterBackend) ServiceFilter(ctx context.Context, ms *bloombits.MatcherSession) {
panic("not supported")
}

View File

@ -24,11 +24,6 @@ import (
// NewTxsEvent is posted when a batch of transactions enter the transaction pool. // NewTxsEvent is posted when a batch of transactions enter the transaction pool.
type NewTxsEvent struct{ Txs []*types.Transaction } type NewTxsEvent struct{ Txs []*types.Transaction }
// PendingLogsEvent is posted pre mining and notifies of pending logs.
type PendingLogsEvent struct {
Logs []*types.Log
}
// NewMinedBlockEvent is posted when a block has been imported. // NewMinedBlockEvent is posted when a block has been imported.
type NewMinedBlockEvent struct{ Block *types.Block } type NewMinedBlockEvent struct{ Block *types.Block }

View File

@ -202,6 +202,10 @@ func (b *EthAPIBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEven
return b.eth.BlockChain().SubscribeRemovedLogsEvent(ch) return b.eth.BlockChain().SubscribeRemovedLogsEvent(ch)
} }
func (b *EthAPIBackend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
return b.eth.miner.SubscribePendingLogs(ch)
}
func (b *EthAPIBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription { func (b *EthAPIBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
return b.eth.BlockChain().SubscribeChainEvent(ch) return b.eth.BlockChain().SubscribeChainEvent(ch)
} }

View File

@ -65,9 +65,8 @@ type PublicFilterAPI struct {
func NewPublicFilterAPI(backend Backend, lightMode bool) *PublicFilterAPI { func NewPublicFilterAPI(backend Backend, lightMode bool) *PublicFilterAPI {
api := &PublicFilterAPI{ api := &PublicFilterAPI{
backend: backend, backend: backend,
mux: backend.EventMux(),
chainDb: backend.ChainDb(), chainDb: backend.ChainDb(),
events: NewEventSystem(backend.EventMux(), backend, lightMode), events: NewEventSystem(backend, lightMode),
filters: make(map[rpc.ID]*filter), filters: make(map[rpc.ID]*filter),
} }
go api.timeoutLoop() go api.timeoutLoop()
@ -428,7 +427,7 @@ func (api *PublicFilterAPI) GetFilterChanges(id rpc.ID) (interface{}, error) {
hashes := f.hashes hashes := f.hashes
f.hashes = nil f.hashes = nil
return returnHashes(hashes), nil return returnHashes(hashes), nil
case LogsSubscription: case LogsSubscription, MinedAndPendingLogsSubscription:
logs := f.logs logs := f.logs
f.logs = nil f.logs = nil
return returnLogs(logs), nil return returnLogs(logs), nil

View File

@ -28,7 +28,6 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/node"
) )
@ -122,14 +121,13 @@ func benchmarkBloomBits(b *testing.B, sectionSize uint64) {
b.Log("Running filter benchmarks...") b.Log("Running filter benchmarks...")
start = time.Now() start = time.Now()
mux := new(event.TypeMux)
var backend *testBackend var backend *testBackend
for i := 0; i < benchFilterCnt; i++ { for i := 0; i < benchFilterCnt; i++ {
if i%20 == 0 { if i%20 == 0 {
db.Close() db.Close()
db, _ = rawdb.NewLevelDBDatabase(benchDataDir, 128, 1024, "") db, _ = rawdb.NewLevelDBDatabase(benchDataDir, 128, 1024, "")
backend = &testBackend{mux, db, cnt, new(event.Feed), new(event.Feed), new(event.Feed), new(event.Feed)} backend = &testBackend{db: db, sections: cnt}
} }
var addr common.Address var addr common.Address
addr[0] = byte(i) addr[0] = byte(i)
@ -173,8 +171,7 @@ func BenchmarkNoBloomBits(b *testing.B) {
b.Log("Running filter benchmarks...") b.Log("Running filter benchmarks...")
start := time.Now() start := time.Now()
mux := new(event.TypeMux) backend := &testBackend{db: db}
backend := &testBackend{mux, db, 0, new(event.Feed), new(event.Feed), new(event.Feed), new(event.Feed)}
filter := NewRangeFilter(backend, 0, int64(*headNum), []common.Address{{}}, nil) filter := NewRangeFilter(backend, 0, int64(*headNum), []common.Address{{}}, nil)
filter.Logs(context.Background()) filter.Logs(context.Background())
d := time.Since(start) d := time.Since(start)

View File

@ -32,7 +32,6 @@ import (
type Backend interface { type Backend interface {
ChainDb() ethdb.Database ChainDb() ethdb.Database
EventMux() *event.TypeMux
HeaderByNumber(ctx context.Context, blockNr rpc.BlockNumber) (*types.Header, error) HeaderByNumber(ctx context.Context, blockNr rpc.BlockNumber) (*types.Header, error)
HeaderByHash(ctx context.Context, blockHash common.Hash) (*types.Header, error) HeaderByHash(ctx context.Context, blockHash common.Hash) (*types.Header, error)
GetReceipts(ctx context.Context, blockHash common.Hash) (types.Receipts, error) GetReceipts(ctx context.Context, blockHash common.Hash) (types.Receipts, error)
@ -42,6 +41,7 @@ type Backend interface {
SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription
SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription
SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription
SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription
BloomStatus() (uint64, uint64) BloomStatus() (uint64, uint64)
ServiceFilter(ctx context.Context, session *bloombits.MatcherSession) ServiceFilter(ctx context.Context, session *bloombits.MatcherSession)

View File

@ -20,7 +20,6 @@ package filters
import ( import (
"context" "context"
"errors"
"fmt" "fmt"
"sync" "sync"
"time" "time"
@ -58,7 +57,6 @@ const (
) )
const ( const (
// txChanSize is the size of channel listening to NewTxsEvent. // txChanSize is the size of channel listening to NewTxsEvent.
// The number is referenced from the size of tx pool. // The number is referenced from the size of tx pool.
txChanSize = 4096 txChanSize = 4096
@ -70,10 +68,6 @@ const (
chainEvChanSize = 10 chainEvChanSize = 10
) )
var (
ErrInvalidSubscriptionID = errors.New("invalid id")
)
type subscription struct { type subscription struct {
id rpc.ID id rpc.ID
typ Type typ Type
@ -89,7 +83,6 @@ type subscription struct {
// EventSystem creates subscriptions, processes events and broadcasts them to the // EventSystem creates subscriptions, processes events and broadcasts them to the
// subscription which match the subscription criteria. // subscription which match the subscription criteria.
type EventSystem struct { type EventSystem struct {
mux *event.TypeMux
backend Backend backend Backend
lightMode bool lightMode bool
lastHead *types.Header lastHead *types.Header
@ -98,14 +91,15 @@ type EventSystem struct {
txsSub event.Subscription // Subscription for new transaction event txsSub event.Subscription // Subscription for new transaction event
logsSub event.Subscription // Subscription for new log event logsSub event.Subscription // Subscription for new log event
rmLogsSub event.Subscription // Subscription for removed log event rmLogsSub event.Subscription // Subscription for removed log event
pendingLogsSub event.Subscription // Subscription for pending log event
chainSub event.Subscription // Subscription for new chain event chainSub event.Subscription // Subscription for new chain event
pendingLogSub *event.TypeMuxSubscription // Subscription for pending log event
// Channels // Channels
install chan *subscription // install filter for event notification install chan *subscription // install filter for event notification
uninstall chan *subscription // remove filter for event notification uninstall chan *subscription // remove filter for event notification
txsCh chan core.NewTxsEvent // Channel to receive new transactions event txsCh chan core.NewTxsEvent // Channel to receive new transactions event
logsCh chan []*types.Log // Channel to receive new log event logsCh chan []*types.Log // Channel to receive new log event
pendingLogsCh chan []*types.Log // Channel to receive new log event
rmLogsCh chan core.RemovedLogsEvent // Channel to receive removed log event rmLogsCh chan core.RemovedLogsEvent // Channel to receive removed log event
chainCh chan core.ChainEvent // Channel to receive new chain event chainCh chan core.ChainEvent // Channel to receive new chain event
} }
@ -116,9 +110,8 @@ type EventSystem struct {
// //
// The returned manager has a loop that needs to be stopped with the Stop function // The returned manager has a loop that needs to be stopped with the Stop function
// or by stopping the given mux. // or by stopping the given mux.
func NewEventSystem(mux *event.TypeMux, backend Backend, lightMode bool) *EventSystem { func NewEventSystem(backend Backend, lightMode bool) *EventSystem {
m := &EventSystem{ m := &EventSystem{
mux: mux,
backend: backend, backend: backend,
lightMode: lightMode, lightMode: lightMode,
install: make(chan *subscription), install: make(chan *subscription),
@ -126,6 +119,7 @@ func NewEventSystem(mux *event.TypeMux, backend Backend, lightMode bool) *EventS
txsCh: make(chan core.NewTxsEvent, txChanSize), txsCh: make(chan core.NewTxsEvent, txChanSize),
logsCh: make(chan []*types.Log, logsChanSize), logsCh: make(chan []*types.Log, logsChanSize),
rmLogsCh: make(chan core.RemovedLogsEvent, rmLogsChanSize), rmLogsCh: make(chan core.RemovedLogsEvent, rmLogsChanSize),
pendingLogsCh: make(chan []*types.Log, logsChanSize),
chainCh: make(chan core.ChainEvent, chainEvChanSize), chainCh: make(chan core.ChainEvent, chainEvChanSize),
} }
@ -134,12 +128,10 @@ func NewEventSystem(mux *event.TypeMux, backend Backend, lightMode bool) *EventS
m.logsSub = m.backend.SubscribeLogsEvent(m.logsCh) m.logsSub = m.backend.SubscribeLogsEvent(m.logsCh)
m.rmLogsSub = m.backend.SubscribeRemovedLogsEvent(m.rmLogsCh) m.rmLogsSub = m.backend.SubscribeRemovedLogsEvent(m.rmLogsCh)
m.chainSub = m.backend.SubscribeChainEvent(m.chainCh) m.chainSub = m.backend.SubscribeChainEvent(m.chainCh)
// TODO(rjl493456442): use feed to subscribe pending log event m.pendingLogsSub = m.backend.SubscribePendingLogsEvent(m.pendingLogsCh)
m.pendingLogSub = m.mux.Subscribe(core.PendingLogsEvent{})
// Make sure none of the subscriptions are empty // Make sure none of the subscriptions are empty
if m.txsSub == nil || m.logsSub == nil || m.rmLogsSub == nil || m.chainSub == nil || if m.txsSub == nil || m.logsSub == nil || m.rmLogsSub == nil || m.chainSub == nil || m.pendingLogsSub == nil {
m.pendingLogSub.Closed() {
log.Crit("Subscribe for event system failed") log.Crit("Subscribe for event system failed")
} }
@ -316,51 +308,55 @@ func (es *EventSystem) SubscribePendingTxs(hashes chan []common.Hash) *Subscript
type filterIndex map[Type]map[rpc.ID]*subscription type filterIndex map[Type]map[rpc.ID]*subscription
// broadcast event to filters that match criteria. func (es *EventSystem) handleLogs(filters filterIndex, ev []*types.Log) {
func (es *EventSystem) broadcast(filters filterIndex, ev interface{}) { if len(ev) == 0 {
if ev == nil {
return return
} }
for _, f := range filters[LogsSubscription] {
matchedLogs := filterLogs(ev, f.logsCrit.FromBlock, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics)
if len(matchedLogs) > 0 {
f.logs <- matchedLogs
}
}
}
switch e := ev.(type) { func (es *EventSystem) handlePendingLogs(filters filterIndex, ev []*types.Log) {
case []*types.Log: if len(ev) == 0 {
if len(e) > 0 { return
for _, f := range filters[LogsSubscription] {
if matchedLogs := filterLogs(e, f.logsCrit.FromBlock, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics); len(matchedLogs) > 0 {
f.logs <- matchedLogs
} }
}
}
case core.RemovedLogsEvent:
for _, f := range filters[LogsSubscription] {
if matchedLogs := filterLogs(e.Logs, f.logsCrit.FromBlock, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics); len(matchedLogs) > 0 {
f.logs <- matchedLogs
}
}
case *event.TypeMuxEvent:
if muxe, ok := e.Data.(core.PendingLogsEvent); ok {
for _, f := range filters[PendingLogsSubscription] { for _, f := range filters[PendingLogsSubscription] {
if e.Time.After(f.created) { matchedLogs := filterLogs(ev, nil, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics)
if matchedLogs := filterLogs(muxe.Logs, nil, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics); len(matchedLogs) > 0 { if len(matchedLogs) > 0 {
f.logs <- matchedLogs f.logs <- matchedLogs
} }
} }
}
func (es *EventSystem) handleRemovedLogs(filters filterIndex, ev core.RemovedLogsEvent) {
for _, f := range filters[LogsSubscription] {
matchedLogs := filterLogs(ev.Logs, f.logsCrit.FromBlock, f.logsCrit.ToBlock, f.logsCrit.Addresses, f.logsCrit.Topics)
if len(matchedLogs) > 0 {
f.logs <- matchedLogs
} }
} }
case core.NewTxsEvent: }
hashes := make([]common.Hash, 0, len(e.Txs))
for _, tx := range e.Txs { func (es *EventSystem) handleTxsEvent(filters filterIndex, ev core.NewTxsEvent) {
hashes := make([]common.Hash, 0, len(ev.Txs))
for _, tx := range ev.Txs {
hashes = append(hashes, tx.Hash()) hashes = append(hashes, tx.Hash())
} }
for _, f := range filters[PendingTransactionsSubscription] { for _, f := range filters[PendingTransactionsSubscription] {
f.hashes <- hashes f.hashes <- hashes
} }
case core.ChainEvent: }
func (es *EventSystem) handleChainEvent(filters filterIndex, ev core.ChainEvent) {
for _, f := range filters[BlocksSubscription] { for _, f := range filters[BlocksSubscription] {
f.headers <- e.Block.Header() f.headers <- ev.Block.Header()
} }
if es.lightMode && len(filters[LogsSubscription]) > 0 { if es.lightMode && len(filters[LogsSubscription]) > 0 {
es.lightFilterNewHead(e.Block.Header(), func(header *types.Header, remove bool) { es.lightFilterNewHead(ev.Block.Header(), func(header *types.Header, remove bool) {
for _, f := range filters[LogsSubscription] { for _, f := range filters[LogsSubscription] {
if matchedLogs := es.lightFilterLogs(header, f.logsCrit.Addresses, f.logsCrit.Topics, remove); len(matchedLogs) > 0 { if matchedLogs := es.lightFilterLogs(header, f.logsCrit.Addresses, f.logsCrit.Topics, remove); len(matchedLogs) > 0 {
f.logs <- matchedLogs f.logs <- matchedLogs
@ -368,7 +364,6 @@ func (es *EventSystem) broadcast(filters filterIndex, ev interface{}) {
} }
}) })
} }
}
} }
func (es *EventSystem) lightFilterNewHead(newHeader *types.Header, callBack func(*types.Header, bool)) { func (es *EventSystem) lightFilterNewHead(newHeader *types.Header, callBack func(*types.Header, bool)) {
@ -448,10 +443,10 @@ func (es *EventSystem) lightFilterLogs(header *types.Header, addresses []common.
func (es *EventSystem) eventLoop() { func (es *EventSystem) eventLoop() {
// Ensure all subscriptions get cleaned up // Ensure all subscriptions get cleaned up
defer func() { defer func() {
es.pendingLogSub.Unsubscribe()
es.txsSub.Unsubscribe() es.txsSub.Unsubscribe()
es.logsSub.Unsubscribe() es.logsSub.Unsubscribe()
es.rmLogsSub.Unsubscribe() es.rmLogsSub.Unsubscribe()
es.pendingLogsSub.Unsubscribe()
es.chainSub.Unsubscribe() es.chainSub.Unsubscribe()
}() }()
@ -462,20 +457,16 @@ func (es *EventSystem) eventLoop() {
for { for {
select { select {
// Handle subscribed events
case ev := <-es.txsCh: case ev := <-es.txsCh:
es.broadcast(index, ev) es.handleTxsEvent(index, ev)
case ev := <-es.logsCh: case ev := <-es.logsCh:
es.broadcast(index, ev) es.handleLogs(index, ev)
case ev := <-es.rmLogsCh: case ev := <-es.rmLogsCh:
es.broadcast(index, ev) es.handleRemovedLogs(index, ev)
case ev := <-es.pendingLogsCh:
es.handlePendingLogs(index, ev)
case ev := <-es.chainCh: case ev := <-es.chainCh:
es.broadcast(index, ev) es.handleChainEvent(index, ev)
case ev, active := <-es.pendingLogSub.Chan():
if !active { // system stopped
return
}
es.broadcast(index, ev)
case f := <-es.install: case f := <-es.install:
if f.typ == MinedAndPendingLogsSubscription { if f.typ == MinedAndPendingLogsSubscription {

View File

@ -42,20 +42,17 @@ type testBackend struct {
mux *event.TypeMux mux *event.TypeMux
db ethdb.Database db ethdb.Database
sections uint64 sections uint64
txFeed *event.Feed txFeed event.Feed
rmLogsFeed *event.Feed logsFeed event.Feed
logsFeed *event.Feed rmLogsFeed event.Feed
chainFeed *event.Feed pendingLogsFeed event.Feed
chainFeed event.Feed
} }
func (b *testBackend) ChainDb() ethdb.Database { func (b *testBackend) ChainDb() ethdb.Database {
return b.db return b.db
} }
func (b *testBackend) EventMux() *event.TypeMux {
return b.mux
}
func (b *testBackend) HeaderByNumber(ctx context.Context, blockNr rpc.BlockNumber) (*types.Header, error) { func (b *testBackend) HeaderByNumber(ctx context.Context, blockNr rpc.BlockNumber) (*types.Header, error) {
var ( var (
hash common.Hash hash common.Hash
@ -116,6 +113,10 @@ func (b *testBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscript
return b.logsFeed.Subscribe(ch) return b.logsFeed.Subscribe(ch)
} }
func (b *testBackend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
return b.pendingLogsFeed.Subscribe(ch)
}
func (b *testBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription { func (b *testBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription {
return b.chainFeed.Subscribe(ch) return b.chainFeed.Subscribe(ch)
} }
@ -160,13 +161,8 @@ func TestBlockSubscription(t *testing.T) {
t.Parallel() t.Parallel()
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
genesis = new(core.Genesis).MustCommit(db) genesis = new(core.Genesis).MustCommit(db)
chain, _ = core.GenerateChain(params.TestChainConfig, genesis, ethash.NewFaker(), db, 10, func(i int, gen *core.BlockGen) {}) chain, _ = core.GenerateChain(params.TestChainConfig, genesis, ethash.NewFaker(), db, 10, func(i int, gen *core.BlockGen) {})
@ -205,7 +201,7 @@ func TestBlockSubscription(t *testing.T) {
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
for _, e := range chainEvents { for _, e := range chainEvents {
chainFeed.Send(e) backend.chainFeed.Send(e)
} }
<-sub0.Err() <-sub0.Err()
@ -217,13 +213,8 @@ func TestPendingTxFilter(t *testing.T) {
t.Parallel() t.Parallel()
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
transactions = []*types.Transaction{ transactions = []*types.Transaction{
@ -240,7 +231,7 @@ func TestPendingTxFilter(t *testing.T) {
fid0 := api.NewPendingTransactionFilter() fid0 := api.NewPendingTransactionFilter()
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
txFeed.Send(core.NewTxsEvent{Txs: transactions}) backend.txFeed.Send(core.NewTxsEvent{Txs: transactions})
timeout := time.Now().Add(1 * time.Second) timeout := time.Now().Add(1 * time.Second)
for { for {
@ -277,13 +268,8 @@ func TestPendingTxFilter(t *testing.T) {
// If not it must return an error. // If not it must return an error.
func TestLogFilterCreation(t *testing.T) { func TestLogFilterCreation(t *testing.T) {
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
testCases = []struct { testCases = []struct {
@ -326,13 +312,8 @@ func TestInvalidLogFilterCreation(t *testing.T) {
t.Parallel() t.Parallel()
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
) )
@ -353,13 +334,8 @@ func TestInvalidLogFilterCreation(t *testing.T) {
func TestInvalidGetLogsRequest(t *testing.T) { func TestInvalidGetLogsRequest(t *testing.T) {
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
blockHash = common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111") blockHash = common.HexToHash("0x1111111111111111111111111111111111111111111111111111111111111111")
) )
@ -383,13 +359,8 @@ func TestLogFilter(t *testing.T) {
t.Parallel() t.Parallel()
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
firstAddr = common.HexToAddress("0x1111111111111111111111111111111111111111") firstAddr = common.HexToAddress("0x1111111111111111111111111111111111111111")
@ -400,7 +371,7 @@ func TestLogFilter(t *testing.T) {
secondTopic = common.HexToHash("0x2222222222222222222222222222222222222222222222222222222222222222") secondTopic = common.HexToHash("0x2222222222222222222222222222222222222222222222222222222222222222")
notUsedTopic = common.HexToHash("0x9999999999999999999999999999999999999999999999999999999999999999") notUsedTopic = common.HexToHash("0x9999999999999999999999999999999999999999999999999999999999999999")
// posted twice, once as vm.Logs and once as core.PendingLogsEvent // posted twice, once as regular logs and once as pending logs.
allLogs = []*types.Log{ allLogs = []*types.Log{
{Address: firstAddr}, {Address: firstAddr},
{Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 1}, {Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 1},
@ -453,11 +424,11 @@ func TestLogFilter(t *testing.T) {
// raise events // raise events
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
if nsend := logsFeed.Send(allLogs); nsend == 0 { if nsend := backend.logsFeed.Send(allLogs); nsend == 0 {
t.Fatal("Shoud have at least one subscription") t.Fatal("Logs event not delivered")
} }
if err := mux.Post(core.PendingLogsEvent{Logs: allLogs}); err != nil { if nsend := backend.pendingLogsFeed.Send(allLogs); nsend == 0 {
t.Fatal(err) t.Fatal("Pending logs event not delivered")
} }
for i, tt := range testCases { for i, tt := range testCases {
@ -502,13 +473,8 @@ func TestPendingLogsSubscription(t *testing.T) {
t.Parallel() t.Parallel()
var ( var (
mux = new(event.TypeMux)
db = rawdb.NewMemoryDatabase() db = rawdb.NewMemoryDatabase()
txFeed = new(event.Feed) backend = &testBackend{db: db}
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
api = NewPublicFilterAPI(backend, false) api = NewPublicFilterAPI(backend, false)
firstAddr = common.HexToAddress("0x1111111111111111111111111111111111111111") firstAddr = common.HexToAddress("0x1111111111111111111111111111111111111111")
@ -521,26 +487,18 @@ func TestPendingLogsSubscription(t *testing.T) {
fourthTopic = common.HexToHash("0x4444444444444444444444444444444444444444444444444444444444444444") fourthTopic = common.HexToHash("0x4444444444444444444444444444444444444444444444444444444444444444")
notUsedTopic = common.HexToHash("0x9999999999999999999999999999999999999999999999999999999999999999") notUsedTopic = common.HexToHash("0x9999999999999999999999999999999999999999999999999999999999999999")
allLogs = []core.PendingLogsEvent{ allLogs = [][]*types.Log{
{Logs: []*types.Log{{Address: firstAddr, Topics: []common.Hash{}, BlockNumber: 0}}}, {{Address: firstAddr, Topics: []common.Hash{}, BlockNumber: 0}},
{Logs: []*types.Log{{Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 1}}}, {{Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 1}},
{Logs: []*types.Log{{Address: secondAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 2}}}, {{Address: secondAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 2}},
{Logs: []*types.Log{{Address: thirdAddress, Topics: []common.Hash{secondTopic}, BlockNumber: 3}}}, {{Address: thirdAddress, Topics: []common.Hash{secondTopic}, BlockNumber: 3}},
{Logs: []*types.Log{{Address: thirdAddress, Topics: []common.Hash{secondTopic}, BlockNumber: 4}}}, {{Address: thirdAddress, Topics: []common.Hash{secondTopic}, BlockNumber: 4}},
{Logs: []*types.Log{ {
{Address: thirdAddress, Topics: []common.Hash{firstTopic}, BlockNumber: 5}, {Address: thirdAddress, Topics: []common.Hash{firstTopic}, BlockNumber: 5},
{Address: thirdAddress, Topics: []common.Hash{thirdTopic}, BlockNumber: 5}, {Address: thirdAddress, Topics: []common.Hash{thirdTopic}, BlockNumber: 5},
{Address: thirdAddress, Topics: []common.Hash{fourthTopic}, BlockNumber: 5}, {Address: thirdAddress, Topics: []common.Hash{fourthTopic}, BlockNumber: 5},
{Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 5}, {Address: firstAddr, Topics: []common.Hash{firstTopic}, BlockNumber: 5},
}}, },
}
convertLogs = func(pl []core.PendingLogsEvent) []*types.Log {
var logs []*types.Log
for _, l := range pl {
logs = append(logs, l.Logs...)
}
return logs
} }
testCases = []struct { testCases = []struct {
@ -550,21 +508,52 @@ func TestPendingLogsSubscription(t *testing.T) {
sub *Subscription sub *Subscription
}{ }{
// match all // match all
{ethereum.FilterQuery{}, convertLogs(allLogs), nil, nil}, {
ethereum.FilterQuery{}, flattenLogs(allLogs),
nil, nil,
},
// match none due to no matching addresses // match none due to no matching addresses
{ethereum.FilterQuery{Addresses: []common.Address{{}, notUsedAddress}, Topics: [][]common.Hash{nil}}, []*types.Log{}, nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{{}, notUsedAddress}, Topics: [][]common.Hash{nil}},
nil,
nil, nil,
},
// match logs based on addresses, ignore topics // match logs based on addresses, ignore topics
{ethereum.FilterQuery{Addresses: []common.Address{firstAddr}}, append(convertLogs(allLogs[:2]), allLogs[5].Logs[3]), nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{firstAddr}},
append(flattenLogs(allLogs[:2]), allLogs[5][3]),
nil, nil,
},
// match none due to no matching topics (match with address) // match none due to no matching topics (match with address)
{ethereum.FilterQuery{Addresses: []common.Address{secondAddr}, Topics: [][]common.Hash{{notUsedTopic}}}, []*types.Log{}, nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{secondAddr}, Topics: [][]common.Hash{{notUsedTopic}}},
nil, nil, nil,
},
// match logs based on addresses and topics // match logs based on addresses and topics
{ethereum.FilterQuery{Addresses: []common.Address{thirdAddress}, Topics: [][]common.Hash{{firstTopic, secondTopic}}}, append(convertLogs(allLogs[3:5]), allLogs[5].Logs[0]), nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{thirdAddress}, Topics: [][]common.Hash{{firstTopic, secondTopic}}},
append(flattenLogs(allLogs[3:5]), allLogs[5][0]),
nil, nil,
},
// match logs based on multiple addresses and "or" topics // match logs based on multiple addresses and "or" topics
{ethereum.FilterQuery{Addresses: []common.Address{secondAddr, thirdAddress}, Topics: [][]common.Hash{{firstTopic, secondTopic}}}, append(convertLogs(allLogs[2:5]), allLogs[5].Logs[0]), nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{secondAddr, thirdAddress}, Topics: [][]common.Hash{{firstTopic, secondTopic}}},
append(flattenLogs(allLogs[2:5]), allLogs[5][0]),
nil,
nil,
},
// block numbers are ignored for filters created with New***Filter, these return all logs that match the given criteria when the state changes // block numbers are ignored for filters created with New***Filter, these return all logs that match the given criteria when the state changes
{ethereum.FilterQuery{Addresses: []common.Address{firstAddr}, FromBlock: big.NewInt(2), ToBlock: big.NewInt(3)}, append(convertLogs(allLogs[:2]), allLogs[5].Logs[3]), nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{firstAddr}, FromBlock: big.NewInt(2), ToBlock: big.NewInt(3)},
append(flattenLogs(allLogs[:2]), allLogs[5][3]),
nil, nil,
},
// multiple pending logs, should match only 2 topics from the logs in block 5 // multiple pending logs, should match only 2 topics from the logs in block 5
{ethereum.FilterQuery{Addresses: []common.Address{thirdAddress}, Topics: [][]common.Hash{{firstTopic, fourthTopic}}}, []*types.Log{allLogs[5].Logs[0], allLogs[5].Logs[2]}, nil, nil}, {
ethereum.FilterQuery{Addresses: []common.Address{thirdAddress}, Topics: [][]common.Hash{{firstTopic, fourthTopic}}},
[]*types.Log{allLogs[5][0], allLogs[5][2]},
nil, nil,
},
} }
) )
@ -607,10 +596,15 @@ func TestPendingLogsSubscription(t *testing.T) {
// raise events // raise events
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
// allLogs are type of core.PendingLogsEvent for _, ev := range allLogs {
for _, l := range allLogs { backend.pendingLogsFeed.Send(ev)
if err := mux.Post(l); err != nil {
t.Fatal(err)
}
} }
} }
func flattenLogs(pl [][]*types.Log) []*types.Log {
var logs []*types.Log
for _, l := range pl {
logs = append(logs, l...)
}
return logs
}

View File

@ -29,7 +29,6 @@ import (
"github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/event"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
) )
@ -51,12 +50,7 @@ func BenchmarkFilters(b *testing.B) {
var ( var (
db, _ = rawdb.NewLevelDBDatabase(dir, 0, 0, "") db, _ = rawdb.NewLevelDBDatabase(dir, 0, 0, "")
mux = new(event.TypeMux) backend = &testBackend{db: db}
txFeed = new(event.Feed)
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
key1, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") key1, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
addr1 = crypto.PubkeyToAddress(key1.PublicKey) addr1 = crypto.PubkeyToAddress(key1.PublicKey)
addr2 = common.BytesToAddress([]byte("jeff")) addr2 = common.BytesToAddress([]byte("jeff"))
@ -110,12 +104,7 @@ func TestFilters(t *testing.T) {
var ( var (
db, _ = rawdb.NewLevelDBDatabase(dir, 0, 0, "") db, _ = rawdb.NewLevelDBDatabase(dir, 0, 0, "")
mux = new(event.TypeMux) backend = &testBackend{db: db}
txFeed = new(event.Feed)
rmLogsFeed = new(event.Feed)
logsFeed = new(event.Feed)
chainFeed = new(event.Feed)
backend = &testBackend{mux, db, 0, txFeed, rmLogsFeed, logsFeed, chainFeed}
key1, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") key1, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
addr = crypto.PubkeyToAddress(key1.PublicKey) addr = crypto.PubkeyToAddress(key1.PublicKey)

View File

@ -43,7 +43,6 @@ type Backend interface {
ProtocolVersion() int ProtocolVersion() int
SuggestPrice(ctx context.Context) (*big.Int, error) SuggestPrice(ctx context.Context) (*big.Int, error)
ChainDb() ethdb.Database ChainDb() ethdb.Database
EventMux() *event.TypeMux
AccountManager() *accounts.Manager AccountManager() *accounts.Manager
ExtRPCEnabled() bool ExtRPCEnabled() bool
RPCGasCap() *big.Int // global gas cap for eth_call over rpc: DoS protection RPCGasCap() *big.Int // global gas cap for eth_call over rpc: DoS protection
@ -80,6 +79,7 @@ type Backend interface {
GetLogs(ctx context.Context, blockHash common.Hash) ([][]*types.Log, error) GetLogs(ctx context.Context, blockHash common.Hash) ([][]*types.Log, error)
ServiceFilter(ctx context.Context, session *bloombits.MatcherSession) ServiceFilter(ctx context.Context, session *bloombits.MatcherSession)
SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscription
SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription
SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription
ChainConfig() *params.ChainConfig ChainConfig() *params.ChainConfig

View File

@ -225,6 +225,13 @@ func (b *LesApiBackend) SubscribeLogsEvent(ch chan<- []*types.Log) event.Subscri
return b.eth.blockchain.SubscribeLogsEvent(ch) return b.eth.blockchain.SubscribeLogsEvent(ch)
} }
func (b *LesApiBackend) SubscribePendingLogsEvent(ch chan<- []*types.Log) event.Subscription {
return event.NewSubscription(func(quit <-chan struct{}) error {
<-quit
return nil
})
}
func (b *LesApiBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription { func (b *LesApiBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription {
return b.eth.blockchain.SubscribeRemovedLogsEvent(ch) return b.eth.blockchain.SubscribeRemovedLogsEvent(ch)
} }
@ -245,10 +252,6 @@ func (b *LesApiBackend) ChainDb() ethdb.Database {
return b.eth.chainDb return b.eth.chainDb
} }
func (b *LesApiBackend) EventMux() *event.TypeMux {
return b.eth.eventMux
}
func (b *LesApiBackend) AccountManager() *accounts.Manager { func (b *LesApiBackend) AccountManager() *accounts.Manager {
return b.eth.accountManager return b.eth.accountManager
} }

View File

@ -182,3 +182,9 @@ func (miner *Miner) SetEtherbase(addr common.Address) {
miner.coinbase = addr miner.coinbase = addr
miner.worker.setEtherbase(addr) miner.worker.setEtherbase(addr)
} }
// SubscribePendingLogs starts delivering logs from pending transactions
// to the given channel.
func (self *Miner) SubscribePendingLogs(ch chan<- []*types.Log) event.Subscription {
return self.worker.pendingLogsFeed.Subscribe(ch)
}

View File

@ -128,6 +128,9 @@ type worker struct {
eth Backend eth Backend
chain *core.BlockChain chain *core.BlockChain
// Feeds
pendingLogsFeed event.Feed
// Subscriptions // Subscriptions
mux *event.TypeMux mux *event.TypeMux
txsCh chan core.NewTxsEvent txsCh chan core.NewTxsEvent
@ -809,7 +812,7 @@ func (w *worker) commitTransactions(txs *types.TransactionsByPriceAndNonce, coin
cpy[i] = new(types.Log) cpy[i] = new(types.Log)
*cpy[i] = *l *cpy[i] = *l
} }
go w.mux.Post(core.PendingLogsEvent{Logs: cpy}) w.pendingLogsFeed.Send(cpy)
} }
// Notify resubmit loop to decrease resubmitting interval if current interval is larger // Notify resubmit loop to decrease resubmitting interval if current interval is larger
// than the user-specified one. // than the user-specified one.