Copy package from geth fork

This commit is contained in:
Roy Crihfield 2023-06-14 20:43:34 +08:00
parent 88a2405805
commit bb00c46735
111 changed files with 18496 additions and 1 deletions

323
README.md
View File

@ -1,3 +1,324 @@
# plugeth-statediff
cerc/go-ethereum statediff functionality packaged as a plugeth plugin
`cerc/go-ethereum` statediff functionality packaged as a plugeth plugin.
## Package
This package provides an auxiliary service that asynchronously processes state diff objects from chain events,
either relaying the state objects to RPC subscribers or writing them directly to Postgres as IPLD objects.
It also exposes RPC endpoints for fetching or writing to Postgres the state diff at a specific block height
or for a specific block hash, this operates on historical block and state data and so depends on a complete state archive.
Data is emitted in this differential format in order to make it feasible to IPLD-ize and index the _entire_ Ethereum state
(including intermediate state and storage trie nodes). If this state diff process is ran continuously from genesis,
the entire state at any block can be materialized from the cumulative differentials up to that point.
## Statediff object
A state diff `StateObject` is the collection of all the state and storage trie nodes that have been updated in a given block.
For convenience, we also associate these nodes with the block number and hash, and optionally the set of code hashes and code for any
contracts deployed in this block.
A complete state diff `StateObject` will include all state and storage intermediate nodes, which is necessary for generating proofs and for
traversing the tries.
```go
// StateObject is a collection of state (and linked storage nodes) as well as the associated block number, block hash,
// and a set of code hashes and their code
type StateObject struct {
BlockNumber *big.Int `json:"blockNumber" gencodec:"required"`
BlockHash common.Hash `json:"blockHash" gencodec:"required"`
Nodes []StateNode `json:"nodes" gencodec:"required"`
CodeAndCodeHashes []CodeAndCodeHash `json:"codeMapping"`
}
// StateNode holds the data for a single state diff node
type StateNode struct {
NodeType NodeType `json:"nodeType" gencodec:"required"`
Path []byte `json:"path" gencodec:"required"`
NodeValue []byte `json:"value" gencodec:"required"`
StorageNodes []StorageNode `json:"storage"`
LeafKey []byte `json:"leafKey"`
}
// StorageNode holds the data for a single storage diff node
type StorageNode struct {
NodeType NodeType `json:"nodeType" gencodec:"required"`
Path []byte `json:"path" gencodec:"required"`
NodeValue []byte `json:"value" gencodec:"required"`
LeafKey []byte `json:"leafKey"`
}
// CodeAndCodeHash struct for holding codehash => code mappings
// we can't use an actual map because they are not rlp serializable
type CodeAndCodeHash struct {
Hash common.Hash `json:"codeHash"`
Code []byte `json:"code"`
}
```
These objects are packed into a `Payload` structure which can additionally associate the `StateObject`
with the block (header, uncles, and transactions), receipts, and total difficulty.
This `Payload` encapsulates all of the differential data at a given block, and allows us to index the entire Ethereum data structure
as hash-linked IPLD objects.
```go
// Payload packages the data to send to state diff subscriptions
type Payload struct {
BlockRlp []byte `json:"blockRlp"`
TotalDifficulty *big.Int `json:"totalDifficulty"`
ReceiptsRlp []byte `json:"receiptsRlp"`
StateObjectRlp []byte `json:"stateObjectRlp" gencodec:"required"`
encoded []byte
err error
}
```
## Usage
This state diffing service runs as an auxiliary service concurrent to the regular syncing process of the geth node.
### CLI configuration
This service introduces a CLI flag namespace `statediff`
`--statediff` flag is used to turn on the service
`--statediff.writing` is used to tell the service to write state diff objects it produces from synced ChainEvents directly to a configured Postgres database
`--statediff.workers` is used to set the number of concurrent workers to process state diff objects and write them into the database
`--statediff.db.type` is the type of database we write out to (current options: postgres, dump, file)
`--statediff.dump.dst` is the destination to write to when operating in database dump mode (stdout, stderr, discard)
`--statediff.db.driver` is the specific driver to use for the database (current options for postgres: pgx and sqlx)
`--statediff.db.host` is the hostname/ip to dial to connect to the database
`--statediff.db.port` is the port to dial to connect to the database
`--statediff.db.name` is the name of the database to connect to
`--statediff.db.user` is the user to connect to the database as
`--statediff.db.password` is the password to use to connect to the database
`--statediff.db.conntimeout` is the connection timeout (in seconds)
`--statediff.db.maxconns` is the maximum number of database connections
`--statediff.db.minconns` is the minimum number of database connections
`--statediff.db.maxidleconns` is the maximum number of idle connections
`--statediff.db.maxconnidletime` is the maximum lifetime for an idle connection (in seconds)
`--statediff.db.maxconnlifetime` is the maximum lifetime for a connection (in seconds)
`--statediff.db.nodeid` is the node id to use in the Postgres database
`--statediff.db.clientname` is the client name to use in the Postgres database
`--statediff.db.upsert` whether or not the service, when operating in a direct database writing mode, should overwrite any existing conflicting data
`--statediff.file.path` full path (including filename) to write statediff data out to when operating in file mode
`--statediff.file.wapath` full path (including filename) to write statediff watched addresses out to when operating in file mode
The service can only operate in full sync mode (`--syncmode=full`), but only the historical RPC endpoints require an archive node (`--gcmode=archive`)
e.g.
`./build/bin/geth --syncmode=full --gcmode=archive --statediff --statediff.writing --statediff.db.type=postgres --statediff.db.driver=sqlx --statediff.db.host=localhost --statediff.db.port=5432 --statediff.db.name=cerc_testing --statediff.db.user=postgres --statediff.db.nodeid=nodeid --statediff.db.clientname=clientname`
When operating in `--statediff.db.type=file` mode, the service will write SQL statements out to the file designated by
`--statediff.file.path`. Please note that it writes out SQL statements with all `ON CONFLICT` constraint checks dropped.
This is done so that we can scale out the production of the SQL statements horizontally, merge the separate SQL files produced,
de-duplicate using unix tools (`sort statediff.sql | uniq` or `sort -u statediff.sql`), bulk load using psql
(`psql db_name --set ON_ERROR_STOP=on -f statediff.sql`), and then add our primary and foreign key constraints and indexes
back afterwards.
### RPC endpoints
The state diffing service exposes both a WS subscription endpoint, and a number of HTTP unary endpoints.
Each of these endpoints requires a set of parameters provided by the caller
```go
// Params is used to carry in parameters from subscribing/requesting clients configuration
type Params struct {
IntermediateStateNodes bool
IntermediateStorageNodes bool
IncludeBlock bool
IncludeReceipts bool
IncludeTD bool
IncludeCode bool
WatchedAddresses []common.Address
}
```
Using these params we can tell the service whether to include state and/or storage intermediate nodes; whether
to include the associated block (header, uncles, and transactions); whether to include the associated receipts;
whether to include the total difficulty for this block; whether to include the set of code hashes and code for
contracts deployed in this block; whether to limit the diffing process to a list of specific addresses.
#### Subscription endpoint
A websocket supporting RPC endpoint is exposed for subscribing to state diff `StateObjects` that come off the head of the chain while the geth node syncs.
```go
// Stream is a subscription endpoint that fires off state diff payloads as they are created
Stream(ctx context.Context, params Params) (*rpc.Subscription, error)
```
To expose this endpoint the node needs to have the websocket server turned on (`--ws`),
and the `statediff` namespace exposed (`--ws.api=statediff`).
Go code subscriptions to this endpoint can be created using the `rpc.Client.Subscribe()` method,
with the "statediff" namespace, a `statediff.Payload` channel, and the name of the statediff api's rpc method: "stream".
e.g.
```go
cli, err := rpc.Dial("ipcPathOrWsURL")
if err != nil {
// handle error
}
stateDiffPayloadChan := make(chan statediff.Payload, 20000)
methodName := "stream"
params := statediff.Params{
IncludeBlock: true,
IncludeTD: true,
IncludeReceipts: true,
IntermediateStorageNodes: true,
IntermediateStateNodes: true,
}
rpcSub, err := cli.Subscribe(context.Background(), statediff.APIName, stateDiffPayloadChan, methodName, params)
if err != nil {
// handle error
}
for {
select {
case stateDiffPayload := <- stateDiffPayloadChan:
// process the payload
case err := <- rpcSub.Err():
// handle rpc subscription error
}
}
```
#### Unary endpoints
The service also exposes unary RPC endpoints for retrieving the state diff `StateObject` for a specific block height/hash.
```go
// StateDiffAt returns a state diff payload at the specific blockheight
StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error)
// StateDiffFor returns a state diff payload for the specific blockhash
StateDiffFor(ctx context.Context, blockHash common.Hash, params Params) (*Payload, error)
```
To expose this endpoint the node needs to have the HTTP server turned on (`--http`),
and the `statediff` namespace exposed (`--http.api=statediff`).
### Direct indexing into Postgres
If `--statediff.writing` is set, the service will convert the state diff `StateObject` data into IPLD objects, persist them directly to Postgres,
and generate secondary indexes around the IPLD data.
The schema and migrations for this Postgres database are provided in `statediff/db/`.
#### Postgres setup
We use [pressly/goose](https://github.com/pressly/goose) as our Postgres migration manager.
You can also load the Postgres schema directly into a database using
`psql database_name < schema.sql`
This will only work on a version 12.4 Postgres database.
#### Schema overview
Our Postgres schemas are built around a single IPFS backing Postgres IPLD blockstore table (`ipld.blocks`) that conforms with [go-ds-sql](https://github.com/ipfs/go-ds-sql/blob/master/postgres/postgres.go).
All IPLD objects are stored in this table, where `key` is the blockstore-prefixed multihash key for the IPLD object and `data` contains
the bytes for the IPLD block (in the case of all Ethereum IPLDs, this is the RLP byte encoding of the Ethereum object).
The IPLD objects in this table can be traversed using an IPLD DAG interface, but since this table only maps multihash to raw IPLD object
it is not particularly useful for searching through the data by looking up Ethereum objects by their constituent fields
(e.g. by block number, tx source/recipient, state/storage trie node path). To improve the accessibility of these objects
we create an Ethereum [advanced data layout](https://github.com/ipld/specs#schemas-and-advanced-data-layouts) (ADL) by generating secondary
indexes on top of the raw IPLDs in other Postgres tables.
These secondary index tables fall under the `eth` schema and follow an `{objectType}_cids` naming convention.
These tables provide a view into individual fields of the underlying Ethereum IPLD objects, allowing lookups on these fields, and reference the raw IPLD objects stored in `ipld.blocks`
by foreign keys to their multihash keys.
Additionally, these tables maintain the hash-linked nature of Ethereum objects to one another. E.g. a storage trie node entry in the `storage_cids`
table contains a `state_id` foreign key which references the `id` for the `state_cids` entry that contains the state leaf node for the contract that storage node belongs to,
and in turn that `state_cids` entry contains a `header_id` foreign key which references the `id` of the `header_cids` entry that contains the header for the block these state and storage nodes were updated (diffed).
### Optimization
On mainnet this process is extremely IO intensive and requires significant resources to allow it to keep up with the head of the chain.
The state diff processing time for a specific block is dependent on the number and complexity of the state changes that occur in a block and
the number of updated state nodes that are available in the in-memory cache vs must be retrieved from disc.
If memory permits, one means of improving the efficiency of this process is to increase the in-memory trie cache allocation.
This can be done by increasing the overall `--cache` allocation and/or by increasing the % of the cache allocated to trie
usage with `--cache.trie`.
## Versioning, Branches, Rebasing, and Releasing
Internal tagged releases are maintained for building the latest version of statediffing geth or using it as a go mod dependency.
When a new core go-ethereum version is released, statediffing geth is rebased onto and adjusted to work with the new tag.
We want to maintain a complete record of our git history, but in order to make frequent and timely rebases feasible we also
need to be able to squash our work before performing a rebase. To this end we retain multiple branches with partial incremental history that culminate in
the full incremental history.
### Versioning
Example: `v1.10.16-statediff-3.0.2`
- The first section, `v1.10.16`, corresponds to the release of the root branch this version is rebased onto (e.g., [](https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16)[https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16](https://github.com/ethereum/go-ethereum/releases/tag/v1.10.16))
- The second section, `3.0.2`, corresponds to the version of our statediffing code. The major version here (3) should always correspond with the major version of the `ipld-eth-db` schema version it works with (e.g., [](https://github.com/cerc-io/ipld-eth-db/releases/tag/v3.0.6)[https://github.com/vulcanize/ipld-eth-db/releases/tag/v3.0.6](https://github.com/vulcanize/ipld-eth-db/releases/tag/v3.0.6)); it is only bumped when we bump the major version of the schema.
- The major version of the schema is only bumped when a breaking change is made to the schema.
- The minor version is bumped when a new feature is added, or a fix is performed that breaks or updates the statediffing API or CLI in some way.
- The patch version is bumped whenever minor fixes/patches/features are done that dont change/break API/CLI compatibility.
- We are very strict about the first section and the major version of the statediffing code, but some discretion is required when deciding to bump minor versus patch version of the statediffing code.
The statediff version is included in the `VersionMeta` in params/version.go
### Branches
We maintain two official kinds of branches:
Major Branch: `{Root Version}-statediff`
Major branches retain the cumulative state of all changes made before the latest root version rebase and track the full incremental history of changes made between the latest root version rebase and the next.
Aside from creating the branch by performing the rebase described in the section below, these branches are never worked off of or committed to directly.
Feature Branch: `{Root Version}-statediff-{Statediff Version}`
Feature branches are checked out from a major branch in order to work on a new feature or fix for the statediffing code.
The statediff version of a feature branch is the new version it affects on the major branch when merged. Internal tagged releases
are cut against these branches after they are merged back to the major branch.
If a developer is unsure what version their patch should affect, they should remain working on an unofficial branch. From there
they can open a PR against the targeted root branch and be directed to the appropriate feature version and branch.
### Rebasing
When a new root tagged release comes out we rebase our statediffing code on top of the new tag using the following process:
1. Checkout a new major branch for the tag from the current major branch
2. On the new major branch, squash all our commits since the last major rebase
3. On the new major branch, perform the rebase against the new tag
4. Push the new major branch to the remote
5. From the new major branch, checkout a new feature branch based on the new major version and the last statediff version
6. On this new feature branch, add the new major branch to the .github/workflows/on-master.yml list of "on push" branches
7. On this new feature branch, make any fixes/adjustments required for all statediffing geth tests to pass
8. PR this feature branch into the new major branch, this PR will trigger CI tests and builds.
9. After merging PR, rebase feature branch onto major branch
10. Cut a new release targeting the feature branch, this release should have the new root version but the same statediff version as the last release

206
api.go Normal file
View File

@ -0,0 +1,206 @@
// Copyright 2019 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package statediff
import (
"context"
"github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rpc"
)
// APIName is the namespace used for the state diffing service API
const APIName = "statediff"
// APIVersion is the version of the state diffing service API
const APIVersion = "0.0.1"
// PublicStateDiffAPI provides an RPC subscription interface
// that can be used to stream out state diffs as they
// are produced by a full node
type PublicStateDiffAPI struct {
sds IService
}
// NewPublicStateDiffAPI creates an rpc subscription interface for the underlying statediff service
func NewPublicStateDiffAPI(sds IService) *PublicStateDiffAPI {
return &PublicStateDiffAPI{
sds: sds,
}
}
// Stream is the public method to setup a subscription that fires off statediff service payloads as they are created
func (api *PublicStateDiffAPI) Stream(ctx context.Context, params Params) (*rpc.Subscription, error) {
// ensure that the RPC connection supports subscriptions
notifier, supported := rpc.NotifierFromContext(ctx)
if !supported {
return nil, rpc.ErrNotificationsUnsupported
}
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
go func() {
// subscribe to events from the statediff service
payloadChannel := make(chan Payload, chainEventChanSize)
quitChan := make(chan bool, 1)
api.sds.Subscribe(rpcSub.ID, payloadChannel, quitChan, params)
// loop and await payloads and relay them to the subscriber with the notifier
for {
select {
case payload := <-payloadChannel:
if err := notifier.Notify(rpcSub.ID, payload); err != nil {
log.Error("Failed to send state diff packet; error: " + err.Error())
if err := api.sds.Unsubscribe(rpcSub.ID); err != nil {
log.Error("Failed to unsubscribe from the state diff service; error: " + err.Error())
}
return
}
case err := <-rpcSub.Err():
if err != nil {
log.Error("State diff service rpcSub error: " + err.Error())
err = api.sds.Unsubscribe(rpcSub.ID)
if err != nil {
log.Error("Failed to unsubscribe from the state diff service; error: " + err.Error())
}
return
}
case <-quitChan:
// don't need to unsubscribe, service does so before sending the quit signal
return
}
}
}()
return rpcSub, nil
}
// StateDiffAt returns a state diff payload at the specific blockheight
func (api *PublicStateDiffAPI) StateDiffAt(ctx context.Context, blockNumber uint64, params Params) (*Payload, error) {
return api.sds.StateDiffAt(blockNumber, params)
}
// StateDiffFor returns a state diff payload for the specific blockhash
func (api *PublicStateDiffAPI) StateDiffFor(ctx context.Context, blockHash common.Hash, params Params) (*Payload, error) {
return api.sds.StateDiffFor(blockHash, params)
}
// StreamCodeAndCodeHash writes all of the codehash=>code pairs out to a websocket channel
func (api *PublicStateDiffAPI) StreamCodeAndCodeHash(ctx context.Context, blockNumber uint64) (*rpc.Subscription, error) {
// ensure that the RPC connection supports subscriptions
notifier, supported := rpc.NotifierFromContext(ctx)
if !supported {
return nil, rpc.ErrNotificationsUnsupported
}
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
payloadChan := make(chan types.CodeAndCodeHash, chainEventChanSize)
quitChan := make(chan bool)
api.sds.StreamCodeAndCodeHash(blockNumber, payloadChan, quitChan)
go func() {
for {
select {
case payload := <-payloadChan:
if err := notifier.Notify(rpcSub.ID, payload); err != nil {
log.Error("Failed to send code and codehash packet", "err", err)
return
}
case err := <-rpcSub.Err():
log.Error("State diff service rpcSub error", "err", err)
return
case <-quitChan:
return
}
}
}()
return rpcSub, nil
}
// WriteStateDiffAt writes a state diff object directly to DB at the specific blockheight
func (api *PublicStateDiffAPI) WriteStateDiffAt(ctx context.Context, blockNumber uint64, params Params) JobID {
var err error
start, logger := countApiRequestBegin("writeStateDiffAt", blockNumber)
defer countApiRequestEnd(start, logger, err)
return api.sds.WriteStateDiffAt(blockNumber, params)
}
// WriteStateDiffFor writes a state diff object directly to DB for the specific block hash
func (api *PublicStateDiffAPI) WriteStateDiffFor(ctx context.Context, blockHash common.Hash, params Params) error {
var err error
start, logger := countApiRequestBegin("writeStateDiffFor", blockHash.Hex())
defer countApiRequestEnd(start, logger, err)
err = api.sds.WriteStateDiffFor(blockHash, params)
return err
}
// WatchAddress changes the list of watched addresses to which the direct indexing is restricted according to given operation
func (api *PublicStateDiffAPI) WatchAddress(operation types.OperationType, args []types.WatchAddressArg) error {
return api.sds.WatchAddress(operation, args)
}
// StreamWrites sets up a subscription that streams the status of completed calls to WriteStateDiff*
func (api *PublicStateDiffAPI) StreamWrites(ctx context.Context) (*rpc.Subscription, error) {
// ensure that the RPC connection supports subscriptions
notifier, supported := rpc.NotifierFromContext(ctx)
if !supported {
return nil, rpc.ErrNotificationsUnsupported
}
// create subscription and start waiting for events
rpcSub := notifier.CreateSubscription()
go func() {
// subscribe to events from the statediff service
statusChan := make(chan JobStatus, chainEventChanSize)
quitChan := make(chan bool, 1)
api.sds.SubscribeWriteStatus(rpcSub.ID, statusChan, quitChan)
var err error
defer func() {
if err = api.sds.UnsubscribeWriteStatus(rpcSub.ID); err != nil {
log.Error("Failed to unsubscribe from job status stream: " + err.Error())
}
}()
// loop and await payloads and relay them to the subscriber with the notifier
for {
select {
case status := <-statusChan:
if err = notifier.Notify(rpcSub.ID, status); err != nil {
log.Error("Failed to send job status; error: " + err.Error())
return
}
case err = <-rpcSub.Err():
if err != nil {
log.Error("statediff_StreamWrites RPC subscription error: " + err.Error())
return
}
case <-quitChan:
// don't need to unsubscribe, service does so before sending the quit signal
return
}
}
}()
return rpcSub, nil
}

677
builder.go Normal file
View File

@ -0,0 +1,677 @@
// Copyright 2019 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
// Contains a batch of utility type declarations used by the tests. As the node
// operates on unique types, a lot of them are needed to check various features.
package statediff
import (
"bytes"
"fmt"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/rlp"
metrics2 "github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ethereum/go-ethereum/statediff/trie_helpers"
types2 "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
)
var (
emptyNode, _ = rlp.EncodeToBytes(&[]byte{})
emptyContractRoot = crypto.Keccak256Hash(emptyNode)
nullCodeHash = crypto.Keccak256Hash([]byte{}).Bytes()
nullNodeHash = common.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000")
)
// Builder interface exposes the method for building a state diff between two blocks
type Builder interface {
BuildStateDiffObject(args Args, params Params) (types2.StateObject, error)
WriteStateDiffObject(args Args, params Params, output types2.StateNodeSink, ipldOutput types2.IPLDSink) error
}
type StateDiffBuilder struct {
StateCache state.Database
}
type IterPair struct {
Older, Newer trie.NodeIterator
}
func StateNodeAppender(nodes *[]types2.StateLeafNode) types2.StateNodeSink {
return func(node types2.StateLeafNode) error {
*nodes = append(*nodes, node)
return nil
}
}
func StorageNodeAppender(nodes *[]types2.StorageLeafNode) types2.StorageNodeSink {
return func(node types2.StorageLeafNode) error {
*nodes = append(*nodes, node)
return nil
}
}
func IPLDMappingAppender(iplds *[]types2.IPLD) types2.IPLDSink {
return func(c types2.IPLD) error {
*iplds = append(*iplds, c)
return nil
}
}
// NewBuilder is used to create a statediff builder
func NewBuilder(stateCache state.Database) Builder {
return &StateDiffBuilder{
StateCache: stateCache, // state cache is safe for concurrent reads
}
}
// BuildStateDiffObject builds a statediff object from two blocks and the provided parameters
func (sdb *StateDiffBuilder) BuildStateDiffObject(args Args, params Params) (types2.StateObject, error) {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStateDiffObjectTimer)
var stateNodes []types2.StateLeafNode
var iplds []types2.IPLD
err := sdb.WriteStateDiffObject(args, params, StateNodeAppender(&stateNodes), IPLDMappingAppender(&iplds))
if err != nil {
return types2.StateObject{}, err
}
return types2.StateObject{
BlockHash: args.BlockHash,
BlockNumber: args.BlockNumber,
Nodes: stateNodes,
IPLDs: iplds,
}, nil
}
// WriteStateDiffObject writes a statediff object to output sinks
func (sdb *StateDiffBuilder) WriteStateDiffObject(args Args, params Params, output types2.StateNodeSink,
ipldOutput types2.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.WriteStateDiffObjectTimer)
// Load tries for old and new states
oldTrie, err := sdb.StateCache.OpenTrie(args.OldStateRoot)
if err != nil {
return fmt.Errorf("error creating trie for oldStateRoot: %v", err)
}
newTrie, err := sdb.StateCache.OpenTrie(args.NewStateRoot)
if err != nil {
return fmt.Errorf("error creating trie for newStateRoot: %v", err)
}
// we do two state trie iterations:
// one for new/updated nodes,
// one for deleted/updated nodes;
// prepare 2 iterator instances for each task
iterPairs := []IterPair{
{
Older: oldTrie.NodeIterator([]byte{}),
Newer: newTrie.NodeIterator([]byte{}),
},
{
Older: oldTrie.NodeIterator([]byte{}),
Newer: newTrie.NodeIterator([]byte{}),
},
}
logger := log.New("hash", args.BlockHash.Hex(), "number", args.BlockNumber)
return sdb.BuildStateDiffWithIntermediateStateNodes(iterPairs, params, output, ipldOutput, logger, nil)
}
func (sdb *StateDiffBuilder) BuildStateDiffWithIntermediateStateNodes(iterPairs []IterPair, params Params,
output types2.StateNodeSink, ipldOutput types2.IPLDSink, logger log.Logger, prefixPath []byte) error {
logger.Debug("statediff BEGIN BuildStateDiffWithIntermediateStateNodes")
defer metrics2.ReportAndUpdateDuration("statediff END BuildStateDiffWithIntermediateStateNodes", time.Now(), logger, metrics2.IndexerMetrics.BuildStateDiffWithIntermediateStateNodesTimer)
// collect a slice of all the nodes that were touched and exist at B (B-A)
// a map of their leafkey to all the accounts that were touched and exist at B
// and a slice of all the paths for the nodes in both of the above sets
diffAccountsAtB, err := sdb.createdAndUpdatedState(
iterPairs[0].Older, iterPairs[0].Newer, params.watchedAddressesLeafPaths, ipldOutput, logger, prefixPath)
if err != nil {
return fmt.Errorf("error collecting createdAndUpdatedNodes: %v", err)
}
// collect a slice of all the nodes that existed at a path in A that doesn't exist in B
// a map of their leafkey to all the accounts that were touched and exist at A
diffAccountsAtA, err := sdb.deletedOrUpdatedState(
iterPairs[1].Older, iterPairs[1].Newer, diffAccountsAtB,
params.watchedAddressesLeafPaths, output, logger, prefixPath)
if err != nil {
return fmt.Errorf("error collecting deletedOrUpdatedNodes: %v", err)
}
// collect and sort the leafkey keys for both account mappings into a slice
t := time.Now()
createKeys := trie_helpers.SortKeys(diffAccountsAtB)
deleteKeys := trie_helpers.SortKeys(diffAccountsAtA)
logger.Debug(fmt.Sprintf("statediff BuildStateDiffWithIntermediateStateNodes sort duration=%dms", time.Since(t).Milliseconds()))
// and then find the intersection of these keys
// these are the leafkeys for the accounts which exist at both A and B but are different
// this also mutates the passed in createKeys and deleteKeys, removing the intersection keys
// and leaving the truly created or deleted keys in place
t = time.Now()
updatedKeys := trie_helpers.FindIntersection(createKeys, deleteKeys)
logger.Debug(fmt.Sprintf("statediff BuildStateDiffWithIntermediateStateNodes intersection count=%d duration=%dms",
len(updatedKeys),
time.Since(t).Milliseconds()))
// build the diff nodes for the updated accounts using the mappings at both A and B as directed by the keys found as the intersection of the two
err = sdb.buildAccountUpdates(diffAccountsAtB, diffAccountsAtA, updatedKeys, output, ipldOutput, logger)
if err != nil {
return fmt.Errorf("error building diff for updated accounts: %v", err)
}
// build the diff nodes for created accounts
err = sdb.buildAccountCreations(diffAccountsAtB, output, ipldOutput, logger)
if err != nil {
return fmt.Errorf("error building diff for created accounts: %v", err)
}
return nil
}
// createdAndUpdatedState returns
// a slice of all the intermediate nodes that exist in a different state at B than A
// a mapping of their leafkeys to all the accounts that exist in a different state at B than A
// and a slice of the paths for all of the nodes included in both
func (sdb *StateDiffBuilder) createdAndUpdatedState(a, b trie.NodeIterator,
watchedAddressesLeafPaths [][]byte, output types2.IPLDSink, logger log.Logger, prefixPath []byte) (types2.AccountMap, error) {
logger.Debug("statediff BEGIN createdAndUpdatedState")
defer metrics2.ReportAndUpdateDuration("statediff END createdAndUpdatedState", time.Now(), logger, metrics2.IndexerMetrics.CreatedAndUpdatedStateTimer)
diffAccountsAtB := make(types2.AccountMap)
watchingAddresses := len(watchedAddressesLeafPaths) > 0
it, itCount := trie.NewDifferenceIterator(a, b)
for it.Next(true) {
// ignore node if it is not along paths of interest
if watchingAddresses && !isValidPrefixPath(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) {
continue
}
// index values by leaf key
if it.Leaf() {
// if it is a "value" node, we will index the value by leaf key
accountW, err := sdb.processStateValueNode(it, watchedAddressesLeafPaths, prefixPath)
if err != nil {
return nil, err
}
if accountW == nil {
continue
}
// for now, just add it to diffAccountsAtB
// we will compare to diffAccountsAtA to determine which diffAccountsAtB
// were creations and which were updates and also identify accounts that were removed going A->B
diffAccountsAtB[common.Bytes2Hex(accountW.LeafKey)] = *accountW
} else { // trie nodes will be written to blockstore only
// reminder that this includes leaf nodes, since the geth iterator.Leaf() actually signifies a "value" node
if bytes.Equal(it.Hash().Bytes(), nullNodeHash) {
continue
}
nodeVal := make([]byte, len(it.NodeBlob()))
copy(nodeVal, it.NodeBlob())
if len(watchedAddressesLeafPaths) > 0 {
var elements []interface{}
if err := rlp.DecodeBytes(nodeVal, &elements); err != nil {
return nil, err
}
ok, err := isLeaf(elements)
if err != nil {
return nil, err
}
if ok {
nodePath := append(prefixPath, it.Path()...)
partialPath := trie.CompactToHex(elements[0].([]byte))
valueNodePath := append(nodePath, partialPath...)
if !isWatchedAddress(watchedAddressesLeafPaths, valueNodePath) {
continue
}
}
}
nodeHash := make([]byte, len(it.Hash().Bytes()))
copy(nodeHash, it.Hash().Bytes())
if err := output(types2.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, nodeHash).String(),
Content: nodeVal,
}); err != nil {
return nil, err
}
}
}
logger.Debug("statediff COUNTS createdAndUpdatedStateWithIntermediateNodes", "it", itCount, "diffAccountsAtB", len(diffAccountsAtB))
metrics2.IndexerMetrics.DifferenceIteratorCounter.Inc(int64(*itCount))
return diffAccountsAtB, it.Error()
}
// reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something that actually exists in an MMPT
func (sdb *StateDiffBuilder) processStateValueNode(it trie.NodeIterator, watchedAddressesLeafPaths [][]byte, prefixPath []byte) (*types2.AccountWrapper, error) {
// skip if it is not a watched address
// If we aren't watching any specific addresses, we are watching everything
if len(watchedAddressesLeafPaths) > 0 && !isWatchedAddress(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) {
return nil, nil
}
// since this is a "value node", we need to move up to the "parent" node which is the actual leaf node
// it should be in the fastcache since it necessarily was recently accessed to reach the current node
parentNodeRLP, err := sdb.StateCache.TrieDB().Node(it.Parent())
if err != nil {
return nil, err
}
var nodeElements []interface{}
if err = rlp.DecodeBytes(parentNodeRLP, &nodeElements); err != nil {
return nil, err
}
parentSubPath := make([]byte, len(it.ParentPath()))
copy(parentSubPath, it.ParentPath())
parentPath := append(prefixPath, parentSubPath...)
partialPath := trie.CompactToHex(nodeElements[0].([]byte))
valueNodePath := append(parentPath, partialPath...)
encodedPath := trie.HexToCompact(valueNodePath)
leafKey := encodedPath[1:]
var account types.StateAccount
accountRLP := make([]byte, len(it.LeafBlob()))
copy(accountRLP, it.LeafBlob())
if err := rlp.DecodeBytes(accountRLP, &account); err != nil {
return nil, fmt.Errorf("error decoding account for leaf value at leaf key %x\nerror: %v", leafKey, err)
}
return &types2.AccountWrapper{
LeafKey: leafKey,
Account: &account,
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(parentNodeRLP)).String(),
}, nil
}
// deletedOrUpdatedState returns a slice of all the pathes that are emptied at B
// and a mapping of their leafkeys to all the accounts that exist in a different state at A than B
func (sdb *StateDiffBuilder) deletedOrUpdatedState(a, b trie.NodeIterator, diffAccountsAtB types2.AccountMap,
watchedAddressesLeafPaths [][]byte, output types2.StateNodeSink, logger log.Logger, prefixPath []byte) (types2.AccountMap, error) {
logger.Debug("statediff BEGIN deletedOrUpdatedState")
defer metrics2.ReportAndUpdateDuration("statediff END deletedOrUpdatedState", time.Now(), logger, metrics2.IndexerMetrics.DeletedOrUpdatedStateTimer)
diffAccountAtA := make(types2.AccountMap)
watchingAddresses := len(watchedAddressesLeafPaths) > 0
it, _ := trie.NewDifferenceIterator(b, a)
for it.Next(true) {
// ignore node if it is not along paths of interest
if watchingAddresses && !isValidPrefixPath(watchedAddressesLeafPaths, append(prefixPath, it.Path()...)) {
continue
}
if it.Leaf() {
accountW, err := sdb.processStateValueNode(it, watchedAddressesLeafPaths, prefixPath)
if err != nil {
return nil, err
}
if accountW == nil {
continue
}
leafKey := common.Bytes2Hex(accountW.LeafKey)
diffAccountAtA[leafKey] = *accountW
// if this node's leaf key did not show up in diffAccountsAtB
// that means the account was deleted
// in that case, emit an empty "removed" diff state node
// include empty "removed" diff storage nodes for all the storage slots
if _, ok := diffAccountsAtB[leafKey]; !ok {
diff := types2.StateLeafNode{
AccountWrapper: types2.AccountWrapper{
Account: nil,
LeafKey: accountW.LeafKey,
CID: shared.RemovedNodeStateCID,
},
Removed: true,
}
storageDiff := make([]types2.StorageLeafNode, 0)
err := sdb.buildRemovedAccountStorageNodes(accountW.Account.Root, StorageNodeAppender(&storageDiff))
if err != nil {
return nil, fmt.Errorf("failed building storage diffs for removed state account with key %x\r\nerror: %v", leafKey, err)
}
diff.StorageDiff = storageDiff
if err := output(diff); err != nil {
return nil, err
}
}
}
}
return diffAccountAtA, it.Error()
}
// buildAccountUpdates uses the account diffs maps for A => B and B => A and the known intersection of their leafkeys
// to generate the statediff node objects for all of the accounts that existed at both A and B but in different states
// needs to be called before building account creations and deletions as this mutates
// those account maps to remove the accounts which were updated
func (sdb *StateDiffBuilder) buildAccountUpdates(creations, deletions types2.AccountMap, updatedKeys []string,
output types2.StateNodeSink, ipldOutput types2.IPLDSink, logger log.Logger) error {
logger.Debug("statediff BEGIN buildAccountUpdates", "creations", len(creations), "deletions", len(deletions), "updatedKeys", len(updatedKeys))
defer metrics2.ReportAndUpdateDuration("statediff END buildAccountUpdates ", time.Now(), logger, metrics2.IndexerMetrics.BuildAccountUpdatesTimer)
var err error
for _, key := range updatedKeys {
createdAcc := creations[key]
deletedAcc := deletions[key]
storageDiff := make([]types2.StorageLeafNode, 0)
if deletedAcc.Account != nil && createdAcc.Account != nil {
oldSR := deletedAcc.Account.Root
newSR := createdAcc.Account.Root
err = sdb.buildStorageNodesIncremental(
oldSR, newSR, StorageNodeAppender(&storageDiff), ipldOutput)
if err != nil {
return fmt.Errorf("failed building incremental storage diffs for account with leafkey %s\r\nerror: %v", key, err)
}
}
if err = output(types2.StateLeafNode{
AccountWrapper: createdAcc,
Removed: false,
StorageDiff: storageDiff,
}); err != nil {
return err
}
delete(creations, key)
delete(deletions, key)
}
return nil
}
// buildAccountCreations returns the statediff node objects for all the accounts that exist at B but not at A
// it also returns the code and codehash for created contract accounts
func (sdb *StateDiffBuilder) buildAccountCreations(accounts types2.AccountMap, output types2.StateNodeSink,
ipldOutput types2.IPLDSink, logger log.Logger) error {
logger.Debug("statediff BEGIN buildAccountCreations")
defer metrics2.ReportAndUpdateDuration("statediff END buildAccountCreations", time.Now(), logger, metrics2.IndexerMetrics.BuildAccountCreationsTimer)
for _, val := range accounts {
diff := types2.StateLeafNode{
AccountWrapper: val,
Removed: false,
}
if !bytes.Equal(val.Account.CodeHash, nullCodeHash) {
// For contract creations, any storage node contained is a diff
storageDiff := make([]types2.StorageLeafNode, 0)
err := sdb.buildStorageNodesEventual(val.Account.Root, StorageNodeAppender(&storageDiff), ipldOutput)
if err != nil {
return fmt.Errorf("failed building eventual storage diffs for node with leaf key %x\r\nerror: %v", val.LeafKey, err)
}
diff.StorageDiff = storageDiff
// emit codehash => code mappings for contract
codeHash := common.BytesToHash(val.Account.CodeHash)
code, err := sdb.StateCache.ContractCode(common.Hash{}, codeHash)
if err != nil {
return fmt.Errorf("failed to retrieve code for codehash %s\r\n error: %v", codeHash.String(), err)
}
if err := ipldOutput(types2.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.RawBinary, codeHash.Bytes()).String(),
Content: code,
}); err != nil {
return err
}
}
if err := output(diff); err != nil {
return err
}
}
return nil
}
// buildStorageNodesEventual builds the storage diff node objects for a created account
// i.e. it returns all the storage nodes at this state, since there is no previous state
func (sdb *StateDiffBuilder) buildStorageNodesEventual(sr common.Hash, output types2.StorageNodeSink,
ipldOutput types2.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesEventualTimer)
if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) {
return nil
}
log.Debug("Storage Root For Eventual Diff", "root", sr.Hex())
sTrie, err := sdb.StateCache.OpenTrie(sr)
if err != nil {
log.Info("error in build storage diff eventual", "error", err)
return err
}
it := sTrie.NodeIterator(make([]byte, 0))
err = sdb.buildStorageNodesFromTrie(it, output, ipldOutput)
if err != nil {
return err
}
return nil
}
// buildStorageNodesFromTrie returns all the storage diff node objects in the provided node interator
// including intermediate nodes can be turned on or off
func (sdb *StateDiffBuilder) buildStorageNodesFromTrie(it trie.NodeIterator, output types2.StorageNodeSink,
ipldOutput types2.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesFromTrieTimer)
for it.Next(true) {
if it.Leaf() {
storageLeafNode, err := sdb.processStorageValueNode(it)
if err != nil {
return err
}
if err := output(storageLeafNode); err != nil {
return err
}
} else {
nodeVal := make([]byte, len(it.NodeBlob()))
copy(nodeVal, it.NodeBlob())
nodeHash := make([]byte, len(it.Hash().Bytes()))
copy(nodeHash, it.Hash().Bytes())
if err := ipldOutput(types2.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, nodeHash).String(),
Content: nodeVal,
}); err != nil {
return err
}
}
}
return it.Error()
}
// reminder: it.Leaf() == true when the iterator is positioned at a "value node" which is not something that actually exists in an MMPT
func (sdb *StateDiffBuilder) processStorageValueNode(it trie.NodeIterator) (types2.StorageLeafNode, error) {
// skip if it is not a watched address
leafKey := make([]byte, len(it.LeafKey()))
copy(leafKey, it.LeafKey())
value := make([]byte, len(it.LeafBlob()))
copy(value, it.LeafBlob())
// since this is a "value node", we need to move up to the "parent" node which is the actual leaf node
// it should be in the fastcache since it necessarily was recently accessed to reach the current node
parentNodeRLP, err := sdb.StateCache.TrieDB().Node(it.Parent())
if err != nil {
return types2.StorageLeafNode{}, err
}
return types2.StorageLeafNode{
LeafKey: leafKey,
Value: value,
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, crypto.Keccak256(parentNodeRLP)).String(),
}, nil
}
// buildRemovedAccountStorageNodes builds the "removed" diffs for all the storage nodes for a destroyed account
func (sdb *StateDiffBuilder) buildRemovedAccountStorageNodes(sr common.Hash, output types2.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildRemovedAccountStorageNodesTimer)
if bytes.Equal(sr.Bytes(), emptyContractRoot.Bytes()) {
return nil
}
log.Debug("Storage Root For Removed Diffs", "root", sr.Hex())
sTrie, err := sdb.StateCache.OpenTrie(sr)
if err != nil {
log.Info("error in build removed account storage diffs", "error", err)
return err
}
it := sTrie.NodeIterator(make([]byte, 0))
err = sdb.buildRemovedStorageNodesFromTrie(it, output)
if err != nil {
return err
}
return nil
}
// buildRemovedStorageNodesFromTrie returns diffs for all the storage nodes in the provided node interator
func (sdb *StateDiffBuilder) buildRemovedStorageNodesFromTrie(it trie.NodeIterator, output types2.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildRemovedStorageNodesFromTrieTimer)
for it.Next(true) {
if it.Leaf() { // only leaf values are indexed, don't need to demarcate removed intermediate nodes
leafKey := make([]byte, len(it.LeafKey()))
copy(leafKey, it.LeafKey())
if err := output(types2.StorageLeafNode{
CID: shared.RemovedNodeStorageCID,
Removed: true,
LeafKey: leafKey,
Value: []byte{},
}); err != nil {
return err
}
}
}
return it.Error()
}
// buildStorageNodesIncremental builds the storage diff node objects for all nodes that exist in a different state at B than A
func (sdb *StateDiffBuilder) buildStorageNodesIncremental(oldSR common.Hash, newSR common.Hash, output types2.StorageNodeSink,
ipldOutput types2.IPLDSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.BuildStorageNodesIncrementalTimer)
if bytes.Equal(newSR.Bytes(), oldSR.Bytes()) {
return nil
}
log.Trace("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex())
oldTrie, err := sdb.StateCache.OpenTrie(oldSR)
if err != nil {
return err
}
newTrie, err := sdb.StateCache.OpenTrie(newSR)
if err != nil {
return err
}
diffSlotsAtB, err := sdb.createdAndUpdatedStorage(
oldTrie.NodeIterator([]byte{}), newTrie.NodeIterator([]byte{}), output, ipldOutput)
if err != nil {
return err
}
err = sdb.deletedOrUpdatedStorage(oldTrie.NodeIterator([]byte{}), newTrie.NodeIterator([]byte{}),
diffSlotsAtB, output)
if err != nil {
return err
}
return nil
}
func (sdb *StateDiffBuilder) createdAndUpdatedStorage(a, b trie.NodeIterator, output types2.StorageNodeSink,
ipldOutput types2.IPLDSink) (map[string]bool, error) {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.CreatedAndUpdatedStorageTimer)
diffSlotsAtB := make(map[string]bool)
it, _ := trie.NewDifferenceIterator(a, b)
for it.Next(true) {
if it.Leaf() {
storageLeafNode, err := sdb.processStorageValueNode(it)
if err != nil {
return nil, err
}
if err := output(storageLeafNode); err != nil {
return nil, err
}
diffSlotsAtB[common.Bytes2Hex(storageLeafNode.LeafKey)] = true
} else {
if bytes.Equal(it.Hash().Bytes(), nullNodeHash) {
continue
}
nodeVal := make([]byte, len(it.NodeBlob()))
copy(nodeVal, it.NodeBlob())
nodeHash := make([]byte, len(it.Hash().Bytes()))
copy(nodeHash, it.Hash().Bytes())
if err := ipldOutput(types2.IPLD{
CID: ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, nodeHash).String(),
Content: nodeVal,
}); err != nil {
return nil, err
}
}
}
return diffSlotsAtB, it.Error()
}
func (sdb *StateDiffBuilder) deletedOrUpdatedStorage(a, b trie.NodeIterator, diffSlotsAtB map[string]bool, output types2.StorageNodeSink) error {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.DeletedOrUpdatedStorageTimer)
it, _ := trie.NewDifferenceIterator(b, a)
for it.Next(true) {
if it.Leaf() {
leafKey := make([]byte, len(it.LeafKey()))
copy(leafKey, it.LeafKey())
// if this node's leaf key did not show up in diffSlotsAtB
// that means the storage slot was vacated
// in that case, emit an empty "removed" diff storage node
if _, ok := diffSlotsAtB[common.Bytes2Hex(leafKey)]; !ok {
if err := output(types2.StorageLeafNode{
CID: shared.RemovedNodeStorageCID,
Removed: true,
LeafKey: leafKey,
Value: []byte{},
}); err != nil {
return err
}
}
}
}
return it.Error()
}
// isValidPrefixPath is used to check if a node at currentPath is a parent | ancestor to one of the addresses the builder is configured to watch
func isValidPrefixPath(watchedAddressesLeafPaths [][]byte, currentPath []byte) bool {
for _, watchedAddressPath := range watchedAddressesLeafPaths {
if bytes.HasPrefix(watchedAddressPath, currentPath) {
return true
}
}
return false
}
// isWatchedAddress is used to check if a state account corresponds to one of the addresses the builder is configured to watch
func isWatchedAddress(watchedAddressesLeafPaths [][]byte, valueNodePath []byte) bool {
defer metrics2.UpdateDuration(time.Now(), metrics2.IndexerMetrics.IsWatchedAddressTimer)
for _, watchedAddressPath := range watchedAddressesLeafPaths {
if bytes.Equal(watchedAddressPath, valueNodePath) {
return true
}
}
return false
}
// isLeaf checks if the node we are at is a leaf
func isLeaf(elements []interface{}) (bool, error) {
if len(elements) > 2 {
return false, nil
}
if len(elements) < 2 {
return false, fmt.Errorf("node cannot be less than two elements in length")
}
switch elements[0].([]byte)[0] / 16 {
case '\x00':
return false, nil
case '\x01':
return false, nil
case '\x02':
return true, nil
case '\x03':
return true, nil
default:
return false, fmt.Errorf("unknown hex prefix")
}
}

3108
builder_test.go Normal file

File diff suppressed because it is too large Load Diff

87
config.go Normal file
View File

@ -0,0 +1,87 @@
// VulcanizeDB
// Copyright © 2021 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 statediff
import (
"context"
"math/big"
"sync"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
)
// Config contains instantiation parameters for the state diffing service
type Config struct {
// The configuration used for the stateDiff Indexer
IndexerConfig interfaces.Config
// A unique ID used for this service
ID string
// Name for the client this service is running
ClientName string
// Whether to enable writing state diffs directly to track blockchain head
EnableWriteLoop bool
// Size of the worker pool
NumWorkers uint
// Should the statediff service wait until geth has synced to the head of the blockchain?
WaitForSync bool
// Context
Context context.Context
}
// Params contains config parameters for the state diff builder
type Params struct {
IncludeBlock bool
IncludeReceipts bool
IncludeTD bool
IncludeCode bool
WatchedAddresses []common.Address
watchedAddressesLeafPaths [][]byte
}
// ComputeWatchedAddressesLeafPaths populates a slice with paths (hex_encoding(Keccak256)) of each of the WatchedAddresses
func (p *Params) ComputeWatchedAddressesLeafPaths() {
p.watchedAddressesLeafPaths = make([][]byte, len(p.WatchedAddresses))
for i, address := range p.WatchedAddresses {
p.watchedAddressesLeafPaths[i] = keybytesToHex(crypto.Keccak256(address.Bytes()))
}
}
// ParamsWithMutex allows to lock the parameters while they are being updated | read from
type ParamsWithMutex struct {
Params
sync.RWMutex
}
// Args bundles the arguments for the state diff builder
type Args struct {
OldStateRoot, NewStateRoot, BlockHash common.Hash
BlockNumber *big.Int
}
// https://github.com/ethereum/go-ethereum/blob/master/trie/encoding.go#L97
func keybytesToHex(str []byte) []byte {
l := len(str)*2 + 1
var nibbles = make([]byte, l)
for i, b := range str {
nibbles[i*2] = b / 16
nibbles[i*2+1] = b % 16
}
nibbles[l-1] = 16
return nibbles
}

17
docs/KnownGaps.md Normal file
View File

@ -0,0 +1,17 @@
# Overview
This document will provide some insight into the `known_gaps` table, their use cases, and implementation. Please refer to the [following PR](https://github.com/vulcanize/go-ethereum/pull/217) and the [following epic](https://github.com/vulcanize/ops/issues/143) to grasp their inception.
![known gaps](diagrams/KnownGapsProcess.png)
# Use Cases
The known gaps table is updated when the following events occur:
1. At start up we check the latest block from the `eth.headers_cid` table. We compare the first block that we are processing with the latest block from the DB. If they are not one unit of expectedDifference away from each other, add the gap between the two blocks.
2. If there is any error in processing a block (db connection, deadlock, etc), add that block to the knownErrorBlocks slice, when the next block is successfully written, write this slice into the DB.
# Glossary
1. `expectedDifference (number)` - This number indicates what the difference between two blocks should be. If we are capturing all events on a geth node then this number would be `1`. But once we scale nodes, the `expectedDifference` might be `2` or greater.
2. `processingKey (number)` - This number can be used to keep track of different geth nodes and their specific `expectedDifference`.

3
docs/README.md Normal file
View File

@ -0,0 +1,3 @@
# Overview
This folder keeps tracks of random documents as they relate to the `statediff` service.

21
docs/database.md Normal file
View File

@ -0,0 +1,21 @@
# Overview
This document will go through some notes on the database component of the statediff service.
# Components
- Indexer: The indexer creates IPLD and DB models to insert to the Postgres DB. It performs the insert utilizing and atomic function.
- Builder: The builder constructs the statediff object that needs to be inserted.
- Known Gaps: Captures any gaps that might have occured and either writes them to the DB, local sql file, to prometeus, or a local error.
# Making Code Changes
## Adding a New Function to the Indexer
If you want to implement a new feature for adding data to the database. Keep the following in mind:
1. You need to handle `sql`, `file`, and `dump`.
1. `sql` - Contains the code needed to write directly to the `sql` db.
2. `file` - Contains all the code required to write the SQL statements to a file.
3. `dump` - Contains all the code for outputting events to the console.
2. You will have to add it to the `interfaces.StateDiffIndexer` interface.

Binary file not shown.

After

Width:  |  Height:  |  Size: 33 KiB

81
indexer/constructor.go Normal file
View File

@ -0,0 +1,81 @@
// VulcanizeDB
// Copyright © 2021 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 indexer
import (
"context"
"fmt"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/dump"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
)
// NewStateDiffIndexer creates and returns an implementation of the StateDiffIndexer interface.
func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, nodeInfo node.Info, config interfaces.Config) (sql.Database, interfaces.StateDiffIndexer, error) {
switch config.Type() {
case shared.FILE:
log.Info("Starting statediff service in SQL file writing mode")
fc, ok := config.(file.Config)
if !ok {
return nil, nil, fmt.Errorf("file config is not the correct type: got %T, expected %T", config, file.Config{})
}
fc.NodeInfo = nodeInfo
ind, err := file.NewStateDiffIndexer(ctx, chainConfig, fc)
return nil, ind, err
case shared.POSTGRES:
log.Info("Starting statediff service in Postgres writing mode")
pgc, ok := config.(postgres.Config)
if !ok {
return nil, nil, fmt.Errorf("postgres config is not the correct type: got %T, expected %T", config, postgres.Config{})
}
var err error
var driver sql.Driver
switch pgc.Driver {
case postgres.PGX:
driver, err = postgres.NewPGXDriver(ctx, pgc, nodeInfo)
if err != nil {
return nil, nil, err
}
case postgres.SQLX:
driver, err = postgres.NewSQLXDriver(ctx, pgc, nodeInfo)
if err != nil {
return nil, nil, err
}
default:
return nil, nil, fmt.Errorf("unrecognized Postgres driver type: %s", pgc.Driver)
}
db := postgres.NewPostgresDB(driver, pgc.Upsert)
ind, err := sql.NewStateDiffIndexer(ctx, chainConfig, db)
return db, ind, err
case shared.DUMP:
log.Info("Starting statediff service in data dump mode")
dumpc, ok := config.(dump.Config)
if !ok {
return nil, nil, fmt.Errorf("dump config is not the correct type: got %T, expected %T", config, dump.Config{})
}
return nil, dump.NewStateDiffIndexer(chainConfig, dumpc), nil
default:
return nil, nil, fmt.Errorf("unrecognized database type: %s", config.Type())
}
}

View File

@ -0,0 +1,80 @@
// VulcanizeDB
// Copyright © 2021 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 dump
import (
"fmt"
"io"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
)
// BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct {
BlockNumber string
dump io.Writer
quit chan struct{}
iplds chan models.IPLDModel
ipldCache models.IPLDBatch
submit func(blockTx *BatchTx, err error) error
}
// Submit satisfies indexer.AtomicTx
func (tx *BatchTx) Submit(err error) error {
return tx.submit(tx, err)
}
func (tx *BatchTx) flush() error {
if _, err := fmt.Fprintf(tx.dump, "%+v\r\n", tx.ipldCache); err != nil {
return err
}
tx.ipldCache = models.IPLDBatch{}
return nil
}
// run in background goroutine to synchronize concurrent appends to the ipldCache
func (tx *BatchTx) cache() {
for {
select {
case i := <-tx.iplds:
tx.ipldCache.Keys = append(tx.ipldCache.Keys, i.Key)
tx.ipldCache.Values = append(tx.ipldCache.Values, i.Data)
case <-tx.quit:
tx.ipldCache = models.IPLDBatch{}
return
}
}
}
func (tx *BatchTx) cacheDirect(key string, value []byte) {
tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber,
Key: key,
Data: value,
}
}
func (tx *BatchTx) cacheIPLD(i ipld.IPLD) {
tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber,
Key: i.Cid().String(),
Data: i.RawData(),
}
}

View File

@ -0,0 +1,79 @@
// VulcanizeDB
// Copyright © 2021 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 dump
import (
"fmt"
"io"
"strings"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
)
// DumpType to explicitly type the dump destination
type DumpType string
const (
STDOUT = "Stdout"
STDERR = "Stderr"
DISCARD = "Discard"
UNKNOWN = "Unknown"
)
// ResolveDumpType resolves the dump type for the provided string
func ResolveDumpType(str string) (DumpType, error) {
switch strings.ToLower(str) {
case "stdout", "out", "std out":
return STDOUT, nil
case "stderr", "err", "std err":
return STDERR, nil
case "discard", "void", "devnull", "dev null":
return DISCARD, nil
default:
return UNKNOWN, fmt.Errorf("unrecognized dump type: %s", str)
}
}
// Config for data dump
type Config struct {
Dump io.WriteCloser
}
// Type satisfies interfaces.Config
func (c Config) Type() shared.DBType {
return shared.DUMP
}
// NewDiscardWriterCloser returns a discardWrapper wrapping io.Discard
func NewDiscardWriterCloser() io.WriteCloser {
return discardWrapper{blackhole: io.Discard}
}
// discardWrapper wraps io.Discard with io.Closer
type discardWrapper struct {
blackhole io.Writer
}
// Write satisfies io.Writer
func (dw discardWrapper) Write(b []byte) (int, error) {
return dw.blackhole.Write(b)
}
// Close satisfies io.Closer
func (dw discardWrapper) Close() error {
return nil
}

View File

@ -0,0 +1,443 @@
// VulcanizeDB
// Copyright © 2021 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 dump
import (
"bytes"
"fmt"
"io"
"math/big"
"time"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of a void
type StateDiffIndexer struct {
dump io.WriteCloser
chainConfig *params.ChainConfig
}
// NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer
func NewStateDiffIndexer(chainConfig *params.ChainConfig, config Config) *StateDiffIndexer {
return &StateDiffIndexer{
dump: config.Dump,
chainConfig: chainConfig,
}
}
// ReportDBMetrics has nothing to report for dump
func (sdi *StateDiffIndexer) ReportDBMetrics(time.Duration, <-chan bool) {}
// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts)
// Returns an initiated DB transaction which must be Closed via defer to commit or rollback
func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) {
start, t := time.Now(), time.Now()
blockHash := block.Hash()
blockHashStr := blockHash.String()
height := block.NumberU64()
traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHashStr)
transactions := block.Transactions()
// Derive any missing fields
if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil {
return nil, err
}
// Generate the block iplds
headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts)
if err != nil {
return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err)
}
if len(txNodes) != len(rctNodes) {
return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d)", len(txNodes), len(rctNodes))
}
// Calculate reward
var reward *big.Int
// in PoA networks block reward is 0
if sdi.chainConfig.Clique != nil {
reward = big.NewInt(0)
} else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts)
}
t = time.Now()
blockTx := &BatchTx{
BlockNumber: block.Number().String(),
dump: sdi.dump,
iplds: make(chan models.IPLDModel),
quit: make(chan struct{}),
ipldCache: models.IPLDBatch{},
submit: func(self *BatchTx, err error) error {
close(self.quit)
close(self.iplds)
tDiff := time.Since(t)
metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String())
t = time.Now()
if err := self.flush(); err != nil {
traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String())
log.Debug(traceMsg)
return err
}
tDiff = time.Since(t)
metrics.IndexerMetrics.PostgresCommitTimer.Update(tDiff)
traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String())
traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String())
log.Debug(traceMsg)
return err
},
}
go blockTx.cache()
tDiff := time.Since(t)
metrics.IndexerMetrics.FreePostgresTimer.Update(tDiff)
traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String())
t = time.Now()
// Publish and index header, collect headerID
var headerID string
headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty)
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String())
t = time.Now()
// Publish and index uncles
err = sdi.processUncles(blockTx, headerID, block.Number(), block.UncleHash(), block.Uncles())
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics.IndexerMetrics.UncleProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String())
t = time.Now()
// Publish and index receipts and txs
err = sdi.processReceiptsAndTxs(blockTx, processArgs{
headerID: headerID,
blockNumber: block.Number(),
receipts: receipts,
txs: transactions,
rctNodes: rctNodes,
txNodes: txNodes,
logNodes: logNodes,
})
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics.IndexerMetrics.TxAndRecProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String())
t = time.Now()
return blockTx, err
}
// processHeader publishes and indexes a header IPLD in Postgres
// it returns the headerID
func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode ipld.IPLD, reward, td *big.Int) (string, error) {
tx.cacheIPLD(headerNode)
headerID := header.Hash().String()
mod := models.HeaderModel{
CID: headerNode.Cid().String(),
ParentHash: header.ParentHash.String(),
BlockNumber: header.Number.String(),
BlockHash: headerID,
TotalDifficulty: td.String(),
Reward: reward.String(),
Bloom: header.Bloom.Bytes(),
StateRoot: header.Root.String(),
RctRoot: header.ReceiptHash.String(),
TxRoot: header.TxHash.String(),
UnclesHash: header.UncleHash.String(),
Timestamp: header.Time,
Coinbase: header.Coinbase.String(),
}
_, err := fmt.Fprintf(sdi.dump, "%+v\r\n", mod)
return headerID, err
}
// processUncles publishes and indexes uncle IPLDs in Postgres
func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber *big.Int, unclesHash common.Hash, uncles []*types.Header) error {
// publish and index uncles
uncleEncoding, err := rlp.EncodeToBytes(uncles)
if err != nil {
return err
}
preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex())
}
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil {
return err
}
tx.cacheDirect(unclesCID.String(), uncleEncoding)
for i, uncle := range uncles {
var uncleReward *big.Int
// in PoA networks uncle reward is 0
if sdi.chainConfig.Clique != nil {
uncleReward = big.NewInt(0)
} else {
uncleReward = shared.CalcUncleMinerReward(blockNumber.Uint64(), uncle.Number.Uint64())
}
uncle := models.UncleModel{
BlockNumber: blockNumber.String(),
HeaderID: headerID,
CID: unclesCID.String(),
ParentHash: uncle.ParentHash.String(),
BlockHash: uncle.Hash().String(),
Reward: uncleReward.String(),
Index: int64(i),
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", uncle); err != nil {
return err
}
}
return nil
}
// processArgs bundles arguments to processReceiptsAndTxs
type processArgs struct {
headerID string
blockNumber *big.Int
receipts types.Receipts
txs types.Transactions
rctNodes []*ipld.EthReceipt
txNodes []*ipld.EthTx
logNodes [][]*ipld.EthLog
}
// processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres
func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs) error {
// Process receipts and txs
signer := types.MakeSigner(sdi.chainConfig, args.blockNumber)
for i, receipt := range args.receipts {
txNode := args.txNodes[i]
tx.cacheIPLD(txNode)
// Indexing
// index tx
trx := args.txs[i]
trxID := trx.Hash().String()
var val string
if trx.Value() != nil {
val = trx.Value().String()
}
// derive sender for the tx that corresponds with this receipt
from, err := types.Sender(signer, trx)
if err != nil {
return fmt.Errorf("error deriving tx sender: %v", err)
}
txModel := models.TxModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
Dst: shared.HandleZeroAddrPointer(trx.To()),
Src: shared.HandleZeroAddr(from),
TxHash: trxID,
Index: int64(i),
CID: txNode.Cid().String(),
Type: trx.Type(),
Value: val,
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", txModel); err != nil {
return err
}
// this is the contract address if this receipt is for a contract creation tx
contract := shared.HandleZeroAddr(receipt.ContractAddress)
// index the receipt
rctModel := &models.ReceiptModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
TxID: trxID,
Contract: contract,
CID: args.rctNodes[i].Cid().String(),
}
if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status
} else {
rctModel.PostState = common.Bytes2Hex(receipt.PostState)
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", rctModel); err != nil {
return err
}
logDataSet := make([]*models.LogsModel, len(receipt.Logs))
for idx, l := range receipt.Logs {
topicSet := make([]string, 4)
for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex()
}
logDataSet[idx] = &models.LogsModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
ReceiptID: trxID,
Address: l.Address.String(),
Index: int64(l.Index),
CID: args.logNodes[i][idx].Cid().String(),
Topic0: topicSet[0],
Topic1: topicSet[1],
Topic2: topicSet[2],
Topic3: topicSet[3],
}
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", logDataSet); err != nil {
return err
}
}
return nil
}
// PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql
func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateLeafNode, headerID string) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("dump: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
// publish the state node
var stateModel models.StateNodeModel
if stateNode.Removed {
// short circuit if it is a Removed node
// this assumes the db has been initialized and a ipld.blocks entry for the Removed node is present
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID,
Removed: true,
}
} else {
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID,
Removed: false,
Balance: stateNode.AccountWrapper.Account.Balance.String(),
Nonce: stateNode.AccountWrapper.Account.Nonce,
CodeHash: common.BytesToHash(stateNode.AccountWrapper.Account.CodeHash).String(),
StorageRoot: stateNode.AccountWrapper.Account.Root.String(),
}
}
// index the state node, collect the stateID to reference by FK
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", stateModel); err != nil {
return err
}
// if there are any storage nodes associated with this node, publish and index them
for _, storageNode := range stateNode.StorageDiff {
if storageNode.Removed {
// short circuit if it is a Removed node
// this assumes the db has been initialized and a ipld.blocks entry for the Removed node is present
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: shared.RemovedNodeStorageCID,
Removed: true,
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", storageModel); err != nil {
return err
}
continue
}
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: storageNode.CID,
Removed: false,
Value: storageNode.Value,
}
if _, err := fmt.Fprintf(sdi.dump, "%+v\r\n", storageModel); err != nil {
return err
}
}
return nil
}
// PushIPLD publishes iplds to ipld.blocks
func (sdi *StateDiffIndexer) PushIPLD(batch interfaces.Batch, ipld sdtypes.IPLD) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("dump: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
tx.cacheDirect(ipld.CID, ipld.Content)
return nil
}
// Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error {
return sdi.dump.Close()
}
// LoadWatchedAddresses satisfies the interfaces.StateDiffIndexer interface
func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) {
return nil, nil
}
// InsertWatchedAddresses satisfies the interfaces.StateDiffIndexer interface
func (sdi *StateDiffIndexer) InsertWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
return nil
}
// RemoveWatchedAddresses satisfies the interfaces.StateDiffIndexer interface
func (sdi *StateDiffIndexer) RemoveWatchedAddresses(args []sdtypes.WatchAddressArg) error {
return nil
}
// SetWatchedAddresses satisfies the interfaces.StateDiffIndexer interface
func (sdi *StateDiffIndexer) SetWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
return nil
}
// ClearWatchedAddresses satisfies the interfaces.StateDiffIndexer interface
func (sdi *StateDiffIndexer) ClearWatchedAddresses() error {
return nil
}

View File

@ -0,0 +1,29 @@
// VulcanizeDB
// Copyright © 2021 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 file
// BatchTx wraps a void with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct {
BlockNumber string
submit func(blockTx *BatchTx, err error) error
}
// Submit satisfies indexer.AtomicTx
func (tx *BatchTx) Submit(err error) error {
return tx.submit(tx, err)
}

View File

@ -0,0 +1,84 @@
// VulcanizeDB
// Copyright © 2021 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 file
import (
"fmt"
"strings"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
)
// FileMode to explicitly type the mode of file writer we are using
type FileMode string
const (
CSV FileMode = "CSV"
SQL FileMode = "SQL"
Unknown FileMode = "Unknown"
)
// ResolveFileMode resolves a FileMode from a provided string
func ResolveFileMode(str string) (FileMode, error) {
switch strings.ToLower(str) {
case "csv":
return CSV, nil
case "sql":
return SQL, nil
default:
return Unknown, fmt.Errorf("unrecognized file type string: %s", str)
}
}
// Config holds params for writing out CSV or SQL files
type Config struct {
Mode FileMode
OutputDir string
FilePath string
WatchedAddressesFilePath string
NodeInfo node.Info
}
// Type satisfies interfaces.Config
func (c Config) Type() shared.DBType {
return shared.FILE
}
var nodeInfo = node.Info{
GenesisBlock: "0xd4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3",
NetworkID: "1",
ChainID: 1,
ID: "mockNodeID",
ClientName: "go-ethereum",
}
// CSVTestConfig config for unit tests
var CSVTestConfig = Config{
Mode: CSV,
OutputDir: "./statediffing_test",
WatchedAddressesFilePath: "./statediffing_watched_addresses_test_file.csv",
NodeInfo: nodeInfo,
}
// SQLTestConfig config for unit tests
var SQLTestConfig = Config{
Mode: SQL,
FilePath: "./statediffing_test_file.sql",
WatchedAddressesFilePath: "./statediffing_watched_addresses_test_file.sql",
NodeInfo: nodeInfo,
}

View File

@ -0,0 +1,118 @@
// VulcanizeDB
// Copyright © 2022 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 file_test
import (
"context"
"errors"
"fmt"
"os"
"path/filepath"
"strings"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
)
const dbDirectory = "/file_indexer"
const pgCopyStatement = `COPY %s FROM '%s' CSV`
func setupLegacyCSVIndexer(t *testing.T) {
if _, err := os.Stat(file.CSVTestConfig.OutputDir); !errors.Is(err, os.ErrNotExist) {
err := os.RemoveAll(file.CSVTestConfig.OutputDir)
require.NoError(t, err)
}
ind, err = file.NewStateDiffIndexer(context.Background(), test.LegacyConfig, file.CSVTestConfig)
require.NoError(t, err)
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
}
func setupLegacyCSV(t *testing.T) {
setupLegacyCSVIndexer(t)
test.SetupLegacyTestData(t, ind)
}
func dumpCSVFileData(t *testing.T) {
outputDir := filepath.Join(dbDirectory, file.CSVTestConfig.OutputDir)
workingDir, err := os.Getwd()
require.NoError(t, err)
localOutputDir := filepath.Join(workingDir, file.CSVTestConfig.OutputDir)
for _, tbl := range file.Tables {
err := test_helpers.DedupFile(file.TableFilePath(localOutputDir, tbl.Name))
require.NoError(t, err)
var stmt string
varcharColumns := tbl.VarcharColumns()
if len(varcharColumns) > 0 {
stmt = fmt.Sprintf(
pgCopyStatement+" FORCE NOT NULL %s",
tbl.Name,
file.TableFilePath(outputDir, tbl.Name),
strings.Join(varcharColumns, ", "),
)
} else {
stmt = fmt.Sprintf(pgCopyStatement, tbl.Name, file.TableFilePath(outputDir, tbl.Name))
}
_, err = db.Exec(context.Background(), stmt)
require.NoError(t, err)
}
}
func resetAndDumpWatchedAddressesCSVFileData(t *testing.T) {
test_helpers.TearDownDB(t, db)
outputFilePath := filepath.Join(dbDirectory, file.CSVTestConfig.WatchedAddressesFilePath)
stmt := fmt.Sprintf(pgCopyStatement, schema.TableWatchedAddresses.Name, outputFilePath)
_, err = db.Exec(context.Background(), stmt)
require.NoError(t, err)
}
func tearDownCSV(t *testing.T) {
test_helpers.TearDownDB(t, db)
require.NoError(t, db.Close())
require.NoError(t, os.RemoveAll(file.CSVTestConfig.OutputDir))
if err := os.Remove(file.CSVTestConfig.WatchedAddressesFilePath); !errors.Is(err, os.ErrNotExist) {
require.NoError(t, err)
}
}
func TestLegacyCSVFileIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs", func(t *testing.T) {
setupLegacyCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestLegacyIndexer(t, db)
})
}

View File

@ -0,0 +1,255 @@
// VulcanizeDB
// Copyright © 2022 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 file_test
import (
"context"
"errors"
"math/big"
"os"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupCSVIndexer(t *testing.T) {
file.CSVTestConfig.OutputDir = "./statediffing_test"
if _, err := os.Stat(file.CSVTestConfig.OutputDir); !errors.Is(err, os.ErrNotExist) {
err := os.RemoveAll(file.CSVTestConfig.OutputDir)
require.NoError(t, err)
}
if _, err := os.Stat(file.CSVTestConfig.WatchedAddressesFilePath); !errors.Is(err, os.ErrNotExist) {
err := os.Remove(file.CSVTestConfig.WatchedAddressesFilePath)
require.NoError(t, err)
}
ind, err = file.NewStateDiffIndexer(context.Background(), mocks.TestConfig, file.CSVTestConfig)
require.NoError(t, err)
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
}
func setupCSV(t *testing.T) {
setupCSVIndexer(t)
test.SetupTestData(t, ind)
}
func setupCSVNonCanonical(t *testing.T) {
setupCSVIndexer(t)
test.SetupTestDataNonCanonical(t, ind)
}
func TestCSVFileIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexHeaderIPLDs(t, db)
})
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexTransactionIPLDs(t, db)
})
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexLogIPLDs(t, db)
})
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexReceiptIPLDs(t, db)
})
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexStateIPLDs(t, db)
})
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
setupCSV(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexStorageIPLDs(t, db)
})
}
func TestCSVFileIndexerNonCanonical(t *testing.T) {
t.Run("Publish and index header", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexHeaderNonCanonical(t, db)
})
t.Run("Publish and index transactions", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexTransactionsNonCanonical(t, db)
})
t.Run("Publish and index receipts", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexReceiptsNonCanonical(t, db)
})
t.Run("Publish and index logs", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexLogsNonCanonical(t, db)
})
t.Run("Publish and index state nodes", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexStateNonCanonical(t, db)
})
t.Run("Publish and index storage nodes", func(t *testing.T) {
setupCSVNonCanonical(t)
dumpCSVFileData(t)
defer tearDownCSV(t)
test.TestPublishAndIndexStorageNonCanonical(t, db)
})
}
func TestCSVFileWatchAddressMethods(t *testing.T) {
setupCSVIndexer(t)
defer tearDownCSV(t)
t.Run("Load watched addresses (empty table)", func(t *testing.T) {
test.TestLoadEmptyWatchedAddresses(t, ind)
})
t.Run("Insert watched addresses", func(t *testing.T) {
args := mocks.GetInsertWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt1)))
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestInsertWatchedAddresses(t, db)
})
t.Run("Insert watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetInsertAlreadyWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestInsertAlreadyWatchedAddresses(t, db)
})
t.Run("Remove watched addresses", func(t *testing.T) {
args := mocks.GetRemoveWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestRemoveWatchedAddresses(t, db)
})
t.Run("Remove watched addresses (some non-watched)", func(t *testing.T) {
args := mocks.GetRemoveNonWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestRemoveNonWatchedAddresses(t, db)
})
t.Run("Set watched addresses", func(t *testing.T) {
args := mocks.GetSetWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestSetWatchedAddresses(t, db)
})
t.Run("Set watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetSetAlreadyWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt3)))
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestSetAlreadyWatchedAddresses(t, db)
})
t.Run("Load watched addresses", func(t *testing.T) {
test.TestLoadWatchedAddresses(t, ind)
})
t.Run("Clear watched addresses", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestClearWatchedAddresses(t, db)
})
t.Run("Clear watched addresses (empty table)", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
resetAndDumpWatchedAddressesCSVFileData(t)
test.TestClearEmptyWatchedAddresses(t, db)
})
}

View File

@ -0,0 +1,418 @@
// VulcanizeDB
// Copyright © 2022 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 file
import (
"encoding/csv"
"errors"
"fmt"
"math/big"
"os"
"path/filepath"
"strconv"
"github.com/thoas/go-funk"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
var (
Tables = []*schema.Table{
&schema.TableIPLDBlock,
&schema.TableNodeInfo,
&schema.TableHeader,
&schema.TableStateNode,
&schema.TableStorageNode,
&schema.TableUncle,
&schema.TableTransaction,
&schema.TableReceipt,
&schema.TableLog,
}
)
type tableRow struct {
table schema.Table
values []interface{}
}
type CSVWriter struct {
// dir containing output files
dir string
writers fileWriters
watchedAddressesWriter fileWriter
rows chan tableRow
flushChan chan struct{}
flushFinished chan struct{}
quitChan chan struct{}
doneChan chan struct{}
}
type fileWriter struct {
*csv.Writer
file *os.File
}
// fileWriters wraps the file writers for each output table
type fileWriters map[string]fileWriter
func newFileWriter(path string) (ret fileWriter, err error) {
file, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644)
if err != nil {
return
}
ret = fileWriter{
Writer: csv.NewWriter(file),
file: file,
}
return
}
func makeFileWriters(dir string, tables []*schema.Table) (fileWriters, error) {
if err := os.MkdirAll(dir, 0755); err != nil {
return nil, err
}
writers := fileWriters{}
for _, tbl := range tables {
w, err := newFileWriter(TableFilePath(dir, tbl.Name))
if err != nil {
return nil, err
}
writers[tbl.Name] = w
}
return writers, nil
}
func (tx fileWriters) write(tbl *schema.Table, args ...interface{}) error {
row := tbl.ToCsvRow(args...)
return tx[tbl.Name].Write(row)
}
func (tx fileWriters) close() error {
for _, w := range tx {
err := w.file.Close()
if err != nil {
return err
}
}
return nil
}
func (tx fileWriters) flush() error {
for _, w := range tx {
w.Flush()
if err := w.Error(); err != nil {
return err
}
}
return nil
}
func NewCSVWriter(path string, watchedAddressesFilePath string) (*CSVWriter, error) {
if err := os.MkdirAll(path, 0777); err != nil {
return nil, fmt.Errorf("unable to make MkdirAll for path: %s err: %s", path, err)
}
writers, err := makeFileWriters(path, Tables)
if err != nil {
return nil, err
}
watchedAddressesWriter, err := newFileWriter(watchedAddressesFilePath)
if err != nil {
return nil, err
}
csvWriter := &CSVWriter{
writers: writers,
watchedAddressesWriter: watchedAddressesWriter,
dir: path,
rows: make(chan tableRow),
flushChan: make(chan struct{}),
flushFinished: make(chan struct{}),
quitChan: make(chan struct{}),
doneChan: make(chan struct{}),
}
return csvWriter, nil
}
func (csw *CSVWriter) Loop() {
go func() {
defer close(csw.doneChan)
for {
select {
case row := <-csw.rows:
err := csw.writers.write(&row.table, row.values...)
if err != nil {
panic(fmt.Sprintf("error writing csv buffer: %v", err))
}
case <-csw.quitChan:
if err := csw.writers.flush(); err != nil {
panic(fmt.Sprintf("error writing csv buffer to file: %v", err))
}
return
case <-csw.flushChan:
if err := csw.writers.flush(); err != nil {
panic(fmt.Sprintf("error writing csv buffer to file: %v", err))
}
csw.flushFinished <- struct{}{}
}
}
}()
}
// Flush sends a flush signal to the looping process
func (csw *CSVWriter) Flush() {
csw.flushChan <- struct{}{}
<-csw.flushFinished
}
func TableFilePath(dir, name string) string { return filepath.Join(dir, name+".csv") }
// Close satisfies io.Closer
func (csw *CSVWriter) Close() error {
close(csw.quitChan)
<-csw.doneChan
close(csw.rows)
close(csw.flushChan)
close(csw.flushFinished)
return csw.writers.close()
}
func (csw *CSVWriter) upsertNode(node nodeinfo.Info) {
var values []interface{}
values = append(values, node.GenesisBlock, node.NetworkID, node.ID, node.ClientName, node.ChainID)
csw.rows <- tableRow{schema.TableNodeInfo, values}
}
func (csw *CSVWriter) upsertIPLD(ipld models.IPLDModel) {
var values []interface{}
values = append(values, ipld.BlockNumber, ipld.Key, ipld.Data)
csw.rows <- tableRow{schema.TableIPLDBlock, values}
}
func (csw *CSVWriter) upsertIPLDDirect(blockNumber, key string, value []byte) {
csw.upsertIPLD(models.IPLDModel{
BlockNumber: blockNumber,
Key: key,
Data: value,
})
}
func (csw *CSVWriter) upsertIPLDNode(blockNumber string, i ipld.IPLD) {
csw.upsertIPLD(models.IPLDModel{
BlockNumber: blockNumber,
Key: i.Cid().String(),
Data: i.RawData(),
})
}
func (csw *CSVWriter) upsertHeaderCID(header models.HeaderModel) {
var values []interface{}
values = append(values, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID,
header.TotalDifficulty, header.NodeIDs, header.Reward, header.StateRoot, header.TxRoot,
header.RctRoot, header.UnclesHash, header.Bloom, strconv.FormatUint(header.Timestamp, 10), header.Coinbase)
csw.rows <- tableRow{schema.TableHeader, values}
metrics.IndexerMetrics.BlocksCounter.Inc(1)
}
func (csw *CSVWriter) upsertUncleCID(uncle models.UncleModel) {
var values []interface{}
values = append(values, uncle.BlockNumber, uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID,
uncle.Reward, uncle.Index)
csw.rows <- tableRow{schema.TableUncle, values}
}
func (csw *CSVWriter) upsertTransactionCID(transaction models.TxModel) {
var values []interface{}
values = append(values, transaction.BlockNumber, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst,
transaction.Src, transaction.Index, transaction.Type, transaction.Value)
csw.rows <- tableRow{schema.TableTransaction, values}
metrics.IndexerMetrics.TransactionsCounter.Inc(1)
}
func (csw *CSVWriter) upsertReceiptCID(rct *models.ReceiptModel) {
var values []interface{}
values = append(values, rct.BlockNumber, rct.HeaderID, rct.TxID, rct.CID, rct.Contract,
rct.PostState, rct.PostStatus)
csw.rows <- tableRow{schema.TableReceipt, values}
metrics.IndexerMetrics.ReceiptsCounter.Inc(1)
}
func (csw *CSVWriter) upsertLogCID(logs []*models.LogsModel) {
for _, l := range logs {
var values []interface{}
values = append(values, l.BlockNumber, l.HeaderID, l.CID, l.ReceiptID, l.Address, l.Index, l.Topic0,
l.Topic1, l.Topic2, l.Topic3)
csw.rows <- tableRow{schema.TableLog, values}
metrics.IndexerMetrics.LogsCounter.Inc(1)
}
}
func (csw *CSVWriter) upsertStateCID(stateNode models.StateNodeModel) {
balance := stateNode.Balance
if stateNode.Removed {
balance = "0"
}
var values []interface{}
values = append(values, stateNode.BlockNumber, stateNode.HeaderID, stateNode.StateKey, stateNode.CID,
true, balance, strconv.FormatUint(stateNode.Nonce, 10), stateNode.CodeHash, stateNode.StorageRoot, stateNode.Removed)
csw.rows <- tableRow{schema.TableStateNode, values}
}
func (csw *CSVWriter) upsertStorageCID(storageCID models.StorageNodeModel) {
var values []interface{}
values = append(values, storageCID.BlockNumber, storageCID.HeaderID, storageCID.StateKey, storageCID.StorageKey, storageCID.CID,
true, storageCID.Value, storageCID.Removed)
csw.rows <- tableRow{schema.TableStorageNode, values}
}
// LoadWatchedAddresses loads watched addresses from a file
func (csw *CSVWriter) loadWatchedAddresses() ([]common.Address, error) {
watchedAddressesFilePath := csw.watchedAddressesWriter.file.Name()
// load csv rows from watched addresses file
rows, err := loadWatchedAddressesRows(watchedAddressesFilePath)
if err != nil {
return nil, err
}
// extract addresses from the csv rows
watchedAddresses := funk.Map(rows, func(row []string) common.Address {
// first column is for address in eth_meta.watched_addresses
addressString := row[0]
return common.HexToAddress(addressString)
}).([]common.Address)
return watchedAddresses, nil
}
// InsertWatchedAddresses inserts the given addresses in a file
func (csw *CSVWriter) insertWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
// load csv rows from watched addresses file
watchedAddresses, err := csw.loadWatchedAddresses()
if err != nil {
return err
}
// append rows for new addresses to existing csv file
for _, arg := range args {
// ignore if already watched
if funk.Contains(watchedAddresses, common.HexToAddress(arg.Address)) {
continue
}
var values []interface{}
values = append(values, arg.Address, strconv.FormatUint(arg.CreatedAt, 10), currentBlockNumber.String(), "0")
row := schema.TableWatchedAddresses.ToCsvRow(values...)
// writing directly instead of using rows channel as it needs to be flushed immediately
err = csw.watchedAddressesWriter.Write(row)
if err != nil {
return err
}
}
// watched addresses need to be flushed immediately to the file to keep them in sync with in-memory watched addresses
csw.watchedAddressesWriter.Flush()
err = csw.watchedAddressesWriter.Error()
if err != nil {
return err
}
return nil
}
// RemoveWatchedAddresses removes the given watched addresses from a file
func (csw *CSVWriter) removeWatchedAddresses(args []sdtypes.WatchAddressArg) error {
// load csv rows from watched addresses file
watchedAddressesFilePath := csw.watchedAddressesWriter.file.Name()
rows, err := loadWatchedAddressesRows(watchedAddressesFilePath)
if err != nil {
return err
}
// get rid of rows having addresses to be removed
filteredRows := funk.Filter(rows, func(row []string) bool {
return !funk.Contains(args, func(arg sdtypes.WatchAddressArg) bool {
// Compare first column in table for address
return arg.Address == row[0]
})
}).([][]string)
return dumpWatchedAddressesRows(csw.watchedAddressesWriter, filteredRows)
}
// SetWatchedAddresses clears and inserts the given addresses in a file
func (csw *CSVWriter) setWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
var rows [][]string
for _, arg := range args {
row := schema.TableWatchedAddresses.ToCsvRow(arg.Address, strconv.FormatUint(arg.CreatedAt, 10), currentBlockNumber.String(), "0")
rows = append(rows, row)
}
return dumpWatchedAddressesRows(csw.watchedAddressesWriter, rows)
}
// loadCSVWatchedAddresses loads csv rows from the given file
func loadWatchedAddressesRows(filePath string) ([][]string, error) {
file, err := os.Open(filePath)
if err != nil {
if errors.Is(err, os.ErrNotExist) {
return [][]string{}, nil
}
return nil, fmt.Errorf("error opening watched addresses file: %v", err)
}
defer file.Close()
reader := csv.NewReader(file)
return reader.ReadAll()
}
// dumpWatchedAddressesRows dumps csv rows to the given file
func dumpWatchedAddressesRows(watchedAddressesWriter fileWriter, filteredRows [][]string) error {
file := watchedAddressesWriter.file
file.Close()
file, err := os.Create(file.Name())
if err != nil {
return fmt.Errorf("error creating watched addresses file: %v", err)
}
watchedAddressesWriter.Writer = csv.NewWriter(file)
watchedAddressesWriter.file = file
for _, row := range filteredRows {
watchedAddressesWriter.Write(row)
}
watchedAddressesWriter.Flush()
return nil
}

View File

@ -0,0 +1,60 @@
// VulcanizeDB
// Copyright © 2021 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 file
import "bytes"
// formatPostgresStringArray parses an array of strings into the proper Postgres string representation of that array
func formatPostgresStringArray(a []string) string {
if a == nil {
return ""
}
if n := len(a); n > 0 {
// There will be at least two curly brackets, 2*N bytes of quotes,
// and N-1 bytes of delimiters.
b := make([]byte, 1, 1+3*n)
b[0] = '{'
b = appendArrayQuotedBytes(b, []byte(a[0]))
for i := 1; i < n; i++ {
b = append(b, ',')
b = appendArrayQuotedBytes(b, []byte(a[i]))
}
return string(append(b, '}'))
}
return "{}"
}
func appendArrayQuotedBytes(b, v []byte) []byte {
b = append(b, '"')
for {
i := bytes.IndexAny(v, `"\`)
if i < 0 {
b = append(b, v...)
break
}
if i > 0 {
b = append(b, v[:i]...)
}
b = append(b, '\\', v[i])
v = v[i+1:]
}
return append(b, '"')
}

View File

@ -0,0 +1,492 @@
// VulcanizeDB
// Copyright © 2021 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 file
import (
"bytes"
"context"
"errors"
"fmt"
"math/big"
"os"
"sync"
"sync/atomic"
"time"
"github.com/lib/pq"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
const defaultCSVOutputDir = "./statediff_output"
const defaultSQLFilePath = "./statediff.sql"
const defaultWatchedAddressesCSVFilePath = "./statediff-watched-addresses.csv"
const defaultWatchedAddressesSQLFilePath = "./statediff-watched-addresses.sql"
const watchedAddressesInsert = "INSERT INTO eth_meta.watched_addresses (address, created_at, watched_at) VALUES ('%s', '%d', '%d') ON CONFLICT (address) DO NOTHING;"
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of a void
type StateDiffIndexer struct {
fileWriter FileWriter
chainConfig *params.ChainConfig
nodeID string
wg *sync.WaitGroup
removedCacheFlag *uint32
}
// NewStateDiffIndexer creates a void implementation of interfaces.StateDiffIndexer
func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, config Config) (*StateDiffIndexer, error) {
var err error
var writer FileWriter
watchedAddressesFilePath := config.WatchedAddressesFilePath
switch config.Mode {
case CSV:
outputDir := config.OutputDir
if outputDir == "" {
outputDir = defaultCSVOutputDir
}
if _, err := os.Stat(outputDir); !errors.Is(err, os.ErrNotExist) {
return nil, fmt.Errorf("cannot create output directory, directory (%s) already exists", outputDir)
}
log.Info("Writing statediff CSV files to directory", "file", outputDir)
if watchedAddressesFilePath == "" {
watchedAddressesFilePath = defaultWatchedAddressesCSVFilePath
}
log.Info("Writing watched addresses to file", "file", watchedAddressesFilePath)
writer, err = NewCSVWriter(outputDir, watchedAddressesFilePath)
if err != nil {
return nil, err
}
case SQL:
filePath := config.FilePath
if filePath == "" {
filePath = defaultSQLFilePath
}
if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) {
return nil, fmt.Errorf("cannot create file, file (%s) already exists", filePath)
}
file, err := os.Create(filePath)
if err != nil {
return nil, fmt.Errorf("unable to create file (%s), err: %v", filePath, err)
}
log.Info("Writing statediff SQL statements to file", "file", filePath)
if watchedAddressesFilePath == "" {
watchedAddressesFilePath = defaultWatchedAddressesSQLFilePath
}
log.Info("Writing watched addresses to file", "file", watchedAddressesFilePath)
writer = NewSQLWriter(file, watchedAddressesFilePath)
default:
return nil, fmt.Errorf("unrecognized file mode: %s", config.Mode)
}
wg := new(sync.WaitGroup)
writer.Loop()
writer.upsertNode(config.NodeInfo)
return &StateDiffIndexer{
fileWriter: writer,
chainConfig: chainConfig,
nodeID: config.NodeInfo.ID,
wg: wg,
}, nil
}
// ReportDBMetrics has nothing to report for dump
func (sdi *StateDiffIndexer) ReportDBMetrics(time.Duration, <-chan bool) {}
// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts)
// Returns an initiated DB transaction which must be Closed via defer to commit or rollback
func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) {
sdi.removedCacheFlag = new(uint32)
start, t := time.Now(), time.Now()
blockHash := block.Hash()
blockHashStr := blockHash.String()
height := block.NumberU64()
traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHashStr)
transactions := block.Transactions()
// Derive any missing fields
if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil {
return nil, err
}
// Generate the block iplds
headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts)
if err != nil {
return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err)
}
if len(txNodes) != len(rctNodes) {
return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d)", len(txNodes), len(rctNodes))
}
// Calculate reward
var reward *big.Int
// in PoA networks block reward is 0
if sdi.chainConfig.Clique != nil {
reward = big.NewInt(0)
} else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts)
}
t = time.Now()
blockTx := &BatchTx{
BlockNumber: block.Number().String(),
submit: func(self *BatchTx, err error) error {
tDiff := time.Since(t)
metrics.IndexerMetrics.StateStoreCodeProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String())
t = time.Now()
sdi.fileWriter.Flush()
tDiff = time.Since(t)
metrics.IndexerMetrics.PostgresCommitTimer.Update(tDiff)
traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String())
traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String())
log.Debug(traceMsg)
return err
},
}
tDiff := time.Since(t)
metrics.IndexerMetrics.FreePostgresTimer.Update(tDiff)
traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String())
t = time.Now()
// write header, collect headerID
headerID := sdi.processHeader(block.Header(), headerNode, reward, totalDifficulty)
tDiff = time.Since(t)
metrics.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String())
t = time.Now()
// write uncles
sdi.processUncles(headerID, block.Number(), block.UncleHash(), block.Uncles())
tDiff = time.Since(t)
metrics.IndexerMetrics.UncleProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String())
t = time.Now()
// write receipts and txs
err = sdi.processReceiptsAndTxs(processArgs{
headerID: headerID,
blockNumber: block.Number(),
receipts: receipts,
txs: transactions,
rctNodes: rctNodes,
txNodes: txNodes,
logNodes: logNodes,
})
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics.IndexerMetrics.TxAndRecProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String())
t = time.Now()
return blockTx, err
}
// processHeader write a header IPLD insert SQL stmt to a file
// it returns the headerID
func (sdi *StateDiffIndexer) processHeader(header *types.Header, headerNode ipld.IPLD, reward, td *big.Int) string {
sdi.fileWriter.upsertIPLDNode(header.Number.String(), headerNode)
var baseFee *string
if header.BaseFee != nil {
baseFee = new(string)
*baseFee = header.BaseFee.String()
}
headerID := header.Hash().String()
sdi.fileWriter.upsertHeaderCID(models.HeaderModel{
NodeIDs: pq.StringArray([]string{sdi.nodeID}),
CID: headerNode.Cid().String(),
ParentHash: header.ParentHash.String(),
BlockNumber: header.Number.String(),
BlockHash: headerID,
TotalDifficulty: td.String(),
Reward: reward.String(),
Bloom: header.Bloom.Bytes(),
StateRoot: header.Root.String(),
RctRoot: header.ReceiptHash.String(),
TxRoot: header.TxHash.String(),
UnclesHash: header.UncleHash.String(),
Timestamp: header.Time,
Coinbase: header.Coinbase.String(),
})
return headerID
}
// processUncles publishes and indexes uncle IPLDs in Postgres
func (sdi *StateDiffIndexer) processUncles(headerID string, blockNumber *big.Int, unclesHash common.Hash, uncles []*types.Header) error {
// publish and index uncles
uncleEncoding, err := rlp.EncodeToBytes(uncles)
if err != nil {
return err
}
preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex())
}
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil {
return err
}
sdi.fileWriter.upsertIPLDDirect(blockNumber.String(), unclesCID.String(), uncleEncoding)
for i, uncle := range uncles {
var uncleReward *big.Int
// in PoA networks uncle reward is 0
if sdi.chainConfig.Clique != nil {
uncleReward = big.NewInt(0)
} else {
uncleReward = shared.CalcUncleMinerReward(blockNumber.Uint64(), uncle.Number.Uint64())
}
sdi.fileWriter.upsertUncleCID(models.UncleModel{
BlockNumber: blockNumber.String(),
HeaderID: headerID,
CID: unclesCID.String(),
ParentHash: uncle.ParentHash.String(),
BlockHash: uncle.Hash().String(),
Reward: uncleReward.String(),
Index: int64(i),
})
}
return nil
}
// processArgs bundles arguments to processReceiptsAndTxs
type processArgs struct {
headerID string
blockNumber *big.Int
receipts types.Receipts
txs types.Transactions
rctNodes []*ipld.EthReceipt
txNodes []*ipld.EthTx
logNodes [][]*ipld.EthLog
}
// processReceiptsAndTxs writes receipt and tx IPLD insert SQL stmts to a file
func (sdi *StateDiffIndexer) processReceiptsAndTxs(args processArgs) error {
// Process receipts and txs
signer := types.MakeSigner(sdi.chainConfig, args.blockNumber)
for i, receipt := range args.receipts {
txNode := args.txNodes[i]
sdi.fileWriter.upsertIPLDNode(args.blockNumber.String(), txNode)
sdi.fileWriter.upsertIPLDNode(args.blockNumber.String(), args.rctNodes[i])
// index tx
trx := args.txs[i]
txID := trx.Hash().String()
var val string
if trx.Value() != nil {
val = trx.Value().String()
}
// derive sender for the tx that corresponds with this receipt
from, err := types.Sender(signer, trx)
if err != nil {
return fmt.Errorf("error deriving tx sender: %v", err)
}
txModel := models.TxModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
Dst: shared.HandleZeroAddrPointer(trx.To()),
Src: shared.HandleZeroAddr(from),
TxHash: txID,
Index: int64(i),
CID: txNode.Cid().String(),
Type: trx.Type(),
Value: val,
}
sdi.fileWriter.upsertTransactionCID(txModel)
// this is the contract address if this receipt is for a contract creation tx
contract := shared.HandleZeroAddr(receipt.ContractAddress)
// index receipt
rctModel := &models.ReceiptModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
TxID: txID,
Contract: contract,
CID: args.rctNodes[i].Cid().String(),
}
if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status
} else {
rctModel.PostState = common.BytesToHash(receipt.PostState).String()
}
sdi.fileWriter.upsertReceiptCID(rctModel)
// index logs
logDataSet := make([]*models.LogsModel, len(receipt.Logs))
for idx, l := range receipt.Logs {
sdi.fileWriter.upsertIPLDNode(args.blockNumber.String(), args.logNodes[i][idx])
topicSet := make([]string, 4)
for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex()
}
logDataSet[idx] = &models.LogsModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
ReceiptID: txID,
Address: l.Address.String(),
Index: int64(l.Index),
CID: args.logNodes[i][idx].Cid().String(),
Topic0: topicSet[0],
Topic1: topicSet[1],
Topic2: topicSet[2],
Topic3: topicSet[3],
}
}
sdi.fileWriter.upsertLogCID(logDataSet)
}
return nil
}
// PushStateNode writes a state diff node object (including any child storage nodes) IPLD insert SQL stmt to a file
func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateLeafNode, headerID string) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("file: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
// publish the state node
var stateModel models.StateNodeModel
if stateNode.Removed {
if atomic.LoadUint32(sdi.removedCacheFlag) == 0 {
atomic.StoreUint32(sdi.removedCacheFlag, 1)
sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStateCID, []byte{})
}
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID,
Removed: true,
}
} else {
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID,
Removed: false,
Balance: stateNode.AccountWrapper.Account.Balance.String(),
Nonce: stateNode.AccountWrapper.Account.Nonce,
CodeHash: common.BytesToHash(stateNode.AccountWrapper.Account.CodeHash).String(),
StorageRoot: stateNode.AccountWrapper.Account.Root.String(),
}
}
// index the state node
sdi.fileWriter.upsertStateCID(stateModel)
// if there are any storage nodes associated with this node, publish and index them
for _, storageNode := range stateNode.StorageDiff {
if storageNode.Removed {
if atomic.LoadUint32(sdi.removedCacheFlag) == 0 {
atomic.StoreUint32(sdi.removedCacheFlag, 1)
sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, shared.RemovedNodeStorageCID, []byte{})
}
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: shared.RemovedNodeStorageCID,
Removed: true,
Value: []byte{},
}
sdi.fileWriter.upsertStorageCID(storageModel)
continue
}
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: storageNode.CID,
Removed: false,
Value: storageNode.Value,
}
sdi.fileWriter.upsertStorageCID(storageModel)
}
return nil
}
// PushIPLD writes iplds to ipld.blocks
func (sdi *StateDiffIndexer) PushIPLD(batch interfaces.Batch, ipld sdtypes.IPLD) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("file: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
sdi.fileWriter.upsertIPLDDirect(tx.BlockNumber, ipld.CID, ipld.Content)
return nil
}
// Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error {
return sdi.fileWriter.Close()
}
// LoadWatchedAddresses loads watched addresses from a file
func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) {
return sdi.fileWriter.loadWatchedAddresses()
}
// InsertWatchedAddresses inserts the given addresses in a file
func (sdi *StateDiffIndexer) InsertWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
return sdi.fileWriter.insertWatchedAddresses(args, currentBlockNumber)
}
// RemoveWatchedAddresses removes the given watched addresses from a file
func (sdi *StateDiffIndexer) RemoveWatchedAddresses(args []sdtypes.WatchAddressArg) error {
return sdi.fileWriter.removeWatchedAddresses(args)
}
// SetWatchedAddresses clears and inserts the given addresses in a file
func (sdi *StateDiffIndexer) SetWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error {
return sdi.fileWriter.setWatchedAddresses(args, currentBlockNumber)
}
// ClearWatchedAddresses clears all the watched addresses from a file
func (sdi *StateDiffIndexer) ClearWatchedAddresses() error {
return sdi.SetWatchedAddresses([]sdtypes.WatchAddressArg{}, big.NewInt(0))
}

View File

@ -0,0 +1,57 @@
// VulcanizeDB
// Copyright © 2022 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 file
import (
"math/big"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/types"
)
// Writer interface required by the file indexer
type FileWriter interface {
// Methods used to control the writer
Loop()
Close() error
Flush()
// Methods to upsert ethereum data model objects
upsertNode(node nodeinfo.Info)
upsertHeaderCID(header models.HeaderModel)
upsertUncleCID(uncle models.UncleModel)
upsertTransactionCID(transaction models.TxModel)
upsertReceiptCID(rct *models.ReceiptModel)
upsertLogCID(logs []*models.LogsModel)
upsertStateCID(stateNode models.StateNodeModel)
upsertStorageCID(storageCID models.StorageNodeModel)
upsertIPLD(ipld models.IPLDModel)
// Methods to upsert IPLD in different ways
upsertIPLDDirect(blockNumber, key string, value []byte)
upsertIPLDNode(blockNumber string, i ipld.IPLD)
// Methods to read and write watched addresses
loadWatchedAddresses() ([]common.Address, error)
insertWatchedAddresses(args []types.WatchAddressArg, currentBlockNumber *big.Int) error
removeWatchedAddresses(args []types.WatchAddressArg) error
setWatchedAddresses(args []types.WatchAddressArg, currentBlockNumber *big.Int) error
}

View File

@ -0,0 +1,112 @@
// VulcanizeDB
// Copyright © 2021 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 mainnet_tests
import (
"context"
"errors"
"fmt"
"math/big"
"os"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
)
var (
err error
db sql.Database
ind interfaces.StateDiffIndexer
chainConf = params.MainnetChainConfig
)
func init() {
if os.Getenv("MODE") != "statediff" {
fmt.Println("Skipping statediff test")
os.Exit(0)
}
if os.Getenv("STATEDIFF_DB") != "file" {
fmt.Println("Skipping statediff .sql file writing mode test")
os.Exit(0)
}
}
func TestPushBlockAndState(t *testing.T) {
conf := test_helpers.GetTestConfig()
for _, blockNumber := range test_helpers.ProblemBlocks {
conf.BlockNumber = big.NewInt(blockNumber)
tb, trs, err := test_helpers.TestBlockAndReceipts(conf)
require.NoError(t, err)
testPushBlockAndState(t, tb, trs)
}
testBlock, testReceipts, err := test_helpers.TestBlockAndReceiptsFromEnv(conf)
require.NoError(t, err)
testPushBlockAndState(t, testBlock, testReceipts)
}
func testPushBlockAndState(t *testing.T, block *types.Block, receipts types.Receipts) {
t.Run("Test PushBlock and PushStateNode", func(t *testing.T) {
setupMainnetIndexer(t)
defer dumpData(t)
defer tearDown(t)
test.TestBlock(t, ind, block, receipts)
})
}
func setupMainnetIndexer(t *testing.T) {
if _, err := os.Stat(file.CSVTestConfig.FilePath); !errors.Is(err, os.ErrNotExist) {
err := os.Remove(file.CSVTestConfig.FilePath)
require.NoError(t, err)
}
ind, err = file.NewStateDiffIndexer(context.Background(), chainConf, file.CSVTestConfig)
require.NoError(t, err)
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
}
func dumpData(t *testing.T) {
sqlFileBytes, err := os.ReadFile(file.CSVTestConfig.FilePath)
require.NoError(t, err)
_, err = db.Exec(context.Background(), string(sqlFileBytes))
require.NoError(t, err)
}
func tearDown(t *testing.T) {
test_helpers.TearDownDB(t, db)
require.NoError(t, db.Close())
require.NoError(t, os.Remove(file.CSVTestConfig.FilePath))
}

View File

@ -0,0 +1,101 @@
// 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 file_test
import (
"context"
"errors"
"os"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
)
var (
db sql.Database
err error
ind interfaces.StateDiffIndexer
)
func setupLegacySQLIndexer(t *testing.T) {
if _, err := os.Stat(file.SQLTestConfig.FilePath); !errors.Is(err, os.ErrNotExist) {
err := os.Remove(file.SQLTestConfig.FilePath)
require.NoError(t, err)
}
ind, err = file.NewStateDiffIndexer(context.Background(), test.LegacyConfig, file.SQLTestConfig)
require.NoError(t, err)
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
}
func setupLegacySQL(t *testing.T) {
setupLegacySQLIndexer(t)
test.SetupLegacyTestData(t, ind)
}
func dumpFileData(t *testing.T) {
err := test_helpers.DedupFile(file.SQLTestConfig.FilePath)
require.NoError(t, err)
sqlFileBytes, err := os.ReadFile(file.SQLTestConfig.FilePath)
require.NoError(t, err)
_, err = db.Exec(context.Background(), string(sqlFileBytes))
require.NoError(t, err)
}
func resetAndDumpWatchedAddressesFileData(t *testing.T) {
test_helpers.TearDownDB(t, db)
sqlFileBytes, err := os.ReadFile(file.SQLTestConfig.WatchedAddressesFilePath)
require.NoError(t, err)
_, err = db.Exec(context.Background(), string(sqlFileBytes))
require.NoError(t, err)
}
func tearDown(t *testing.T) {
test_helpers.TearDownDB(t, db)
require.NoError(t, db.Close())
require.NoError(t, os.Remove(file.SQLTestConfig.FilePath))
if err := os.Remove(file.SQLTestConfig.WatchedAddressesFilePath); !errors.Is(err, os.ErrNotExist) {
require.NoError(t, err)
}
}
func TestLegacySQLFileIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs", func(t *testing.T) {
setupLegacySQL(t)
dumpFileData(t)
defer tearDown(t)
test.TestLegacyIndexer(t, db)
})
}

View File

@ -0,0 +1,253 @@
// 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 file_test
import (
"context"
"errors"
"math/big"
"os"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupIndexer(t *testing.T) {
if _, err := os.Stat(file.SQLTestConfig.FilePath); !errors.Is(err, os.ErrNotExist) {
err := os.Remove(file.SQLTestConfig.FilePath)
require.NoError(t, err)
}
if _, err := os.Stat(file.SQLTestConfig.WatchedAddressesFilePath); !errors.Is(err, os.ErrNotExist) {
err := os.Remove(file.SQLTestConfig.WatchedAddressesFilePath)
require.NoError(t, err)
}
ind, err = file.NewStateDiffIndexer(context.Background(), mocks.TestConfig, file.SQLTestConfig)
require.NoError(t, err)
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
}
func setup(t *testing.T) {
setupIndexer(t)
test.SetupTestData(t, ind)
}
func setupSQLNonCanonical(t *testing.T) {
setupIndexer(t)
test.SetupTestDataNonCanonical(t, ind)
}
func TestSQLFileIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexHeaderIPLDs(t, db)
})
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexTransactionIPLDs(t, db)
})
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexLogIPLDs(t, db)
})
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexReceiptIPLDs(t, db)
})
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexStateIPLDs(t, db)
})
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
setup(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexStorageIPLDs(t, db)
})
}
func TestSQLFileIndexerNonCanonical(t *testing.T) {
t.Run("Publish and index header", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexHeaderNonCanonical(t, db)
})
t.Run("Publish and index transactions", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexTransactionsNonCanonical(t, db)
})
t.Run("Publish and index receipts", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexReceiptsNonCanonical(t, db)
})
t.Run("Publish and index logs", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexLogsNonCanonical(t, db)
})
t.Run("Publish and index state nodes", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexStateNonCanonical(t, db)
})
t.Run("Publish and index storage nodes", func(t *testing.T) {
setupSQLNonCanonical(t)
dumpFileData(t)
defer tearDown(t)
test.TestPublishAndIndexStorageNonCanonical(t, db)
})
}
func TestSQLFileWatchAddressMethods(t *testing.T) {
setupIndexer(t)
defer tearDown(t)
t.Run("Load watched addresses (empty table)", func(t *testing.T) {
test.TestLoadEmptyWatchedAddresses(t, ind)
})
t.Run("Insert watched addresses", func(t *testing.T) {
args := mocks.GetInsertWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt1)))
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestInsertWatchedAddresses(t, db)
})
t.Run("Insert watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetInsertAlreadyWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestInsertAlreadyWatchedAddresses(t, db)
})
t.Run("Remove watched addresses", func(t *testing.T) {
args := mocks.GetRemoveWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestRemoveWatchedAddresses(t, db)
})
t.Run("Remove watched addresses (some non-watched)", func(t *testing.T) {
args := mocks.GetRemoveNonWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestRemoveNonWatchedAddresses(t, db)
})
t.Run("Set watched addresses", func(t *testing.T) {
args := mocks.GetSetWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestSetWatchedAddresses(t, db)
})
t.Run("Set watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetSetAlreadyWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt3)))
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestSetAlreadyWatchedAddresses(t, db)
})
t.Run("Load watched addresses", func(t *testing.T) {
test.TestLoadWatchedAddresses(t, ind)
})
t.Run("Clear watched addresses", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestClearWatchedAddresses(t, db)
})
t.Run("Clear watched addresses (empty table)", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
resetAndDumpWatchedAddressesFileData(t)
test.TestClearEmptyWatchedAddresses(t, db)
})
}

View File

@ -0,0 +1,391 @@
// 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 file
import (
"bufio"
"errors"
"fmt"
"io"
"math/big"
"os"
pg_query "github.com/pganalyze/pg_query_go/v2"
"github.com/thoas/go-funk"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
nodeinfo "github.com/ethereum/go-ethereum/statediff/indexer/node"
"github.com/ethereum/go-ethereum/statediff/types"
)
var (
pipeSize = 65336 // min(linuxPipeSize, macOSPipeSize)
writeBufferSize = pipeSize * 16 * 96
)
// SQLWriter writes sql statements to a file
type SQLWriter struct {
wc io.WriteCloser
stmts chan []byte
collatedStmt []byte
collationIndex int
flushChan chan struct{}
flushFinished chan struct{}
quitChan chan struct{}
doneChan chan struct{}
watchedAddressesFilePath string
}
// NewSQLWriter creates a new pointer to a Writer
func NewSQLWriter(wc io.WriteCloser, watchedAddressesFilePath string) *SQLWriter {
return &SQLWriter{
wc: wc,
stmts: make(chan []byte),
collatedStmt: make([]byte, writeBufferSize),
flushChan: make(chan struct{}),
flushFinished: make(chan struct{}),
quitChan: make(chan struct{}),
doneChan: make(chan struct{}),
watchedAddressesFilePath: watchedAddressesFilePath,
}
}
// Loop enables concurrent writes to the underlying os.File
// since os.File does not buffer, it utilizes an internal buffer that is the size of a unix pipe
// by using copy() and tracking the index/size of the buffer, we require only the initial memory allocation
func (sqw *SQLWriter) Loop() {
sqw.collationIndex = 0
go func() {
defer close(sqw.doneChan)
var l int
for {
select {
case stmt := <-sqw.stmts:
l = len(stmt)
if sqw.collationIndex+l > writeBufferSize {
if err := sqw.flush(); err != nil {
panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err))
}
if l > writeBufferSize {
if _, err := sqw.wc.Write(stmt); err != nil {
panic(fmt.Sprintf("error writing large sql stmt to file: %v", err))
}
continue
}
}
copy(sqw.collatedStmt[sqw.collationIndex:sqw.collationIndex+l], stmt)
sqw.collationIndex += l
case <-sqw.quitChan:
if err := sqw.flush(); err != nil {
panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err))
}
return
case <-sqw.flushChan:
if err := sqw.flush(); err != nil {
panic(fmt.Sprintf("error writing sql stmts buffer to file: %v", err))
}
sqw.flushFinished <- struct{}{}
}
}
}()
}
// Close satisfies io.Closer
func (sqw *SQLWriter) Close() error {
close(sqw.quitChan)
<-sqw.doneChan
close(sqw.stmts)
close(sqw.flushChan)
close(sqw.flushFinished)
return sqw.wc.Close()
}
// Flush sends a flush signal to the looping process
func (sqw *SQLWriter) Flush() {
sqw.flushChan <- struct{}{}
<-sqw.flushFinished
}
func (sqw *SQLWriter) flush() error {
if _, err := sqw.wc.Write(sqw.collatedStmt[0:sqw.collationIndex]); err != nil {
return err
}
sqw.collationIndex = 0
return nil
}
const (
nodeInsert = "INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES " +
"('%s', '%s', '%s', '%s', %d);\n"
ipldInsert = "INSERT INTO ipld.blocks (block_number, key, data) VALUES ('%s', '%s', '\\x%x');\n"
headerInsert = "INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_ids, reward, " +
"state_root, tx_root, receipt_root, uncles_hash, bloom, timestamp, coinbase) VALUES " +
"('%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '%s', '\\x%x', %d, '%s');\n"
uncleInsert = "INSERT INTO eth.uncle_cids (block_number, block_hash, header_id, parent_hash, cid, reward, index) VALUES " +
"('%s', '%s', '%s', '%s', '%s', '%s', %d);\n"
txInsert = "INSERT INTO eth.transaction_cids (block_number, header_id, tx_hash, cid, dst, src, index, tx_type, " +
"value) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', %d, %d, '%s');\n"
rctInsert = "INSERT INTO eth.receipt_cids (block_number, header_id, tx_id, cid, contract, post_state, " +
"post_status) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', %d);\n"
logInsert = "INSERT INTO eth.log_cids (block_number, header_id, cid, rct_id, address, index, topic0, topic1, topic2, " +
"topic3) VALUES ('%s', '%s', '%s', '%s', '%s', %d, '%s', '%s', '%s', '%s');\n"
stateInsert = "INSERT INTO eth.state_cids (block_number, header_id, state_leaf_key, cid, removed, diff, " +
"balance, nonce, code_hash, storage_root) VALUES ('%s', '%s', '%s', '%s', %t, %t, '%s', %d, '%s', '%s');\n"
storageInsert = "INSERT INTO eth.storage_cids (block_number, header_id, state_leaf_key, storage_leaf_key, cid, " +
"removed, diff, val) VALUES ('%s', '%s', '%s', '%s', '%s', %t, %t, '\\x%x');\n"
)
func (sqw *SQLWriter) upsertNode(node nodeinfo.Info) {
sqw.stmts <- []byte(fmt.Sprintf(nodeInsert, node.GenesisBlock, node.NetworkID, node.ID, node.ClientName, node.ChainID))
}
func (sqw *SQLWriter) upsertIPLD(ipld models.IPLDModel) {
sqw.stmts <- []byte(fmt.Sprintf(ipldInsert, ipld.BlockNumber, ipld.Key, ipld.Data))
}
func (sqw *SQLWriter) upsertIPLDDirect(blockNumber, key string, value []byte) {
sqw.upsertIPLD(models.IPLDModel{
BlockNumber: blockNumber,
Key: key,
Data: value,
})
}
func (sqw *SQLWriter) upsertIPLDNode(blockNumber string, i ipld.IPLD) {
sqw.upsertIPLD(models.IPLDModel{
BlockNumber: blockNumber,
Key: i.Cid().String(),
Data: i.RawData(),
})
}
func (sqw *SQLWriter) upsertHeaderCID(header models.HeaderModel) {
stmt := fmt.Sprintf(headerInsert, header.BlockNumber, header.BlockHash, header.ParentHash, header.CID,
header.TotalDifficulty, formatPostgresStringArray(header.NodeIDs), header.Reward, header.StateRoot, header.TxRoot,
header.RctRoot, header.UnclesHash, header.Bloom, header.Timestamp, header.Coinbase)
sqw.stmts <- []byte(stmt)
metrics.IndexerMetrics.BlocksCounter.Inc(1)
}
func (sqw *SQLWriter) upsertUncleCID(uncle models.UncleModel) {
sqw.stmts <- []byte(fmt.Sprintf(uncleInsert, uncle.BlockNumber, uncle.BlockHash, uncle.HeaderID, uncle.ParentHash, uncle.CID,
uncle.Reward, uncle.Index))
}
func (sqw *SQLWriter) upsertTransactionCID(transaction models.TxModel) {
sqw.stmts <- []byte(fmt.Sprintf(txInsert, transaction.BlockNumber, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst,
transaction.Src, transaction.Index, transaction.Type, transaction.Value))
metrics.IndexerMetrics.TransactionsCounter.Inc(1)
}
func (sqw *SQLWriter) upsertReceiptCID(rct *models.ReceiptModel) {
sqw.stmts <- []byte(fmt.Sprintf(rctInsert, rct.BlockNumber, rct.HeaderID, rct.TxID, rct.CID, rct.Contract,
rct.PostState, rct.PostStatus))
metrics.IndexerMetrics.ReceiptsCounter.Inc(1)
}
func (sqw *SQLWriter) upsertLogCID(logs []*models.LogsModel) {
for _, l := range logs {
sqw.stmts <- []byte(fmt.Sprintf(logInsert, l.BlockNumber, l.HeaderID, l.CID, l.ReceiptID, l.Address, l.Index, l.Topic0,
l.Topic1, l.Topic2, l.Topic3))
metrics.IndexerMetrics.LogsCounter.Inc(1)
}
}
func (sqw *SQLWriter) upsertStateCID(stateNode models.StateNodeModel) {
balance := stateNode.Balance
if stateNode.Removed {
balance = "0"
}
sqw.stmts <- []byte(fmt.Sprintf(stateInsert, stateNode.BlockNumber, stateNode.HeaderID, stateNode.StateKey, stateNode.CID,
stateNode.Removed, true, balance, stateNode.Nonce, stateNode.CodeHash, stateNode.StorageRoot))
}
func (sqw *SQLWriter) upsertStorageCID(storageCID models.StorageNodeModel) {
sqw.stmts <- []byte(fmt.Sprintf(storageInsert, storageCID.BlockNumber, storageCID.HeaderID, storageCID.StateKey, storageCID.StorageKey, storageCID.CID,
storageCID.Removed, true, storageCID.Value))
}
// LoadWatchedAddresses loads watched addresses from a file
func (sqw *SQLWriter) loadWatchedAddresses() ([]common.Address, error) {
// load sql statements from watched addresses file
stmts, err := loadWatchedAddressesStatements(sqw.watchedAddressesFilePath)
if err != nil {
return nil, err
}
// extract addresses from the sql statements
watchedAddresses := []common.Address{}
for _, stmt := range stmts {
addressString, err := parseWatchedAddressStatement(stmt)
if err != nil {
return nil, err
}
watchedAddresses = append(watchedAddresses, common.HexToAddress(addressString))
}
return watchedAddresses, nil
}
// InsertWatchedAddresses inserts the given addresses in a file
func (sqw *SQLWriter) insertWatchedAddresses(args []types.WatchAddressArg, currentBlockNumber *big.Int) error {
// load sql statements from watched addresses file
stmts, err := loadWatchedAddressesStatements(sqw.watchedAddressesFilePath)
if err != nil {
return err
}
// get already watched addresses
var watchedAddresses []string
for _, stmt := range stmts {
addressString, err := parseWatchedAddressStatement(stmt)
if err != nil {
return err
}
watchedAddresses = append(watchedAddresses, addressString)
}
// append statements for new addresses to existing statements
for _, arg := range args {
// ignore if already watched
if funk.Contains(watchedAddresses, arg.Address) {
continue
}
stmt := fmt.Sprintf(watchedAddressesInsert, arg.Address, arg.CreatedAt, currentBlockNumber.Uint64())
stmts = append(stmts, stmt)
}
return dumpWatchedAddressesStatements(sqw.watchedAddressesFilePath, stmts)
}
// RemoveWatchedAddresses removes the given watched addresses from a file
func (sqw *SQLWriter) removeWatchedAddresses(args []types.WatchAddressArg) error {
// load sql statements from watched addresses file
stmts, err := loadWatchedAddressesStatements(sqw.watchedAddressesFilePath)
if err != nil {
return err
}
// get rid of statements having addresses to be removed
var filteredStmts []string
for _, stmt := range stmts {
addressString, err := parseWatchedAddressStatement(stmt)
if err != nil {
return err
}
toRemove := funk.Contains(args, func(arg types.WatchAddressArg) bool {
return arg.Address == addressString
})
if !toRemove {
filteredStmts = append(filteredStmts, stmt)
}
}
return dumpWatchedAddressesStatements(sqw.watchedAddressesFilePath, filteredStmts)
}
// SetWatchedAddresses clears and inserts the given addresses in a file
func (sqw *SQLWriter) setWatchedAddresses(args []types.WatchAddressArg, currentBlockNumber *big.Int) error {
var stmts []string
for _, arg := range args {
stmt := fmt.Sprintf(watchedAddressesInsert, arg.Address, arg.CreatedAt, currentBlockNumber.Uint64())
stmts = append(stmts, stmt)
}
return dumpWatchedAddressesStatements(sqw.watchedAddressesFilePath, stmts)
}
// loadWatchedAddressesStatements loads sql statements from the given file in a string slice
func loadWatchedAddressesStatements(filePath string) ([]string, error) {
file, err := os.Open(filePath)
if err != nil {
if errors.Is(err, os.ErrNotExist) {
return []string{}, nil
}
return nil, fmt.Errorf("error opening watched addresses file: %v", err)
}
defer file.Close()
stmts := []string{}
scanner := bufio.NewScanner(file)
for scanner.Scan() {
stmts = append(stmts, scanner.Text())
}
if err := scanner.Err(); err != nil {
return nil, fmt.Errorf("error loading watched addresses: %v", err)
}
return stmts, nil
}
// dumpWatchedAddressesStatements dumps sql statements to the given file
func dumpWatchedAddressesStatements(filePath string, stmts []string) error {
file, err := os.Create(filePath)
if err != nil {
return fmt.Errorf("error creating watched addresses file: %v", err)
}
defer file.Close()
for _, stmt := range stmts {
_, err := file.Write([]byte(stmt + "\n"))
if err != nil {
return fmt.Errorf("error inserting watched_addresses entry: %v", err)
}
}
return nil
}
// parseWatchedAddressStatement parses given sql insert statement to extract the address argument
func parseWatchedAddressStatement(stmt string) (string, error) {
parseResult, err := pg_query.Parse(stmt)
if err != nil {
return "", fmt.Errorf("error parsing sql stmt: %v", err)
}
// extract address argument from parse output for a SQL statement of form
// "INSERT INTO eth_meta.watched_addresses (address, created_at, watched_at)
// VALUES ('0xabc', '123', '130') ON CONFLICT (address) DO NOTHING;"
addressString := parseResult.Stmts[0].Stmt.GetInsertStmt().
SelectStmt.GetSelectStmt().
ValuesLists[0].GetList().
Items[0].GetAConst().
GetVal().
GetString_().
Str
return addressString, nil
}

View File

@ -0,0 +1,263 @@
// VulcanizeDB
// Copyright © 2021 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 metrics
import (
"fmt"
"strings"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
)
const (
namespace = "statediff"
)
var (
IndexerMetrics = RegisterIndexerMetrics(metrics.DefaultRegistry)
DBMetrics = RegisterDBMetrics(metrics.DefaultRegistry)
)
// Build a fully qualified metric name
func metricName(subsystem, name string) string {
if name == "" {
return ""
}
parts := []string{namespace, name}
if subsystem != "" {
parts = []string{namespace, subsystem, name}
}
// Prometheus uses _ but geth metrics uses / and replaces
return strings.Join(parts, "/")
}
type IndexerMetricsHandles struct {
// The total number of processed BlocksCounter
BlocksCounter metrics.Counter
// The total number of processed transactions
TransactionsCounter metrics.Counter
// The total number of processed receipts
ReceiptsCounter metrics.Counter
// The total number of processed logs
LogsCounter metrics.Counter
// The total number of access list entries processed
AccessListEntriesCounter metrics.Counter
// Time spent waiting for free postgres tx
FreePostgresTimer metrics.Timer
// Postgres transaction commit duration
PostgresCommitTimer metrics.Timer
// Header processing time
HeaderProcessingTimer metrics.Timer
// Uncle processing time
UncleProcessingTimer metrics.Timer
// Tx and receipt processing time
TxAndRecProcessingTimer metrics.Timer
// State, storage, and code combined processing time
StateStoreCodeProcessingTimer metrics.Timer
// Fine-grained code timers
BuildStateDiffWithIntermediateStateNodesTimer metrics.Timer
BuildStateDiffWithoutIntermediateStateNodesTimer metrics.Timer
CreatedAndUpdatedStateWithIntermediateNodesTimer metrics.Timer
DeletedOrUpdatedStateTimer metrics.Timer
BuildAccountUpdatesTimer metrics.Timer
BuildAccountCreationsTimer metrics.Timer
ResolveNodeTimer metrics.Timer
SortKeysTimer metrics.Timer
FindIntersectionTimer metrics.Timer
OutputTimer metrics.Timer
IPLDOutputTimer metrics.Timer
DifferenceIteratorNextTimer metrics.Timer
DifferenceIteratorCounter metrics.Counter
DeletedOrUpdatedStorageTimer metrics.Timer
CreatedAndUpdatedStorageTimer metrics.Timer
BuildStorageNodesIncrementalTimer metrics.Timer
BuildStateTrieObjectTimer metrics.Timer
BuildStateTrieTimer metrics.Timer
BuildStateDiffObjectTimer metrics.Timer
WriteStateDiffObjectTimer metrics.Timer
CreatedAndUpdatedStateTimer metrics.Timer
BuildStorageNodesEventualTimer metrics.Timer
BuildStorageNodesFromTrieTimer metrics.Timer
BuildRemovedAccountStorageNodesTimer metrics.Timer
BuildRemovedStorageNodesFromTrieTimer metrics.Timer
IsWatchedAddressTimer metrics.Timer
}
func RegisterIndexerMetrics(reg metrics.Registry) IndexerMetricsHandles {
ctx := IndexerMetricsHandles{
BlocksCounter: metrics.NewCounter(),
TransactionsCounter: metrics.NewCounter(),
ReceiptsCounter: metrics.NewCounter(),
LogsCounter: metrics.NewCounter(),
AccessListEntriesCounter: metrics.NewCounter(),
FreePostgresTimer: metrics.NewTimer(),
PostgresCommitTimer: metrics.NewTimer(),
HeaderProcessingTimer: metrics.NewTimer(),
UncleProcessingTimer: metrics.NewTimer(),
TxAndRecProcessingTimer: metrics.NewTimer(),
StateStoreCodeProcessingTimer: metrics.NewTimer(),
BuildStateDiffWithIntermediateStateNodesTimer: metrics.NewTimer(),
BuildStateDiffWithoutIntermediateStateNodesTimer: metrics.NewTimer(),
CreatedAndUpdatedStateWithIntermediateNodesTimer: metrics.NewTimer(),
DeletedOrUpdatedStateTimer: metrics.NewTimer(),
BuildAccountUpdatesTimer: metrics.NewTimer(),
BuildAccountCreationsTimer: metrics.NewTimer(),
ResolveNodeTimer: metrics.NewTimer(),
SortKeysTimer: metrics.NewTimer(),
FindIntersectionTimer: metrics.NewTimer(),
OutputTimer: metrics.NewTimer(),
IPLDOutputTimer: metrics.NewTimer(),
DifferenceIteratorNextTimer: metrics.NewTimer(),
DifferenceIteratorCounter: metrics.NewCounter(),
DeletedOrUpdatedStorageTimer: metrics.NewTimer(),
CreatedAndUpdatedStorageTimer: metrics.NewTimer(),
BuildStorageNodesIncrementalTimer: metrics.NewTimer(),
BuildStateTrieObjectTimer: metrics.NewTimer(),
BuildStateTrieTimer: metrics.NewTimer(),
BuildStateDiffObjectTimer: metrics.NewTimer(),
WriteStateDiffObjectTimer: metrics.NewTimer(),
CreatedAndUpdatedStateTimer: metrics.NewTimer(),
BuildStorageNodesEventualTimer: metrics.NewTimer(),
BuildStorageNodesFromTrieTimer: metrics.NewTimer(),
BuildRemovedAccountStorageNodesTimer: metrics.NewTimer(),
BuildRemovedStorageNodesFromTrieTimer: metrics.NewTimer(),
IsWatchedAddressTimer: metrics.NewTimer(),
}
subsys := "indexer"
reg.Register(metricName(subsys, "blocks"), ctx.BlocksCounter)
reg.Register(metricName(subsys, "transactions"), ctx.TransactionsCounter)
reg.Register(metricName(subsys, "receipts"), ctx.ReceiptsCounter)
reg.Register(metricName(subsys, "logs"), ctx.LogsCounter)
reg.Register(metricName(subsys, "access_list_entries"), ctx.AccessListEntriesCounter)
reg.Register(metricName(subsys, "t_free_postgres"), ctx.FreePostgresTimer)
reg.Register(metricName(subsys, "t_postgres_commit"), ctx.PostgresCommitTimer)
reg.Register(metricName(subsys, "t_header_processing"), ctx.HeaderProcessingTimer)
reg.Register(metricName(subsys, "t_uncle_processing"), ctx.UncleProcessingTimer)
reg.Register(metricName(subsys, "t_tx_receipt_processing"), ctx.TxAndRecProcessingTimer)
reg.Register(metricName(subsys, "t_state_store_code_processing"), ctx.StateStoreCodeProcessingTimer)
reg.Register(metricName(subsys, "t_build_statediff_with_intermediate_state_nodes"), ctx.BuildStateDiffWithIntermediateStateNodesTimer)
reg.Register(metricName(subsys, "t_build_statediff_without_intermediate_state_nodes"), ctx.BuildStateDiffWithoutIntermediateStateNodesTimer)
reg.Register(metricName(subsys, "t_created_and_update_state_with_intermediate_nodes"), ctx.CreatedAndUpdatedStateWithIntermediateNodesTimer)
reg.Register(metricName(subsys, "t_deleted_or_updated_state"), ctx.DeletedOrUpdatedStateTimer)
reg.Register(metricName(subsys, "t_build_account_updates"), ctx.BuildAccountUpdatesTimer)
reg.Register(metricName(subsys, "t_build_account_creations"), ctx.BuildAccountCreationsTimer)
reg.Register(metricName(subsys, "t_resolve_node"), ctx.ResolveNodeTimer)
reg.Register(metricName(subsys, "t_sort_keys"), ctx.SortKeysTimer)
reg.Register(metricName(subsys, "t_find_intersection"), ctx.FindIntersectionTimer)
reg.Register(metricName(subsys, "t_output_fn"), ctx.OutputTimer)
reg.Register(metricName(subsys, "t_ipld_output_fn"), ctx.IPLDOutputTimer)
reg.Register(metricName(subsys, "t_difference_iterator_next"), ctx.DifferenceIteratorNextTimer)
reg.Register(metricName(subsys, "difference_iterator_counter"), ctx.DifferenceIteratorCounter)
reg.Register(metricName(subsys, "t_created_and_updated_storage"), ctx.CreatedAndUpdatedStorageTimer)
reg.Register(metricName(subsys, "t_deleted_or_updated_storage"), ctx.DeletedOrUpdatedStorageTimer)
reg.Register(metricName(subsys, "t_build_storage_nodes_incremental"), ctx.BuildStorageNodesIncrementalTimer)
reg.Register(metricName(subsys, "t_build_state_trie_object"), ctx.BuildStateTrieObjectTimer)
reg.Register(metricName(subsys, "t_build_state_trie"), ctx.BuildStateTrieTimer)
reg.Register(metricName(subsys, "t_build_statediff_object"), ctx.BuildStateDiffObjectTimer)
reg.Register(metricName(subsys, "t_write_statediff_object"), ctx.WriteStateDiffObjectTimer)
reg.Register(metricName(subsys, "t_created_and_updated_state"), ctx.CreatedAndUpdatedStateTimer)
reg.Register(metricName(subsys, "t_build_storage_nodes_eventual"), ctx.BuildStorageNodesEventualTimer)
reg.Register(metricName(subsys, "t_build_storage_nodes_from_trie"), ctx.BuildStorageNodesFromTrieTimer)
reg.Register(metricName(subsys, "t_build_removed_accounts_storage_nodes"), ctx.BuildRemovedAccountStorageNodesTimer)
reg.Register(metricName(subsys, "t_build_removed_storage_nodes_from_trie"), ctx.BuildRemovedStorageNodesFromTrieTimer)
reg.Register(metricName(subsys, "t_is_watched_address"), ctx.IsWatchedAddressTimer)
log.Debug("Registering statediff indexer metrics.")
return ctx
}
type dbMetricsHandles struct {
// Maximum number of open connections to the sql
maxOpen metrics.Gauge
// The number of established connections both in use and idle
open metrics.Gauge
// The number of connections currently in use
inUse metrics.Gauge
// The number of idle connections
idle metrics.Gauge
// The total number of connections waited for
waitedFor metrics.Counter
// The total time blocked waiting for a new connection
blockedMilliseconds metrics.Counter
// The total number of connections closed due to SetMaxIdleConns
closedMaxIdle metrics.Counter
// The total number of connections closed due to SetConnMaxLifetime
closedMaxLifetime metrics.Counter
}
func RegisterDBMetrics(reg metrics.Registry) dbMetricsHandles {
ctx := dbMetricsHandles{
maxOpen: metrics.NewGauge(),
open: metrics.NewGauge(),
inUse: metrics.NewGauge(),
idle: metrics.NewGauge(),
waitedFor: metrics.NewCounter(),
blockedMilliseconds: metrics.NewCounter(),
closedMaxIdle: metrics.NewCounter(),
closedMaxLifetime: metrics.NewCounter(),
}
subsys := "connections"
reg.Register(metricName(subsys, "max_open"), ctx.maxOpen)
reg.Register(metricName(subsys, "open"), ctx.open)
reg.Register(metricName(subsys, "in_use"), ctx.inUse)
reg.Register(metricName(subsys, "idle"), ctx.idle)
reg.Register(metricName(subsys, "waited_for"), ctx.waitedFor)
reg.Register(metricName(subsys, "blocked_milliseconds"), ctx.blockedMilliseconds)
reg.Register(metricName(subsys, "closed_max_idle"), ctx.closedMaxIdle)
reg.Register(metricName(subsys, "closed_max_lifetime"), ctx.closedMaxLifetime)
log.Debug("Registering statediff DB metrics.")
return ctx
}
// DbStats interface to accommodate different concrete sql stats types
type DbStats interface {
MaxOpen() int64
Open() int64
InUse() int64
Idle() int64
WaitCount() int64
WaitDuration() time.Duration
MaxIdleClosed() int64
MaxLifetimeClosed() int64
}
func (met *dbMetricsHandles) Update(stats DbStats) {
met.maxOpen.Update(stats.MaxOpen())
met.open.Update(stats.Open())
met.inUse.Update(stats.InUse())
met.idle.Update(stats.Idle())
met.waitedFor.Inc(stats.WaitCount())
met.blockedMilliseconds.Inc(stats.WaitDuration().Milliseconds())
met.closedMaxIdle.Inc(stats.MaxIdleClosed())
met.closedMaxLifetime.Inc(stats.MaxLifetimeClosed())
}
func ReportAndUpdateDuration(msg string, start time.Time, logger log.Logger, timer metrics.Timer) {
since := UpdateDuration(start, timer)
logger.Trace(fmt.Sprintf("%s duration=%dms", msg, since.Milliseconds()))
}
func UpdateDuration(start time.Time, timer metrics.Timer) time.Duration {
since := time.Since(start)
timer.Update(since)
return since
}

View File

@ -0,0 +1,126 @@
// VulcanizeDB
// Copyright © 2021 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 sql
import (
"context"
"sync"
"sync/atomic"
"github.com/lib/pq"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
)
const startingCacheCapacity = 1024 * 24
// BatchTx wraps a sql tx with the state necessary for building the tx concurrently during trie difference iteration
type BatchTx struct {
BlockNumber string
ctx context.Context
dbtx Tx
stm string
quit chan struct{}
iplds chan models.IPLDModel
ipldCache models.IPLDBatch
removedCacheFlag *uint32
// Tracks expected cache size and ensures cache is caught up before flush
cacheWg sync.WaitGroup
submit func(blockTx *BatchTx, err error) error
}
// Submit satisfies indexer.AtomicTx
func (tx *BatchTx) Submit(err error) error {
return tx.submit(tx, err)
}
func (tx *BatchTx) flush() error {
tx.cacheWg.Wait()
_, err := tx.dbtx.Exec(tx.ctx, tx.stm, pq.Array(tx.ipldCache.BlockNumbers), pq.Array(tx.ipldCache.Keys),
pq.Array(tx.ipldCache.Values))
if err != nil {
log.Debug(insertError{"ipld.blocks", err, tx.stm,
struct {
blockNumbers []string
keys []string
values [][]byte
}{
tx.ipldCache.BlockNumbers,
tx.ipldCache.Keys,
tx.ipldCache.Values,
}}.Error())
return insertError{"ipld.blocks", err, tx.stm, "too many arguments; use debug mode for full list"}
}
tx.ipldCache = models.IPLDBatch{}
return nil
}
// run in background goroutine to synchronize concurrent appends to the ipldCache
func (tx *BatchTx) cache() {
for {
select {
case i := <-tx.iplds:
tx.ipldCache.BlockNumbers = append(tx.ipldCache.BlockNumbers, i.BlockNumber)
tx.ipldCache.Keys = append(tx.ipldCache.Keys, i.Key)
tx.ipldCache.Values = append(tx.ipldCache.Values, i.Data)
tx.cacheWg.Done()
case <-tx.quit:
tx.ipldCache = models.IPLDBatch{}
return
}
}
}
func (tx *BatchTx) cacheDirect(key string, value []byte) {
tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber,
Key: key,
Data: value,
}
}
func (tx *BatchTx) cacheIPLD(i ipld.IPLD) {
tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber,
Key: i.Cid().String(),
Data: i.RawData(),
}
}
func (tx *BatchTx) cacheRemoved(key string, value []byte) {
if atomic.LoadUint32(tx.removedCacheFlag) == 0 {
atomic.StoreUint32(tx.removedCacheFlag, 1)
tx.cacheWg.Add(1)
tx.iplds <- models.IPLDModel{
BlockNumber: tx.BlockNumber,
Key: key,
Data: value,
}
}
}
// rollback sql transaction and log any error
func rollback(ctx context.Context, tx Tx) {
if err := tx.Rollback(ctx); err != nil {
log.Error(err.Error())
}
}

View File

@ -0,0 +1,583 @@
// 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 sql provides an interface for pushing and indexing IPLD objects into a sql database
// Metrics for reporting processing and connection stats are defined in ./metrics.go
package sql
import (
"bytes"
"context"
"fmt"
"math/big"
"time"
"github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp"
metrics2 "github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
var _ interfaces.StateDiffIndexer = &StateDiffIndexer{}
// StateDiffIndexer satisfies the indexer.StateDiffIndexer interface for ethereum statediff objects on top of an SQL sql
type StateDiffIndexer struct {
ctx context.Context
chainConfig *params.ChainConfig
dbWriter *Writer
}
// NewStateDiffIndexer creates a sql implementation of interfaces.StateDiffIndexer
func NewStateDiffIndexer(ctx context.Context, chainConfig *params.ChainConfig, db Database) (*StateDiffIndexer, error) {
return &StateDiffIndexer{
ctx: ctx,
chainConfig: chainConfig,
dbWriter: NewWriter(db),
}, nil
}
// ReportDBMetrics is a reporting function to run as goroutine
func (sdi *StateDiffIndexer) ReportDBMetrics(delay time.Duration, quit <-chan bool) {
if !metrics.Enabled {
return
}
ticker := time.NewTicker(delay)
go func() {
for {
select {
case <-ticker.C:
metrics2.DBMetrics.Update(sdi.dbWriter.db.Stats())
case <-quit:
ticker.Stop()
return
}
}
}()
}
// PushBlock pushes and indexes block data in sql, except state & storage nodes (includes header, uncles, transactions & receipts)
// Returns an initiated DB transaction which must be Closed via defer to commit or rollback
func (sdi *StateDiffIndexer) PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (interfaces.Batch, error) {
start, t := time.Now(), time.Now()
blockHash := block.Hash()
blockHashStr := blockHash.String()
height := block.NumberU64()
traceMsg := fmt.Sprintf("indexer stats for statediff at %d with hash %s:\r\n", height, blockHashStr)
transactions := block.Transactions()
// Derive any missing fields
if err := receipts.DeriveFields(sdi.chainConfig, blockHash, height, block.BaseFee(), transactions); err != nil {
return nil, err
}
// Generate the block iplds
headerNode, txNodes, rctNodes, logNodes, err := ipld.FromBlockAndReceipts(block, receipts)
if err != nil {
return nil, fmt.Errorf("error creating IPLD nodes from block and receipts: %v", err)
}
if len(txNodes) != len(rctNodes) {
return nil, fmt.Errorf("expected number of transactions (%d), receipts (%d)", len(txNodes), len(rctNodes))
}
// Calculate reward
var reward *big.Int
// in PoA networks block reward is 0
if sdi.chainConfig.Clique != nil {
reward = big.NewInt(0)
} else {
reward = shared.CalcEthBlockReward(block.Header(), block.Uncles(), block.Transactions(), receipts)
}
t = time.Now()
// Begin new DB tx for everything
tx := NewDelayedTx(sdi.dbWriter.db)
defer func() {
if p := recover(); p != nil {
rollback(sdi.ctx, tx)
panic(p)
} else if err != nil {
rollback(sdi.ctx, tx)
}
}()
blockTx := &BatchTx{
removedCacheFlag: new(uint32),
ctx: sdi.ctx,
BlockNumber: block.Number().String(),
stm: sdi.dbWriter.db.InsertIPLDsStm(),
iplds: make(chan models.IPLDModel),
quit: make(chan struct{}),
ipldCache: models.IPLDBatch{
BlockNumbers: make([]string, 0, startingCacheCapacity),
Keys: make([]string, 0, startingCacheCapacity),
Values: make([][]byte, 0, startingCacheCapacity),
},
dbtx: tx,
// handle transaction commit or rollback for any return case
submit: func(self *BatchTx, err error) error {
defer func() {
close(self.quit)
close(self.iplds)
}()
if p := recover(); p != nil {
log.Info("panic detected before tx submission, rolling back the tx", "panic", p)
rollback(sdi.ctx, tx)
panic(p)
} else if err != nil {
log.Info("error detected before tx submission, rolling back the tx", "error", err)
rollback(sdi.ctx, tx)
} else {
tDiff := time.Since(t)
metrics2.IndexerMetrics.StateStoreCodeProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("state, storage, and code storage processing time: %s\r\n", tDiff.String())
t = time.Now()
if err := self.flush(); err != nil {
rollback(sdi.ctx, tx)
traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String())
log.Debug(traceMsg)
return err
}
err = tx.Commit(sdi.ctx)
tDiff = time.Since(t)
metrics2.IndexerMetrics.PostgresCommitTimer.Update(tDiff)
traceMsg += fmt.Sprintf("postgres transaction commit duration: %s\r\n", tDiff.String())
}
traceMsg += fmt.Sprintf(" TOTAL PROCESSING DURATION: %s\r\n", time.Since(start).String())
log.Debug(traceMsg)
return err
},
}
go blockTx.cache()
tDiff := time.Since(t)
metrics2.IndexerMetrics.FreePostgresTimer.Update(tDiff)
traceMsg += fmt.Sprintf("time spent waiting for free postgres tx: %s:\r\n", tDiff.String())
t = time.Now()
// Publish and index header, collect headerID
var headerID string
headerID, err = sdi.processHeader(blockTx, block.Header(), headerNode, reward, totalDifficulty)
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics2.IndexerMetrics.HeaderProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("header processing time: %s\r\n", tDiff.String())
t = time.Now()
// Publish and index uncles
err = sdi.processUncles(blockTx, headerID, block.Number(), block.UncleHash(), block.Uncles())
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics2.IndexerMetrics.UncleProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("uncle processing time: %s\r\n", tDiff.String())
t = time.Now()
// Publish and index receipts and txs
err = sdi.processReceiptsAndTxs(blockTx, processArgs{
headerID: headerID,
blockNumber: block.Number(),
receipts: receipts,
txs: transactions,
rctNodes: rctNodes,
txNodes: txNodes,
logNodes: logNodes,
})
if err != nil {
return nil, err
}
tDiff = time.Since(t)
metrics2.IndexerMetrics.TxAndRecProcessingTimer.Update(tDiff)
traceMsg += fmt.Sprintf("tx and receipt processing time: %s\r\n", tDiff.String())
t = time.Now()
return blockTx, err
}
// processHeader publishes and indexes a header IPLD in Postgres
// it returns the headerID
func (sdi *StateDiffIndexer) processHeader(tx *BatchTx, header *types.Header, headerNode ipld.IPLD, reward, td *big.Int) (string, error) {
tx.cacheIPLD(headerNode)
var baseFee *string
if header.BaseFee != nil {
baseFee = new(string)
*baseFee = header.BaseFee.String()
}
headerID := header.Hash().String()
// index header
return headerID, sdi.dbWriter.upsertHeaderCID(tx.dbtx, models.HeaderModel{
CID: headerNode.Cid().String(),
ParentHash: header.ParentHash.String(),
BlockNumber: header.Number.String(),
BlockHash: headerID,
TotalDifficulty: td.String(),
Reward: reward.String(),
Bloom: header.Bloom.Bytes(),
StateRoot: header.Root.String(),
RctRoot: header.ReceiptHash.String(),
TxRoot: header.TxHash.String(),
UnclesHash: header.UncleHash.String(),
Timestamp: header.Time,
Coinbase: header.Coinbase.String(),
})
}
// processUncles publishes and indexes uncle IPLDs in Postgres
func (sdi *StateDiffIndexer) processUncles(tx *BatchTx, headerID string, blockNumber *big.Int, unclesHash common.Hash, uncles []*types.Header) error {
// publish and index uncles
uncleEncoding, err := rlp.EncodeToBytes(uncles)
if err != nil {
return err
}
preparedHash := crypto.Keccak256Hash(uncleEncoding)
if !bytes.Equal(preparedHash.Bytes(), unclesHash.Bytes()) {
return fmt.Errorf("derived uncles hash (%s) does not match the hash in the header (%s)", preparedHash.Hex(), unclesHash.Hex())
}
unclesCID, err := ipld.RawdataToCid(ipld.MEthHeaderList, uncleEncoding, multihash.KECCAK_256)
if err != nil {
return err
}
tx.cacheDirect(unclesCID.String(), uncleEncoding)
for i, uncle := range uncles {
var uncleReward *big.Int
// in PoA networks uncle reward is 0
if sdi.chainConfig.Clique != nil {
uncleReward = big.NewInt(0)
} else {
uncleReward = shared.CalcUncleMinerReward(blockNumber.Uint64(), uncle.Number.Uint64())
}
uncle := models.UncleModel{
BlockNumber: blockNumber.String(),
HeaderID: headerID,
CID: unclesCID.String(),
ParentHash: uncle.ParentHash.String(),
BlockHash: uncle.Hash().String(),
Reward: uncleReward.String(),
Index: int64(i),
}
if err := sdi.dbWriter.upsertUncleCID(tx.dbtx, uncle); err != nil {
return err
}
}
return nil
}
// processArgs bundles arguments to processReceiptsAndTxs
type processArgs struct {
headerID string
blockNumber *big.Int
receipts types.Receipts
txs types.Transactions
rctNodes []*ipld.EthReceipt
txNodes []*ipld.EthTx
logNodes [][]*ipld.EthLog
}
// processReceiptsAndTxs publishes and indexes receipt and transaction IPLDs in Postgres
func (sdi *StateDiffIndexer) processReceiptsAndTxs(tx *BatchTx, args processArgs) error {
// Process receipts and txs
signer := types.MakeSigner(sdi.chainConfig, args.blockNumber)
for i, receipt := range args.receipts {
txNode := args.txNodes[i]
tx.cacheIPLD(txNode)
tx.cacheIPLD(args.rctNodes[i])
// index tx
trx := args.txs[i]
txID := trx.Hash().String()
var val string
if trx.Value() != nil {
val = trx.Value().String()
}
// derive sender for the tx that corresponds with this receipt
from, err := types.Sender(signer, trx)
if err != nil {
return fmt.Errorf("error deriving tx sender: %v", err)
}
txModel := models.TxModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
Dst: shared.HandleZeroAddrPointer(trx.To()),
Src: shared.HandleZeroAddr(from),
TxHash: txID,
Index: int64(i),
CID: txNode.Cid().String(),
Type: trx.Type(),
Value: val,
}
if err := sdi.dbWriter.upsertTransactionCID(tx.dbtx, txModel); err != nil {
return err
}
// this is the contract address if this receipt is for a contract creation tx
contract := shared.HandleZeroAddr(receipt.ContractAddress)
rctModel := &models.ReceiptModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
TxID: txID,
Contract: contract,
CID: args.rctNodes[i].Cid().String(),
}
if len(receipt.PostState) == 0 {
rctModel.PostStatus = receipt.Status
} else {
rctModel.PostState = common.BytesToHash(receipt.PostState).String()
}
if err := sdi.dbWriter.upsertReceiptCID(tx.dbtx, rctModel); err != nil {
return err
}
// index logs
logDataSet := make([]*models.LogsModel, len(receipt.Logs))
for idx, l := range receipt.Logs {
tx.cacheIPLD(args.logNodes[i][idx])
topicSet := make([]string, 4)
for ti, topic := range l.Topics {
topicSet[ti] = topic.Hex()
}
logDataSet[idx] = &models.LogsModel{
BlockNumber: args.blockNumber.String(),
HeaderID: args.headerID,
ReceiptID: txID,
Address: l.Address.String(),
Index: int64(l.Index),
CID: args.logNodes[i][idx].Cid().String(),
Topic0: topicSet[0],
Topic1: topicSet[1],
Topic2: topicSet[2],
Topic3: topicSet[3],
}
}
if err := sdi.dbWriter.upsertLogCID(tx.dbtx, logDataSet); err != nil {
return err
}
}
return nil
}
// PushStateNode publishes and indexes a state diff node object (including any child storage nodes) in the IPLD sql
func (sdi *StateDiffIndexer) PushStateNode(batch interfaces.Batch, stateNode sdtypes.StateLeafNode, headerID string) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("sql: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
// publish the state node
var stateModel models.StateNodeModel
if stateNode.Removed {
tx.cacheRemoved(shared.RemovedNodeStateCID, []byte{})
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: shared.RemovedNodeStateCID,
Removed: true,
}
} else {
stateModel = models.StateNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
CID: stateNode.AccountWrapper.CID,
Removed: false,
Balance: stateNode.AccountWrapper.Account.Balance.String(),
Nonce: stateNode.AccountWrapper.Account.Nonce,
CodeHash: common.BytesToHash(stateNode.AccountWrapper.Account.CodeHash).String(),
StorageRoot: stateNode.AccountWrapper.Account.Root.String(),
}
}
// index the state node
if err := sdi.dbWriter.upsertStateCID(tx.dbtx, stateModel); err != nil {
return err
}
// if there are any storage nodes associated with this node, publish and index them
for _, storageNode := range stateNode.StorageDiff {
if storageNode.Removed {
tx.cacheRemoved(shared.RemovedNodeStorageCID, []byte{})
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: shared.RemovedNodeStorageCID,
Removed: true,
Value: []byte{},
}
if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel); err != nil {
return err
}
continue
}
storageModel := models.StorageNodeModel{
BlockNumber: tx.BlockNumber,
HeaderID: headerID,
StateKey: common.BytesToHash(stateNode.AccountWrapper.LeafKey).String(),
StorageKey: common.BytesToHash(storageNode.LeafKey).String(),
CID: storageNode.CID,
Removed: false,
Value: storageNode.Value,
}
if err := sdi.dbWriter.upsertStorageCID(tx.dbtx, storageModel); err != nil {
return err
}
}
return nil
}
// PushIPLD publishes iplds to ipld.blocks
func (sdi *StateDiffIndexer) PushIPLD(batch interfaces.Batch, ipld sdtypes.IPLD) error {
tx, ok := batch.(*BatchTx)
if !ok {
return fmt.Errorf("sql: batch is expected to be of type %T, got %T", &BatchTx{}, batch)
}
tx.cacheDirect(ipld.CID, ipld.Content)
return nil
}
// Close satisfies io.Closer
func (sdi *StateDiffIndexer) Close() error {
return sdi.dbWriter.Close()
}
// Update the known gaps table with the gap information.
// LoadWatchedAddresses reads watched addresses from the database
func (sdi *StateDiffIndexer) LoadWatchedAddresses() ([]common.Address, error) {
addressStrings := make([]string, 0)
pgStr := "SELECT address FROM eth_meta.watched_addresses"
err := sdi.dbWriter.db.Select(sdi.ctx, &addressStrings, pgStr)
if err != nil {
return nil, fmt.Errorf("error loading watched addresses: %v", err)
}
watchedAddresses := []common.Address{}
for _, addressString := range addressStrings {
watchedAddresses = append(watchedAddresses, common.HexToAddress(addressString))
}
return watchedAddresses, nil
}
// InsertWatchedAddresses inserts the given addresses in the database
func (sdi *StateDiffIndexer) InsertWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) (err error) {
tx := NewDelayedTx(sdi.dbWriter.db)
defer func() {
if p := recover(); p != nil {
rollback(sdi.ctx, tx)
panic(p)
} else if err != nil {
rollback(sdi.ctx, tx)
} else {
err = tx.Commit(sdi.ctx)
}
}()
for _, arg := range args {
_, err = tx.Exec(sdi.ctx, `INSERT INTO eth_meta.watched_addresses (address, created_at, watched_at) VALUES ($1, $2, $3) ON CONFLICT (address) DO NOTHING`,
arg.Address, arg.CreatedAt, currentBlockNumber.Uint64())
if err != nil {
return fmt.Errorf("error inserting watched_addresses entry: %v", err)
}
}
return err
}
// RemoveWatchedAddresses removes the given watched addresses from the database
func (sdi *StateDiffIndexer) RemoveWatchedAddresses(args []sdtypes.WatchAddressArg) (err error) {
tx := NewDelayedTx(sdi.dbWriter.db)
defer func() {
if p := recover(); p != nil {
rollback(sdi.ctx, tx)
panic(p)
} else if err != nil {
rollback(sdi.ctx, tx)
} else {
err = tx.Commit(sdi.ctx)
}
}()
for _, arg := range args {
_, err = tx.Exec(sdi.ctx, `DELETE FROM eth_meta.watched_addresses WHERE address = $1`, arg.Address)
if err != nil {
return fmt.Errorf("error removing watched_addresses entry: %v", err)
}
}
return err
}
// SetWatchedAddresses clears and inserts the given addresses in the database
func (sdi *StateDiffIndexer) SetWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) (err error) {
tx := NewDelayedTx(sdi.dbWriter.db)
defer func() {
if p := recover(); p != nil {
rollback(sdi.ctx, tx)
panic(p)
} else if err != nil {
rollback(sdi.ctx, tx)
} else {
err = tx.Commit(sdi.ctx)
}
}()
_, err = tx.Exec(sdi.ctx, `DELETE FROM eth_meta.watched_addresses`)
if err != nil {
return fmt.Errorf("error setting watched_addresses table: %v", err)
}
for _, arg := range args {
_, err = tx.Exec(sdi.ctx, `INSERT INTO eth_meta.watched_addresses (address, created_at, watched_at) VALUES ($1, $2, $3) ON CONFLICT (address) DO NOTHING`,
arg.Address, arg.CreatedAt, currentBlockNumber.Uint64())
if err != nil {
return fmt.Errorf("error setting watched_addresses table: %v", err)
}
}
return err
}
// ClearWatchedAddresses clears all the watched addresses from the database
func (sdi *StateDiffIndexer) ClearWatchedAddresses() error {
_, err := sdi.dbWriter.db.Exec(sdi.ctx, `DELETE FROM eth_meta.watched_addresses`)
if err != nil {
return fmt.Errorf("error clearing watched_addresses table: %v", err)
}
return nil
}

View File

@ -0,0 +1,28 @@
package sql_test
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
)
var (
db sql.Database
err error
ind interfaces.StateDiffIndexer
)
func checkTxClosure(t *testing.T, idle, inUse, open int64) {
require.Equal(t, idle, db.Stats().Idle())
require.Equal(t, inUse, db.Stats().InUse())
require.Equal(t, open, db.Stats().Open())
}
func tearDown(t *testing.T) {
test_helpers.TearDownDB(t, db)
require.NoError(t, ind.Close())
}

View File

@ -0,0 +1,88 @@
// VulcanizeDB
// Copyright © 2021 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 sql
import (
"context"
"io"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
)
// Database interfaces required by the sql indexer
type Database interface {
Driver
Statements
}
// Driver interface has all the methods required by a driver implementation to support the sql indexer
type Driver interface {
UseCopyFrom() bool
QueryRow(ctx context.Context, sql string, args ...interface{}) ScannableRow
Exec(ctx context.Context, sql string, args ...interface{}) (Result, error)
Select(ctx context.Context, dest interface{}, query string, args ...interface{}) error
Get(ctx context.Context, dest interface{}, query string, args ...interface{}) error
Begin(ctx context.Context) (Tx, error)
Stats() metrics.DbStats
NodeID() string
Context() context.Context
io.Closer
}
// Statements interface to accommodate different SQL query syntax
type Statements interface {
InsertHeaderStm() string
InsertUncleStm() string
InsertTxStm() string
InsertRctStm() string
InsertLogStm() string
InsertStateStm() string
InsertStorageStm() string
InsertIPLDStm() string
InsertIPLDsStm() string
// Table/column descriptions for use with CopyFrom and similar commands.
LogTableName() []string
LogColumnNames() []string
RctTableName() []string
RctColumnNames() []string
StateTableName() []string
StateColumnNames() []string
StorageTableName() []string
StorageColumnNames() []string
TxTableName() []string
TxColumnNames() []string
}
// Tx interface to accommodate different concrete SQL transaction types
type Tx interface {
QueryRow(ctx context.Context, sql string, args ...interface{}) ScannableRow
Exec(ctx context.Context, sql string, args ...interface{}) (Result, error)
CopyFrom(ctx context.Context, tableName []string, columnNames []string, rows [][]interface{}) (int64, error)
Commit(ctx context.Context) error
Rollback(ctx context.Context) error
}
// ScannableRow interface to accommodate different concrete row types
type ScannableRow interface {
Scan(dest ...interface{}) error
}
// Result interface to accommodate different concrete result types
type Result interface {
RowsAffected() (int64, error)
}

View File

@ -0,0 +1,106 @@
package sql
import (
"context"
"reflect"
"github.com/ethereum/go-ethereum/log"
)
// Changing this to 1 would make sure only sequential COPYs were combined.
const copyFromCheckLimit = 100
type DelayedTx struct {
cache []interface{}
db Database
}
type cachedStmt struct {
sql string
args []interface{}
}
type copyFrom struct {
tableName []string
columnNames []string
rows [][]interface{}
}
func (cf *copyFrom) appendRows(rows [][]interface{}) {
cf.rows = append(cf.rows, rows...)
}
func (cf *copyFrom) matches(tableName []string, columnNames []string) bool {
return reflect.DeepEqual(cf.tableName, tableName) && reflect.DeepEqual(cf.columnNames, columnNames)
}
func NewDelayedTx(db Database) *DelayedTx {
return &DelayedTx{db: db}
}
func (tx *DelayedTx) QueryRow(ctx context.Context, sql string, args ...interface{}) ScannableRow {
return tx.db.QueryRow(ctx, sql, args...)
}
func (tx *DelayedTx) findPrevCopyFrom(tableName []string, columnNames []string, limit int) (*copyFrom, int) {
for pos, count := len(tx.cache)-1, 0; pos >= 0 && count < limit; pos, count = pos-1, count+1 {
prevCopy, ok := tx.cache[pos].(*copyFrom)
if ok && prevCopy.matches(tableName, columnNames) {
return prevCopy, count
}
}
return nil, -1
}
func (tx *DelayedTx) CopyFrom(ctx context.Context, tableName []string, columnNames []string, rows [][]interface{}) (int64, error) {
if prevCopy, distance := tx.findPrevCopyFrom(tableName, columnNames, copyFromCheckLimit); nil != prevCopy {
log.Trace("statediff lazy_tx : Appending to COPY", "table", tableName,
"current", len(prevCopy.rows), "new", len(rows), "distance", distance)
prevCopy.appendRows(rows)
} else {
tx.cache = append(tx.cache, &copyFrom{tableName, columnNames, rows})
}
return 0, nil
}
func (tx *DelayedTx) Exec(ctx context.Context, sql string, args ...interface{}) (Result, error) {
tx.cache = append(tx.cache, cachedStmt{sql, args})
return nil, nil
}
func (tx *DelayedTx) Commit(ctx context.Context) error {
base, err := tx.db.Begin(ctx)
if err != nil {
return err
}
defer func() {
if p := recover(); p != nil {
rollback(ctx, base)
panic(p)
} else if err != nil {
rollback(ctx, base)
}
}()
for _, item := range tx.cache {
switch item := item.(type) {
case *copyFrom:
_, err := base.CopyFrom(ctx, item.tableName, item.columnNames, item.rows)
if err != nil {
log.Error("COPY error", "table", item.tableName, "err", err)
return err
}
case cachedStmt:
_, err := base.Exec(ctx, item.sql, item.args...)
if err != nil {
return err
}
}
}
tx.cache = nil
return base.Commit(ctx)
}
func (tx *DelayedTx) Rollback(ctx context.Context) error {
tx.cache = nil
return nil
}

View File

@ -0,0 +1,95 @@
// VulcanizeDB
// Copyright © 2021 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 mainnet_tests
import (
"context"
"fmt"
"math/big"
"os"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
"github.com/ethereum/go-ethereum/statediff/indexer/test_helpers"
)
var (
err error
db sql.Database
ind interfaces.StateDiffIndexer
chainConf = params.MainnetChainConfig
)
func init() {
if os.Getenv("MODE") != "statediff" {
fmt.Println("Skipping statediff test")
os.Exit(0)
}
}
func TestMainnetIndexer(t *testing.T) {
conf := test_helpers.GetTestConfig()
for _, blockNumber := range test_helpers.ProblemBlocks {
conf.BlockNumber = big.NewInt(blockNumber)
tb, trs, err := test_helpers.TestBlockAndReceipts(conf)
require.NoError(t, err)
testPushBlockAndState(t, tb, trs)
}
testBlock, testReceipts, err := test_helpers.TestBlockAndReceiptsFromEnv(conf)
require.NoError(t, err)
testPushBlockAndState(t, testBlock, testReceipts)
}
func testPushBlockAndState(t *testing.T, block *types.Block, receipts types.Receipts) {
t.Run("Test PushBlock and PushStateNode", func(t *testing.T) {
setupMainnetIndexer(t)
defer checkTxClosure(t, 0, 0, 0)
defer tearDown(t)
test.TestBlock(t, ind, block, receipts)
})
}
func setupMainnetIndexer(t *testing.T) {
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
ind, err = sql.NewStateDiffIndexer(context.Background(), chainConf, db)
}
func checkTxClosure(t *testing.T, idle, inUse, open int64) {
require.Equal(t, idle, db.Stats().Idle())
require.Equal(t, inUse, db.Stats().InUse())
require.Equal(t, open, db.Stats().Open())
}
func tearDown(t *testing.T) {
test_helpers.TearDownDB(t, db)
require.NoError(t, ind.Close())
}

View File

@ -0,0 +1,52 @@
// 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 sql_test
import (
"context"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupLegacyPGXIndexer(t *testing.T) {
db, err = postgres.SetupPGXDB(postgres.TestConfig)
if err != nil {
t.Fatal(err)
}
ind, err = sql.NewStateDiffIndexer(context.Background(), test.LegacyConfig, db)
require.NoError(t, err)
}
func setupLegacyPGX(t *testing.T) {
setupLegacyPGXIndexer(t)
test.SetupLegacyTestData(t, ind)
}
func TestLegacyPGXIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs", func(t *testing.T) {
setupLegacyPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestLegacyIndexer(t, db)
})
}

View File

@ -0,0 +1,245 @@
// 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 sql_test
import (
"context"
"math/big"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupPGXIndexer(t *testing.T, config postgres.Config) {
db, err = postgres.SetupPGXDB(config)
if err != nil {
t.Fatal(err)
}
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db)
require.NoError(t, err)
}
func setupPGX(t *testing.T) {
setupPGXWithConfig(t, postgres.TestConfig)
}
func setupPGXWithConfig(t *testing.T, config postgres.Config) {
setupPGXIndexer(t, config)
test.SetupTestData(t, ind)
}
func setupPGXNonCanonical(t *testing.T) {
setupPGXIndexer(t, postgres.TestConfig)
test.SetupTestDataNonCanonical(t, ind)
}
// Test indexer for a canonical block
func TestPGXIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexHeaderIPLDs(t, db)
})
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexTransactionIPLDs(t, db)
})
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexLogIPLDs(t, db)
})
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexReceiptIPLDs(t, db)
})
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateIPLDs(t, db)
})
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
setupPGX(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStorageIPLDs(t, db)
})
t.Run("Publish and index with CopyFrom enabled.", func(t *testing.T) {
config := postgres.TestConfig
config.CopyFrom = true
setupPGXWithConfig(t, config)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateIPLDs(t, db)
test.TestPublishAndIndexStorageIPLDs(t, db)
test.TestPublishAndIndexReceiptIPLDs(t, db)
test.TestPublishAndIndexLogIPLDs(t, db)
})
}
// Test indexer for a canonical + a non-canonical block at London height + a non-canonical block at London height + 1
func TestPGXIndexerNonCanonical(t *testing.T) {
t.Run("Publish and index header", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexHeaderNonCanonical(t, db)
})
t.Run("Publish and index transactions", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexTransactionsNonCanonical(t, db)
})
t.Run("Publish and index receipts", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexReceiptsNonCanonical(t, db)
})
t.Run("Publish and index logs", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexLogsNonCanonical(t, db)
})
t.Run("Publish and index state nodes", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStateNonCanonical(t, db)
})
t.Run("Publish and index storage nodes", func(t *testing.T) {
setupPGXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
test.TestPublishAndIndexStorageNonCanonical(t, db)
})
}
func TestPGXWatchAddressMethods(t *testing.T) {
setupPGXIndexer(t, postgres.TestConfig)
defer tearDown(t)
defer checkTxClosure(t, 1, 0, 1)
t.Run("Load watched addresses (empty table)", func(t *testing.T) {
test.TestLoadEmptyWatchedAddresses(t, ind)
})
t.Run("Insert watched addresses", func(t *testing.T) {
args := mocks.GetInsertWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt1)))
require.NoError(t, err)
test.TestInsertWatchedAddresses(t, db)
})
t.Run("Insert watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetInsertAlreadyWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
test.TestInsertAlreadyWatchedAddresses(t, db)
})
t.Run("Remove watched addresses", func(t *testing.T) {
args := mocks.GetRemoveWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
test.TestRemoveWatchedAddresses(t, db)
})
t.Run("Remove watched addresses (some non-watched)", func(t *testing.T) {
args := mocks.GetRemoveNonWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
test.TestRemoveNonWatchedAddresses(t, db)
})
t.Run("Set watched addresses", func(t *testing.T) {
args := mocks.GetSetWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
test.TestSetWatchedAddresses(t, db)
})
t.Run("Set watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetSetAlreadyWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt3)))
require.NoError(t, err)
test.TestSetAlreadyWatchedAddresses(t, db)
})
t.Run("Load watched addresses", func(t *testing.T) {
test.TestLoadWatchedAddresses(t, ind)
})
t.Run("Clear watched addresses", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
test.TestClearWatchedAddresses(t, db)
})
t.Run("Clear watched addresses (empty table)", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
test.TestClearEmptyWatchedAddresses(t, db)
})
}

View File

@ -0,0 +1,140 @@
// VulcanizeDB
// Copyright © 2021 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 postgres
import (
"fmt"
"os"
"strconv"
"strings"
"time"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
)
// DriverType to explicitly type the kind of sql driver we are using
type DriverType string
const (
PGX DriverType = "PGX"
SQLX DriverType = "SQLX"
Unknown DriverType = "Unknown"
)
// Env variables
const (
DATABASE_NAME = "DATABASE_NAME"
DATABASE_HOSTNAME = "DATABASE_HOSTNAME"
DATABASE_PORT = "DATABASE_PORT"
DATABASE_USER = "DATABASE_USER"
DATABASE_PASSWORD = "DATABASE_PASSWORD"
)
// ResolveDriverType resolves a DriverType from a provided string
func ResolveDriverType(str string) (DriverType, error) {
switch strings.ToLower(str) {
case "pgx", "pgxpool":
return PGX, nil
case "sqlx":
return SQLX, nil
default:
return Unknown, fmt.Errorf("unrecognized driver type string: %s", str)
}
}
// TestConfig specifies default parameters for connecting to a testing DB
var TestConfig = Config{
Hostname: "localhost",
Port: 8077,
DatabaseName: "cerc_testing",
Username: "vdbm",
Password: "password",
Driver: SQLX,
}
// Config holds params for a Postgres db
type Config struct {
// conn string params
Hostname string
Port int
DatabaseName string
Username string
Password string
// conn settings
MaxConns int
MaxIdle int
MinConns int
MaxConnIdleTime time.Duration
MaxConnLifetime time.Duration
ConnTimeout time.Duration
LogStatements bool
// node info params
ID string
ClientName string
// driver type
Driver DriverType
// toggle on/off upserts
Upsert bool
// toggle on/off CopyFrom
CopyFrom bool
}
// Type satisfies interfaces.Config
func (c Config) Type() shared.DBType {
return shared.POSTGRES
}
// DbConnectionString constructs and returns the connection string from the config
func (c Config) DbConnectionString() string {
if len(c.Username) > 0 && len(c.Password) > 0 {
return fmt.Sprintf("postgresql://%s:%s@%s:%d/%s?sslmode=disable",
c.Username, c.Password, c.Hostname, c.Port, c.DatabaseName)
}
if len(c.Username) > 0 && len(c.Password) == 0 {
return fmt.Sprintf("postgresql://%s@%s:%d/%s?sslmode=disable",
c.Username, c.Hostname, c.Port, c.DatabaseName)
}
return fmt.Sprintf("postgresql://%s:%d/%s?sslmode=disable", c.Hostname, c.Port, c.DatabaseName)
}
func (c Config) WithEnv() (Config, error) {
if val := os.Getenv(DATABASE_NAME); val != "" {
c.DatabaseName = val
}
if val := os.Getenv(DATABASE_HOSTNAME); val != "" {
c.Hostname = val
}
if val := os.Getenv(DATABASE_PORT); val != "" {
port, err := strconv.Atoi(val)
if err != nil {
return c, err
}
c.Port = port
}
if val := os.Getenv(DATABASE_USER); val != "" {
c.Username = val
}
if val := os.Getenv(DATABASE_PASSWORD); val != "" {
c.Password = val
}
return c, nil
}

View File

@ -0,0 +1,126 @@
// VulcanizeDB
// Copyright © 2021 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 postgres
import (
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/shared/schema"
)
var _ sql.Database = &DB{}
const (
createNodeStm = `INSERT INTO nodes (genesis_block, network_id, node_id, client_name, chain_id) VALUES ($1, $2, $3, $4, $5)
ON CONFLICT (node_id) DO NOTHING`
)
// NewPostgresDB returns a postgres.DB using the provided driver
func NewPostgresDB(driver sql.Driver, upsert bool) *DB {
return &DB{upsert, driver}
}
// DB implements sql.Database using a configured driver and Postgres statement syntax
type DB struct {
upsert bool
sql.Driver
}
// InsertHeaderStm satisfies the sql.Statements interface
// Stm == Statement
func (db *DB) InsertHeaderStm() string {
return schema.TableHeader.ToInsertStatement(db.upsert)
}
// InsertUncleStm satisfies the sql.Statements interface
func (db *DB) InsertUncleStm() string {
return schema.TableUncle.ToInsertStatement(db.upsert)
}
// InsertTxStm satisfies the sql.Statements interface
func (db *DB) InsertTxStm() string {
return schema.TableTransaction.ToInsertStatement(db.upsert)
}
// InsertRctStm satisfies the sql.Statements interface
func (db *DB) InsertRctStm() string {
return schema.TableReceipt.ToInsertStatement(db.upsert)
}
// InsertLogStm satisfies the sql.Statements interface
func (db *DB) InsertLogStm() string {
return schema.TableLog.ToInsertStatement(db.upsert)
}
// InsertStateStm satisfies the sql.Statements interface
func (db *DB) InsertStateStm() string {
return schema.TableStateNode.ToInsertStatement(db.upsert)
}
// InsertStorageStm satisfies the sql.Statements interface
func (db *DB) InsertStorageStm() string {
return schema.TableStorageNode.ToInsertStatement(db.upsert)
}
// InsertIPLDStm satisfies the sql.Statements interface
func (db *DB) InsertIPLDStm() string {
return schema.TableIPLDBlock.ToInsertStatement(db.upsert)
}
// InsertIPLDsStm satisfies the sql.Statements interface
func (db *DB) InsertIPLDsStm() string {
return `INSERT INTO ipld.blocks (block_number, key, data) VALUES (unnest($1::BIGINT[]), unnest($2::TEXT[]), unnest($3::BYTEA[])) ON CONFLICT DO NOTHING`
}
func (db *DB) LogTableName() []string {
return []string{"eth", "log_cids"}
}
func (db *DB) LogColumnNames() []string {
return []string{"block_number", "header_id", "cid", "rct_id", "address", "index", "topic0", "topic1", "topic2", "topic3"}
}
func (db *DB) RctTableName() []string {
return []string{"eth", "receipt_cids"}
}
func (db *DB) RctColumnNames() []string {
return []string{"block_number", "header_id", "tx_id", "cid", "contract", "post_state", "post_status"}
}
func (db *DB) StateTableName() []string {
return []string{"eth", "state_cids"}
}
func (db *DB) StateColumnNames() []string {
return []string{"block_number", "header_id", "state_leaf_key", "cid", "diff", "balance", "nonce", "code_hash", "storage_root", "removed"}
}
func (db *DB) StorageTableName() []string {
return []string{"eth", "storage_cids"}
}
func (db *DB) StorageColumnNames() []string {
return []string{"block_number", "header_id", "state_leaf_key", "storage_leaf_key", "cid", "diff", "val", "removed"}
}
func (db *DB) TxTableName() []string {
return []string{"eth", "transaction_cids"}
}
func (db *DB) TxColumnNames() []string {
return []string{"block_number", "header_id", "tx_hash", "cid", "dst", "src", "index", "tx_type", "value"}
}

View File

@ -0,0 +1,38 @@
// 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 postgres
import (
"fmt"
)
const (
DbConnectionFailedMsg = "db connection failed"
SettingNodeFailedMsg = "unable to set db node"
)
func ErrDBConnectionFailed(connectErr error) error {
return formatError(DbConnectionFailedMsg, connectErr)
}
func ErrUnableToSetNode(setErr error) error {
return formatError(SettingNodeFailedMsg, setErr)
}
func formatError(msg string, err error) error {
return fmt.Errorf("%s: %w", msg, err)
}

View File

@ -0,0 +1,61 @@
// Copyright © 2023 Cerc
// 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 postgres
import (
"context"
"github.com/ethereum/go-ethereum/log"
"github.com/jackc/pgx/v4"
)
type LogAdapter struct {
l log.Logger
}
func NewLogAdapter(l log.Logger) *LogAdapter {
return &LogAdapter{l: l}
}
func (l *LogAdapter) Log(ctx context.Context, level pgx.LogLevel, msg string, data map[string]interface{}) {
var logger log.Logger
if data != nil {
var args = make([]interface{}, 0)
for key, value := range data {
if value != nil {
args = append(args, key, value)
}
}
logger = l.l.New(args...)
} else {
logger = l.l
}
switch level {
case pgx.LogLevelTrace:
logger.Trace(msg)
case pgx.LogLevelDebug:
logger.Debug(msg)
case pgx.LogLevelInfo:
logger.Info(msg)
case pgx.LogLevelWarn:
logger.Warn(msg)
case pgx.LogLevelError:
logger.Error(msg)
default:
logger.New("INVALID_PGX_LOG_LEVEL", level).Error(msg)
}
}

View File

@ -0,0 +1,256 @@
// 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 postgres
import (
"context"
"time"
"github.com/ethereum/go-ethereum/log"
"github.com/georgysavva/scany/pgxscan"
"github.com/jackc/pgconn"
"github.com/jackc/pgx/v4"
"github.com/jackc/pgx/v4/pgxpool"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
)
// PGXDriver driver, implements sql.Driver
type PGXDriver struct {
ctx context.Context
pool *pgxpool.Pool
nodeInfo node.Info
nodeID string
config Config
}
// ConnectPGX initializes and returns a PGX connection pool
func ConnectPGX(ctx context.Context, config Config) (*pgxpool.Pool, error) {
pgConf, err := MakeConfig(config)
if err != nil {
return nil, err
}
return pgxpool.ConnectConfig(ctx, pgConf)
}
// NewPGXDriver returns a new pgx driver
// it initializes the connection pool and creates the node info table
func NewPGXDriver(ctx context.Context, config Config, node node.Info) (*PGXDriver, error) {
dbPool, err := ConnectPGX(ctx, config)
if err != nil {
return nil, ErrDBConnectionFailed(err)
}
pg := &PGXDriver{ctx: ctx, pool: dbPool, nodeInfo: node, config: config}
nodeErr := pg.createNode()
if nodeErr != nil {
return &PGXDriver{}, ErrUnableToSetNode(nodeErr)
}
return pg, nil
}
// MakeConfig creates a pgxpool.Config from the provided Config
func MakeConfig(config Config) (*pgxpool.Config, error) {
conf, err := pgxpool.ParseConfig("")
if err != nil {
return nil, err
}
//conf.ConnConfig.BuildStatementCache = nil
conf.ConnConfig.Config.Host = config.Hostname
conf.ConnConfig.Config.Port = uint16(config.Port)
conf.ConnConfig.Config.Database = config.DatabaseName
conf.ConnConfig.Config.User = config.Username
conf.ConnConfig.Config.Password = config.Password
if config.ConnTimeout != 0 {
conf.ConnConfig.Config.ConnectTimeout = config.ConnTimeout
}
if config.MaxConns != 0 {
conf.MaxConns = int32(config.MaxConns)
}
if config.MinConns != 0 {
conf.MinConns = int32(config.MinConns)
}
if config.MaxConnLifetime != 0 {
conf.MaxConnLifetime = config.MaxConnLifetime
}
if config.MaxConnIdleTime != 0 {
conf.MaxConnIdleTime = config.MaxConnIdleTime
}
if config.LogStatements {
conf.ConnConfig.Logger = NewLogAdapter(log.New())
}
return conf, nil
}
func (pgx *PGXDriver) createNode() error {
_, err := pgx.pool.Exec(
pgx.ctx,
createNodeStm,
pgx.nodeInfo.GenesisBlock, pgx.nodeInfo.NetworkID,
pgx.nodeInfo.ID, pgx.nodeInfo.ClientName,
pgx.nodeInfo.ChainID)
if err != nil {
return ErrUnableToSetNode(err)
}
pgx.nodeID = pgx.nodeInfo.ID
return nil
}
// QueryRow satisfies sql.Database
func (pgx *PGXDriver) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow {
return pgx.pool.QueryRow(ctx, sql, args...)
}
// Exec satisfies sql.Database
func (pgx *PGXDriver) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) {
res, err := pgx.pool.Exec(ctx, sql, args...)
return resultWrapper{ct: res}, err
}
// Select satisfies sql.Database
func (pgx *PGXDriver) Select(ctx context.Context, dest interface{}, query string, args ...interface{}) error {
return pgxscan.Select(ctx, pgx.pool, dest, query, args...)
}
// Get satisfies sql.Database
func (pgx *PGXDriver) Get(ctx context.Context, dest interface{}, query string, args ...interface{}) error {
return pgxscan.Get(ctx, pgx.pool, dest, query, args...)
}
// Begin satisfies sql.Database
func (pgx *PGXDriver) Begin(ctx context.Context) (sql.Tx, error) {
tx, err := pgx.pool.Begin(ctx)
if err != nil {
return nil, err
}
return pgxTxWrapper{tx: tx}, nil
}
func (pgx *PGXDriver) Stats() metrics.DbStats {
stats := pgx.pool.Stat()
return pgxStatsWrapper{stats: stats}
}
// NodeID satisfies sql.Database
func (pgx *PGXDriver) NodeID() string {
return pgx.nodeID
}
// Close satisfies sql.Database/io.Closer
func (pgx *PGXDriver) Close() error {
pgx.pool.Close()
return nil
}
// Context satisfies sql.Database
func (pgx *PGXDriver) Context() context.Context {
return pgx.ctx
}
// HasCopy satisfies sql.Database
func (pgx *PGXDriver) UseCopyFrom() bool {
return pgx.config.CopyFrom
}
type resultWrapper struct {
ct pgconn.CommandTag
}
// RowsAffected satisfies sql.Result
func (r resultWrapper) RowsAffected() (int64, error) {
return r.ct.RowsAffected(), nil
}
type pgxStatsWrapper struct {
stats *pgxpool.Stat
}
// MaxOpen satisfies metrics.DbStats
func (s pgxStatsWrapper) MaxOpen() int64 {
return int64(s.stats.MaxConns())
}
// Open satisfies metrics.DbStats
func (s pgxStatsWrapper) Open() int64 {
return int64(s.stats.TotalConns())
}
// InUse satisfies metrics.DbStats
func (s pgxStatsWrapper) InUse() int64 {
return int64(s.stats.AcquiredConns())
}
// Idle satisfies metrics.DbStats
func (s pgxStatsWrapper) Idle() int64 {
return int64(s.stats.IdleConns())
}
// WaitCount satisfies metrics.DbStats
func (s pgxStatsWrapper) WaitCount() int64 {
return s.stats.EmptyAcquireCount()
}
// WaitDuration satisfies metrics.DbStats
func (s pgxStatsWrapper) WaitDuration() time.Duration {
return s.stats.AcquireDuration()
}
// MaxIdleClosed satisfies metrics.DbStats
func (s pgxStatsWrapper) MaxIdleClosed() int64 {
// this stat isn't supported by pgxpool, but we don't want to panic
return 0
}
// MaxLifetimeClosed satisfies metrics.DbStats
func (s pgxStatsWrapper) MaxLifetimeClosed() int64 {
return s.stats.CanceledAcquireCount()
}
type pgxTxWrapper struct {
tx pgx.Tx
}
// QueryRow satisfies sql.Tx
func (t pgxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow {
return t.tx.QueryRow(ctx, sql, args...)
}
// Exec satisfies sql.Tx
func (t pgxTxWrapper) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) {
res, err := t.tx.Exec(ctx, sql, args...)
return resultWrapper{ct: res}, err
}
// Commit satisfies sql.Tx
func (t pgxTxWrapper) Commit(ctx context.Context) error {
return t.tx.Commit(ctx)
}
// Rollback satisfies sql.Tx
func (t pgxTxWrapper) Rollback(ctx context.Context) error {
return t.tx.Rollback(ctx)
}
func (t pgxTxWrapper) CopyFrom(ctx context.Context, tableName []string, columnNames []string, rows [][]interface{}) (int64, error) {
return t.tx.CopyFrom(ctx, tableName, columnNames, pgx.CopyFromRows(rows))
}

View File

@ -0,0 +1,121 @@
// 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 postgres_test
import (
"context"
"fmt"
"math/big"
"strings"
"testing"
"github.com/jackc/pgx/v4/pgxpool"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
)
var (
pgConfig, _ = postgres.MakeConfig(postgres.TestConfig)
ctx = context.Background()
)
func expectContainsSubstring(t *testing.T, full string, sub string) {
if !strings.Contains(full, sub) {
t.Fatalf("Expected \"%v\" to contain substring \"%v\"\n", full, sub)
}
}
func TestPostgresPGX(t *testing.T) {
t.Run("connects to the sql", func(t *testing.T) {
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig)
if err != nil {
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err)
}
if dbPool == nil {
t.Fatal("DB pool is nil")
}
dbPool.Close()
})
t.Run("serializes big.Int to db", func(t *testing.T) {
// postgres driver doesn't support go big.Int type
// various casts in golang uint64, int64, overflow for
// transaction value (in wei) even though
// postgres numeric can handle an arbitrary
// sized int, so use string representation of big.Int
// and cast on insert
dbPool, err := pgxpool.ConnectConfig(context.Background(), pgConfig)
if err != nil {
t.Fatalf("failed to connect to db with connection string: %s err: %v", pgConfig.ConnString(), err)
}
defer dbPool.Close()
bi := new(big.Int)
bi.SetString("34940183920000000000", 10)
require.Equal(t, "34940183920000000000", bi.String())
defer dbPool.Exec(ctx, `DROP TABLE IF EXISTS example`)
_, err = dbPool.Exec(ctx, "CREATE TABLE example ( id INTEGER, data NUMERIC )")
if err != nil {
t.Fatal(err)
}
sqlStatement := `
INSERT INTO example (id, data)
VALUES (1, cast($1 AS NUMERIC))`
_, err = dbPool.Exec(ctx, sqlStatement, bi.String())
if err != nil {
t.Fatal(err)
}
var data string
err = dbPool.QueryRow(ctx, `SELECT cast(data AS TEXT) FROM example WHERE id = 1`).Scan(&data)
if err != nil {
t.Fatal(err)
}
require.Equal(t, data, bi.String())
actual := new(big.Int)
actual.SetString(data, 10)
require.Equal(t, bi, actual)
})
t.Run("throws error when can't connect to the database", func(t *testing.T) {
goodInfo := node.Info{GenesisBlock: "GENESIS", NetworkID: "1", ID: "x123", ClientName: "geth"}
_, err := postgres.NewPGXDriver(ctx, postgres.Config{}, goodInfo)
if err == nil {
t.Fatal("Expected an error")
}
expectContainsSubstring(t, err.Error(), postgres.DbConnectionFailedMsg)
})
t.Run("throws error when can't create node", func(t *testing.T) {
badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100))
badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"}
_, err := postgres.NewPGXDriver(ctx, postgres.TestConfig, badInfo)
if err == nil {
t.Fatal("Expected an error")
}
expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg)
})
}

View File

@ -0,0 +1,33 @@
// 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 postgres_test
import (
"fmt"
"os"
"github.com/ethereum/go-ethereum/log"
)
func init() {
if os.Getenv("MODE") != "statediff" {
fmt.Println("Skipping statediff test")
os.Exit(0)
}
log.Root().SetHandler(log.DiscardHandler())
}

View File

@ -0,0 +1,210 @@
// 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 postgres
import (
"context"
coresql "database/sql"
"errors"
"time"
"github.com/jmoiron/sqlx"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
)
// SQLXDriver driver, implements sql.Driver
type SQLXDriver struct {
ctx context.Context
db *sqlx.DB
nodeInfo node.Info
nodeID string
}
// ConnectSQLX initializes and returns a SQLX connection pool for postgres
func ConnectSQLX(ctx context.Context, config Config) (*sqlx.DB, error) {
db, err := sqlx.ConnectContext(ctx, "postgres", config.DbConnectionString())
if err != nil {
return nil, ErrDBConnectionFailed(err)
}
if config.MaxConns > 0 {
db.SetMaxOpenConns(config.MaxConns)
}
if config.MaxConnLifetime > 0 {
db.SetConnMaxLifetime(config.MaxConnLifetime)
}
db.SetMaxIdleConns(config.MaxIdle)
return db, nil
}
// NewSQLXDriver returns a new sqlx driver for Postgres
// it initializes the connection pool and creates the node info table
func NewSQLXDriver(ctx context.Context, config Config, node node.Info) (*SQLXDriver, error) {
db, err := ConnectSQLX(ctx, config)
if err != nil {
return nil, err
}
driver := &SQLXDriver{ctx: ctx, db: db, nodeInfo: node}
if err := driver.createNode(); err != nil {
return nil, err
}
return driver, nil
}
func (driver *SQLXDriver) createNode() error {
_, err := driver.db.Exec(
createNodeStm,
driver.nodeInfo.GenesisBlock,
driver.nodeInfo.NetworkID,
driver.nodeInfo.ID,
driver.nodeInfo.ClientName,
driver.nodeInfo.ChainID)
if err != nil {
return ErrUnableToSetNode(err)
}
driver.nodeID = driver.nodeInfo.ID
return nil
}
// QueryRow satisfies sql.Database
func (driver *SQLXDriver) QueryRow(_ context.Context, sql string, args ...interface{}) sql.ScannableRow {
return driver.db.QueryRowx(sql, args...)
}
// Exec satisfies sql.Database
func (driver *SQLXDriver) Exec(_ context.Context, sql string, args ...interface{}) (sql.Result, error) {
return driver.db.Exec(sql, args...)
}
// Select satisfies sql.Database
func (driver *SQLXDriver) Select(_ context.Context, dest interface{}, query string, args ...interface{}) error {
return driver.db.Select(dest, query, args...)
}
// Get satisfies sql.Database
func (driver *SQLXDriver) Get(_ context.Context, dest interface{}, query string, args ...interface{}) error {
return driver.db.Get(dest, query, args...)
}
// Begin satisfies sql.Database
func (driver *SQLXDriver) Begin(_ context.Context) (sql.Tx, error) {
tx, err := driver.db.Beginx()
if err != nil {
return nil, err
}
return sqlxTxWrapper{tx: tx}, nil
}
func (driver *SQLXDriver) Stats() metrics.DbStats {
stats := driver.db.Stats()
return sqlxStatsWrapper{stats: stats}
}
// NodeID satisfies sql.Database
func (driver *SQLXDriver) NodeID() string {
return driver.nodeID
}
// Close satisfies sql.Database/io.Closer
func (driver *SQLXDriver) Close() error {
return driver.db.Close()
}
// Context satisfies sql.Database
func (driver *SQLXDriver) Context() context.Context {
return driver.ctx
}
// HasCopy satisfies sql.Database
func (driver *SQLXDriver) UseCopyFrom() bool {
// sqlx does not currently support COPY.
return false
}
type sqlxStatsWrapper struct {
stats coresql.DBStats
}
// MaxOpen satisfies metrics.DbStats
func (s sqlxStatsWrapper) MaxOpen() int64 {
return int64(s.stats.MaxOpenConnections)
}
// Open satisfies metrics.DbStats
func (s sqlxStatsWrapper) Open() int64 {
return int64(s.stats.OpenConnections)
}
// InUse satisfies metrics.DbStats
func (s sqlxStatsWrapper) InUse() int64 {
return int64(s.stats.InUse)
}
// Idle satisfies metrics.DbStats
func (s sqlxStatsWrapper) Idle() int64 {
return int64(s.stats.Idle)
}
// WaitCount satisfies metrics.DbStats
func (s sqlxStatsWrapper) WaitCount() int64 {
return s.stats.WaitCount
}
// WaitDuration satisfies metrics.DbStats
func (s sqlxStatsWrapper) WaitDuration() time.Duration {
return s.stats.WaitDuration
}
// MaxIdleClosed satisfies metrics.DbStats
func (s sqlxStatsWrapper) MaxIdleClosed() int64 {
return s.stats.MaxIdleClosed
}
// MaxLifetimeClosed satisfies metrics.DbStats
func (s sqlxStatsWrapper) MaxLifetimeClosed() int64 {
return s.stats.MaxLifetimeClosed
}
type sqlxTxWrapper struct {
tx *sqlx.Tx
}
// QueryRow satisfies sql.Tx
func (t sqlxTxWrapper) QueryRow(ctx context.Context, sql string, args ...interface{}) sql.ScannableRow {
return t.tx.QueryRowx(sql, args...)
}
// Exec satisfies sql.Tx
func (t sqlxTxWrapper) Exec(ctx context.Context, sql string, args ...interface{}) (sql.Result, error) {
return t.tx.Exec(sql, args...)
}
// Commit satisfies sql.Tx
func (t sqlxTxWrapper) Commit(ctx context.Context) error {
return t.tx.Commit()
}
// Rollback satisfies sql.Tx
func (t sqlxTxWrapper) Rollback(ctx context.Context) error {
return t.tx.Rollback()
}
func (t sqlxTxWrapper) CopyFrom(ctx context.Context, tableName []string, columnNames []string, rows [][]interface{}) (int64, error) {
return 0, errors.New("Unsupported Operation")
}

View File

@ -0,0 +1,119 @@
// 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 postgres_test
import (
"fmt"
"math/big"
"strings"
"testing"
"github.com/jmoiron/sqlx"
_ "github.com/lib/pq"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
)
func TestPostgresSQLX(t *testing.T) {
var sqlxdb *sqlx.DB
t.Run("connects to the database", func(t *testing.T) {
var err error
connStr := postgres.TestConfig.DbConnectionString()
sqlxdb, err = sqlx.Connect("postgres", connStr)
if err != nil {
t.Fatalf("failed to connect to db with connection string: %s err: %v", connStr, err)
}
if sqlxdb == nil {
t.Fatal("DB is nil")
}
err = sqlxdb.Close()
if err != nil {
t.Fatal(err)
}
})
t.Run("serializes big.Int to db", func(t *testing.T) {
// postgres driver doesn't support go big.Int type
// various casts in golang uint64, int64, overflow for
// transaction value (in wei) even though
// postgres numeric can handle an arbitrary
// sized int, so use string representation of big.Int
// and cast on insert
connStr := postgres.TestConfig.DbConnectionString()
db, err := sqlx.Connect("postgres", connStr)
if err != nil {
t.Fatal(err)
}
defer db.Close()
bi := new(big.Int)
bi.SetString("34940183920000000000", 10)
require.Equal(t, "34940183920000000000", bi.String())
defer db.Exec(`DROP TABLE IF EXISTS example`)
_, err = db.Exec("CREATE TABLE example ( id INTEGER, data NUMERIC )")
if err != nil {
t.Fatal(err)
}
sqlStatement := `
INSERT INTO example (id, data)
VALUES (1, cast($1 AS NUMERIC))`
_, err = db.Exec(sqlStatement, bi.String())
if err != nil {
t.Fatal(err)
}
var data string
err = db.QueryRow(`SELECT data FROM example WHERE id = 1`).Scan(&data)
if err != nil {
t.Fatal(err)
}
require.Equal(t, data, bi.String())
actual := new(big.Int)
actual.SetString(data, 10)
require.Equal(t, bi, actual)
})
t.Run("throws error when can't connect to the database", func(t *testing.T) {
goodInfo := node.Info{GenesisBlock: "GENESIS", NetworkID: "1", ID: "x123", ClientName: "geth"}
_, err := postgres.NewSQLXDriver(ctx, postgres.Config{}, goodInfo)
if err == nil {
t.Fatal("Expected an error")
}
expectContainsSubstring(t, err.Error(), postgres.DbConnectionFailedMsg)
})
t.Run("throws error when can't create node", func(t *testing.T) {
badHash := fmt.Sprintf("x %s", strings.Repeat("1", 100))
badInfo := node.Info{GenesisBlock: badHash, NetworkID: "1", ID: "x123", ClientName: "geth"}
_, err := postgres.NewSQLXDriver(ctx, postgres.TestConfig, badInfo)
if err == nil {
t.Fatal("Expected an error")
}
expectContainsSubstring(t, err.Error(), postgres.SettingNodeFailedMsg)
})
}

View File

@ -0,0 +1,44 @@
// VulcanizeDB
// Copyright © 2021 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 postgres
import (
"context"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/node"
)
// SetupSQLXDB is used to setup a sqlx db for tests
func SetupSQLXDB() (sql.Database, error) {
conf := TestConfig
conf.MaxIdle = 0
driver, err := NewSQLXDriver(context.Background(), conf, node.Info{})
if err != nil {
return nil, err
}
return NewPostgresDB(driver, false), nil
}
// SetupPGXDB is used to setup a pgx db for tests
func SetupPGXDB(config Config) (sql.Database, error) {
driver, err := NewPGXDriver(context.Background(), config, node.Info{})
if err != nil {
return nil, err
}
return NewPostgresDB(driver, false), nil
}

View File

@ -0,0 +1,52 @@
// 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 sql_test
import (
"context"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupLegacySQLXIndexer(t *testing.T) {
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
ind, err = sql.NewStateDiffIndexer(context.Background(), test.LegacyConfig, db)
require.NoError(t, err)
}
func setupLegacySQLX(t *testing.T) {
setupLegacySQLXIndexer(t)
test.SetupLegacyTestData(t, ind)
}
func TestLegacySQLXIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs", func(t *testing.T) {
setupLegacySQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestLegacyIndexer(t, db)
})
}

View File

@ -0,0 +1,227 @@
// 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 sql_test
import (
"context"
"math/big"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql/postgres"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/test"
)
func setupSQLXIndexer(t *testing.T) {
db, err = postgres.SetupSQLXDB()
if err != nil {
t.Fatal(err)
}
ind, err = sql.NewStateDiffIndexer(context.Background(), mocks.TestConfig, db)
require.NoError(t, err)
}
func setupSQLX(t *testing.T) {
setupSQLXIndexer(t)
test.SetupTestData(t, ind)
}
func setupSQLXNonCanonical(t *testing.T) {
setupSQLXIndexer(t)
test.SetupTestDataNonCanonical(t, ind)
}
// Test indexer for a canonical block
func TestSQLXIndexer(t *testing.T) {
t.Run("Publish and index header IPLDs in a single tx", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexHeaderIPLDs(t, db)
})
t.Run("Publish and index transaction IPLDs in a single tx", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexTransactionIPLDs(t, db)
})
t.Run("Publish and index log IPLDs for multiple receipt of a specific block", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexLogIPLDs(t, db)
})
t.Run("Publish and index receipt IPLDs in a single tx", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexReceiptIPLDs(t, db)
})
t.Run("Publish and index state IPLDs in a single tx", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStateIPLDs(t, db)
})
t.Run("Publish and index storage IPLDs in a single tx", func(t *testing.T) {
setupSQLX(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStorageIPLDs(t, db)
})
}
// Test indexer for a canonical + a non-canonical block at London height + a non-canonical block at London height + 1
func TestSQLXIndexerNonCanonical(t *testing.T) {
t.Run("Publish and index header", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexHeaderNonCanonical(t, db)
})
t.Run("Publish and index transactions", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexTransactionsNonCanonical(t, db)
})
t.Run("Publish and index receipts", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexReceiptsNonCanonical(t, db)
})
t.Run("Publish and index logs", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexLogsNonCanonical(t, db)
})
t.Run("Publish and index state nodes", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStateNonCanonical(t, db)
})
t.Run("Publish and index storage nodes", func(t *testing.T) {
setupSQLXNonCanonical(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
test.TestPublishAndIndexStorageNonCanonical(t, db)
})
}
func TestSQLXWatchAddressMethods(t *testing.T) {
setupSQLXIndexer(t)
defer tearDown(t)
defer checkTxClosure(t, 0, 0, 0)
t.Run("Load watched addresses (empty table)", func(t *testing.T) {
test.TestLoadEmptyWatchedAddresses(t, ind)
})
t.Run("Insert watched addresses", func(t *testing.T) {
args := mocks.GetInsertWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt1)))
require.NoError(t, err)
test.TestInsertWatchedAddresses(t, db)
})
t.Run("Insert watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetInsertAlreadyWatchedAddressesArgs()
err = ind.InsertWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
test.TestInsertAlreadyWatchedAddresses(t, db)
})
t.Run("Remove watched addresses", func(t *testing.T) {
args := mocks.GetRemoveWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
test.TestRemoveWatchedAddresses(t, db)
})
t.Run("Remove watched addresses (some non-watched)", func(t *testing.T) {
args := mocks.GetRemoveNonWatchedAddressesArgs()
err = ind.RemoveWatchedAddresses(args)
require.NoError(t, err)
test.TestRemoveNonWatchedAddresses(t, db)
})
t.Run("Set watched addresses", func(t *testing.T) {
args := mocks.GetSetWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt2)))
require.NoError(t, err)
test.TestSetWatchedAddresses(t, db)
})
t.Run("Set watched addresses (some already watched)", func(t *testing.T) {
args := mocks.GetSetAlreadyWatchedAddressesArgs()
err = ind.SetWatchedAddresses(args, big.NewInt(int64(mocks.WatchedAt3)))
require.NoError(t, err)
test.TestSetAlreadyWatchedAddresses(t, db)
})
t.Run("Load watched addresses", func(t *testing.T) {
test.TestLoadWatchedAddresses(t, ind)
})
t.Run("Clear watched addresses", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
test.TestClearWatchedAddresses(t, db)
})
t.Run("Clear watched addresses (empty table)", func(t *testing.T) {
err = ind.ClearWatchedAddresses()
require.NoError(t, err)
test.TestClearEmptyWatchedAddresses(t, db)
})
}

View File

@ -0,0 +1,345 @@
// 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 sql
import (
"fmt"
"strconv"
"github.com/jackc/pgtype"
shopspring "github.com/jackc/pgtype/ext/shopspring-numeric"
"github.com/lib/pq"
"github.com/shopspring/decimal"
"github.com/ethereum/go-ethereum/statediff/indexer/database/metrics"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
)
// Writer handles processing and writing of indexed IPLD objects to Postgres
type Writer struct {
db Database
}
// NewWriter creates a new pointer to a Writer
func NewWriter(db Database) *Writer {
return &Writer{
db: db,
}
}
// Close satisfies io.Closer
func (w *Writer) Close() error {
return w.db.Close()
}
/*
INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_ids, reward, state_root, tx_root, receipt_root, uncles_hash, bloom, timestamp, coinbase)
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14)
ON CONFLICT (block_hash, block_number) DO NOTHING
*/
func (w *Writer) upsertHeaderCID(tx Tx, header models.HeaderModel) error {
nodeIDs := pq.StringArray([]string{w.db.NodeID()})
_, err := tx.Exec(w.db.Context(), w.db.InsertHeaderStm(),
header.BlockNumber,
header.BlockHash,
header.ParentHash,
header.CID,
header.TotalDifficulty,
nodeIDs,
header.Reward,
header.StateRoot,
header.TxRoot,
header.RctRoot,
header.UnclesHash,
header.Bloom,
header.Timestamp,
header.Coinbase)
if err != nil {
return insertError{"eth.header_cids", err, w.db.InsertHeaderStm(), header}
}
metrics.IndexerMetrics.BlocksCounter.Inc(1)
return nil
}
/*
INSERT INTO eth.uncle_cids (block_number, block_hash, header_id, parent_hash, cid, reward, index) VALUES ($1, $2, $3, $4, $5, $6, $7)
ON CONFLICT (block_hash, block_number) DO NOTHING
*/
func (w *Writer) upsertUncleCID(tx Tx, uncle models.UncleModel) error {
_, err := tx.Exec(w.db.Context(), w.db.InsertUncleStm(),
uncle.BlockNumber,
uncle.BlockHash,
uncle.HeaderID,
uncle.ParentHash,
uncle.CID,
uncle.Reward,
uncle.Index)
if err != nil {
return insertError{"eth.uncle_cids", err, w.db.InsertUncleStm(), uncle}
}
return nil
}
/*
INSERT INTO eth.transaction_cids (block_number, header_id, tx_hash, cid, dst, src, index, tx_type, value) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9)
ON CONFLICT (tx_hash, header_id, block_number) DO NOTHING
*/
func (w *Writer) upsertTransactionCID(tx Tx, transaction models.TxModel) error {
val := transaction.Value
if val == "" {
val = "0"
}
if w.useCopyForTx(tx) {
blockNum, err := strconv.ParseInt(transaction.BlockNumber, 10, 64)
if err != nil {
return insertError{"eth.transaction_cids", err, "COPY", transaction}
}
value, err := toNumeric(val)
if err != nil {
return insertError{"eth.transaction_cids", err, "COPY", transaction}
}
_, err = tx.CopyFrom(w.db.Context(), w.db.TxTableName(), w.db.TxColumnNames(),
toRows(toRow(blockNum, transaction.HeaderID, transaction.TxHash, transaction.CID, transaction.Dst,
transaction.Src, transaction.Index, int(transaction.Type), value)))
if err != nil {
return insertError{"eth.transaction_cids", err, "COPY", transaction}
}
} else {
_, err := tx.Exec(w.db.Context(), w.db.InsertTxStm(),
transaction.BlockNumber,
transaction.HeaderID,
transaction.TxHash,
transaction.CID,
transaction.Dst,
transaction.Src,
transaction.Index,
transaction.Type,
transaction.Value)
if err != nil {
return insertError{"eth.transaction_cids", err, w.db.InsertTxStm(), transaction}
}
}
metrics.IndexerMetrics.TransactionsCounter.Inc(1)
return nil
}
/*
INSERT INTO eth.receipt_cids (block_number, header_id, tx_id, cid, contract, post_state, post_status) VALUES ($1, $2, $3, $4, $5, $6, $7, $8)
ON CONFLICT (tx_id, header_id, block_number) DO NOTHING
*/
func (w *Writer) upsertReceiptCID(tx Tx, rct *models.ReceiptModel) error {
if w.useCopyForTx(tx) {
blockNum, err := strconv.ParseUint(rct.BlockNumber, 10, 64)
if err != nil {
return insertError{"eth.receipt_cids", err, "COPY", rct}
}
_, err = tx.CopyFrom(w.db.Context(), w.db.RctTableName(), w.db.RctColumnNames(),
toRows(toRow(blockNum, rct.HeaderID, rct.TxID, rct.CID, rct.Contract,
rct.PostState, int(rct.PostStatus))))
if err != nil {
return insertError{"eth.receipt_cids", err, "COPY", rct}
}
} else {
_, err := tx.Exec(w.db.Context(), w.db.InsertRctStm(),
rct.BlockNumber,
rct.HeaderID,
rct.TxID,
rct.CID,
rct.Contract,
rct.PostState,
rct.PostStatus)
if err != nil {
return insertError{"eth.receipt_cids", err, w.db.InsertRctStm(), *rct}
}
}
metrics.IndexerMetrics.ReceiptsCounter.Inc(1)
return nil
}
/*
INSERT INTO eth.log_cids (block_number, header_id, cid, rct_id, address, index, topic0, topic1, topic2, topic3) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)
ON CONFLICT (rct_id, index, header_id, block_number) DO NOTHING
*/
func (w *Writer) upsertLogCID(tx Tx, logs []*models.LogsModel) error {
if w.useCopyForTx(tx) {
var rows [][]interface{}
for _, log := range logs {
blockNum, err := strconv.ParseUint(log.BlockNumber, 10, 64)
if err != nil {
return insertError{"eth.log_cids", err, "COPY", log}
}
rows = append(rows, toRow(blockNum, log.HeaderID, log.CID, log.ReceiptID,
log.Address, log.Index, log.Topic0, log.Topic1, log.Topic2, log.Topic3))
}
if nil != rows && len(rows) >= 0 {
_, err := tx.CopyFrom(w.db.Context(), w.db.LogTableName(), w.db.LogColumnNames(), rows)
if err != nil {
return insertError{"eth.log_cids", err, "COPY", rows}
}
metrics.IndexerMetrics.LogsCounter.Inc(int64(len(rows)))
}
} else {
for _, log := range logs {
_, err := tx.Exec(w.db.Context(), w.db.InsertLogStm(),
log.BlockNumber,
log.HeaderID,
log.CID,
log.ReceiptID,
log.Address,
log.Index,
log.Topic0,
log.Topic1,
log.Topic2,
log.Topic3)
if err != nil {
return insertError{"eth.log_cids", err, w.db.InsertLogStm(), *log}
}
metrics.IndexerMetrics.LogsCounter.Inc(1)
}
}
return nil
}
/*
INSERT INTO eth.state_cids (block_number, header_id, state_leaf_key, cid, removed, diff, balance, nonce, code_hash, storage_root) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)
ON CONFLICT (header_id, state_leaf_key, block_number) DO NOTHING
*/
func (w *Writer) upsertStateCID(tx Tx, stateNode models.StateNodeModel) error {
bal := stateNode.Balance
if stateNode.Removed {
bal = "0"
}
if w.useCopyForTx(tx) {
blockNum, err := strconv.ParseUint(stateNode.BlockNumber, 10, 64)
if err != nil {
return insertError{"eth.state_cids", err, "COPY", stateNode}
}
balance, err := toNumeric(bal)
if err != nil {
return insertError{"eth.state_cids", err, "COPY", stateNode}
}
_, err = tx.CopyFrom(w.db.Context(), w.db.StateTableName(), w.db.StateColumnNames(),
toRows(toRow(blockNum, stateNode.HeaderID, stateNode.StateKey, stateNode.CID,
true, balance, stateNode.Nonce, stateNode.CodeHash, stateNode.StorageRoot, stateNode.Removed)))
if err != nil {
return insertError{"eth.state_cids", err, "COPY", stateNode}
}
} else {
_, err := tx.Exec(w.db.Context(), w.db.InsertStateStm(),
stateNode.BlockNumber,
stateNode.HeaderID,
stateNode.StateKey,
stateNode.CID,
true,
bal,
stateNode.Nonce,
stateNode.CodeHash,
stateNode.StorageRoot,
stateNode.Removed,
)
if err != nil {
return insertError{"eth.state_cids", err, w.db.InsertStateStm(), stateNode}
}
}
return nil
}
/*
INSERT INTO eth.storage_cids (block_number, header_id, state_leaf_key, storage_leaf_key, cid, removed, diff, val) VALUES ($1, $2, $3, $4, $5, $6, $7, $8)
ON CONFLICT (header_id, state_leaf_key, storage_leaf_key, block_number) DO NOTHING
*/
func (w *Writer) upsertStorageCID(tx Tx, storageCID models.StorageNodeModel) error {
if w.useCopyForTx(tx) {
blockNum, err := strconv.ParseUint(storageCID.BlockNumber, 10, 64)
if err != nil {
return insertError{"eth.storage_cids", err, "COPY", storageCID}
}
_, err = tx.CopyFrom(w.db.Context(), w.db.StorageTableName(), w.db.StorageColumnNames(),
toRows(toRow(blockNum, storageCID.HeaderID, storageCID.StateKey, storageCID.StorageKey, storageCID.CID,
true, storageCID.Value, storageCID.Removed)))
if err != nil {
return insertError{"eth.storage_cids", err, "COPY", storageCID}
}
} else {
_, err := tx.Exec(w.db.Context(), w.db.InsertStorageStm(),
storageCID.BlockNumber,
storageCID.HeaderID,
storageCID.StateKey,
storageCID.StorageKey,
storageCID.CID,
true,
storageCID.Value,
storageCID.Removed,
)
if err != nil {
return insertError{"eth.storage_cids", err, w.db.InsertStorageStm(), storageCID}
}
}
return nil
}
func (w *Writer) useCopyForTx(tx Tx) bool {
// Using COPY instead of INSERT only makes much sense if also using a DelayedTx, so that operations
// can be collected over time and then all submitted within in a single TX.
if _, ok := tx.(*DelayedTx); ok {
return w.db.UseCopyFrom()
}
return false
}
// combine args into a row
func toRow(args ...interface{}) []interface{} {
var row []interface{}
row = append(row, args...)
return row
}
func toNumeric(value string) (*shopspring.Numeric, error) {
decimalValue, err := decimal.NewFromString(value)
if nil != err {
return nil, err
}
return &shopspring.Numeric{Decimal: decimalValue, Status: pgtype.Present}, nil
}
// combine row (or rows) into a slice of rows for CopyFrom
func toRows(rows ...[]interface{}) [][]interface{} {
return rows
}
type insertError struct {
table string
err error
stmt string
arguments interface{}
}
var _ error = insertError{}
func (dbe insertError) Error() string {
return fmt.Sprintf("error inserting %s entry: %v\r\nstatement: %s\r\narguments: %+v",
dbe.table, dbe.err, dbe.stmt, dbe.arguments)
}

View File

@ -0,0 +1,55 @@
// VulcanizeDB
// Copyright © 2021 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 interfaces
import (
"io"
"math/big"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
// StateDiffIndexer interface required to index statediff data
type StateDiffIndexer interface {
PushBlock(block *types.Block, receipts types.Receipts, totalDifficulty *big.Int) (Batch, error)
PushStateNode(tx Batch, stateNode sdtypes.StateLeafNode, headerID string) error
PushIPLD(tx Batch, ipld sdtypes.IPLD) error
ReportDBMetrics(delay time.Duration, quit <-chan bool)
// Methods used by WatchAddress API/functionality
LoadWatchedAddresses() ([]common.Address, error)
InsertWatchedAddresses(addresses []sdtypes.WatchAddressArg, currentBlock *big.Int) error
RemoveWatchedAddresses(addresses []sdtypes.WatchAddressArg) error
SetWatchedAddresses(args []sdtypes.WatchAddressArg, currentBlockNumber *big.Int) error
ClearWatchedAddresses() error
io.Closer
}
// Batch required for indexing data atomically
type Batch interface {
Submit(err error) error
}
// Config used to configure different underlying implementations
type Config interface {
Type() shared.DBType
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

View File

@ -0,0 +1,60 @@
// 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 ipld
import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
)
// EthHeader (eth-block, codec 0x90), represents an ethereum block header
type EthHeader struct {
cid cid.Cid
rawdata []byte
}
// Static (compile time) check that EthHeader satisfies the node.Node interface.
var _ IPLD = (*EthHeader)(nil)
// NewEthHeader converts a *types.Header into an EthHeader IPLD node
func NewEthHeader(header *types.Header) (*EthHeader, error) {
headerRLP, err := rlp.EncodeToBytes(header)
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthHeader, headerRLP, mh.KECCAK_256)
if err != nil {
return nil, err
}
return &EthHeader{
cid: c,
rawdata: headerRLP,
}, nil
}
// RawData returns the binary of the RLP encode of the block header.
func (b *EthHeader) RawData() []byte {
return b.rawdata
}
// Cid returns the cid of the block header.
func (b *EthHeader) Cid() cid.Cid {
return b.cid
}

44
indexer/ipld/eth_log.go Normal file
View File

@ -0,0 +1,44 @@
package ipld
import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
)
// EthLog (eth-log, codec 0x9a), represents an ethereum block header
type EthLog struct {
rawData []byte
cid cid.Cid
}
// Static (compile time) check that EthLog satisfies the node.Node interface.
var _ IPLD = (*EthLog)(nil)
// NewLog create a new EthLog IPLD node
func NewLog(log *types.Log) (*EthLog, error) {
logRaw, err := rlp.EncodeToBytes(log)
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthLog, logRaw, mh.KECCAK_256)
if err != nil {
return nil, err
}
return &EthLog{
cid: c,
rawData: logRaw,
}, nil
}
// RawData returns the binary of the RLP encode of the log.
func (l *EthLog) RawData() []byte {
return l.rawData
}
// Cid returns the cid of the receipt log.
func (l *EthLog) Cid() cid.Cid {
return l.cid
}

View File

@ -0,0 +1,94 @@
// 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 ipld
import (
"github.com/ethereum/go-ethereum/core/types"
)
// FromBlockAndReceipts takes a block and processes it
// to return it a set of IPLD nodes for further processing.
func FromBlockAndReceipts(block *types.Block, receipts []*types.Receipt) (*EthHeader, []*EthTx, []*EthReceipt, [][]*EthLog, error) {
// Process the header
headerNode, err := NewEthHeader(block.Header())
if err != nil {
return nil, nil, nil, nil, err
}
// Process the txs
txNodes, err := processTransactions(block.Transactions())
if err != nil {
return nil, nil, nil, nil, err
}
// Process the receipts and logs
rctNodes, logNodes, err := processReceiptsAndLogs(receipts)
return headerNode, txNodes, rctNodes, logNodes, err
}
// processTransactions will take the found transactions in a parsed block body
// to return IPLD node slices for eth-tx
func processTransactions(txs []*types.Transaction) ([]*EthTx, error) {
var ethTxNodes []*EthTx
for _, tx := range txs {
ethTx, err := NewEthTx(tx)
if err != nil {
return nil, err
}
ethTxNodes = append(ethTxNodes, ethTx)
}
return ethTxNodes, nil
}
// processReceiptsAndLogs will take in receipts
// to return IPLD node slices for eth-rct and eth-log
func processReceiptsAndLogs(rcts []*types.Receipt) ([]*EthReceipt, [][]*EthLog, error) {
// Pre allocating memory.
ethRctNodes := make([]*EthReceipt, len(rcts))
ethLogNodes := make([][]*EthLog, len(rcts))
for idx, rct := range rcts {
logNodes, err := processLogs(rct.Logs)
if err != nil {
return nil, nil, err
}
ethRct, err := NewReceipt(rct)
if err != nil {
return nil, nil, err
}
ethRctNodes[idx] = ethRct
ethLogNodes[idx] = logNodes
}
return ethRctNodes, ethLogNodes, nil
}
func processLogs(logs []*types.Log) ([]*EthLog, error) {
logNodes := make([]*EthLog, len(logs))
for idx, log := range logs {
logNode, err := NewLog(log)
if err != nil {
return nil, err
}
logNodes[idx] = logNode
}
return logNodes, nil
}

View File

@ -0,0 +1,126 @@
// 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 ipld
import (
"os"
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rlp"
)
type kind string
const (
legacy kind = "legacy"
eip1559 kind = "eip2930"
)
var blockFileNames = []string{
"eth-block-12252078",
"eth-block-12365585",
"eth-block-12365586",
}
var receiptsFileNames = []string{
"eth-receipts-12252078",
"eth-receipts-12365585",
"eth-receipts-12365586",
}
var kinds = []kind{
eip1559,
eip1559,
legacy,
}
type testCase struct {
kind kind
block *types.Block
receipts types.Receipts
}
func loadBlockData(t *testing.T) []testCase {
fileDir := "./eip2930_test_data"
testCases := make([]testCase, len(blockFileNames))
for i, blockFileName := range blockFileNames {
blockRLP, err := os.ReadFile(filepath.Join(fileDir, blockFileName))
if err != nil {
t.Fatalf("failed to load blockRLP from file, err %v", err)
}
block := new(types.Block)
if err := rlp.DecodeBytes(blockRLP, block); err != nil {
t.Fatalf("failed to decode blockRLP, err %v", err)
}
receiptsFileName := receiptsFileNames[i]
receiptsRLP, err := os.ReadFile(filepath.Join(fileDir, receiptsFileName))
if err != nil {
t.Fatalf("failed to load receiptsRLP from file, err %s", err)
}
receipts := make(types.Receipts, 0)
if err := rlp.DecodeBytes(receiptsRLP, &receipts); err != nil {
t.Fatalf("failed to decode receiptsRLP, err %s", err)
}
testCases[i] = testCase{
block: block,
receipts: receipts,
kind: kinds[i],
}
}
return testCases
}
func TestFromBlockAndReceipts(t *testing.T) {
testCases := loadBlockData(t)
for _, tc := range testCases {
_, _, _, _, err := FromBlockAndReceipts(tc.block, tc.receipts)
if err != nil {
t.Fatalf("error generating IPLDs from block and receipts, err %v, kind %s, block hash %s", err, tc.kind, tc.block.Hash())
}
}
}
func TestProcessLogs(t *testing.T) {
logs := []*types.Log{mockLog1, mockLog2}
nodes, err := processLogs(logs)
require.NoError(t, err)
require.GreaterOrEqual(t, len(nodes), len(logs))
}
var (
address = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476592")
anotherAddress = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476593")
mockTopic11 = common.HexToHash("0x04")
mockTopic12 = common.HexToHash("0x06")
mockTopic21 = common.HexToHash("0x05")
mockTopic22 = common.HexToHash("0x07")
mockLog1 = &types.Log{
Address: address,
Topics: []common.Hash{mockTopic11, mockTopic12},
Data: []byte{},
}
mockLog2 = &types.Log{
Address: anotherAddress,
Topics: []common.Hash{mockTopic21, mockTopic22},
Data: []byte{},
}
)

View File

@ -0,0 +1,58 @@
// 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 ipld
import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types"
)
type EthReceipt struct {
rawdata []byte
cid cid.Cid
}
// Static (compile time) check that EthReceipt satisfies the node.Node interface.
var _ IPLD = (*EthReceipt)(nil)
// NewReceipt converts a types.ReceiptForStorage to an EthReceipt IPLD node
func NewReceipt(receipt *types.Receipt) (*EthReceipt, error) {
rctRaw, err := receipt.MarshalBinary()
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthTxReceipt, rctRaw, mh.KECCAK_256)
if err != nil {
return nil, err
}
return &EthReceipt{
cid: c,
rawdata: rctRaw,
}, nil
}
// RawData returns the binary of the RLP encode of the receipt.
func (r *EthReceipt) RawData() []byte {
return r.rawdata
}
// Cid returns the cid of the receipt.
func (r *EthReceipt) Cid() cid.Cid {
return r.cid
}

59
indexer/ipld/eth_tx.go Normal file
View File

@ -0,0 +1,59 @@
// 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 ipld
import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
"github.com/ethereum/go-ethereum/core/types"
)
// EthTx (eth-tx codec 0x93) represents an ethereum transaction
type EthTx struct {
cid cid.Cid
rawdata []byte
}
// Static (compile time) check that EthTx satisfies the node.Node interface.
var _ IPLD = (*EthTx)(nil)
// NewEthTx converts a *types.Transaction to an EthTx IPLD node
func NewEthTx(tx *types.Transaction) (*EthTx, error) {
txRaw, err := tx.MarshalBinary()
if err != nil {
return nil, err
}
c, err := RawdataToCid(MEthTx, txRaw, mh.KECCAK_256)
if err != nil {
return nil, err
}
return &EthTx{
cid: c,
rawdata: txRaw,
}, nil
}
// RawData returns the binary of the RLP encode of the transaction.
func (t *EthTx) RawData() []byte {
return t.rawdata
}
// Cid returns the cid of the transaction.
func (t *EthTx) Cid() cid.Cid {
return t.cid
}

View File

@ -0,0 +1,8 @@
package ipld
import "github.com/ipfs/go-cid"
type IPLD interface {
Cid() cid.Cid
RawData() []byte
}

66
indexer/ipld/shared.go Normal file
View File

@ -0,0 +1,66 @@
// 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 ipld
import (
"github.com/ipfs/go-cid"
mh "github.com/multiformats/go-multihash"
)
// IPLD Codecs for Ethereum
// See the authoritative document:
// https://github.com/multiformats/multicodec/blob/master/table.csv
const (
RawBinary = 0x55
MEthHeader = 0x90
MEthHeaderList = 0x91
MEthTxTrie = 0x92
MEthTx = 0x93
MEthTxReceiptTrie = 0x94
MEthTxReceipt = 0x95
MEthStateTrie = 0x96
MEthAccountSnapshot = 0x97
MEthStorageTrie = 0x98
MEthLogTrie = 0x99
MEthLog = 0x9a
)
// RawdataToCid takes the desired codec and a slice of bytes
// and returns the proper cid of the object.
func RawdataToCid(codec uint64, rawdata []byte, multiHash uint64) (cid.Cid, error) {
c, err := cid.Prefix{
Codec: codec,
Version: 1,
MhType: multiHash,
MhLength: -1,
}.Sum(rawdata)
if err != nil {
return cid.Cid{}, err
}
return c, nil
}
// Keccak256ToCid takes a keccak256 hash and returns its cid based on
// the codec given.
func Keccak256ToCid(codec uint64, h []byte) cid.Cid {
buf, err := mh.Encode(h, mh.KECCAK_256)
if err != nil {
panic(err)
}
return cid.NewCidV1(codec, buf)
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

638
indexer/mocks/test_data.go Normal file
View File

@ -0,0 +1,638 @@
// 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 (
"crypto/ecdsa"
"crypto/elliptic"
"crypto/rand"
"math/big"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff/test_helpers"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
"github.com/ethereum/go-ethereum/trie"
)
// Test variables
var (
// block data
TestConfig = params.MainnetChainConfig
BlockNumber = TestConfig.LondonBlock
// canonical block at London height
// includes 5 transactions: 3 Legacy + 1 EIP-2930 + 1 EIP-1559
MockHeader = types.Header{
Time: 0,
Number: new(big.Int).Set(BlockNumber),
Root: common.HexToHash("0x0"),
TxHash: common.HexToHash("0x0"),
ReceiptHash: common.HexToHash("0x0"),
Difficulty: big.NewInt(5000000),
Extra: []byte{},
BaseFee: big.NewInt(params.InitialBaseFee),
Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476777"),
}
MockTransactions, MockReceipts, SenderAddr = createTransactionsAndReceipts(TestConfig, BlockNumber)
MockBlock = types.NewBlock(&MockHeader, MockTransactions, nil, MockReceipts, trie.NewEmpty(nil))
MockHeaderRlp, _ = rlp.EncodeToBytes(MockBlock.Header())
// non-canonical block at London height
// includes 2nd and 5th transactions from the canonical block
MockNonCanonicalHeader = MockHeader
MockNonCanonicalBlockTransactions = types.Transactions{MockTransactions[1], MockTransactions[4]}
MockNonCanonicalBlockReceipts = createNonCanonicalBlockReceipts(TestConfig, BlockNumber, MockNonCanonicalBlockTransactions)
MockNonCanonicalBlock = types.NewBlock(&MockNonCanonicalHeader, MockNonCanonicalBlockTransactions, nil, MockNonCanonicalBlockReceipts, trie.NewEmpty(nil))
MockNonCanonicalHeaderRlp, _ = rlp.EncodeToBytes(MockNonCanonicalBlock.Header())
// non-canonical block at London height + 1
// includes 3rd and 5th transactions from the canonical block
Block2Number = big.NewInt(BlockNumber.Int64() + 1)
MockNonCanonicalHeader2 = types.Header{
Time: 0,
Number: new(big.Int).Set(Block2Number),
Root: common.HexToHash("0x0"),
TxHash: common.HexToHash("0x0"),
ReceiptHash: common.HexToHash("0x0"),
Difficulty: big.NewInt(6000000),
Extra: []byte{},
BaseFee: big.NewInt(params.InitialBaseFee),
Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476777"),
}
MockNonCanonicalBlock2Transactions = types.Transactions{MockTransactions[2], MockTransactions[4]}
MockNonCanonicalBlock2Receipts = createNonCanonicalBlockReceipts(TestConfig, Block2Number, MockNonCanonicalBlock2Transactions)
MockNonCanonicalBlock2 = types.NewBlock(&MockNonCanonicalHeader2, MockNonCanonicalBlock2Transactions, nil, MockNonCanonicalBlock2Receipts, trie.NewEmpty(nil))
MockNonCanonicalHeader2Rlp, _ = rlp.EncodeToBytes(MockNonCanonicalBlock2.Header())
Address = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476592")
AnotherAddress = common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476593")
ContractAddress = crypto.CreateAddress(SenderAddr, MockTransactions[2].Nonce())
ContractAddress2 = crypto.CreateAddress(SenderAddr, MockTransactions[3].Nonce())
MockContractByteCode = []byte{0, 1, 2, 3, 4, 5}
mockTopic11 = common.HexToHash("0x04")
mockTopic12 = common.HexToHash("0x06")
mockTopic21 = common.HexToHash("0x05")
mockTopic22 = common.HexToHash("0x07")
ExpectedPostStatus uint64 = 1
ExpectedPostState1 = common.HexToHash("0x1").String()
ExpectedPostState2 = common.HexToHash("0x2").String()
ExpectedPostState3 = common.HexToHash("0x3").String()
MockLog1 = &types.Log{
Address: Address,
Topics: []common.Hash{mockTopic11, mockTopic12},
Data: []byte{},
}
MockLog2 = &types.Log{
Address: AnotherAddress,
Topics: []common.Hash{mockTopic21, mockTopic22},
Data: []byte{},
}
MockLog3 = &types.Log{
Address: Address,
Topics: []common.Hash{mockTopic11, mockTopic22},
Data: []byte{},
}
MockLog4 = &types.Log{
Address: AnotherAddress,
Topics: []common.Hash{mockTopic21, mockTopic12},
Data: []byte{},
}
ShortLog1 = &types.Log{
Address: AnotherAddress,
Topics: []common.Hash{},
Data: []byte{},
}
ShortLog2 = &types.Log{
Address: Address,
Topics: []common.Hash{},
Data: []byte{},
}
// access list entries
AccessListEntry1 = types.AccessTuple{
Address: Address,
}
AccessListEntry2 = types.AccessTuple{
Address: AnotherAddress,
StorageKeys: []common.Hash{common.BytesToHash(StorageLeafKey), common.BytesToHash(MockStorageLeafKey)},
}
// statediff data
storageLocation = common.HexToHash("0")
StorageLeafKey = crypto.Keccak256Hash(storageLocation[:]).Bytes()
mockStorageLocation = common.HexToHash("1")
MockStorageLeafKey = crypto.Keccak256Hash(mockStorageLocation[:]).Bytes()
StorageValue = common.Hex2Bytes("01")
StoragePartialPath = common.Hex2Bytes("20290decd9548b62a8d60345a988386fc84ba6bc95484008f6362f93160ef3e563")
StorageLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
StoragePartialPath,
StorageValue,
})
StorageLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStorageTrie, crypto.Keccak256(StorageLeafNode)).String()
nonce1 = uint64(1)
ContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0"
ContractCodeHash = common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea")
ContractLeafKey = test_helpers.AddressToLeafKey(ContractAddress)
ContractAccount = &types.StateAccount{
Nonce: nonce1,
Balance: big.NewInt(0),
CodeHash: ContractCodeHash.Bytes(),
Root: common.HexToHash(ContractRoot),
}
ContractAccountRLP, _ = rlp.EncodeToBytes(ContractAccount)
ContractPartialPath = common.Hex2Bytes("3114658a74d9cc9f7acf2c5cd696c3494d7c344d78bfec3add0d91ec4e8d1c45")
ContractLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
ContractPartialPath,
ContractAccount,
})
ContractLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(ContractLeafNode)).String()
Contract2LeafKey = test_helpers.AddressToLeafKey(ContractAddress2)
storage2Location = common.HexToHash("2")
Storage2LeafKey = crypto.Keccak256Hash(storage2Location[:]).Bytes()
storage3Location = common.HexToHash("3")
Storage3LeafKey = crypto.Keccak256Hash(storage3Location[:]).Bytes()
nonce0 = uint64(0)
AccountRoot = "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"
AccountCodeHash = common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470")
AccountLeafKey = test_helpers.Account2LeafKey
RemovedLeafKey = test_helpers.Account1LeafKey
Balance, _ = new(big.Int).SetString("106387458790507306766", 10)
Account = &types.StateAccount{
Nonce: nonce0,
Balance: Balance,
CodeHash: AccountCodeHash.Bytes(),
Root: common.HexToHash(AccountRoot),
}
AccountRLP, _ = rlp.EncodeToBytes(Account)
AccountPartialPath = common.Hex2Bytes("3957f3e2f04a0764c3a0491b175f69926da61efbcc8f61fa1455fd2d2b4cdd45")
AccountLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
AccountPartialPath,
Account,
})
AccountLeafNodeCID = ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(AccountLeafNode)).String()
StateDiffs = []sdtypes.StateLeafNode{
{
AccountWrapper: sdtypes.AccountWrapper{
Account: ContractAccount,
LeafKey: ContractLeafKey,
CID: ContractLeafNodeCID,
},
Removed: false,
StorageDiff: []sdtypes.StorageLeafNode{
{
Removed: false,
LeafKey: StorageLeafKey,
Value: StorageValue,
CID: StorageLeafNodeCID,
},
{
Removed: true,
LeafKey: RemovedLeafKey,
CID: shared.RemovedNodeStorageCID,
Value: []byte{},
},
},
},
{
AccountWrapper: sdtypes.AccountWrapper{
Account: Account,
LeafKey: AccountLeafKey,
CID: AccountLeafNodeCID,
},
Removed: false,
StorageDiff: []sdtypes.StorageLeafNode{},
},
{
AccountWrapper: sdtypes.AccountWrapper{
Account: nil,
LeafKey: RemovedLeafKey,
CID: shared.RemovedNodeStateCID,
},
Removed: true,
StorageDiff: []sdtypes.StorageLeafNode{},
},
{
AccountWrapper: sdtypes.AccountWrapper{
Account: nil,
LeafKey: Contract2LeafKey,
CID: shared.RemovedNodeStateCID,
},
Removed: true,
StorageDiff: []sdtypes.StorageLeafNode{
{
Removed: true,
CID: shared.RemovedNodeStorageCID,
LeafKey: Storage2LeafKey,
Value: []byte{},
},
{
Removed: true,
CID: shared.RemovedNodeStorageCID,
LeafKey: Storage3LeafKey,
Value: []byte{},
},
},
},
}
IPLDs = []sdtypes.IPLD{
{
CID: ContractLeafNodeCID,
Content: ContractLeafNode,
},
{
CID: StorageLeafNodeCID,
Content: StorageLeafNode,
},
{
CID: shared.RemovedNodeStorageCID,
Content: []byte{},
},
{
CID: AccountLeafNodeCID,
Content: AccountLeafNode,
},
{
CID: shared.RemovedNodeStateCID,
Content: []byte{},
},
}
// Mock data for testing watched addresses methods
Contract1Address = "0x5d663F5269090bD2A7DC2390c911dF6083D7b28F"
Contract2Address = "0x6Eb7e5C66DB8af2E96159AC440cbc8CDB7fbD26B"
Contract3Address = "0xcfeB164C328CA13EFd3C77E1980d94975aDfedfc"
Contract4Address = "0x0Edf0c4f393a628DE4828B228C48175b3EA297fc"
Contract1CreatedAt = uint64(1)
Contract2CreatedAt = uint64(2)
Contract3CreatedAt = uint64(3)
Contract4CreatedAt = uint64(4)
LastFilledAt = uint64(0)
WatchedAt1 = uint64(10)
WatchedAt2 = uint64(15)
WatchedAt3 = uint64(20)
)
type LegacyData struct {
Config *params.ChainConfig
BlockNumber *big.Int
MockHeader types.Header
MockTransactions types.Transactions
MockReceipts types.Receipts
SenderAddr common.Address
MockBlock *types.Block
MockHeaderRlp []byte
Address []byte
AnotherAddress []byte
ContractAddress common.Address
MockContractByteCode []byte
MockLog1 *types.Log
MockLog2 *types.Log
StorageLeafKey []byte
MockStorageLeafKey []byte
StorageLeafNode []byte
ContractLeafKey []byte
ContractAccount []byte
ContractPartialPath []byte
ContractLeafNode []byte
AccountRoot string
AccountLeafNode []byte
StateDiffs []sdtypes.StateLeafNode
}
func NewLegacyData(config *params.ChainConfig) *LegacyData {
// Block number before london fork.
blockNumber := config.EIP155Block
mockHeader := types.Header{
Time: 0,
Number: new(big.Int).Set(blockNumber),
Root: common.HexToHash("0x0"),
TxHash: common.HexToHash("0x0"),
ReceiptHash: common.HexToHash("0x0"),
Difficulty: big.NewInt(5000000),
Extra: []byte{},
Coinbase: common.HexToAddress("0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476888"),
}
mockTransactions, mockReceipts, senderAddr := createLegacyTransactionsAndReceipts(config, blockNumber)
mockBlock := types.NewBlock(&mockHeader, mockTransactions, nil, mockReceipts, trie.NewEmpty(nil))
mockHeaderRlp, _ := rlp.EncodeToBytes(mockBlock.Header())
contractAddress := crypto.CreateAddress(senderAddr, mockTransactions[2].Nonce())
return &LegacyData{
Config: config,
BlockNumber: blockNumber,
MockHeader: mockHeader,
MockTransactions: mockTransactions,
MockReceipts: mockReceipts,
SenderAddr: senderAddr,
MockBlock: mockBlock,
MockHeaderRlp: mockHeaderRlp,
ContractAddress: contractAddress,
MockContractByteCode: MockContractByteCode,
MockLog1: MockLog1,
MockLog2: MockLog2,
StorageLeafKey: StorageLeafKey,
MockStorageLeafKey: MockStorageLeafKey,
StorageLeafNode: StorageLeafNode,
ContractLeafKey: ContractLeafKey,
ContractAccount: ContractAccountRLP,
ContractPartialPath: ContractPartialPath,
ContractLeafNode: ContractLeafNode,
AccountRoot: AccountRoot,
AccountLeafNode: AccountLeafKey,
StateDiffs: StateDiffs,
}
}
// createLegacyTransactionsAndReceipts is a helper function to generate signed mock legacy transactions and mock receipts with mock logs
func createLegacyTransactionsAndReceipts(config *params.ChainConfig, blockNumber *big.Int) (types.Transactions, types.Receipts, common.Address) {
// make transactions
trx1 := types.NewTransaction(0, Address, big.NewInt(1000), 50, big.NewInt(100), []byte{})
trx2 := types.NewTransaction(1, AnotherAddress, big.NewInt(2000), 100, big.NewInt(200), []byte{})
trx3 := types.NewContractCreation(2, big.NewInt(1500), 75, big.NewInt(150), MockContractByteCode)
transactionSigner := types.MakeSigner(config, blockNumber)
mockCurve := elliptic.P256()
mockPrvKey, err := ecdsa.GenerateKey(mockCurve, rand.Reader)
if err != nil {
log.Crit(err.Error())
}
signedTrx1, err := types.SignTx(trx1, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx2, err := types.SignTx(trx2, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx3, err := types.SignTx(trx3, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
senderAddr, err := types.Sender(transactionSigner, signedTrx1) // same for both trx
if err != nil {
log.Crit(err.Error())
}
// make receipts
mockReceipt1 := types.NewReceipt(nil, false, 50)
mockReceipt1.Logs = []*types.Log{MockLog1}
mockReceipt1.TxHash = signedTrx1.Hash()
mockReceipt2 := types.NewReceipt(common.HexToHash("0x1").Bytes(), false, 100)
mockReceipt2.Logs = []*types.Log{MockLog2, ShortLog1}
mockReceipt2.TxHash = signedTrx2.Hash()
mockReceipt3 := types.NewReceipt(common.HexToHash("0x2").Bytes(), false, 75)
mockReceipt3.Logs = []*types.Log{}
mockReceipt3.TxHash = signedTrx3.Hash()
return types.Transactions{signedTrx1, signedTrx2, signedTrx3}, types.Receipts{mockReceipt1, mockReceipt2, mockReceipt3}, senderAddr
}
// createTransactionsAndReceipts is a helper function to generate signed mock transactions and mock receipts with mock logs
func createTransactionsAndReceipts(config *params.ChainConfig, blockNumber *big.Int) (types.Transactions, types.Receipts, common.Address) {
// make transactions
trx1 := types.NewTransaction(0, Address, big.NewInt(1000), 50, big.NewInt(100), []byte{})
trx2 := types.NewTransaction(1, AnotherAddress, big.NewInt(2000), 100, big.NewInt(200), []byte{})
trx3 := types.NewContractCreation(2, big.NewInt(1500), 75, big.NewInt(150), MockContractByteCode)
trx4 := types.NewTx(&types.AccessListTx{
ChainID: config.ChainID,
Nonce: 0,
GasPrice: big.NewInt(100),
Gas: 50,
To: &AnotherAddress,
Value: big.NewInt(999),
Data: []byte{},
AccessList: types.AccessList{
AccessListEntry1,
AccessListEntry2,
},
})
trx5 := types.NewTx(&types.DynamicFeeTx{
ChainID: config.ChainID,
Nonce: 0,
GasTipCap: big.NewInt(100),
GasFeeCap: big.NewInt(100),
Gas: 50,
To: &AnotherAddress,
Value: big.NewInt(1000),
Data: []byte{},
AccessList: types.AccessList{
AccessListEntry1,
AccessListEntry2,
},
})
transactionSigner := types.MakeSigner(config, blockNumber)
mockCurve := elliptic.P256()
mockPrvKey, err := ecdsa.GenerateKey(mockCurve, rand.Reader)
if err != nil {
log.Crit(err.Error())
}
signedTrx1, err := types.SignTx(trx1, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx2, err := types.SignTx(trx2, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx3, err := types.SignTx(trx3, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx4, err := types.SignTx(trx4, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx5, err := types.SignTx(trx5, transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
senderAddr, err := types.Sender(transactionSigner, signedTrx1) // same for both trx
if err != nil {
log.Crit(err.Error())
}
// make receipts
mockReceipt1 := types.NewReceipt(nil, false, 50)
mockReceipt1.Logs = []*types.Log{MockLog1}
mockReceipt1.TxHash = signedTrx1.Hash()
mockReceipt2 := types.NewReceipt(common.HexToHash("0x1").Bytes(), false, 100)
mockReceipt2.Logs = []*types.Log{MockLog2, ShortLog1}
mockReceipt2.TxHash = signedTrx2.Hash()
mockReceipt3 := types.NewReceipt(common.HexToHash("0x2").Bytes(), false, 75)
mockReceipt3.Logs = []*types.Log{}
mockReceipt3.TxHash = signedTrx3.Hash()
mockReceipt4 := &types.Receipt{
Type: types.AccessListTxType,
PostState: common.HexToHash("0x3").Bytes(),
Status: types.ReceiptStatusSuccessful,
CumulativeGasUsed: 175,
Logs: []*types.Log{MockLog3, MockLog4, ShortLog2},
TxHash: signedTrx4.Hash(),
}
mockReceipt5 := &types.Receipt{
Type: types.DynamicFeeTxType,
PostState: common.HexToHash("0x3").Bytes(),
Status: types.ReceiptStatusSuccessful,
CumulativeGasUsed: 175,
Logs: []*types.Log{},
TxHash: signedTrx5.Hash(),
}
return types.Transactions{signedTrx1, signedTrx2, signedTrx3, signedTrx4, signedTrx5}, types.Receipts{mockReceipt1, mockReceipt2, mockReceipt3, mockReceipt4, mockReceipt5}, senderAddr
}
// createNonCanonicalBlockReceipts is a helper function to generate mock receipts with mock logs for non-canonical blocks
func createNonCanonicalBlockReceipts(config *params.ChainConfig, blockNumber *big.Int, transactions types.Transactions) types.Receipts {
transactionSigner := types.MakeSigner(config, blockNumber)
mockCurve := elliptic.P256()
mockPrvKey, err := ecdsa.GenerateKey(mockCurve, rand.Reader)
if err != nil {
log.Crit(err.Error())
}
signedTrx0, err := types.SignTx(transactions[0], transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
signedTrx1, err := types.SignTx(transactions[1], transactionSigner, mockPrvKey)
if err != nil {
log.Crit(err.Error())
}
mockReceipt0 := types.NewReceipt(common.HexToHash("0x3").Bytes(), false, 300)
mockReceipt0.Logs = []*types.Log{MockLog1, ShortLog1}
mockReceipt0.TxHash = signedTrx0.Hash()
mockReceipt1 := &types.Receipt{
Type: types.DynamicFeeTxType,
PostState: common.HexToHash("0x4").Bytes(),
Status: types.ReceiptStatusSuccessful,
CumulativeGasUsed: 300,
Logs: []*types.Log{},
TxHash: signedTrx1.Hash(),
}
return types.Receipts{mockReceipt0, mockReceipt1}
}
// Helper methods for testing watched addresses methods
func GetInsertWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract1Address,
CreatedAt: Contract1CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
}
}
func GetInsertAlreadyWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract3Address,
CreatedAt: Contract3CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
}
}
func GetRemoveWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract3Address,
CreatedAt: Contract3CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
}
}
func GetRemoveNonWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract1Address,
CreatedAt: Contract1CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
}
}
func GetSetWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract1Address,
CreatedAt: Contract1CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
{
Address: Contract3Address,
CreatedAt: Contract3CreatedAt,
},
}
}
func GetSetAlreadyWatchedAddressesArgs() []sdtypes.WatchAddressArg {
return []sdtypes.WatchAddressArg{
{
Address: Contract4Address,
CreatedAt: Contract4CreatedAt,
},
{
Address: Contract2Address,
CreatedAt: Contract2CreatedAt,
},
{
Address: Contract3Address,
CreatedAt: Contract3CreatedAt,
},
}
}

24
indexer/models/batch.go Normal file
View File

@ -0,0 +1,24 @@
// VulcanizeDB
// Copyright © 2021 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 models
// IPLDBatch holds the arguments for a batch insert of IPLD data
type IPLDBatch struct {
BlockNumbers []string
Keys []string
Values [][]byte
}

119
indexer/models/models.go Normal file
View File

@ -0,0 +1,119 @@
// 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 models
import "github.com/lib/pq"
// IPLDModel is the db model for ipld.blocks
type IPLDModel struct {
BlockNumber string `db:"block_number"`
Key string `db:"key"`
Data []byte `db:"data"`
}
// HeaderModel is the db model for eth.header_cids
type HeaderModel struct {
BlockNumber string `db:"block_number"`
BlockHash string `db:"block_hash"`
ParentHash string `db:"parent_hash"`
CID string `db:"cid"`
TotalDifficulty string `db:"td"`
NodeIDs pq.StringArray `db:"node_ids"`
Reward string `db:"reward"`
StateRoot string `db:"state_root"`
UnclesHash string `db:"uncles_hash"`
TxRoot string `db:"tx_root"`
RctRoot string `db:"receipt_root"`
Bloom []byte `db:"bloom"`
Timestamp uint64 `db:"timestamp"`
Coinbase string `db:"coinbase"`
}
// UncleModel is the db model for eth.uncle_cids
type UncleModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
BlockHash string `db:"block_hash"`
ParentHash string `db:"parent_hash"`
CID string `db:"cid"`
Reward string `db:"reward"`
Index int64 `db:"index"`
}
// TxModel is the db model for eth.transaction_cids
type TxModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
Index int64 `db:"index"`
TxHash string `db:"tx_hash"`
CID string `db:"cid"`
Dst string `db:"dst"`
Src string `db:"src"`
Type uint8 `db:"tx_type"`
Value string `db:"value"`
}
// ReceiptModel is the db model for eth.receipt_cids
type ReceiptModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
TxID string `db:"tx_id"`
CID string `db:"cid"`
PostStatus uint64 `db:"post_status"`
PostState string `db:"post_state"`
Contract string `db:"contract"`
}
// StateNodeModel is the db model for eth.state_cids
type StateNodeModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
StateKey string `db:"state_leaf_key"`
Removed bool `db:"removed"`
CID string `db:"cid"`
Diff bool `db:"diff"`
Balance string `db:"balance"`
Nonce uint64 `db:"nonce"`
CodeHash string `db:"code_hash"`
StorageRoot string `db:"storage_root"`
}
// StorageNodeModel is the db model for eth.storage_cids
type StorageNodeModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
StateKey string `db:"state_leaf_key"`
StorageKey string `db:"storage_leaf_key"`
Removed bool `db:"removed"`
CID string `db:"cid"`
Diff bool `db:"diff"`
Value []byte `db:"val"`
}
// LogsModel is the db model for eth.logs
type LogsModel struct {
BlockNumber string `db:"block_number"`
HeaderID string `db:"header_id"`
ReceiptID string `db:"rct_id"`
CID string `db:"cid"`
Address string `db:"address"`
Index int64 `db:"index"`
Topic0 string `db:"topic0"`
Topic1 string `db:"topic1"`
Topic2 string `db:"topic2"`
Topic3 string `db:"topic3"`
}

25
indexer/node/node.go Normal file
View File

@ -0,0 +1,25 @@
// 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 node
type Info struct {
GenesisBlock string
NetworkID string
ChainID uint64
ID string
ClientName string
}

View File

@ -0,0 +1,22 @@
// VulcanizeDB
// Copyright © 2021 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 shared
const (
RemovedNodeStorageCID = "bagmacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya"
RemovedNodeStateCID = "baglacgzayxjemamg64rtzet6pwznzrydydsqbnstzkbcoo337lmaixmfurya"
)

46
indexer/shared/db_kind.go Normal file
View File

@ -0,0 +1,46 @@
// VulcanizeDB
// Copyright © 2021 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 shared
import (
"fmt"
"strings"
)
// DBType to explicitly type the kind of DB
type DBType string
const (
POSTGRES DBType = "Postgres"
DUMP DBType = "Dump"
FILE DBType = "File"
UNKNOWN DBType = "Unknown"
)
// ResolveDBType resolves a DBType from a provided string
func ResolveDBType(str string) (DBType, error) {
switch strings.ToLower(str) {
case "postgres", "pg":
return POSTGRES, nil
case "dump", "d":
return DUMP, nil
case "file", "f", "fs":
return FILE, nil
default:
return UNKNOWN, fmt.Errorf("unrecognized db type string: %s", str)
}
}

View File

@ -0,0 +1,37 @@
// 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 shared
import (
"github.com/ethereum/go-ethereum/common"
)
// HandleZeroAddrPointer will return an empty string for a nil address pointer
func HandleZeroAddrPointer(to *common.Address) string {
if to == nil {
return ""
}
return to.Hex()
}
// HandleZeroAddr will return an empty string for a 0 value address
func HandleZeroAddr(to common.Address) string {
if to.Hex() == "0x0000000000000000000000000000000000000000" {
return ""
}
return to.Hex()
}

76
indexer/shared/reward.go Normal file
View File

@ -0,0 +1,76 @@
// 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 shared
import (
"math/big"
"github.com/ethereum/go-ethereum/core/types"
)
func CalcEthBlockReward(header *types.Header, uncles []*types.Header, txs types.Transactions, receipts types.Receipts) *big.Int {
staticBlockReward := staticRewardByBlockNumber(header.Number.Uint64())
transactionFees := calcEthTransactionFees(txs, receipts)
uncleInclusionRewards := calcEthUncleInclusionRewards(header, uncles)
tmp := transactionFees.Add(transactionFees, uncleInclusionRewards)
return tmp.Add(tmp, staticBlockReward)
}
func CalcUncleMinerReward(blockNumber, uncleBlockNumber uint64) *big.Int {
staticBlockReward := staticRewardByBlockNumber(blockNumber)
rewardDiv8 := staticBlockReward.Div(staticBlockReward, big.NewInt(8))
mainBlock := new(big.Int).SetUint64(blockNumber)
uncleBlock := new(big.Int).SetUint64(uncleBlockNumber)
uncleBlockPlus8 := uncleBlock.Add(uncleBlock, big.NewInt(8))
uncleBlockPlus8MinusMainBlock := uncleBlockPlus8.Sub(uncleBlockPlus8, mainBlock)
return rewardDiv8.Mul(rewardDiv8, uncleBlockPlus8MinusMainBlock)
}
func staticRewardByBlockNumber(blockNumber uint64) *big.Int {
staticBlockReward := new(big.Int)
//https://blog.ethereum.org/2017/10/12/byzantium-hf-announcement/
if blockNumber >= 7280000 {
staticBlockReward.SetString("2000000000000000000", 10)
} else if blockNumber >= 4370000 {
staticBlockReward.SetString("3000000000000000000", 10)
} else {
staticBlockReward.SetString("5000000000000000000", 10)
}
return staticBlockReward
}
func calcEthTransactionFees(txs types.Transactions, receipts types.Receipts) *big.Int {
transactionFees := new(big.Int)
for i, transaction := range txs {
receipt := receipts[i]
gasPrice := big.NewInt(transaction.GasPrice().Int64())
gasUsed := big.NewInt(int64(receipt.GasUsed))
transactionFee := gasPrice.Mul(gasPrice, gasUsed)
transactionFees = transactionFees.Add(transactionFees, transactionFee)
}
return transactionFees
}
func calcEthUncleInclusionRewards(header *types.Header, uncles []*types.Header) *big.Int {
uncleInclusionRewards := new(big.Int)
for range uncles {
staticBlockReward := staticRewardByBlockNumber(header.Number.Uint64())
staticBlockReward.Div(staticBlockReward, big.NewInt(32))
uncleInclusionRewards.Add(uncleInclusionRewards, staticBlockReward)
}
return uncleInclusionRewards
}

View File

@ -0,0 +1,174 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package schema
var TableIPLDBlock = Table{
Name: `ipld.blocks`,
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "key", Type: Dtext},
{Name: "data", Type: Dbytea},
},
UpsertClause: OnConflict("block_number", "key"),
}
var TableNodeInfo = Table{
Name: `public.nodes`,
Columns: []Column{
{Name: "genesis_block", Type: Dvarchar},
{Name: "network_id", Type: Dvarchar},
{Name: "node_id", Type: Dvarchar},
{Name: "client_name", Type: Dvarchar},
{Name: "chain_id", Type: Dinteger},
},
}
var TableHeader = Table{
Name: "eth.header_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "block_hash", Type: Dvarchar},
{Name: "parent_hash", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "td", Type: Dnumeric},
{Name: "node_ids", Type: Dvarchar, Array: true},
{Name: "reward", Type: Dnumeric},
{Name: "state_root", Type: Dvarchar},
{Name: "tx_root", Type: Dvarchar},
{Name: "receipt_root", Type: Dvarchar},
{Name: "uncles_hash", Type: Dvarchar},
{Name: "bloom", Type: Dbytea},
{Name: "timestamp", Type: Dnumeric},
{Name: "coinbase", Type: Dvarchar},
},
UpsertClause: OnConflict("block_number", "block_hash").Set(
"parent_hash",
"cid",
"td",
"node_ids",
"reward",
"state_root",
"tx_root",
"receipt_root",
"uncles_hash",
"bloom",
"timestamp",
"coinbase",
)}
var TableStateNode = Table{
Name: "eth.state_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "header_id", Type: Dvarchar},
{Name: "state_leaf_key", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "diff", Type: Dboolean},
{Name: "balance", Type: Dnumeric},
{Name: "nonce", Type: Dbigint},
{Name: "code_hash", Type: Dvarchar},
{Name: "storage_root", Type: Dvarchar},
{Name: "removed", Type: Dboolean},
},
UpsertClause: OnConflict("block_number", "header_id", "state_leaf_key"),
}
var TableStorageNode = Table{
Name: "eth.storage_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "header_id", Type: Dvarchar},
{Name: "state_leaf_key", Type: Dvarchar},
{Name: "storage_leaf_key", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "diff", Type: Dboolean},
{Name: "val", Type: Dbytea},
{Name: "removed", Type: Dboolean},
},
UpsertClause: OnConflict("block_number", "header_id", "state_leaf_key", "storage_leaf_key"),
}
var TableUncle = Table{
Name: "eth.uncle_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "block_hash", Type: Dvarchar},
{Name: "header_id", Type: Dvarchar},
{Name: "parent_hash", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "reward", Type: Dnumeric},
{Name: "index", Type: Dinteger},
},
UpsertClause: OnConflict("block_number", "block_hash"),
}
var TableTransaction = Table{
Name: "eth.transaction_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "header_id", Type: Dvarchar},
{Name: "tx_hash", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "dst", Type: Dvarchar},
{Name: "src", Type: Dvarchar},
{Name: "index", Type: Dinteger},
{Name: "tx_type", Type: Dinteger},
{Name: "value", Type: Dnumeric},
},
UpsertClause: OnConflict("block_number", "header_id", "tx_hash"),
}
var TableReceipt = Table{
Name: "eth.receipt_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "header_id", Type: Dvarchar},
{Name: "tx_id", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "contract", Type: Dvarchar},
{Name: "post_state", Type: Dvarchar},
{Name: "post_status", Type: Dinteger},
},
UpsertClause: OnConflict("block_number", "header_id", "tx_id"),
}
var TableLog = Table{
Name: "eth.log_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "header_id", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "rct_id", Type: Dvarchar},
{Name: "address", Type: Dvarchar},
{Name: "index", Type: Dinteger},
{Name: "topic0", Type: Dvarchar},
{Name: "topic1", Type: Dvarchar},
{Name: "topic2", Type: Dvarchar},
{Name: "topic3", Type: Dvarchar},
},
UpsertClause: OnConflict("block_number", "header_id", "rct_id", "index"),
}
var TableWatchedAddresses = Table{
Name: "eth_meta.watched_addresses",
Columns: []Column{
{Name: "address", Type: Dvarchar},
{Name: "created_at", Type: Dbigint},
{Name: "watched_at", Type: Dbigint},
{Name: "last_filled_at", Type: Dbigint},
},
}

View File

@ -0,0 +1,147 @@
// Copyright 2022 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package schema
import (
"fmt"
"strings"
"github.com/thoas/go-funk"
)
type colType int
const (
Dinteger colType = iota
Dboolean
Dbigint
Dnumeric
Dbytea
Dvarchar
Dtext
)
type ConflictClause struct {
Target []string
Update []string
}
type Column struct {
Name string
Type colType
Array bool
}
type Table struct {
Name string
Columns []Column
UpsertClause ConflictClause
}
type colfmt = func(interface{}) string
func (tbl *Table) ToCsvRow(args ...interface{}) []string {
var row []string
for i, col := range tbl.Columns {
value := col.Type.formatter()(args[i])
if col.Array {
valueList := funk.Map(args[i], col.Type.formatter()).([]string)
value = fmt.Sprintf("{%s}", strings.Join(valueList, ","))
}
row = append(row, value)
}
return row
}
func (tbl *Table) VarcharColumns() []string {
columns := funk.Filter(tbl.Columns, func(col Column) bool {
return col.Type == Dvarchar
}).([]Column)
columnNames := funk.Map(columns, func(col Column) string {
return col.Name
}).([]string)
return columnNames
}
func OnConflict(target ...string) ConflictClause {
return ConflictClause{Target: target}
}
func (c ConflictClause) Set(fields ...string) ConflictClause {
c.Update = fields
return c
}
// ToInsertStatement returns a Postgres-compatible SQL insert statement for the table
// using positional placeholders
func (tbl *Table) ToInsertStatement(upsert bool) string {
var colnames, placeholders []string
for i, col := range tbl.Columns {
colnames = append(colnames, col.Name)
placeholders = append(placeholders, fmt.Sprintf("$%d", i+1))
}
suffix := fmt.Sprintf("ON CONFLICT (%s)", strings.Join(tbl.UpsertClause.Target, ", "))
if upsert && len(tbl.UpsertClause.Update) != 0 {
var update_placeholders []string
for _, name := range tbl.UpsertClause.Update {
i := funk.IndexOf(tbl.Columns, func(col Column) bool { return col.Name == name })
update_placeholders = append(update_placeholders, fmt.Sprintf("$%d", i+1))
}
suffix += fmt.Sprintf(
" DO UPDATE SET (%s) = (%s)",
strings.Join(tbl.UpsertClause.Update, ", "), strings.Join(update_placeholders, ", "),
)
} else {
suffix += " DO NOTHING"
}
return fmt.Sprintf(
"INSERT INTO %s (%s) VALUES (%s) %s",
tbl.Name, strings.Join(colnames, ", "), strings.Join(placeholders, ", "), suffix,
)
}
func sprintf(f string) colfmt {
return func(x interface{}) string { return fmt.Sprintf(f, x) }
}
func (typ colType) formatter() colfmt {
switch typ {
case Dinteger:
return sprintf("%d")
case Dboolean:
return func(x interface{}) string {
if x.(bool) {
return "t"
}
return "f"
}
case Dbigint:
return sprintf("%s")
case Dnumeric:
return sprintf("%s")
case Dbytea:
return sprintf(`\x%x`)
case Dvarchar:
return sprintf("%s")
case Dtext:
return sprintf("%s")
}
panic("unreachable")
}

View File

@ -0,0 +1,53 @@
package schema_test
import (
"testing"
"github.com/stretchr/testify/require"
. "github.com/ethereum/go-ethereum/statediff/indexer/shared/schema"
)
var testHeaderTable = Table{
Name: "eth.header_cids",
Columns: []Column{
{Name: "block_number", Type: Dbigint},
{Name: "block_hash", Type: Dvarchar},
{Name: "parent_hash", Type: Dvarchar},
{Name: "cid", Type: Dtext},
{Name: "td", Type: Dnumeric},
{Name: "node_id", Type: Dvarchar},
{Name: "reward", Type: Dnumeric},
{Name: "state_root", Type: Dvarchar},
{Name: "tx_root", Type: Dvarchar},
{Name: "receipt_root", Type: Dvarchar},
{Name: "uncle_root", Type: Dvarchar},
{Name: "bloom", Type: Dbytea},
{Name: "timestamp", Type: Dnumeric},
{Name: "mh_key", Type: Dtext},
{Name: "times_validated", Type: Dinteger},
{Name: "coinbase", Type: Dvarchar},
},
UpsertClause: OnConflict("block_hash", "block_number").Set(
"parent_hash",
"cid",
"td",
"node_id",
"reward",
"state_root",
"tx_root",
"receipt_root",
"uncle_root",
"bloom",
"timestamp",
"mh_key",
"times_validated",
"coinbase",
)}
func TestTable(t *testing.T) {
headerUpsert := `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) ON CONFLICT (block_hash, block_number) DO UPDATE SET (parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) = ($3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16)`
headerNoUpsert := `INSERT INTO eth.header_cids (block_number, block_hash, parent_hash, cid, td, node_id, reward, state_root, tx_root, receipt_root, uncle_root, bloom, timestamp, mh_key, times_validated, coinbase) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, $14, $15, $16) ON CONFLICT (block_hash, block_number) DO NOTHING`
require.Equal(t, headerNoUpsert, testHeaderTable.ToInsertStatement(false))
require.Equal(t, headerUpsert, testHeaderTable.ToInsertStatement(true))
}

1157
indexer/test/test.go Normal file

File diff suppressed because it is too large Load Diff

194
indexer/test/test_init.go Normal file
View File

@ -0,0 +1,194 @@
// VulcanizeDB
// Copyright © 2022 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 test
import (
"bytes"
"fmt"
"os"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ethereum/go-ethereum/statediff/indexer/models"
"github.com/ethereum/go-ethereum/statediff/indexer/shared"
"github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash"
)
var (
err error
ipfsPgGet = `SELECT data FROM ipld.blocks
WHERE key = $1 AND block_number = $2`
watchedAddressesPgGet = `SELECT *
FROM eth_meta.watched_addresses`
tx1, tx2, tx3, tx4, tx5, rct1, rct2, rct3, rct4, rct5 []byte
nonCanonicalBlockRct1, nonCanonicalBlockRct2 []byte
nonCanonicalBlock2Rct1, nonCanonicalBlock2Rct2 []byte
mockBlock, mockNonCanonicalBlock, mockNonCanonicalBlock2 *types.Block
headerCID, mockNonCanonicalHeaderCID, mockNonCanonicalHeader2CID cid.Cid
trx1CID, trx2CID, trx3CID, trx4CID, trx5CID cid.Cid
rct1CID, rct2CID, rct3CID, rct4CID, rct5CID cid.Cid
nonCanonicalBlockRct1CID, nonCanonicalBlockRct2CID cid.Cid
nonCanonicalBlock2Rct1CID, nonCanonicalBlock2Rct2CID cid.Cid
state1CID, state2CID, storageCID cid.Cid
)
func init() {
if os.Getenv("MODE") != "statediff" {
fmt.Println("Skipping statediff test")
os.Exit(0)
}
// canonical block at LondonBlock height
mockBlock = mocks.MockBlock
txs, rcts := mocks.MockBlock.Transactions(), mocks.MockReceipts
// non-canonical block at LondonBlock height
mockNonCanonicalBlock = mocks.MockNonCanonicalBlock
nonCanonicalBlockRcts := mocks.MockNonCanonicalBlockReceipts
// non-canonical block at LondonBlock height + 1
mockNonCanonicalBlock2 = mocks.MockNonCanonicalBlock2
nonCanonicalBlock2Rcts := mocks.MockNonCanonicalBlock2Receipts
// encode mock receipts
buf := new(bytes.Buffer)
txs.EncodeIndex(0, buf)
tx1 = make([]byte, buf.Len())
copy(tx1, buf.Bytes())
buf.Reset()
txs.EncodeIndex(1, buf)
tx2 = make([]byte, buf.Len())
copy(tx2, buf.Bytes())
buf.Reset()
txs.EncodeIndex(2, buf)
tx3 = make([]byte, buf.Len())
copy(tx3, buf.Bytes())
buf.Reset()
txs.EncodeIndex(3, buf)
tx4 = make([]byte, buf.Len())
copy(tx4, buf.Bytes())
buf.Reset()
txs.EncodeIndex(4, buf)
tx5 = make([]byte, buf.Len())
copy(tx5, buf.Bytes())
buf.Reset()
rcts.EncodeIndex(0, buf)
rct1 = make([]byte, buf.Len())
copy(rct1, buf.Bytes())
buf.Reset()
rcts.EncodeIndex(1, buf)
rct2 = make([]byte, buf.Len())
copy(rct2, buf.Bytes())
buf.Reset()
rcts.EncodeIndex(2, buf)
rct3 = make([]byte, buf.Len())
copy(rct3, buf.Bytes())
buf.Reset()
rcts.EncodeIndex(3, buf)
rct4 = make([]byte, buf.Len())
copy(rct4, buf.Bytes())
buf.Reset()
rcts.EncodeIndex(4, buf)
rct5 = make([]byte, buf.Len())
copy(rct5, buf.Bytes())
buf.Reset()
// encode mock receipts for non-canonical blocks
nonCanonicalBlockRcts.EncodeIndex(0, buf)
nonCanonicalBlockRct1 = make([]byte, buf.Len())
copy(nonCanonicalBlockRct1, buf.Bytes())
buf.Reset()
nonCanonicalBlockRcts.EncodeIndex(1, buf)
nonCanonicalBlockRct2 = make([]byte, buf.Len())
copy(nonCanonicalBlockRct2, buf.Bytes())
buf.Reset()
nonCanonicalBlock2Rcts.EncodeIndex(0, buf)
nonCanonicalBlock2Rct1 = make([]byte, buf.Len())
copy(nonCanonicalBlock2Rct1, buf.Bytes())
buf.Reset()
nonCanonicalBlock2Rcts.EncodeIndex(1, buf)
nonCanonicalBlock2Rct2 = make([]byte, buf.Len())
copy(nonCanonicalBlock2Rct2, buf.Bytes())
buf.Reset()
headerCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockHeaderRlp, multihash.KECCAK_256)
mockNonCanonicalHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockNonCanonicalHeaderRlp, multihash.KECCAK_256)
mockNonCanonicalHeader2CID, _ = ipld.RawdataToCid(ipld.MEthHeader, mocks.MockNonCanonicalHeader2Rlp, multihash.KECCAK_256)
trx1CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx1, multihash.KECCAK_256)
trx2CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx2, multihash.KECCAK_256)
trx3CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx3, multihash.KECCAK_256)
trx4CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx4, multihash.KECCAK_256)
trx5CID, _ = ipld.RawdataToCid(ipld.MEthTx, tx5, multihash.KECCAK_256)
state1CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.ContractLeafNode, multihash.KECCAK_256)
state2CID, _ = ipld.RawdataToCid(ipld.MEthStateTrie, mocks.AccountLeafNode, multihash.KECCAK_256)
storageCID, _ = ipld.RawdataToCid(ipld.MEthStorageTrie, mocks.StorageLeafNode, multihash.KECCAK_256)
rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct1, multihash.KECCAK_256)
rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct2, multihash.KECCAK_256)
rct3CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct3, multihash.KECCAK_256)
rct4CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct4, multihash.KECCAK_256)
rct5CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, rct5, multihash.KECCAK_256)
// create raw receipts for non-canonical blocks
nonCanonicalBlockRct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, nonCanonicalBlockRct1, multihash.KECCAK_256)
nonCanonicalBlockRct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, nonCanonicalBlockRct2, multihash.KECCAK_256)
nonCanonicalBlock2Rct1CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, nonCanonicalBlock2Rct1, multihash.KECCAK_256)
nonCanonicalBlock2Rct2CID, _ = ipld.RawdataToCid(ipld.MEthTxReceipt, nonCanonicalBlock2Rct2, multihash.KECCAK_256)
}
// createRctModel creates a models.ReceiptModel object from a given ethereum receipt
func createRctModel(rct *types.Receipt, cid cid.Cid, blockNumber string) models.ReceiptModel {
rctModel := models.ReceiptModel{
BlockNumber: blockNumber,
HeaderID: rct.BlockHash.String(),
TxID: rct.TxHash.String(),
CID: cid.String(),
}
contract := shared.HandleZeroAddr(rct.ContractAddress)
rctModel.Contract = contract
if len(rct.PostState) == 0 {
rctModel.PostStatus = rct.Status
} else {
rctModel.PostState = common.BytesToHash(rct.PostState).String()
}
return rctModel
}
func expectTrue(t *testing.T, value bool) {
if !value {
t.Fatalf("Assertion failed")
}
}

View File

@ -0,0 +1,96 @@
// VulcanizeDB
// Copyright © 2022 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 test
import (
"context"
"testing"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/ipfs/go-cid"
"github.com/multiformats/go-multihash"
"github.com/stretchr/testify/require"
)
var (
LegacyConfig = params.MainnetChainConfig
legacyData = mocks.NewLegacyData(LegacyConfig)
mockLegacyBlock *types.Block
legacyHeaderCID cid.Cid
)
func SetupLegacyTestData(t *testing.T, ind interfaces.StateDiffIndexer) {
mockLegacyBlock = legacyData.MockBlock
legacyHeaderCID, _ = ipld.RawdataToCid(ipld.MEthHeader, legacyData.MockHeaderRlp, multihash.KECCAK_256)
var tx interfaces.Batch
tx, err = ind.PushBlock(
mockLegacyBlock,
legacyData.MockReceipts,
legacyData.MockBlock.Difficulty())
require.NoError(t, err)
defer func() {
if err := tx.Submit(err); err != nil {
t.Fatal(err)
}
}()
for _, node := range legacyData.StateDiffs {
err = ind.PushStateNode(tx, node, mockLegacyBlock.Hash().String())
require.NoError(t, err)
}
if batchTx, ok := tx.(*sql.BatchTx); ok {
require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber)
} else if batchTx, ok := tx.(*file.BatchTx); ok {
require.Equal(t, legacyData.BlockNumber.String(), batchTx.BlockNumber)
}
}
func TestLegacyIndexer(t *testing.T, db sql.Database) {
pgStr := `SELECT cid, cast(td AS TEXT), cast(reward AS TEXT), block_hash, coinbase
FROM eth.header_cids
WHERE block_number = $1`
// check header was properly indexed
type res struct {
CID string
TD string
Reward string
BlockHash string `db:"block_hash"`
Coinbase string `db:"coinbase"`
}
header := new(res)
err = db.QueryRow(context.Background(), pgStr, legacyData.BlockNumber.Uint64()).Scan(
&header.CID,
&header.TD,
&header.Reward,
&header.BlockHash,
&header.Coinbase)
require.NoError(t, err)
require.Equal(t, legacyHeaderCID.String(), header.CID)
require.Equal(t, legacyData.MockBlock.Difficulty().String(), header.TD)
require.Equal(t, "5000000000000011250", header.Reward)
require.Equal(t, legacyData.MockHeader.Coinbase.String(), header.Coinbase)
require.Nil(t, legacyData.MockHeader.BaseFee)
}

View File

@ -0,0 +1,53 @@
// VulcanizeDB
// Copyright © 2022 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 test
import (
"testing"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/statediff/indexer/database/file"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/stretchr/testify/require"
)
func TestBlock(t *testing.T, ind interfaces.StateDiffIndexer, testBlock *types.Block, testReceipts types.Receipts) {
var tx interfaces.Batch
tx, err = ind.PushBlock(
testBlock,
testReceipts,
testBlock.Difficulty())
require.NoError(t, err)
defer func() {
if err := tx.Submit(err); err != nil {
t.Fatal(err)
}
}()
for _, node := range mocks.StateDiffs {
err = ind.PushStateNode(tx, node, testBlock.Hash().String())
require.NoError(t, err)
}
if batchTx, ok := tx.(*sql.BatchTx); ok {
require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber)
} else if batchTx, ok := tx.(*file.BatchTx); ok {
require.Equal(t, testBlock.Number().String(), batchTx.BlockNumber)
}
}

View File

@ -0,0 +1,258 @@
// VulcanizeDB
// Copyright © 2022 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 test
import (
"context"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
"github.com/ethereum/go-ethereum/statediff/indexer/interfaces"
"github.com/ethereum/go-ethereum/statediff/indexer/mocks"
"github.com/stretchr/testify/require"
)
type res struct {
Address string `db:"address"`
CreatedAt uint64 `db:"created_at"`
WatchedAt uint64 `db:"watched_at"`
LastFilledAt uint64 `db:"last_filled_at"`
}
func TestLoadEmptyWatchedAddresses(t *testing.T, ind interfaces.StateDiffIndexer) {
expectedData := []common.Address{}
rows, err := ind.LoadWatchedAddresses()
require.NoError(t, err)
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestInsertWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{
{
Address: mocks.Contract1Address,
CreatedAt: mocks.Contract1CreatedAt,
WatchedAt: mocks.WatchedAt1,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract2Address,
CreatedAt: mocks.Contract2CreatedAt,
WatchedAt: mocks.WatchedAt1,
LastFilledAt: mocks.LastFilledAt,
},
}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestInsertAlreadyWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{
{
Address: mocks.Contract1Address,
CreatedAt: mocks.Contract1CreatedAt,
WatchedAt: mocks.WatchedAt1,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract2Address,
CreatedAt: mocks.Contract2CreatedAt,
WatchedAt: mocks.WatchedAt1,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract3Address,
CreatedAt: mocks.Contract3CreatedAt,
WatchedAt: mocks.WatchedAt2,
LastFilledAt: mocks.LastFilledAt,
},
}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestRemoveWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{
{
Address: mocks.Contract1Address,
CreatedAt: mocks.Contract1CreatedAt,
WatchedAt: mocks.WatchedAt1,
LastFilledAt: mocks.LastFilledAt,
},
}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestRemoveNonWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestSetWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{
{
Address: mocks.Contract1Address,
CreatedAt: mocks.Contract1CreatedAt,
WatchedAt: mocks.WatchedAt2,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract2Address,
CreatedAt: mocks.Contract2CreatedAt,
WatchedAt: mocks.WatchedAt2,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract3Address,
CreatedAt: mocks.Contract3CreatedAt,
WatchedAt: mocks.WatchedAt2,
LastFilledAt: mocks.LastFilledAt,
},
}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestSetAlreadyWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{
{
Address: mocks.Contract4Address,
CreatedAt: mocks.Contract4CreatedAt,
WatchedAt: mocks.WatchedAt3,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract2Address,
CreatedAt: mocks.Contract2CreatedAt,
WatchedAt: mocks.WatchedAt3,
LastFilledAt: mocks.LastFilledAt,
},
{
Address: mocks.Contract3Address,
CreatedAt: mocks.Contract3CreatedAt,
WatchedAt: mocks.WatchedAt3,
LastFilledAt: mocks.LastFilledAt,
},
}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestLoadWatchedAddresses(t *testing.T, ind interfaces.StateDiffIndexer) {
expectedData := []common.Address{
common.HexToAddress(mocks.Contract4Address),
common.HexToAddress(mocks.Contract2Address),
common.HexToAddress(mocks.Contract3Address),
}
rows, err := ind.LoadWatchedAddresses()
require.NoError(t, err)
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestClearWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}
func TestClearEmptyWatchedAddresses(t *testing.T, db sql.Database) {
expectedData := []res{}
rows := []res{}
err = db.Select(context.Background(), &rows, watchedAddressesPgGet)
if err != nil {
t.Fatal(err)
}
require.Equal(t, len(expectedData), len(rows))
for idx, row := range rows {
require.Equal(t, expectedData[idx], row)
}
}

View File

@ -0,0 +1,248 @@
// VulcanizeDB
// Copyright © 2021 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 test_helpers
import (
"context"
"errors"
"fmt"
"math/big"
"os"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/rlp"
)
const (
defaultBlockFilePath = "../../../mainnet_data/block"
defaultReceiptsFilePath = "../../../mainnet_data/receipts"
)
const (
TEST_RAW_URL = "TEST_RAW_URL"
TEST_BLOCK_NUMBER = "TEST_BLOCK_NUMBER"
)
// ProblemBlocks list of known problem blocks, with funky edge cases
var ProblemBlocks = []int64{
12600011,
12619985,
12625121,
12655432,
12579670,
12914664,
}
// TestConfig holds configuration params for mainnet tests
type TestConfig struct {
RawURL string
BlockNumber *big.Int
LocalCache bool
}
// DefaultTestConfig is the default TestConfig
var DefaultTestConfig = TestConfig{
RawURL: "http://127.0.0.1:8545",
BlockNumber: big.NewInt(12914664),
LocalCache: true,
}
func GetTestConfig() TestConfig {
conf := DefaultTestConfig
rawURL := os.Getenv(TEST_RAW_URL)
if rawURL == "" {
fmt.Printf("Warning: no raw url configured for statediffing mainnet tests, will look for local file and"+
"then try default endpoint (%s)\r\n", DefaultTestConfig.RawURL)
} else {
conf.RawURL = rawURL
}
return conf
}
// TestBlockAndReceiptsFromEnv retrieves the block and receipts using env variables to override default config block number
func TestBlockAndReceiptsFromEnv(conf TestConfig) (*types.Block, types.Receipts, error) {
blockNumberStr := os.Getenv(TEST_BLOCK_NUMBER)
blockNumber, ok := new(big.Int).SetString(blockNumberStr, 10)
if !ok {
fmt.Printf("Warning: no blockNumber configured for statediffing mainnet tests, using default (%d)\r\n",
DefaultTestConfig.BlockNumber)
} else {
conf.BlockNumber = blockNumber
}
return TestBlockAndReceipts(conf)
}
// TestBlockAndReceipts retrieves the block and receipts for the provided test config
// It first tries to load files from the local system before setting up and using an ethclient.Client to pull the data
func TestBlockAndReceipts(conf TestConfig) (*types.Block, types.Receipts, error) {
var cli *ethclient.Client
var err error
var block *types.Block
var receipts types.Receipts
blockFilePath := fmt.Sprintf("%s_%s.rlp", defaultBlockFilePath, conf.BlockNumber.String())
if _, err = os.Stat(blockFilePath); !errors.Is(err, os.ErrNotExist) {
fmt.Printf("local file (%s) found for block %s\n", blockFilePath, conf.BlockNumber.String())
block, err = LoadBlockRLP(blockFilePath)
if err != nil {
fmt.Printf("loading local file (%s) failed (%s), dialing remote client at %s\n", blockFilePath, err.Error(), conf.RawURL)
cli, err = ethclient.Dial(conf.RawURL)
if err != nil {
return nil, nil, err
}
block, err = FetchBlock(cli, conf.BlockNumber)
if err != nil {
return nil, nil, err
}
if conf.LocalCache {
if err := WriteBlockRLP(blockFilePath, block); err != nil {
return nil, nil, err
}
}
}
} else {
fmt.Printf("no local file found for block %s, dialing remote client at %s\n", conf.BlockNumber.String(), conf.RawURL)
cli, err = ethclient.Dial(conf.RawURL)
if err != nil {
return nil, nil, err
}
block, err = FetchBlock(cli, conf.BlockNumber)
if err != nil {
return nil, nil, err
}
if conf.LocalCache {
if err := WriteBlockRLP(blockFilePath, block); err != nil {
return nil, nil, err
}
}
}
receiptsFilePath := fmt.Sprintf("%s_%s.rlp", defaultReceiptsFilePath, conf.BlockNumber.String())
if _, err = os.Stat(receiptsFilePath); !errors.Is(err, os.ErrNotExist) {
fmt.Printf("local file (%s) found for block %s receipts\n", receiptsFilePath, conf.BlockNumber.String())
receipts, err = LoadReceiptsEncoding(receiptsFilePath, len(block.Transactions()))
if err != nil {
fmt.Printf("loading local file (%s) failed (%s), dialing remote client at %s\n", receiptsFilePath, err.Error(), conf.RawURL)
if cli == nil {
cli, err = ethclient.Dial(conf.RawURL)
if err != nil {
return nil, nil, err
}
}
receipts, err = FetchReceipts(cli, block)
if err != nil {
return nil, nil, err
}
if conf.LocalCache {
if err := WriteReceiptsEncoding(receiptsFilePath, block.Number(), receipts); err != nil {
return nil, nil, err
}
}
}
} else {
fmt.Printf("no local file found for block %s receipts, dialing remote client at %s\n", conf.BlockNumber.String(), conf.RawURL)
if cli == nil {
cli, err = ethclient.Dial(conf.RawURL)
if err != nil {
return nil, nil, err
}
}
receipts, err = FetchReceipts(cli, block)
if err != nil {
return nil, nil, err
}
if conf.LocalCache {
if err := WriteReceiptsEncoding(receiptsFilePath, block.Number(), receipts); err != nil {
return nil, nil, err
}
}
}
return block, receipts, nil
}
// FetchBlock fetches the block at the provided height using the ethclient.Client
func FetchBlock(cli *ethclient.Client, blockNumber *big.Int) (*types.Block, error) {
return cli.BlockByNumber(context.Background(), blockNumber)
}
// FetchReceipts fetches the receipts for the provided block using the ethclient.Client
func FetchReceipts(cli *ethclient.Client, block *types.Block) (types.Receipts, error) {
receipts := make(types.Receipts, len(block.Transactions()))
for i, tx := range block.Transactions() {
rct, err := cli.TransactionReceipt(context.Background(), tx.Hash())
if err != nil {
return nil, err
}
receipts[i] = rct
}
return receipts, nil
}
// WriteBlockRLP writes out the RLP encoding of the block to the provided filePath
func WriteBlockRLP(filePath string, block *types.Block) error {
if filePath == "" {
filePath = fmt.Sprintf("%s_%s.rlp", defaultBlockFilePath, block.Number().String())
}
if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) {
return fmt.Errorf("cannot create file, file (%s) already exists", filePath)
}
file, err := os.Create(filePath)
if err != nil {
return fmt.Errorf("unable to create file (%s), err: %v", filePath, err)
}
fmt.Printf("writing block rlp to file at %s\r\n", filePath)
if err := block.EncodeRLP(file); err != nil {
return err
}
return file.Close()
}
// LoadBlockRLP loads block from the rlp at filePath
func LoadBlockRLP(filePath string) (*types.Block, error) {
blockBytes, err := os.ReadFile(filePath)
if err != nil {
return nil, err
}
block := new(types.Block)
return block, rlp.DecodeBytes(blockBytes, block)
}
// LoadReceiptsEncoding loads receipts from the encoding at filePath
func LoadReceiptsEncoding(filePath string, cap int) (types.Receipts, error) {
rctsBytes, err := os.ReadFile(filePath)
if err != nil {
return nil, err
}
receipts := new(types.Receipts)
return *receipts, rlp.DecodeBytes(rctsBytes, receipts)
}
// WriteReceiptsEncoding writes out the consensus encoding of the receipts to the provided io.WriteCloser
func WriteReceiptsEncoding(filePath string, blockNumber *big.Int, receipts types.Receipts) error {
if filePath == "" {
filePath = fmt.Sprintf("%s_%s.rlp", defaultReceiptsFilePath, blockNumber.String())
}
if _, err := os.Stat(filePath); !errors.Is(err, os.ErrNotExist) {
return fmt.Errorf("cannot create file, file (%s) already exists", filePath)
}
file, err := os.Create(filePath)
if err != nil {
return fmt.Errorf("unable to create file (%s), err: %v", filePath, err)
}
defer file.Close()
fmt.Printf("writing receipts rlp to file at %s\r\n", filePath)
return rlp.Encode(file, receipts)
}

View File

@ -0,0 +1,123 @@
// 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 test_helpers
import (
"bufio"
"context"
"os"
"testing"
"github.com/ethereum/go-ethereum/statediff/indexer/database/sql"
)
// ListContainsString used to check if a list of strings contains a particular string
func ListContainsString(sss []string, s string) bool {
for _, str := range sss {
if s == str {
return true
}
}
return false
}
// DedupFile removes duplicates from the given file
func DedupFile(filePath string) error {
f, err := os.OpenFile(filePath, os.O_CREATE|os.O_RDONLY, os.ModePerm)
if err != nil {
return err
}
stmts := make(map[string]struct{}, 0)
sc := bufio.NewScanner(f)
for sc.Scan() {
s := sc.Text()
stmts[s] = struct{}{}
}
if err != nil {
return err
}
f.Close()
f, err = os.Create(filePath)
if err != nil {
return err
}
defer f.Close()
for stmt := range stmts {
f.Write([]byte(stmt + "\n"))
}
return nil
}
// TearDownDB is used to tear down the watcher dbs after tests
func TearDownDB(t *testing.T, db sql.Database) {
ctx := context.Background()
tx, err := db.Begin(ctx)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.header_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.uncle_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.transaction_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.receipt_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.state_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.storage_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth.log_cids`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM ipld.blocks`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM nodes`)
if err != nil {
t.Fatal(err)
}
_, err = tx.Exec(ctx, `DELETE FROM eth_meta.watched_addresses`)
if err != nil {
t.Fatal(err)
}
err = tx.Commit(ctx)
if err != nil {
t.Fatal(err)
}
}

BIN
mainnet_tests/block0_rlp Normal file

Binary file not shown.

BIN
mainnet_tests/block1_rlp Normal file

Binary file not shown.

BIN
mainnet_tests/block2_rlp Normal file

Binary file not shown.

BIN
mainnet_tests/block3_rlp Normal file

Binary file not shown.

View File

@ -0,0 +1,704 @@
// Copyright 2019 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package statediff_test
import (
"bytes"
"encoding/json"
"fmt"
"io"
"log"
"math/big"
"os"
"sort"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/core/rawdb"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/statediff"
ipld2 "github.com/ethereum/go-ethereum/statediff/indexer/ipld"
"github.com/ethereum/go-ethereum/statediff/test_helpers"
sdtypes "github.com/ethereum/go-ethereum/statediff/types"
)
var (
db ethdb.Database
genesisBlock, block0, block1, block2, block3 *types.Block
block1CoinbaseAddr, block2CoinbaseAddr, block3CoinbaseAddr common.Address
block1CoinbaseHash, block2CoinbaseHash, block3CoinbaseHash common.Hash
builder statediff.Builder
emptyStorage = make([]sdtypes.StorageLeafNode, 0)
// block 1 data
block1CoinbaseAccount = &types.StateAccount{
Nonce: 0,
Balance: big.NewInt(5000000000000000000),
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block1CoinbaseAccountRLP, _ = rlp.EncodeToBytes(block1CoinbaseAccount)
block1CoinbaseLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("38251692195afc818c92b485fcb8a4691af89cbe5a2ab557b83a4261be2a9a"),
block1CoinbaseAccountRLP,
})
block1CoinbaseLeafNodeHash = crypto.Keccak256(block1CoinbaseLeafNode)
block1x040bBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("cc947d5ebb80600bad471f12c6ad5e4981e3525ecf8a2d982cc032536ae8b66d"),
common.Hex2Bytes("e80e52462e635a834e90e86ccf7673a6430384aac17004d626f4db831f0624bc"),
common.Hex2Bytes("59a8f11f60cb0a8488831f242da02944a26fd269d0608a44b8b873ded9e59e1b"),
common.Hex2Bytes("1ffb51e987e3cbd2e1dc1a64508d2e2b265477e21698b0d10fdf137f35027f40"),
[]byte{},
common.Hex2Bytes("ce5077f49a13ff8199d0e77715fdd7bfd6364774effcd5499bd93cba54b3c644"),
common.Hex2Bytes("f5146783c048e66ce1a776ae990b4255e5fba458ece77fcb83ff6e91d6637a88"),
common.Hex2Bytes("6a0558b6c38852e985cf01c2156517c1c6a1e64c787a953c347825f050b236c6"),
common.Hex2Bytes("56b6e93958b99aaae158cc2329e71a1865ba6f39c67b096922c5cf3ed86b0ae5"),
[]byte{},
common.Hex2Bytes("50d317a89a3405367d66668902f2c9f273a8d0d7d5d790dc516bca142f4a84af"),
common.Hex2Bytes("c72ca72750fdc1af3e6da5c7c5d82c54e4582f15b488a8aa1674058a99825dae"),
common.Hex2Bytes("e1a489df7b18cde818da6d38e235b026c2e61bcd3d34880b3ed0d67e0e4f0159"),
common.Hex2Bytes("b58d5062f2609fd2d68f00d14ab33fef2b373853877cf40bf64729e85b8fdc54"),
block1CoinbaseLeafNodeHash,
[]byte{},
[]byte{},
})
block1x040bBranchNodeHash = crypto.Keccak256(block1x040bBranchNode)
block1x04BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("a9317a59365ca09cefcd384018696590afffc432e35a97e8f85aa48907bf3247"),
common.Hex2Bytes("e0bc229254ce7a6a736c3953e570ab18b4a7f5f2a9aa3c3057b5f17d250a1cad"),
common.Hex2Bytes("a2484ec8884dbe0cf24ece99d67df0d1fe78992d67cc777636a817cb2ef205aa"),
common.Hex2Bytes("12b78d4078c607747f06bb88bd08f839eaae0e3ac6854e5f65867d4f78abb84e"),
common.Hex2Bytes("359a51862df5462e4cd302f69cb338512f21eb37ce0791b9a562e72ec48b7dbf"),
common.Hex2Bytes("13f8d617b6a734da9235b6ac80bdd7aeaff6120c39aa223638d88f22d4ba4007"),
common.Hex2Bytes("02055c6400e0ec3440a8bb8fdfd7d6b6c57b7bf83e37d7e4e983d416fdd8314e"),
common.Hex2Bytes("4b1cca9eb3e47e805e7f4c80671a9fcd589fd6ddbe1790c3f3e177e8ede01b9e"),
common.Hex2Bytes("70c3815efb23b986018089e009a38e6238b8850b3efd33831913ca6fa9240249"),
common.Hex2Bytes("7084699d2e72a193fd75bb6108ae797b4661696eba2d631d521fc94acc7b3247"),
common.Hex2Bytes("b2b3cd9f1e46eb583a6185d9a96b4e80125e3d75e6191fdcf684892ef52935cb"),
block1x040bBranchNodeHash,
common.Hex2Bytes("34d9ff0fee6c929424e52268dedbc596d10786e909c5a68d6466c2aba17387ce"),
common.Hex2Bytes("7484d5e44b6ee6b10000708c37e035b42b818475620f9316beffc46531d1eebf"),
common.Hex2Bytes("30c8a283adccf2742272563cd3d6710c89ba21eac0118bf5310cfb231bcca77f"),
common.Hex2Bytes("4bae8558d2385b8d3bc6e6ede20bdbc5dbb0b5384c316ba8985682f88d2e506d"),
[]byte{},
})
block1x04BranchNodeHash = crypto.Keccak256(block1x04BranchNode)
block1RootBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("90dcaf88c40c7bbc95a912cbdde67c175767b31173df9ee4b0d733bfdd511c43"),
common.Hex2Bytes("babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bd"),
common.Hex2Bytes("473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021"),
common.Hex2Bytes("bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0"),
block1x04BranchNodeHash,
common.Hex2Bytes("a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7"),
common.Hex2Bytes("e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92d"),
common.Hex2Bytes("f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721"),
common.Hex2Bytes("7117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681"),
common.Hex2Bytes("69eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472"),
common.Hex2Bytes("203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8e"),
common.Hex2Bytes("9287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208"),
common.Hex2Bytes("6fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94ec"),
common.Hex2Bytes("7b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475"),
common.Hex2Bytes("51f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0"),
common.Hex2Bytes("89d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb"),
[]byte{},
})
// block 2 data
block2CoinbaseAccount = &types.StateAccount{
Nonce: 0,
Balance: big.NewInt(5000000000000000000),
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block2CoinbaseAccountRLP, _ = rlp.EncodeToBytes(block2CoinbaseAccount)
block2CoinbaseLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("20679cbcf198c1741a6f4e4473845659a30caa8b26f8d37a0be2e2bc0d8892"),
block2CoinbaseAccountRLP,
})
block2CoinbaseLeafNodeHash = crypto.Keccak256(block2CoinbaseLeafNode)
block2MovedPremineBalance, _ = new(big.Int).SetString("4000000000000000000000", 10)
block2MovedPremineAccount = &types.StateAccount{
Nonce: 0,
Balance: block2MovedPremineBalance,
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block2MovedPremineAccountRLP, _ = rlp.EncodeToBytes(block2MovedPremineAccount)
block2MovedPremineLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("20f2e24db7943eab4415f99e109698863b0fecca1cf9ffc500f38cefbbe29e"),
block2MovedPremineAccountRLP,
})
block2MovedPremineLeafNodeHash = crypto.Keccak256(block2MovedPremineLeafNode)
block2x00080dBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
block2MovedPremineLeafNodeHash,
[]byte{},
[]byte{},
[]byte{},
block2CoinbaseLeafNodeHash,
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
})
block2x00080dBranchNodeHash = crypto.Keccak256(block2x00080dBranchNode)
block2x0008BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("def97a26f824fc3911cf7f8c41dfc9bc93cc36ae2248de22ecae01d6950b2dc9"),
common.Hex2Bytes("234a575e2c5badab8de0f6515b6723195323a0562fbe1316255888637043f1c1"),
common.Hex2Bytes("29659740af1c23306ee8f8294c71a5632ace8c80b1eb61cfdf7022f47ff52305"),
common.Hex2Bytes("cf2681d23bb666d89dec8123bce9e626240a7e2ce7a1e8316b1ee88181c9471c"),
common.Hex2Bytes("18d8de6967fe34b9fd411c74fecc45f8a737961791e70d8ece967bb07cf4d4dc"),
common.Hex2Bytes("7cad60c7cbca8c79c2db5a8fc1baa9381484d43d6c37dfb97718c3a109d47dfc"),
common.Hex2Bytes("2138f5a9062b750b6320e5fac5b134da90a9edbda06ef3e1ae64fb1366ca998c"),
common.Hex2Bytes("532826502a9661fcae7c0f5d2a4c8cb287dfc521e828349543c5a461a9d591ed"),
common.Hex2Bytes("30543537413dd086d4b1560f46b90e8da0f43de5584a138ab036d74e84657523"),
common.Hex2Bytes("c98042928af640bfa1142aca895cd76e146332dce94ddad3426e74ed519ca1e0"),
common.Hex2Bytes("43de3e62cc3148193899d018dff813c04c5b636ce95bd7e828416204292d9ff9"),
[]byte{},
common.Hex2Bytes("78d533b9182bb42f6c16e9ebd5734f0d280179ba1c9b6316c2c1df73f7dd8a54"),
block2x00080dBranchNodeHash,
common.Hex2Bytes("934b736b57a892aaa15a03c7e37746bb096313727135f9841cb64c263785cf81"),
common.Hex2Bytes("38ce97150e90dfd7258901a0ddee72d8e30760a3d0419dbb80135c66588739a2"),
[]byte{},
})
block2x0008BranchNodeHash = crypto.Keccak256(block2x0008BranchNode)
block2x00BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("e45a9e85cab1b6eb18b30df2c6acc448bbac6a30d81646823b31223e16e5063e"),
common.Hex2Bytes("33bd7171d556b981f6849064eb09412b24fedc0812127db936067043f53db1b9"),
common.Hex2Bytes("ca56945f074da4f15587404593faf3a50d17ea0e21a418ad6ec99bdf4bf3f914"),
common.Hex2Bytes("da23e9004f782df128eea1adff77952dc85f91b7f7ca4893aac5f21d24c3a1c9"),
common.Hex2Bytes("ba5ec61fa780ee02af19db99677c37560fc4f0df5c278d9dfa2837f30f72bc6b"),
common.Hex2Bytes("8310ad91625c2e3429a74066b7e2e0c958325e4e7fa3ec486b73b7c8300cfef7"),
common.Hex2Bytes("732e5c103bf4d5adfef83773026809d9405539b67e93293a02342e83ad2fb766"),
common.Hex2Bytes("30d14ff0c2aab57d1fbaf498ab14519b4e9d94f149a3dc15f0eec5adf8df25e1"),
block2x0008BranchNodeHash,
common.Hex2Bytes("5a43bd92e55aa78df60e70b6b53b6366c4080fd6a5bdd7b533b46aff4a75f6f2"),
common.Hex2Bytes("a0c410aa59efe416b1213166fab680ce330bd46c3ebf877ff14609ee6a383600"),
common.Hex2Bytes("2f41e918786e557293068b1eda9b3f9f86ed4e65a6a5363ee3262109f6e08b17"),
common.Hex2Bytes("01f42a40f02f6f24bb97b09c4d3934e8b03be7cfbb902acc1c8fd67a7a5abace"),
common.Hex2Bytes("0acbdce2787a6ea177209bd13bfc9d0779d7e2b5249e0211a2974164e14312f5"),
common.Hex2Bytes("dadbe113e4132e0c0c3cd4867e0a2044d0e5a3d44b350677ed42fc9244d004d4"),
common.Hex2Bytes("aa7441fefc17d76aedfcaf692fe71014b94c1547b6d129562b34fc5995ca0d1a"),
[]byte{},
})
block2x00BranchNodeHash = crypto.Keccak256(block2x00BranchNode)
block2RootBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
block2x00BranchNodeHash,
common.Hex2Bytes("babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bd"),
common.Hex2Bytes("473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021"),
common.Hex2Bytes("bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0"),
block1x04BranchNodeHash,
common.Hex2Bytes("a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7"),
common.Hex2Bytes("e823850f50bf72baae9d1733a36a444ab65d0a6faaba404f0583ce0ca4dad92d"),
common.Hex2Bytes("f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721"),
common.Hex2Bytes("7117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681"),
common.Hex2Bytes("69eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472"),
common.Hex2Bytes("203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8e"),
common.Hex2Bytes("9287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208"),
common.Hex2Bytes("6fc2d754e304c48ce6a517753c62b1a9c1d5925b89707486d7fc08919e0a94ec"),
common.Hex2Bytes("7b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475"),
common.Hex2Bytes("51f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0"),
common.Hex2Bytes("89d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb"),
[]byte{},
})
// block3 data
// path 060e0f
blcok3CoinbaseBalance, _ = new(big.Int).SetString("5156250000000000000", 10)
block3CoinbaseAccount = &types.StateAccount{
Nonce: 0,
Balance: blcok3CoinbaseBalance,
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block3CoinbaseAccountRLP, _ = rlp.EncodeToBytes(block3CoinbaseAccount)
block3CoinbaseLeafNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("3a174f00e64521a535f35e67c1aa241951c791639b2f3d060f49c5d9fa8b9e"),
block3CoinbaseAccountRLP,
})
block3CoinbaseLeafNodeHash = crypto.Keccak256(block3CoinbaseLeafNode)
// path 0c0e050703
block3MovedPremineBalance1, _ = new(big.Int).SetString("3750000000000000000", 10)
block3MovedPremineAccount1 = &types.StateAccount{
Nonce: 0,
Balance: block3MovedPremineBalance1,
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block3MovedPremineAccount1RLP, _ = rlp.EncodeToBytes(block3MovedPremineAccount1)
block3MovedPremineLeafNode1, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("3ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190"), // ce573ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190
block3MovedPremineAccount1RLP,
})
block3MovedPremineLeafNodeHash1 = crypto.Keccak256(block3MovedPremineLeafNode1)
// path 0c0e050708
block3MovedPremineBalance2, _ = new(big.Int).SetString("1999944000000000000000", 10)
block3MovedPremineAccount2 = &types.StateAccount{
Nonce: 0,
Balance: block3MovedPremineBalance2,
CodeHash: test_helpers.NullCodeHash.Bytes(),
Root: test_helpers.EmptyContractRoot,
}
block3MovedPremineAccount2RLP, _ = rlp.EncodeToBytes(block3MovedPremineAccount2)
block3MovedPremineLeafNode2, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("33bc1e69eedf90f402e11f6862da14ed8e50156635a04d6393bbae154012"), // ce5783bc1e69eedf90f402e11f6862da14ed8e50156635a04d6393bbae154012
block3MovedPremineAccount2RLP,
})
block3MovedPremineLeafNodeHash2 = crypto.Keccak256(block3MovedPremineLeafNode2)
block3x0c0e0507BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
[]byte{},
[]byte{},
[]byte{},
block3MovedPremineLeafNodeHash1,
[]byte{},
[]byte{},
[]byte{},
[]byte{},
block3MovedPremineLeafNodeHash2,
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
})
block3x0c0e0507BranchNodeHash = crypto.Keccak256(block3x0c0e0507BranchNode)
block3x0c0e05BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("452e3beb503b1d87ae7c672b98a8e3fd043a671405502562ae1043dc97151a50"),
[]byte{},
common.Hex2Bytes("2f5bb16f77086f67ce8c4258cb9061cb299e597b2ad4ad6d7ccc474d6d88e85e"),
[]byte{},
[]byte{},
[]byte{},
[]byte{},
block3x0c0e0507BranchNodeHash,
[]byte{},
common.Hex2Bytes("44623e5a9319f83870db0ea4611a25fca1e1da3eeea2be4a091dfc15ab45689e"),
common.Hex2Bytes("b41e047a97f44fa4cb8146467b88c8f4705811029d9e170abb0aba7d0af9f0da"),
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
[]byte{},
})
block3x0c0e05BranchNodeHash = crypto.Keccak256(block3x0c0e05BranchNode)
block3x060eBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("94d77c7c30b88829c9989948b206cda5e532b38b49534261c517aebf4a3e6fdb"),
common.Hex2Bytes("a5cf57a50da8204964e834a12a53f9bed7afc9b700a4a81b440122d60c7603a7"),
[]byte{},
common.Hex2Bytes("3730ec0571f34b6c3b178dc26ccb31a3f50c29da9b1921e41b9477ddab41b0fe"),
[]byte{},
common.Hex2Bytes("543952bb9566c2018cf8d7b90d6a7903cdfff3d79ac36189be5322de42fc3fc0"),
[]byte{},
common.Hex2Bytes("c4a49b66f0bcc08531e50cdea5577a281d111fa542eaefd9a9aead8febb0735e"),
common.Hex2Bytes("362ad58916c71463b98c079649fc486c5f082c4f548bd4ab501515f0c5641cb4"),
common.Hex2Bytes("36aae109f6f55f0bd05eb05bb365af2332dfe5f06d3d17903e88534c319eb709"),
common.Hex2Bytes("430dcfc5cc49a6b490dd54138920e8f94e427239c2bccc14705cfd4ff6cc4383"),
common.Hex2Bytes("73ed77563dfed2fdb38900b474db88b2270f449167e0d877fda9e2229f119fe8"),
common.Hex2Bytes("5dfe06013f2a41f1779194ceb07769d019f518b2a694a82fa1661e60fd973eaa"),
common.Hex2Bytes("80bdfd85fbb6b45850bad6e34136aaa1b04711e47469fa2f0d19eca52089efb5"),
[]byte{},
block3CoinbaseLeafNodeHash,
[]byte{},
})
block3x060eBranchNodeHash = crypto.Keccak256(block3x060eBranchNode)
block3x0c0eBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("70647f11b2b995d718f9e8aceb44c8839e0055641930d216fa6090280a9d63d5"),
common.Hex2Bytes("fdfb17cd2fba2a14219981cb7886a1977cd85dbef5c767c562f4a5f547febff0"),
common.Hex2Bytes("ff87313253ec6f860142b7bf62efb4cb07ea668c57aa90cbe9ef22b72fee15c7"),
common.Hex2Bytes("3a77b3c26a54ad37bdf4e19c1bce93493ec0f79d9ad90190b70bc840b54918e1"),
common.Hex2Bytes("af1b3b14324561b68f2e24dbcc28673ab35ce3fd0230fe2bc86b3d1931745195"),
block3x0c0e05BranchNodeHash,
common.Hex2Bytes("647dcbfe6aabcd9d219ff40422af4326bfc1ec66703195a78eb48618ddef248d"),
common.Hex2Bytes("2d2bf06159cc8928283c3419a03f08ea34c493a9d002a0ec76d5c429508ccaf4"),
common.Hex2Bytes("d7147251b3f48f25e1e4c6d8f83a00b1eca66e99a4ea0d238942ce72d0ba6414"),
common.Hex2Bytes("cb859370869967594fb29f4e2904413310146733d7fcbd11407f3e47626e0e34"),
common.Hex2Bytes("b93ab9b0bd83963860fbe0b7d543879cfde756ea1618d2a40d85483058cc5a26"),
common.Hex2Bytes("45aee096499d209931457ce251c5c7e5543f22524f67785ff8f0f3f02588b0ed"),
[]byte{},
common.Hex2Bytes("aa2ae9379797c5066bba646108074ae8677e82c923d584b6d1c1268ca3708c5c"),
common.Hex2Bytes("e6eb055f0d8e194c083471479a3de87fa0f90c0f4aaa518416ec1e469ec32e3a"),
common.Hex2Bytes("0cc9c50fc7eba162fb17f2e04e3599c13abbf210d9781864d0edec401ecaebba"),
[]byte{},
})
block3x0c0eBranchNodeHash = crypto.Keccak256(block3x0c0eBranchNode)
block3x06BranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("68f7ff8c074d6e4cccd55b5b1c2116a6dd7047d4332090e6db8839362991b0ae"),
common.Hex2Bytes("c446eb4377c750701374c56e50759e6ba68b7adf4d543e718c8b28a99ae3b6ad"),
common.Hex2Bytes("ef2c49ec64cb65eae0d99684e74c8af2bd0206c9a0214d9d3eddf0881dd8412a"),
common.Hex2Bytes("7096c4cc7e8125f0b142d8644ad681f8a8142e210c806f33f3f7004f0e9d6002"),
common.Hex2Bytes("bc9a8ae647b234cd6607b6b0245e3b3d5ec4f7ea006e7eda1f92d02f0ea91116"),
common.Hex2Bytes("a87720deb92ff2f899e809befab9970a61c86148c4fa09d04b77505ee4a5bda5"),
common.Hex2Bytes("2460e5b6ded7c0001de29c15db124614432fef6486370cc9970f63b0d95fd5e2"),
common.Hex2Bytes("ed1c447d4a32bc31e9e32259dc63da10df91231e786332e3df122b301b1f8fc3"),
common.Hex2Bytes("0d27dfc201d995c2323b792860dbca087da7cc56d1698c39b7c4b9277729c5ca"),
common.Hex2Bytes("f6d2be168d9c17643c9ea80c29322b364604cdfd36eef40123d83fad364e43fa"),
common.Hex2Bytes("004bf1c30a5730f464de1a0ba4ac5b5618df66d6106073d08742166e33a7eeb5"),
common.Hex2Bytes("7298d019a57a1b04ac31ed874d654ba0d3c249704c5d9efa1d08959fc89e0779"),
common.Hex2Bytes("fb3d50b7af6f839e371ff8ebd0322e94e6b6fb7888416737f88cf55bcf5859ec"),
common.Hex2Bytes("4e7a2618fa1fc560a73c24839657adf7e48d600ecfb12333678115936597a913"),
block3x060eBranchNodeHash,
common.Hex2Bytes("1909706c5db040f54c19f4050659ad484982145b02474653917de379f15ebb36"),
[]byte{},
})
block3x06BranchNodeHash = crypto.Keccak256(block3x06BranchNode)
block3x0cBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("dae48f5b47930c28bb116fbd55e52cd47242c71bf55373b55eb2805ee2e4a929"),
common.Hex2Bytes("0f1f37f337ec800e2e5974e2e7355f10f1a4832b39b846d916c3597a460e0676"),
common.Hex2Bytes("da8f627bb8fbeead17b318e0a8e4f528db310f591bb6ab2deda4a9f7ca902ab5"),
common.Hex2Bytes("971c662648d58295d0d0aa4b8055588da0037619951217c22052802549d94a2f"),
common.Hex2Bytes("ccc701efe4b3413fd6a61a6c9f40e955af774649a8d9fd212d046a5a39ddbb67"),
common.Hex2Bytes("d607cdb32e2bd635ee7f2f9e07bc94ddbd09b10ec0901b66628e15667aec570b"),
common.Hex2Bytes("5b89203dc940e6fa70ec19ad4e01d01849d3a5baa0a8f9c0525256ed490b159f"),
common.Hex2Bytes("b84227d48df68aecc772939a59afa9e1a4ab578f7b698bdb1289e29b6044668e"),
common.Hex2Bytes("fd1c992070b94ace57e48cbf6511a16aa770c645f9f5efba87bbe59d0a042913"),
common.Hex2Bytes("e16a7ccea6748ae90de92f8aef3b3dc248a557b9ac4e296934313f24f7fced5f"),
common.Hex2Bytes("42373cf4a00630d94de90d0a23b8f38ced6b0f7cb818b8925fee8f0c2a28a25a"),
common.Hex2Bytes("5f89d2161c1741ff428864f7889866484cef622de5023a46e795dfdec336319f"),
common.Hex2Bytes("7597a017664526c8c795ce1da27b8b72455c49657113e0455552dbc068c5ba31"),
common.Hex2Bytes("d5be9089012fda2c585a1b961e988ea5efcd3a06988e150a8682091f694b37c5"),
block3x0c0eBranchNodeHash,
common.Hex2Bytes("49bf6e8df0acafd0eff86defeeb305568e44d52d2235cf340ae15c6034e2b241"),
[]byte{},
})
block3x0cBranchNodeHash = crypto.Keccak256(block3x0cBranchNode)
block3RootBranchNode, _ = rlp.EncodeToBytes(&[]interface{}{
common.Hex2Bytes("f646da473c426e79f1c796b00d4873f47de1dbe1c9d19d63993a05eeb8b4041d"),
common.Hex2Bytes("babe369f6b12092f49181ae04ca173fb68d1a5456f18d20fa32cba73954052bd"),
common.Hex2Bytes("473ecf8a7e36a829e75039a3b055e51b8332cbf03324ab4af2066bbd6fbf0021"),
common.Hex2Bytes("bbda34753d7aa6c38e603f360244e8f59611921d9e1f128372fec0d586d4f9e0"),
common.Hex2Bytes("d9cff5d5f2418afd16a4da5c221fdc8bd47520c5927922f69a68177b64da6ac0"),
common.Hex2Bytes("a5f3f2f7542148c973977c8a1e154c4300fec92f755f7846f1b734d3ab1d90e7"),
block3x06BranchNodeHash,
common.Hex2Bytes("f7a00cbe7d4b30b11faea3ae61b7f1f2b315b61d9f6bd68bfe587ad0eeceb721"),
common.Hex2Bytes("7117ef9fc932f1a88e908eaead8565c19b5645dc9e5b1b6e841c5edbdfd71681"),
common.Hex2Bytes("69eb2de283f32c11f859d7bcf93da23990d3e662935ed4d6b39ce3673ec84472"),
common.Hex2Bytes("203d26456312bbc4da5cd293b75b840fc5045e493d6f904d180823ec22bfed8e"),
common.Hex2Bytes("9287b5c21f2254af4e64fca76acc5cd87399c7f1ede818db4326c98ce2dc2208"),
block3x0cBranchNodeHash,
common.Hex2Bytes("7b1c54f15e299bd58bdfef9741538c7828b5d7d11a489f9c20d052b3471df475"),
common.Hex2Bytes("51f9dd3739a927c89e357580a4c97b40234aa01ed3d5e0390dc982a7975880a0"),
common.Hex2Bytes("89d613f26159af43616fd9455bb461f4869bfede26f2130835ed067a8b967bfb"),
[]byte{},
})
)
func init() {
if os.Getenv("MODE") != "statediff" {
fmt.Println("Skipping statediff test")
os.Exit(0)
}
db = rawdb.NewMemoryDatabase()
genesisBlock = core.DefaultGenesisBlock().MustCommit(db)
genBy, err := rlp.EncodeToBytes(genesisBlock)
if err != nil {
log.Fatal(err)
}
var block0RLP []byte
block0, block0RLP, err = loadBlockFromRLPFile("./block0_rlp")
if err != nil {
log.Fatal(err)
}
if !bytes.Equal(genBy, block0RLP) {
log.Fatal("mainnet genesis blocks do not match")
}
block1, _, err = loadBlockFromRLPFile("./block1_rlp")
if err != nil {
log.Fatal(err)
}
block1CoinbaseAddr = block1.Coinbase()
block1CoinbaseHash = crypto.Keccak256Hash(block1CoinbaseAddr.Bytes())
block2, _, err = loadBlockFromRLPFile("./block2_rlp")
if err != nil {
log.Fatal(err)
}
block2CoinbaseAddr = block2.Coinbase()
block2CoinbaseHash = crypto.Keccak256Hash(block2CoinbaseAddr.Bytes()) // 0x08d4679cbcf198c1741a6f4e4473845659a30caa8b26f8d37a0be2e2bc0d8892
block3, _, err = loadBlockFromRLPFile("./block3_rlp")
if err != nil {
log.Fatal(err)
}
block3CoinbaseAddr = block3.Coinbase()
block3CoinbaseHash = crypto.Keccak256Hash(block3CoinbaseAddr.Bytes())
}
func loadBlockFromRLPFile(filename string) (*types.Block, []byte, error) {
f, err := os.Open(filename)
if err != nil {
return nil, nil, err
}
defer f.Close()
blockRLP, err := io.ReadAll(f)
if err != nil {
return nil, nil, err
}
block := new(types.Block)
return block, blockRLP, rlp.DecodeBytes(blockRLP, block)
}
func TestBuilderOnMainnetBlocks(t *testing.T) {
chain, _ := core.NewBlockChain(db, nil, nil, nil, ethash.NewFaker(), vm.Config{}, nil, nil)
_, err := chain.InsertChain([]*types.Block{block1, block2, block3})
if err != nil {
t.Error(err)
}
params := statediff.Params{}
builder = statediff.NewBuilder(chain.StateCache())
var tests = []struct {
name string
startingArguments statediff.Args
expected *sdtypes.StateObject
}{
// note that block0 (genesis) has over 1000 nodes due to the pre-allocation for the crowd-sale
// it is not feasible to write a unit test of that size at this time
{
"testBlock1",
//10000 transferred from testBankAddress to account1Addr
statediff.Args{
OldStateRoot: block0.Root(),
NewStateRoot: block1.Root(),
BlockNumber: block1.Number(),
BlockHash: block1.Hash(),
},
&sdtypes.StateObject{
BlockNumber: block1.Number(),
BlockHash: block1.Hash(),
Nodes: []sdtypes.StateLeafNode{
{
Removed: false,
AccountWrapper: sdtypes.AccountWrapper{
Account: block1CoinbaseAccount,
LeafKey: block1CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(),
},
StorageDiff: emptyStorage,
},
},
IPLDs: []sdtypes.IPLD{
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1RootBranchNode)).String(),
Content: block1RootBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1x04BranchNode)).String(),
Content: block1x04BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1x040bBranchNode)).String(),
Content: block1x040bBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block1CoinbaseLeafNode)).String(),
Content: block1CoinbaseLeafNode,
},
},
},
},
{
"testBlock2",
// 1000 transferred from testBankAddress to account1Addr
// 1000 transferred from account1Addr to account2Addr
// account1addr creates a new contract
statediff.Args{
OldStateRoot: block1.Root(),
NewStateRoot: block2.Root(),
BlockNumber: block2.Number(),
BlockHash: block2.Hash(),
},
&sdtypes.StateObject{
BlockNumber: block2.Number(),
BlockHash: block2.Hash(),
Nodes: []sdtypes.StateLeafNode{
{
Removed: false,
AccountWrapper: sdtypes.AccountWrapper{
Account: block2CoinbaseAccount,
LeafKey: block2CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(),
},
StorageDiff: emptyStorage,
},
},
IPLDs: []sdtypes.IPLD{
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2RootBranchNode)).String(),
Content: block2RootBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x00BranchNode)).String(),
Content: block2x00BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x0008BranchNode)).String(),
Content: block2x0008BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2x00080dBranchNode)).String(),
Content: block2x00080dBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2MovedPremineLeafNode)).String(),
Content: block2MovedPremineLeafNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block2CoinbaseLeafNode)).String(),
Content: block2CoinbaseLeafNode,
},
},
},
},
{
"testBlock3",
//the contract's storage is changed
//and the block is mined by account 2
statediff.Args{
OldStateRoot: block2.Root(),
NewStateRoot: block3.Root(),
BlockNumber: block3.Number(),
BlockHash: block3.Hash(),
},
&sdtypes.StateObject{
BlockNumber: block3.Number(),
BlockHash: block3.Hash(),
Nodes: []sdtypes.StateLeafNode{
{ // How was this account created???
Removed: false,
AccountWrapper: sdtypes.AccountWrapper{
Account: block3MovedPremineAccount1,
LeafKey: common.HexToHash("ce573ced93917e658d10e2d9009470dad72b63c898d173721194a12f2ae5e190").Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(),
},
StorageDiff: emptyStorage,
},
{ // this is the new account created due to the coinbase mining a block, it's creation shouldn't affect 0x 0e 05 07
Removed: false,
AccountWrapper: sdtypes.AccountWrapper{
Account: block3CoinbaseAccount,
LeafKey: block3CoinbaseHash.Bytes(),
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(),
},
StorageDiff: emptyStorage,
},
},
IPLDs: []sdtypes.IPLD{
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3RootBranchNode)).String(),
Content: block3RootBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x06BranchNode)).String(),
Content: block3x06BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x060eBranchNode)).String(),
Content: block3x060eBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0cBranchNode)).String(),
Content: block3x0cBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0eBranchNode)).String(),
Content: block3x0c0eBranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0e05BranchNode)).String(),
Content: block3x0c0e05BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3x0c0e0507BranchNode)).String(),
Content: block3x0c0e0507BranchNode,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode1)).String(),
Content: block3MovedPremineLeafNode1,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3MovedPremineLeafNode2)).String(),
Content: block3MovedPremineLeafNode2,
},
{
CID: ipld2.Keccak256ToCid(ipld2.MEthStateTrie, crypto.Keccak256(block3CoinbaseLeafNode)).String(),
Content: block3CoinbaseLeafNode,
},
},
},
},
}
for _, test := range tests {
diff, err := builder.BuildStateDiffObject(test.startingArguments, params)
if err != nil {
t.Error(err)
}
receivedStateDiffRlp, err := rlp.EncodeToBytes(diff)
if err != nil {
t.Error(err)
}
expectedStateDiffRlp, err := rlp.EncodeToBytes(&test.expected)
if err != nil {
t.Error(err)
}
sort.Slice(receivedStateDiffRlp, func(i, j int) bool { return receivedStateDiffRlp[i] < receivedStateDiffRlp[j] })
sort.Slice(expectedStateDiffRlp, func(i, j int) bool { return expectedStateDiffRlp[i] < expectedStateDiffRlp[j] })
if !bytes.Equal(receivedStateDiffRlp, expectedStateDiffRlp) {
actual, err := json.Marshal(diff)
if err != nil {
t.Error(err)
}
expected, err := json.Marshal(test.expected)
if err != nil {
t.Error(err)
}
t.Logf("Test failed: %s", test.name)
t.Errorf("actual state diff: %s\r\n\r\n\r\nexpected state diff: %s", actual, expected)
}
}
if !bytes.Equal(crypto.Keccak256(block1RootBranchNode), block1.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block1RootBranchNode), block1.Root().Bytes())
}
if !bytes.Equal(crypto.Keccak256(block2RootBranchNode), block2.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block2RootBranchNode), block2.Root().Bytes())
}
if !bytes.Equal(crypto.Keccak256(block3RootBranchNode), block3.Root().Bytes()) {
t.Errorf("actual state root: %s\r\nexpected state root: %s", crypto.Keccak256(block3RootBranchNode), block3.Root().Bytes())
}
}

108
metrics.go Normal file
View File

@ -0,0 +1,108 @@
// VulcanizeDB
// Copyright © 2021 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 statediff
import (
"strings"
"github.com/ethereum/go-ethereum/metrics"
)
const (
namespace = "statediff"
)
var defaultStatediffMetrics = RegisterStatediffMetrics(metrics.DefaultRegistry)
// Build a fully qualified metric name
func metricName(subsystem, name string) string {
if name == "" {
return ""
}
parts := []string{namespace, name}
if subsystem != "" {
parts = []string{namespace, subsystem, name}
}
// Prometheus uses _ but geth metrics uses / and replaces
return strings.Join(parts, "/")
}
type statediffMetricsHandles struct {
// Height of latest synced by core.BlockChain
// FIXME
lastSyncHeight metrics.Gauge
// Height of the latest block received from chainEvent channel
lastEventHeight metrics.Gauge
// Height of latest state diff
lastStatediffHeight metrics.Gauge
// Current length of chainEvent channels
serviceLoopChannelLen metrics.Gauge
writeLoopChannelLen metrics.Gauge
// The start block of the known gap
knownGapStart metrics.Gauge
// The end block of the known gap
knownGapEnd metrics.Gauge
// A known gaps start block which had an error being written to the DB
knownGapErrorStart metrics.Gauge
// A known gaps end block which had an error being written to the DB
knownGapErrorEnd metrics.Gauge
apiRequests metrics.Counter
apiRequestsUnderway metrics.Counter
failed metrics.Counter
succeeded metrics.Counter
underway metrics.Counter
totalProcessingTime metrics.Gauge
}
func RegisterStatediffMetrics(reg metrics.Registry) statediffMetricsHandles {
ctx := statediffMetricsHandles{
lastSyncHeight: metrics.NewGauge(),
lastEventHeight: metrics.NewGauge(),
lastStatediffHeight: metrics.NewGauge(),
serviceLoopChannelLen: metrics.NewGauge(),
writeLoopChannelLen: metrics.NewGauge(),
knownGapStart: metrics.NewGauge(),
knownGapEnd: metrics.NewGauge(),
knownGapErrorStart: metrics.NewGauge(),
knownGapErrorEnd: metrics.NewGauge(),
apiRequests: metrics.NewCounter(),
apiRequestsUnderway: metrics.NewCounter(),
failed: metrics.NewCounter(),
succeeded: metrics.NewCounter(),
underway: metrics.NewCounter(),
totalProcessingTime: metrics.NewGauge(),
}
subsys := "service"
reg.Register(metricName(subsys, "last_sync_height"), ctx.lastSyncHeight)
reg.Register(metricName(subsys, "last_event_height"), ctx.lastEventHeight)
reg.Register(metricName(subsys, "last_statediff_height"), ctx.lastStatediffHeight)
reg.Register(metricName(subsys, "service_loop_channel_len"), ctx.serviceLoopChannelLen)
reg.Register(metricName(subsys, "write_loop_channel_len"), ctx.writeLoopChannelLen)
reg.Register(metricName(subsys, "known_gaps_start"), ctx.knownGapStart)
reg.Register(metricName(subsys, "known_gaps_end"), ctx.knownGapEnd)
reg.Register(metricName(subsys, "known_gaps_error_start"), ctx.knownGapErrorStart)
reg.Register(metricName(subsys, "known_gaps_error_end"), ctx.knownGapErrorEnd)
reg.Register(metricName(subsys, "api_requests"), ctx.apiRequests)
reg.Register(metricName(subsys, "api_requests_underway"), ctx.apiRequestsUnderway)
reg.Register(metricName(subsys, "failed"), ctx.failed)
reg.Register(metricName(subsys, "succeeded"), ctx.succeeded)
reg.Register(metricName(subsys, "underway"), ctx.underway)
reg.Register(metricName(subsys, "total_processing_time"), ctx.totalProcessingTime)
return ctx
}

89
metrics_helpers.go Normal file
View File

@ -0,0 +1,89 @@
// Copyright 2019 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library 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 Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package statediff
import (
"fmt"
"time"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/log"
)
func countStateDiffBegin(block *types.Block) (time.Time, log.Logger) {
start := time.Now()
logger := log.New("hash", block.Hash().Hex(), "number", block.NumberU64())
defaultStatediffMetrics.underway.Inc(1)
logger.Debug(fmt.Sprintf("writeStateDiff BEGIN [underway=%d, succeeded=%d, failed=%d, total_time=%dms]",
defaultStatediffMetrics.underway.Count(),
defaultStatediffMetrics.succeeded.Count(),
defaultStatediffMetrics.failed.Count(),
defaultStatediffMetrics.totalProcessingTime.Value(),
))
return start, logger
}
func countStateDiffEnd(start time.Time, logger log.Logger, err error) time.Duration {
duration := time.Since(start)
defaultStatediffMetrics.underway.Dec(1)
if nil == err {
defaultStatediffMetrics.succeeded.Inc(1)
} else {
defaultStatediffMetrics.failed.Inc(1)
}
defaultStatediffMetrics.totalProcessingTime.Inc(duration.Milliseconds())
logger.Debug(fmt.Sprintf("writeStateDiff END (duration=%dms, err=%t) [underway=%d, succeeded=%d, failed=%d, total_time=%dms]",
duration.Milliseconds(), nil != err,
defaultStatediffMetrics.underway.Count(),
defaultStatediffMetrics.succeeded.Count(),
defaultStatediffMetrics.failed.Count(),
defaultStatediffMetrics.totalProcessingTime.Value(),
))
return duration
}
func countApiRequestBegin(methodName string, blockHashOrNumber interface{}) (time.Time, log.Logger) {
start := time.Now()
logger := log.New(methodName, blockHashOrNumber)
defaultStatediffMetrics.apiRequests.Inc(1)
defaultStatediffMetrics.apiRequestsUnderway.Inc(1)
logger.Debug(fmt.Sprintf("statediff API BEGIN [underway=%d, requests=%d])",
defaultStatediffMetrics.apiRequestsUnderway.Count(),
defaultStatediffMetrics.apiRequests.Count(),
))
return start, logger
}
func countApiRequestEnd(start time.Time, logger log.Logger, err error) time.Duration {
duration := time.Since(start)
defaultStatediffMetrics.apiRequestsUnderway.Dec(1)
logger.Debug(fmt.Sprintf("statediff API END (duration=%dms, err=%t) [underway=%d, requests=%d]",
duration.Milliseconds(), nil != err,
defaultStatediffMetrics.apiRequestsUnderway.Count(),
defaultStatediffMetrics.apiRequests.Count(),
))
return duration
}

57
payload.go Normal file
View File

@ -0,0 +1,57 @@
// VulcanizeDB
// Copyright © 2021 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 statediff
import (
"encoding/json"
"math/big"
)
// Payload packages the data to send to statediff subscriptions
type Payload struct {
BlockRlp []byte `json:"blockRlp"`
TotalDifficulty *big.Int `json:"totalDifficulty"`
ReceiptsRlp []byte `json:"receiptsRlp"`
StateObjectRlp []byte `json:"stateObjectRlp" gencodec:"required"`
encoded []byte
err error
}
func (sd *Payload) ensureEncoded() {
if sd.encoded == nil && sd.err == nil {
sd.encoded, sd.err = json.Marshal(sd)
}
}
// Length to implement Encoder interface for Payload
func (sd *Payload) Length() int {
sd.ensureEncoded()
return len(sd.encoded)
}
// Encode to implement Encoder interface for Payload
func (sd *Payload) Encode() ([]byte, error) {
sd.ensureEncoded()
return sd.encoded, sd.err
}
// Subscription struct holds our subscription channels
type Subscription struct {
PayloadChan chan<- Payload
QuitChan chan<- bool
}

Some files were not shown because too many files have changed in this diff Show More