Merge pull request #86 from vulcanize/vdb-371-recheck-queued-storage

(VDB-371) recheck queued storage
This commit is contained in:
Rob Mulholand 2019-05-01 12:49:02 -05:00 committed by GitHub
commit 782e3fd095
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 733 additions and 236 deletions

View File

@ -16,16 +16,20 @@
package cmd
import (
"os"
"plugin"
syn "sync"
"time"
log "github.com/sirupsen/logrus"
"github.com/spf13/cobra"
"github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
"github.com/vulcanize/vulcanizedb/libraries/shared/watcher"
"github.com/vulcanize/vulcanizedb/pkg/fs"
p2 "github.com/vulcanize/vulcanizedb/pkg/plugin"
"github.com/vulcanize/vulcanizedb/pkg/plugin/helpers"
"github.com/vulcanize/vulcanizedb/utils"
"os"
"plugin"
syn "sync"
)
// composeAndExecuteCmd represents the composeAndExecute command
@ -170,7 +174,8 @@ func composeAndExecute() {
if len(ethStorageInitializers) > 0 {
tailer := fs.FileTailer{Path: storageDiffsPath}
sw := watcher.NewStorageWatcher(tailer, &db)
storageFetcher := fetcher.NewCsvTailStorageFetcher(tailer)
sw := watcher.NewStorageWatcher(storageFetcher, &db)
sw.AddTransformers(ethStorageInitializers)
wg.Add(1)
go watchEthStorage(&sw, &wg)
@ -187,5 +192,6 @@ func composeAndExecute() {
func init() {
rootCmd.AddCommand(composeAndExecuteCmd)
composeAndExecuteCmd.Flags().BoolVar(&recheckHeadersArg, "recheckHeaders", false, "checks headers that are already checked for each transformer.")
composeAndExecuteCmd.Flags().BoolVarP(&recheckHeadersArg, "recheck-headers", "r", false, "whether to re-check headers for watched events")
composeAndExecuteCmd.Flags().DurationVarP(&queueRecheckInterval, "queue-recheck-interval", "q", 5 * time.Minute, "how often to recheck queued storage diffs")
}

View File

@ -26,6 +26,8 @@ import (
"github.com/spf13/cobra"
"github.com/vulcanize/vulcanizedb/libraries/shared/constants"
"github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
storageUtils "github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/libraries/shared/transformer"
"github.com/vulcanize/vulcanizedb/libraries/shared/watcher"
"github.com/vulcanize/vulcanizedb/pkg/fs"
@ -118,7 +120,8 @@ func execute() {
if len(ethStorageInitializers) > 0 {
tailer := fs.FileTailer{Path: storageDiffsPath}
sw := watcher.NewStorageWatcher(tailer, &db)
storageFetcher := fetcher.NewCsvTailStorageFetcher(tailer)
sw := watcher.NewStorageWatcher(storageFetcher, &db)
sw.AddTransformers(ethStorageInitializers)
wg.Add(1)
go watchEthStorage(&sw, &wg)
@ -135,7 +138,8 @@ func execute() {
func init() {
rootCmd.AddCommand(executeCmd)
executeCmd.Flags().BoolVar(&recheckHeadersArg, "recheckHeaders", false, "checks headers that are already checked for each transformer.")
executeCmd.Flags().BoolVarP(&recheckHeadersArg, "recheck-headers", "r", false, "whether to re-check headers for watched events")
executeCmd.Flags().DurationVarP(&queueRecheckInterval, "queue-recheck-interval", "q", 5 * time.Minute, "how often to recheck queued storage diffs")
}
type Exporter interface {
@ -166,7 +170,9 @@ func watchEthStorage(w *watcher.StorageWatcher, wg *syn.WaitGroup) {
ticker := time.NewTicker(pollingInterval)
defer ticker.Stop()
for range ticker.C {
w.Execute()
errs := make(chan error)
rows := make(chan storageUtils.StorageDiffRow)
w.Execute(rows, errs, queueRecheckInterval)
}
}

View File

@ -41,6 +41,7 @@ var (
genConfig config.Plugin
ipc string
levelDbPath string
queueRecheckInterval time.Duration
startingBlockNumber int64
storageDiffsPath string
syncAll bool
@ -49,8 +50,8 @@ var (
)
const (
pollingInterval = 7 * time.Second
validationWindow = 15
pollingInterval = 7 * time.Second
validationWindow = 15
)
var rootCmd = &cobra.Command{

View File

@ -36,6 +36,20 @@ composeAndExecute:
`./vulcanizedb composeAndExecute --config=./environments/config_name.toml`
## Flags
The `compose` and `composeAndExecute` commands can be passed optional flags to specify the operation of the watchers:
- `--recheck-headers`/`-r` - specifies whether to re-check headers for events after the header has already been queried for watched logs.
Can be useful for redundancy if you suspect that your node is not always returning all desired logs on every query.
Argument is expected to be a boolean: e.g. `-r=true`.
Defaults to `false`.
- `query-recheck-interval`/`-q` - specifies interval for re-checking storage diffs that haven been queued for later processing
(by default, the storage watched queues storage diffs if transformer execution fails, on the assumption that subsequent data derived from the event transformers may enable us to decode storage keys that we don't recognize right now).
Argument is expected to be a duration (integer measured in nanoseconds): e.g. `-q=10m30s` (for 10 minute, 30 second intervals).
Defaults to `5m` (5 minutes).
## Configuration
A .toml config file is specified when executing the commands.
The config provides information for composing a set of transformers from external repositories:

View File

@ -24,22 +24,22 @@ import (
"github.com/vulcanize/vulcanizedb/pkg/core"
)
type LogFetcher interface {
type ILogFetcher interface {
FetchLogs(contractAddresses []common.Address, topics []common.Hash, missingHeader core.Header) ([]types.Log, error)
}
type Fetcher struct {
type LogFetcher struct {
blockChain core.BlockChain
}
func NewFetcher(blockchain core.BlockChain) *Fetcher {
return &Fetcher{
func NewLogFetcher(blockchain core.BlockChain) *LogFetcher {
return &LogFetcher{
blockChain: blockchain,
}
}
// Checks all topic0s, on all addresses, fetching matching logs for the given header
func (fetcher Fetcher) FetchLogs(addresses []common.Address, topic0s []common.Hash, header core.Header) ([]types.Log, error) {
func (logFetcher LogFetcher) FetchLogs(addresses []common.Address, topic0s []common.Hash, header core.Header) ([]types.Log, error) {
blockHash := common.HexToHash(header.Hash)
query := ethereum.FilterQuery{
BlockHash: &blockHash,
@ -48,7 +48,7 @@ func (fetcher Fetcher) FetchLogs(addresses []common.Address, topic0s []common.Ha
Topics: [][]common.Hash{topic0s},
}
logs, err := fetcher.blockChain.GetEthLogsWithCustomQuery(query)
logs, err := logFetcher.blockChain.GetEthLogsWithCustomQuery(query)
if err != nil {
// TODO review aggregate fetching error handling
return []types.Log{}, err

View File

@ -22,16 +22,16 @@ import (
. "github.com/onsi/ginkgo"
. "github.com/onsi/gomega"
fetch "github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
"github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
"github.com/vulcanize/vulcanizedb/pkg/core"
"github.com/vulcanize/vulcanizedb/pkg/fakes"
)
var _ = Describe("Fetcher", func() {
var _ = Describe("LogFetcher", func() {
Describe("FetchLogs", func() {
It("fetches logs based on the given query", func() {
blockChain := fakes.NewMockBlockChain()
fetcher := fetch.NewFetcher(blockChain)
logFetcher := fetcher.NewLogFetcher(blockChain)
header := fakes.FakeHeader
addresses := []common.Address{
@ -41,7 +41,7 @@ var _ = Describe("Fetcher", func() {
topicZeros := []common.Hash{common.BytesToHash([]byte{1, 2, 3, 4, 5})}
_, err := fetcher.FetchLogs(addresses, topicZeros, header)
_, err := logFetcher.FetchLogs(addresses, topicZeros, header)
address1 := common.HexToAddress("0xfakeAddress")
address2 := common.HexToAddress("0xanotherFakeAddress")
@ -59,9 +59,9 @@ var _ = Describe("Fetcher", func() {
It("returns an error if fetching the logs fails", func() {
blockChain := fakes.NewMockBlockChain()
blockChain.SetGetEthLogsWithCustomQueryErr(fakes.FakeError)
fetcher := fetch.NewFetcher(blockChain)
logFetcher := fetcher.NewLogFetcher(blockChain)
_, err := fetcher.FetchLogs([]common.Address{}, []common.Hash{}, core.Header{})
_, err := logFetcher.FetchLogs([]common.Address{}, []common.Hash{}, core.Header{})
Expect(err).To(HaveOccurred())
Expect(err).To(MatchError(fakes.FakeError))

View File

@ -0,0 +1,50 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fetcher
import (
"github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/pkg/fs"
"strings"
)
type IStorageFetcher interface {
FetchStorageDiffs(chan<- utils.StorageDiffRow, chan<- error)
}
type CsvTailStorageFetcher struct {
tailer fs.Tailer
}
func NewCsvTailStorageFetcher(tailer fs.Tailer) CsvTailStorageFetcher {
return CsvTailStorageFetcher{tailer: tailer}
}
func (storageFetcher CsvTailStorageFetcher) FetchStorageDiffs(out chan<- utils.StorageDiffRow, errs chan<- error) {
t, tailErr := storageFetcher.tailer.Tail()
if tailErr != nil {
errs <- tailErr
}
for line := range t.Lines {
row, parseErr := utils.FromStrings(strings.Split(line.Text, ","))
if parseErr != nil {
errs <- parseErr
} else {
out <- row
}
}
}

View File

@ -0,0 +1,99 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fetcher_test
import (
"fmt"
"strings"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/hpcloud/tail"
. "github.com/onsi/ginkgo"
. "github.com/onsi/gomega"
"github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
"github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/pkg/fakes"
)
var _ = Describe("Csv Tail Storage Fetcher", func() {
var (
errorsChannel chan error
mockTailer *fakes.MockTailer
rowsChannel chan utils.StorageDiffRow
storageFetcher fetcher.CsvTailStorageFetcher
)
BeforeEach(func() {
errorsChannel = make(chan error)
rowsChannel = make(chan utils.StorageDiffRow)
mockTailer = fakes.NewMockTailer()
storageFetcher = fetcher.NewCsvTailStorageFetcher(mockTailer)
})
It("adds error to errors channel if tailing file fails", func(done Done) {
mockTailer.TailErr = fakes.FakeError
go storageFetcher.FetchStorageDiffs(rowsChannel, errorsChannel)
Expect(<-errorsChannel).To(MatchError(fakes.FakeError))
close(done)
})
It("adds parsed csv row to rows channel for storage diff", func(done Done) {
line := getFakeLine()
go storageFetcher.FetchStorageDiffs(rowsChannel, errorsChannel)
mockTailer.Lines <- line
expectedRow, err := utils.FromStrings(strings.Split(line.Text, ","))
Expect(err).NotTo(HaveOccurred())
Expect(<-rowsChannel).To(Equal(expectedRow))
close(done)
})
It("adds error to errors channel if parsing csv fails", func(done Done) {
line := &tail.Line{Text: "invalid"}
go storageFetcher.FetchStorageDiffs(rowsChannel, errorsChannel)
mockTailer.Lines <- line
Expect(<-errorsChannel).To(HaveOccurred())
select {
case <-rowsChannel:
Fail("value passed to rows channel on error")
default:
Succeed()
}
close(done)
})
})
func getFakeLine() *tail.Line {
address := common.HexToAddress("0x1234567890abcdef")
blockHash := []byte{4, 5, 6}
blockHeight := int64(789)
storageKey := []byte{9, 8, 7}
storageValue := []byte{6, 5, 4}
return &tail.Line{
Text: fmt.Sprintf("%s,%s,%d,%s,%s", common.Bytes2Hex(address.Bytes()), common.Bytes2Hex(blockHash),
blockHeight, common.Bytes2Hex(storageKey), common.Bytes2Hex(storageValue)),
Time: time.Time{},
Err: nil,
}
}

View File

@ -53,6 +53,6 @@ func (converter *MockConverter) SetToEntityConverterError(err error) {
converter.entityConverterError = err
}
func (c *MockConverter) SetToModelConverterError(err error) {
c.modelConverterError = err
func (converter *MockConverter) SetToModelConverterError(err error) {
converter.modelConverterError = err
}

View File

@ -0,0 +1,39 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package mocks
import "github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
type MockStorageFetcher struct {
RowsToReturn []utils.StorageDiffRow
ErrsToReturn []error
}
func NewMockStorageFetcher() *MockStorageFetcher {
return &MockStorageFetcher{}
}
func (fetcher *MockStorageFetcher) FetchStorageDiffs(out chan<- utils.StorageDiffRow, errs chan<- error) {
defer close(out)
defer close(errs)
for _, err := range fetcher.ErrsToReturn {
errs <- err
}
for _, row := range fetcher.RowsToReturn {
out <- row
}
}

View File

@ -21,11 +21,26 @@ import (
)
type MockStorageQueue struct {
AddCalled bool
AddError error
AddCalled bool
AddError error
AddPassedRow utils.StorageDiffRow
DeleteErr error
DeletePassedId int
GetAllErr error
RowsToReturn []utils.StorageDiffRow
}
func (queue *MockStorageQueue) Add(row utils.StorageDiffRow) error {
queue.AddCalled = true
queue.AddPassedRow = row
return queue.AddError
}
func (queue *MockStorageQueue) Delete(id int) error {
queue.DeletePassedId = id
return queue.DeleteErr
}
func (queue *MockStorageQueue) GetAll() ([]utils.StorageDiffRow, error) {
return queue.RowsToReturn, queue.GetAllErr
}

View File

@ -23,6 +23,8 @@ import (
type IStorageQueue interface {
Add(row utils.StorageDiffRow) error
Delete(id int) error
GetAll() ([]utils.StorageDiffRow, error)
}
type StorageQueue struct {
@ -40,3 +42,14 @@ func (queue StorageQueue) Add(row utils.StorageDiffRow) error {
row.BlockHeight, row.StorageKey.Bytes(), row.StorageValue.Bytes())
return err
}
func (queue StorageQueue) Delete(id int) error {
_, err := queue.db.Exec(`DELETE FROM public.queued_storage WHERE id = $1`, id)
return err
}
func (queue StorageQueue) GetAll() ([]utils.StorageDiffRow, error) {
var result []utils.StorageDiffRow
err := queue.db.Select(&result, `SELECT * FROM public.queued_storage`)
return result, err
}

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package storage_test
import (
@ -7,27 +23,79 @@ import (
"github.com/vulcanize/vulcanizedb/libraries/shared/storage"
"github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/pkg/datastore/postgres"
"github.com/vulcanize/vulcanizedb/test_config"
)
var _ = Describe("Storage queue", func() {
It("adds a storage row to the db", func() {
row := utils.StorageDiffRow{
var (
db *postgres.DB
row utils.StorageDiffRow
queue storage.IStorageQueue
)
BeforeEach(func() {
row = utils.StorageDiffRow{
Contract: common.HexToAddress("0x123456"),
BlockHash: common.HexToHash("0x678901"),
BlockHeight: 987,
StorageKey: common.HexToHash("0x654321"),
StorageValue: common.HexToHash("0x198765"),
}
db := test_config.NewTestDB(test_config.NewTestNode())
queue := storage.NewStorageQueue(db)
db = test_config.NewTestDB(test_config.NewTestNode())
test_config.CleanTestDB(db)
queue = storage.NewStorageQueue(db)
addErr := queue.Add(row)
Expect(addErr).NotTo(HaveOccurred())
})
It("adds a storage row to the db", func() {
var result utils.StorageDiffRow
getErr := db.Get(&result, `SELECT contract, block_hash, block_height, storage_key, storage_value FROM public.queued_storage`)
Expect(getErr).NotTo(HaveOccurred())
Expect(result).To(Equal(row))
})
It("deletes storage row from db", func() {
rows, getErr := queue.GetAll()
Expect(getErr).NotTo(HaveOccurred())
Expect(len(rows)).To(Equal(1))
err := queue.Delete(rows[0].Id)
Expect(err).NotTo(HaveOccurred())
remainingRows, secondGetErr := queue.GetAll()
Expect(secondGetErr).NotTo(HaveOccurred())
Expect(len(remainingRows)).To(BeZero())
})
It("gets all storage rows from db", func() {
rowTwo := utils.StorageDiffRow{
Contract: common.HexToAddress("0x123456"),
BlockHash: common.HexToHash("0x678902"),
BlockHeight: 988,
StorageKey: common.HexToHash("0x654322"),
StorageValue: common.HexToHash("0x198766"),
}
addErr := queue.Add(rowTwo)
Expect(addErr).NotTo(HaveOccurred())
rows, err := queue.GetAll()
Expect(err).NotTo(HaveOccurred())
Expect(len(rows)).To(Equal(2))
Expect(rows[0]).NotTo(Equal(rows[1]))
Expect(rows[0].Id).NotTo(BeZero())
Expect(rows[0].Contract).To(Or(Equal(row.Contract), Equal(rowTwo.Contract)))
Expect(rows[0].BlockHash).To(Or(Equal(row.BlockHash), Equal(rowTwo.BlockHash)))
Expect(rows[0].BlockHeight).To(Or(Equal(row.BlockHeight), Equal(rowTwo.BlockHeight)))
Expect(rows[0].StorageKey).To(Or(Equal(row.StorageKey), Equal(rowTwo.StorageKey)))
Expect(rows[0].StorageValue).To(Or(Equal(row.StorageValue), Equal(rowTwo.StorageValue)))
Expect(rows[1].Id).NotTo(BeZero())
Expect(rows[1].Contract).To(Or(Equal(row.Contract), Equal(rowTwo.Contract)))
Expect(rows[1].BlockHash).To(Or(Equal(row.BlockHash), Equal(rowTwo.BlockHash)))
Expect(rows[1].BlockHeight).To(Or(Equal(row.BlockHeight), Equal(rowTwo.BlockHeight)))
Expect(rows[1].StorageKey).To(Or(Equal(row.StorageKey), Equal(rowTwo.StorageKey)))
Expect(rows[1].StorageValue).To(Or(Equal(row.StorageValue), Equal(rowTwo.StorageValue)))
})
})

View File

@ -25,6 +25,7 @@ import (
const ExpectedRowLength = 5
type StorageDiffRow struct {
Id int
Contract common.Address
BlockHash common.Hash `db:"block_hash"`
BlockHeight int `db:"block_height"`

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package transactions
import (

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package transactions_test
import (

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package transactions_test
import (

View File

@ -54,10 +54,10 @@ func (watcher *ContractWatcher) AddTransformers(inits interface{}) error {
watcher.Transformers = append(watcher.Transformers, t)
}
for _, transformer := range watcher.Transformers {
err := transformer.Init()
for _, contractTransformer := range watcher.Transformers {
err := contractTransformer.Init()
if err != nil {
log.Print("Unable to initialize transformer:", transformer.GetConfig().Name, err)
log.Print("Unable to initialize transformer:", contractTransformer.GetConfig().Name, err)
return err
}
}
@ -65,10 +65,10 @@ func (watcher *ContractWatcher) AddTransformers(inits interface{}) error {
}
func (watcher *ContractWatcher) Execute() error {
for _, transformer := range watcher.Transformers {
err := transformer.Execute()
for _, contractTransformer := range watcher.Transformers {
err := contractTransformer.Execute()
if err != nil {
log.Error("Unable to execute transformer:", transformer.GetConfig().Name, err)
log.Error("Unable to execute transformer:", contractTransformer.GetConfig().Name, err)
return err
}
}

View File

@ -37,7 +37,7 @@ type EventWatcher struct {
Transformers []transformer.EventTransformer
BlockChain core.BlockChain
DB *postgres.DB
Fetcher fetcher.LogFetcher
Fetcher fetcher.ILogFetcher
Chunker chunker.Chunker
Addresses []common.Address
Topics []common.Hash
@ -47,7 +47,7 @@ type EventWatcher struct {
func NewEventWatcher(db *postgres.DB, bc core.BlockChain) EventWatcher {
logChunker := chunker.NewLogChunker()
logFetcher := fetcher.NewFetcher(bc)
logFetcher := fetcher.NewLogFetcher(bc)
transactionSyncer := transactions.NewTransactionsSyncer(db, bc)
return EventWatcher{
BlockChain: bc,

View File

@ -17,73 +17,100 @@
package watcher
import (
"fmt"
"reflect"
"strings"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/sirupsen/logrus"
"github.com/vulcanize/vulcanizedb/libraries/shared/fetcher"
"github.com/vulcanize/vulcanizedb/libraries/shared/storage"
"github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/libraries/shared/transformer"
"github.com/vulcanize/vulcanizedb/pkg/datastore/postgres"
"github.com/vulcanize/vulcanizedb/pkg/fs"
)
type StorageWatcher struct {
db *postgres.DB
tailer fs.Tailer
Queue storage.IStorageQueue
Transformers map[common.Address]transformer.StorageTransformer
db *postgres.DB
StorageFetcher fetcher.IStorageFetcher
Queue storage.IStorageQueue
Transformers map[common.Address]transformer.StorageTransformer
}
func NewStorageWatcher(tailer fs.Tailer, db *postgres.DB) StorageWatcher {
func NewStorageWatcher(fetcher fetcher.IStorageFetcher, db *postgres.DB) StorageWatcher {
transformers := make(map[common.Address]transformer.StorageTransformer)
queue := storage.NewStorageQueue(db)
return StorageWatcher{
db: db,
tailer: tailer,
Queue: queue,
Transformers: transformers,
db: db,
StorageFetcher: fetcher,
Queue: queue,
Transformers: transformers,
}
}
func (watcher StorageWatcher) AddTransformers(initializers []transformer.StorageTransformerInitializer) {
func (storageWatcher StorageWatcher) AddTransformers(initializers []transformer.StorageTransformerInitializer) {
for _, initializer := range initializers {
storageTransformer := initializer(watcher.db)
watcher.Transformers[storageTransformer.ContractAddress()] = storageTransformer
storageTransformer := initializer(storageWatcher.db)
storageWatcher.Transformers[storageTransformer.ContractAddress()] = storageTransformer
}
}
func (watcher StorageWatcher) Execute() error {
t, tailErr := watcher.tailer.Tail()
if tailErr != nil {
return tailErr
}
for line := range t.Lines {
row, parseErr := utils.FromStrings(strings.Split(line.Text, ","))
if parseErr != nil {
return parseErr
func (storageWatcher StorageWatcher) Execute(rows chan utils.StorageDiffRow, errs chan error, queueRecheckInterval time.Duration) {
ticker := time.NewTicker(queueRecheckInterval)
go storageWatcher.StorageFetcher.FetchStorageDiffs(rows, errs)
for {
select {
case fetchErr := <-errs:
logrus.Warn(fmt.Sprintf("error fetching storage diffs: %s", fetchErr))
case row := <-rows:
storageWatcher.processRow(row)
case <-ticker.C:
storageWatcher.processQueue()
}
storageTransformer, ok := watcher.Transformers[row.Contract]
}
}
func (storageWatcher StorageWatcher) processRow(row utils.StorageDiffRow) {
storageTransformer, ok := storageWatcher.Transformers[row.Contract]
if !ok {
// ignore rows from unwatched contracts
return
}
executeErr := storageTransformer.Execute(row)
if executeErr != nil {
logrus.Warn(fmt.Sprintf("error executing storage transformer: %s", executeErr))
queueErr := storageWatcher.Queue.Add(row)
if queueErr != nil {
logrus.Warn(fmt.Sprintf("error queueing storage diff: %s", queueErr))
}
}
}
func (storageWatcher StorageWatcher) processQueue() {
rows, fetchErr := storageWatcher.Queue.GetAll()
if fetchErr != nil {
logrus.Warn(fmt.Sprintf("error getting queued storage: %s", fetchErr))
}
for _, row := range rows {
storageTransformer, ok := storageWatcher.Transformers[row.Contract]
if !ok {
logrus.Warn(utils.ErrContractNotFound{Contract: row.Contract.Hex()}.Error())
// delete row from queue if address no longer watched
storageWatcher.deleteRow(row.Id)
continue
}
executeErr := storageTransformer.Execute(row)
if executeErr != nil {
if isKeyNotFound(executeErr) {
queueErr := watcher.Queue.Add(row)
if queueErr != nil {
logrus.Warn(queueErr.Error())
}
} else {
logrus.Warn(executeErr.Error())
}
continue
if executeErr == nil {
storageWatcher.deleteRow(row.Id)
}
}
return nil
}
func (storageWatcher StorageWatcher) deleteRow(id int) {
deleteErr := storageWatcher.Queue.Delete(id)
if deleteErr != nil {
logrus.Warn(fmt.Sprintf("error deleting persisted row from queue: %s", deleteErr))
}
}
func isKeyNotFound(executeErr error) bool {

View File

@ -17,15 +17,11 @@
package watcher_test
import (
"errors"
"fmt"
"io/ioutil"
"os"
"strings"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/hpcloud/tail"
. "github.com/onsi/ginkgo"
. "github.com/onsi/gomega"
"github.com/sirupsen/logrus"
@ -34,7 +30,6 @@ import (
"github.com/vulcanize/vulcanizedb/libraries/shared/storage/utils"
"github.com/vulcanize/vulcanizedb/libraries/shared/transformer"
"github.com/vulcanize/vulcanizedb/libraries/shared/watcher"
"github.com/vulcanize/vulcanizedb/pkg/core"
"github.com/vulcanize/vulcanizedb/pkg/fakes"
"github.com/vulcanize/vulcanizedb/test_config"
)
@ -43,170 +38,207 @@ var _ = Describe("Storage Watcher", func() {
It("adds transformers", func() {
fakeAddress := common.HexToAddress("0x12345")
fakeTransformer := &mocks.MockStorageTransformer{Address: fakeAddress}
w := watcher.NewStorageWatcher(&fakes.MockTailer{}, test_config.NewTestDB(core.Node{}))
w := watcher.NewStorageWatcher(mocks.NewMockStorageFetcher(), test_config.NewTestDB(test_config.NewTestNode()))
w.AddTransformers([]transformer.StorageTransformerInitializer{fakeTransformer.FakeTransformerInitializer})
Expect(w.Transformers[fakeAddress]).To(Equal(fakeTransformer))
})
It("reads the tail of the storage diffs file", func() {
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
Describe("executing watcher", func() {
var (
errs chan error
mockFetcher *mocks.MockStorageFetcher
mockQueue *mocks.MockStorageQueue
mockTransformer *mocks.MockStorageTransformer
row utils.StorageDiffRow
rows chan utils.StorageDiffRow
storageWatcher watcher.StorageWatcher
)
assert(func(err error) {
Expect(err).To(BeNil())
Expect(mockTailer.TailCalled).To(BeTrue())
}, w, mockTailer, []*tail.Line{})
})
It("returns error if row parsing fails", func() {
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
line := &tail.Line{Text: "oops"}
assert(func(err error) {
Expect(err).To(HaveOccurred())
Expect(err).To(MatchError(utils.ErrRowMalformed{Length: 1}))
}, w, mockTailer, []*tail.Line{line})
})
It("logs error if no transformer can parse storage row", func() {
mockTailer := fakes.NewMockTailer()
address := common.HexToAddress("0x12345")
line := getFakeLine(address.Bytes())
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
tempFile, err := ioutil.TempFile("", "log")
defer os.Remove(tempFile.Name())
Expect(err).NotTo(HaveOccurred())
logrus.SetOutput(tempFile)
assert(func(err error) {
Expect(err).NotTo(HaveOccurred())
logContent, readErr := ioutil.ReadFile(tempFile.Name())
Expect(readErr).NotTo(HaveOccurred())
Expect(string(logContent)).To(ContainSubstring(utils.ErrContractNotFound{Contract: address.Hex()}.Error()))
}, w, mockTailer, []*tail.Line{line})
})
It("executes transformer with storage row", func() {
address := []byte{1, 2, 3}
line := getFakeLine(address)
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
fakeTransformer := &mocks.MockStorageTransformer{Address: common.BytesToAddress(address)}
w.AddTransformers([]transformer.StorageTransformerInitializer{fakeTransformer.FakeTransformerInitializer})
assert(func(err error) {
Expect(err).To(BeNil())
expectedRow, err := utils.FromStrings(strings.Split(line.Text, ","))
Expect(err).NotTo(HaveOccurred())
Expect(fakeTransformer.PassedRow).To(Equal(expectedRow))
}, w, mockTailer, []*tail.Line{line})
})
Describe("when executing transformer fails", func() {
It("queues row when error is storage key not found", func() {
address := []byte{1, 2, 3}
line := getFakeLine(address)
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
mockQueue := &mocks.MockStorageQueue{}
w.Queue = mockQueue
keyNotFoundError := utils.ErrStorageKeyNotFound{Key: "unknown_storage_key"}
fakeTransformer := &mocks.MockStorageTransformer{Address: common.BytesToAddress(address), ExecuteErr: keyNotFoundError}
w.AddTransformers([]transformer.StorageTransformerInitializer{fakeTransformer.FakeTransformerInitializer})
assert(func(err error) {
Expect(err).NotTo(HaveOccurred())
Expect(mockQueue.AddCalled).To(BeTrue())
}, w, mockTailer, []*tail.Line{line})
BeforeEach(func() {
errs = make(chan error)
rows = make(chan utils.StorageDiffRow)
address := common.HexToAddress("0x0123456789abcdef")
mockFetcher = mocks.NewMockStorageFetcher()
mockQueue = &mocks.MockStorageQueue{}
mockTransformer = &mocks.MockStorageTransformer{Address: address}
row = utils.StorageDiffRow{
Id: 1337,
Contract: address,
BlockHash: common.HexToHash("0xfedcba9876543210"),
BlockHeight: 0,
StorageKey: common.HexToHash("0xabcdef1234567890"),
StorageValue: common.HexToHash("0x9876543210abcdef"),
}
})
It("logs error if queuing row fails", func() {
address := []byte{1, 2, 3}
line := getFakeLine(address)
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
mockQueue := &mocks.MockStorageQueue{}
mockQueue.AddError = fakes.FakeError
w.Queue = mockQueue
keyNotFoundError := utils.ErrStorageKeyNotFound{Key: "unknown_storage_key"}
fakeTransformer := &mocks.MockStorageTransformer{Address: common.BytesToAddress(address), ExecuteErr: keyNotFoundError}
w.AddTransformers([]transformer.StorageTransformerInitializer{fakeTransformer.FakeTransformerInitializer})
tempFile, err := ioutil.TempFile("", "log")
It("logs error if fetching storage diffs fails", func(done Done) {
mockFetcher.ErrsToReturn = []error{fakes.FakeError}
storageWatcher = watcher.NewStorageWatcher(mockFetcher, test_config.NewTestDB(test_config.NewTestNode()))
storageWatcher.Queue = mockQueue
storageWatcher.AddTransformers([]transformer.StorageTransformerInitializer{mockTransformer.FakeTransformerInitializer})
tempFile, fileErr := ioutil.TempFile("", "log")
Expect(fileErr).NotTo(HaveOccurred())
defer os.Remove(tempFile.Name())
Expect(err).NotTo(HaveOccurred())
logrus.SetOutput(tempFile)
assert(func(err error) {
Expect(err).NotTo(HaveOccurred())
Expect(mockQueue.AddCalled).To(BeTrue())
logContent, readErr := ioutil.ReadFile(tempFile.Name())
Expect(readErr).NotTo(HaveOccurred())
Expect(string(logContent)).To(ContainSubstring(fakes.FakeError.Error()))
}, w, mockTailer, []*tail.Line{line})
go storageWatcher.Execute(rows, errs, time.Hour)
Eventually(func() (string, error) {
logContent, err := ioutil.ReadFile(tempFile.Name())
return string(logContent), err
}).Should(ContainSubstring(fakes.FakeError.Error()))
close(done)
})
It("logs any other error", func() {
address := []byte{1, 2, 3}
line := getFakeLine(address)
mockTailer := fakes.NewMockTailer()
w := watcher.NewStorageWatcher(mockTailer, test_config.NewTestDB(core.Node{}))
executionError := errors.New("storage watcher failed attempting to execute transformer")
fakeTransformer := &mocks.MockStorageTransformer{Address: common.BytesToAddress(address), ExecuteErr: executionError}
w.AddTransformers([]transformer.StorageTransformerInitializer{fakeTransformer.FakeTransformerInitializer})
tempFile, err := ioutil.TempFile("", "log")
defer os.Remove(tempFile.Name())
Expect(err).NotTo(HaveOccurred())
logrus.SetOutput(tempFile)
Describe("transforming new storage diffs", func() {
BeforeEach(func() {
mockFetcher.RowsToReturn = []utils.StorageDiffRow{row}
storageWatcher = watcher.NewStorageWatcher(mockFetcher, test_config.NewTestDB(test_config.NewTestNode()))
storageWatcher.Queue = mockQueue
storageWatcher.AddTransformers([]transformer.StorageTransformerInitializer{mockTransformer.FakeTransformerInitializer})
})
assert(func(err error) {
Expect(err).NotTo(HaveOccurred())
logContent, readErr := ioutil.ReadFile(tempFile.Name())
Expect(readErr).NotTo(HaveOccurred())
Expect(string(logContent)).To(ContainSubstring(executionError.Error()))
}, w, mockTailer, []*tail.Line{line})
It("executes transformer for recognized storage row", func(done Done) {
go storageWatcher.Execute(rows, errs, time.Hour)
Eventually(func() utils.StorageDiffRow {
return mockTransformer.PassedRow
}).Should(Equal(row))
close(done)
})
It("queues row for later processing if transformer execution fails", func(done Done) {
mockTransformer.ExecuteErr = fakes.FakeError
go storageWatcher.Execute(rows, errs, time.Hour)
Expect(<-errs).To(BeNil())
Eventually(func() bool {
return mockQueue.AddCalled
}).Should(BeTrue())
Eventually(func() utils.StorageDiffRow {
return mockQueue.AddPassedRow
}).Should(Equal(row))
close(done)
})
It("logs error if queueing row fails", func(done Done) {
mockTransformer.ExecuteErr = utils.ErrStorageKeyNotFound{}
mockQueue.AddError = fakes.FakeError
tempFile, fileErr := ioutil.TempFile("", "log")
Expect(fileErr).NotTo(HaveOccurred())
defer os.Remove(tempFile.Name())
logrus.SetOutput(tempFile)
go storageWatcher.Execute(rows, errs, time.Hour)
Eventually(func() bool {
return mockQueue.AddCalled
}).Should(BeTrue())
Eventually(func() (string, error) {
logContent, err := ioutil.ReadFile(tempFile.Name())
return string(logContent), err
}).Should(ContainSubstring(fakes.FakeError.Error()))
close(done)
})
})
Describe("transforming queued storage diffs", func() {
BeforeEach(func() {
mockQueue.RowsToReturn = []utils.StorageDiffRow{row}
storageWatcher = watcher.NewStorageWatcher(mockFetcher, test_config.NewTestDB(test_config.NewTestNode()))
storageWatcher.Queue = mockQueue
storageWatcher.AddTransformers([]transformer.StorageTransformerInitializer{mockTransformer.FakeTransformerInitializer})
})
It("logs error if getting queued storage fails", func(done Done) {
mockQueue.GetAllErr = fakes.FakeError
tempFile, fileErr := ioutil.TempFile("", "log")
Expect(fileErr).NotTo(HaveOccurred())
defer os.Remove(tempFile.Name())
logrus.SetOutput(tempFile)
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() (string, error) {
logContent, err := ioutil.ReadFile(tempFile.Name())
return string(logContent), err
}).Should(ContainSubstring(fakes.FakeError.Error()))
close(done)
})
It("executes transformer for storage row", func(done Done) {
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() utils.StorageDiffRow {
return mockTransformer.PassedRow
}).Should(Equal(row))
close(done)
})
It("deletes row from queue if transformer execution successful", func(done Done) {
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() int {
return mockQueue.DeletePassedId
}).Should(Equal(row.Id))
close(done)
})
It("logs error if deleting persisted row fails", func(done Done) {
mockQueue.DeleteErr = fakes.FakeError
tempFile, fileErr := ioutil.TempFile("", "log")
Expect(fileErr).NotTo(HaveOccurred())
defer os.Remove(tempFile.Name())
logrus.SetOutput(tempFile)
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() (string, error) {
logContent, err := ioutil.ReadFile(tempFile.Name())
return string(logContent), err
}).Should(ContainSubstring(fakes.FakeError.Error()))
close(done)
})
It("deletes obsolete row from queue if contract not recognized", func(done Done) {
obsoleteRow := utils.StorageDiffRow{
Id: row.Id + 1,
Contract: common.HexToAddress("0xfedcba9876543210"),
}
mockQueue.RowsToReturn = []utils.StorageDiffRow{obsoleteRow}
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() int {
return mockQueue.DeletePassedId
}).Should(Equal(obsoleteRow.Id))
close(done)
})
It("logs error if deleting obsolete row fails", func(done Done) {
obsoleteRow := utils.StorageDiffRow{
Id: row.Id + 1,
Contract: common.HexToAddress("0xfedcba9876543210"),
}
mockQueue.RowsToReturn = []utils.StorageDiffRow{obsoleteRow}
mockQueue.DeleteErr = fakes.FakeError
tempFile, fileErr := ioutil.TempFile("", "log")
Expect(fileErr).NotTo(HaveOccurred())
defer os.Remove(tempFile.Name())
logrus.SetOutput(tempFile)
go storageWatcher.Execute(rows, errs, time.Nanosecond)
Eventually(func() (string, error) {
logContent, err := ioutil.ReadFile(tempFile.Name())
return string(logContent), err
}).Should(ContainSubstring(fakes.FakeError.Error()))
close(done)
})
})
})
})
func assert(assertion func(err error), watcher watcher.StorageWatcher, mockTailer *fakes.MockTailer, lines []*tail.Line) {
errs := make(chan error, 1)
done := make(chan bool, 1)
go execute(watcher, errs, done)
for _, line := range lines {
mockTailer.Lines <- line
}
close(mockTailer.Lines)
select {
case err := <-errs:
assertion(err)
break
case <-done:
assertion(nil)
break
}
}
func execute(w watcher.StorageWatcher, errs chan error, done chan bool) {
err := w.Execute()
if err != nil {
errs <- err
} else {
done <- true
}
}
func getFakeLine(address []byte) *tail.Line {
blockHash := []byte{4, 5, 6}
blockHeight := int64(789)
storageKey := []byte{9, 8, 7}
storageValue := []byte{6, 5, 4}
return &tail.Line{
Text: fmt.Sprintf("%s,%s,%d,%s,%s", common.Bytes2Hex(address), common.Bytes2Hex(blockHash), blockHeight, common.Bytes2Hex(storageKey), common.Bytes2Hex(storageValue)),
Time: time.Time{},
Err: nil,
}
}

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fakes
import "github.com/vulcanize/vulcanizedb/pkg/filters"

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fakes
import (

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fakes
import (

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fakes
import (
@ -6,24 +22,22 @@ import (
)
type MockTailer struct {
Lines chan *tail.Line
TailCalled bool
Lines chan *tail.Line
TailErr error
}
func NewMockTailer() *MockTailer {
return &MockTailer{
Lines: make(chan *tail.Line, 1),
TailCalled: false,
Lines: make(chan *tail.Line, 1),
}
}
func (mock *MockTailer) Tail() (*tail.Tail, error) {
mock.TailCalled = true
fakeTail := &tail.Tail{
Filename: "",
Lines: mock.Lines,
Config: tail.Config{},
Tomb: tomb.Tomb{},
}
return fakeTail, nil
return fakeTail, mock.TailErr
}

View File

@ -1,3 +1,19 @@
// VulcanizeDB
// Copyright © 2019 Vulcanize
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package fakes
import "github.com/ethereum/go-ethereum/core/types"