diff --git a/Gopkg.toml b/Gopkg.toml index 14d43eb8..dda6c81e 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -52,7 +52,7 @@ [[constraint]] name = "github.com/ethereum/go-ethereum" source = "github.com/vulcanize/go-ethereum" - branch = "rpc_statediffs_at_head" + branch = "rpc_statediffing" [[constraint]] name = "github.com/vulcanize/eth-block-extractor" diff --git a/db/migrations/00028_create_header_cids_table.sql b/db/migrations/00028_create_header_cids_table.sql index e282fe01..8d63f299 100644 --- a/db/migrations/00028_create_header_cids_table.sql +++ b/db/migrations/00028_create_header_cids_table.sql @@ -4,6 +4,7 @@ CREATE TABLE public.header_cids ( block_number BIGINT NOT NULL, block_hash VARCHAR(66) NOT NULL, cid TEXT NOT NULL, + uncle BOOLEAN NOT NULL, UNIQUE (block_number, block_hash) ); diff --git a/db/migrations/00031_create_state_cids_table.sql b/db/migrations/00031_create_state_cids_table.sql index e7b9e48f..b040bdee 100644 --- a/db/migrations/00031_create_state_cids_table.sql +++ b/db/migrations/00031_create_state_cids_table.sql @@ -2,9 +2,10 @@ CREATE TABLE public.state_cids ( id SERIAL PRIMARY KEY, header_id INTEGER NOT NULL REFERENCES header_cids (id) ON DELETE CASCADE, - account_key VARCHAR(66) NOT NULL, + state_key VARCHAR(66) NOT NULL, + leaf BOOLEAN NOT NULL, cid TEXT NOT NULL, - UNIQUE (header_id, account_key) + UNIQUE (header_id, state_key) ); -- +goose Down diff --git a/db/migrations/00032_create_storage_cids_table.sql b/db/migrations/00032_create_storage_cids_table.sql index 319fd0ec..607ccb7c 100644 --- a/db/migrations/00032_create_storage_cids_table.sql +++ b/db/migrations/00032_create_storage_cids_table.sql @@ -3,6 +3,7 @@ CREATE TABLE public.storage_cids ( id SERIAL PRIMARY KEY, state_id INTEGER NOT NULL REFERENCES state_cids (id) ON DELETE CASCADE, storage_key VARCHAR(66) NOT NULL, + leaf BOOLEAN NOT NULL, cid TEXT NOT NULL, UNIQUE (state_id, storage_key) ); diff --git a/pkg/ipfs/converter.go b/pkg/ipfs/converter.go index f4e8e371..8fdc5788 100644 --- a/pkg/ipfs/converter.go +++ b/pkg/ipfs/converter.go @@ -63,8 +63,8 @@ func (pc *Converter) Convert(payload statediff.Payload) (*IPLDPayload, error) { TrxMetaData: make([]*TrxMetaData, 0, trxLen), Receipts: make(types.Receipts, 0, trxLen), ReceiptMetaData: make([]*ReceiptMetaData, 0, trxLen), - StateLeafs: make(map[common.Hash][]byte), - StorageLeafs: make(map[common.Hash]map[common.Hash][]byte), + StateNodes: make(map[common.Hash]StateNode), + StorageNodes: make(map[common.Hash][]StorageNode), } for gethTransactionIndex, trx := range block.Transactions() { // Extract to and from data from the the transactions for indexing @@ -105,25 +105,49 @@ func (pc *Converter) Convert(payload statediff.Payload) (*IPLDPayload, error) { if err != nil { return nil, err } - for addr, createdAccount := range stateDiff.CreatedAccounts { - convertedPayload.StateLeafs[addr] = createdAccount.Value - convertedPayload.StorageLeafs[addr] = make(map[common.Hash][]byte) + for _, createdAccount := range stateDiff.CreatedAccounts { + hashKey := common.BytesToHash(createdAccount.Key) + convertedPayload.StateNodes[hashKey] = StateNode{ + Value: createdAccount.Value, + Leaf: createdAccount.Leaf, + } + convertedPayload.StorageNodes[hashKey] = make([]StorageNode, 0) for _, storageDiff := range createdAccount.Storage { - convertedPayload.StorageLeafs[addr][common.BytesToHash(storageDiff.Key)] = storageDiff.Value + convertedPayload.StorageNodes[hashKey] = append(convertedPayload.StorageNodes[hashKey], StorageNode{ + Key: common.BytesToHash(storageDiff.Key), + Value: storageDiff.Value, + Leaf: storageDiff.Leaf, + }) } } - for addr, deletedAccount := range stateDiff.DeletedAccounts { - convertedPayload.StateLeafs[addr] = deletedAccount.Value - convertedPayload.StorageLeafs[addr] = make(map[common.Hash][]byte) + for _, deletedAccount := range stateDiff.DeletedAccounts { + hashKey := common.BytesToHash(deletedAccount.Key) + convertedPayload.StateNodes[hashKey] = StateNode{ + Value: deletedAccount.Value, + Leaf: deletedAccount.Leaf, + } + convertedPayload.StorageNodes[hashKey] = make([]StorageNode, 0) for _, storageDiff := range deletedAccount.Storage { - convertedPayload.StorageLeafs[addr][common.BytesToHash(storageDiff.Key)] = storageDiff.Value + convertedPayload.StorageNodes[hashKey] = append(convertedPayload.StorageNodes[hashKey], StorageNode{ + Key: common.BytesToHash(storageDiff.Key), + Value: storageDiff.Value, + Leaf: storageDiff.Leaf, + }) } } - for addr, updatedAccount := range stateDiff.UpdatedAccounts { - convertedPayload.StateLeafs[addr] = updatedAccount.Value - convertedPayload.StorageLeafs[addr] = make(map[common.Hash][]byte) + for _, updatedAccount := range stateDiff.UpdatedAccounts { + hashKey := common.BytesToHash(updatedAccount.Key) + convertedPayload.StateNodes[hashKey] = StateNode{ + Value: updatedAccount.Value, + Leaf: updatedAccount.Leaf, + } + convertedPayload.StorageNodes[hashKey] = make([]StorageNode, 0) for _, storageDiff := range updatedAccount.Storage { - convertedPayload.StorageLeafs[addr][common.BytesToHash(storageDiff.Key)] = storageDiff.Value + convertedPayload.StorageNodes[hashKey] = append(convertedPayload.StorageNodes[hashKey], StorageNode{ + Key: common.BytesToHash(storageDiff.Key), + Value: storageDiff.Value, + Leaf: storageDiff.Leaf, + }) } } return convertedPayload, nil diff --git a/pkg/ipfs/converter_test.go b/pkg/ipfs/converter_test.go index b670e88f..859d215d 100644 --- a/pkg/ipfs/converter_test.go +++ b/pkg/ipfs/converter_test.go @@ -34,9 +34,5 @@ var _ = Describe("Converter", func() { Expect(ipldPayload).To(Equal(&test_helpers.MockIPLDPayload)) Expect(mockConverter.PassedStatediffPayload).To(Equal(test_helpers.MockStatediffPayload)) }) - - It("Fails if", func() { - - }) }) }) diff --git a/pkg/ipfs/fetcher.go b/pkg/ipfs/fetcher.go index 83635050..bdadfb57 100644 --- a/pkg/ipfs/fetcher.go +++ b/pkg/ipfs/fetcher.go @@ -24,37 +24,31 @@ import ( "github.com/ipfs/go-cid" ) -// IPLDFetcher is the interface for fetching IPLD objects from IPFS -type IPLDFetcher interface { - Fetch(cid cid.Cid) (blocks.Block, error) - FetchBatch(cids []cid.Cid) []blocks.Block -} - -// Fetcher is the underlying struct which supports the IPLDFetcher interface -type Fetcher struct { +// IPLDFetcher is the underlying struct which supports a IPLD fetching interface +type IPLDFetcher struct { BlockService blockservice.BlockService } -// NewIPLDFetcher creates a pointer to a new Fetcher which satisfies the IPLDFetcher interface -func NewIPLDFetcher(ipfsPath string) (*Fetcher, error) { +// NewIPLDFetcher creates a pointer to a new IPLDFetcher +func NewIPLDFetcher(ipfsPath string) (*IPLDFetcher, error) { blockService, err := InitIPFSBlockService(ipfsPath) if err != nil { return nil, err } - return &Fetcher{ + return &IPLDFetcher{ BlockService: blockService, }, nil } -// Fetch is used to fetch a batch of IPFS data blocks by cid -func (f *Fetcher) Fetch(cid cid.Cid) (blocks.Block, error) { +// Fetch is used to fetch a single block of IPFS data by cid +func (f *IPLDFetcher) Fetch(cid cid.Cid) (blocks.Block, error) { return f.BlockService.GetBlock(context.Background(), cid) } // FetchBatch is used to fetch a batch of IPFS data blocks by cid // There is no guarantee all are fetched, and no error in such a case, so // downstream we will need to confirm which CIDs were fetched in the result set -func (f *Fetcher) FetchBatch(cids []cid.Cid) []blocks.Block { +func (f *IPLDFetcher) FetchBatch(cids []cid.Cid) []blocks.Block { fetchedBlocks := make([]blocks.Block, 0, len(cids)) blockChan := f.BlockService.GetBlocks(context.Background(), cids) for block := range blockChan { diff --git a/pkg/ipfs/ipfs_suite_test.go b/pkg/ipfs/ipfs_suite_test.go index f4369d4c..98eb91fa 100644 --- a/pkg/ipfs/ipfs_suite_test.go +++ b/pkg/ipfs/ipfs_suite_test.go @@ -17,10 +17,11 @@ package ipfs_test import ( - "github.com/sirupsen/logrus" "io/ioutil" "testing" + "github.com/sirupsen/logrus" + . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" ) diff --git a/pkg/ipfs/processor.go b/pkg/ipfs/processor.go index 038b12d6..451784cf 100644 --- a/pkg/ipfs/processor.go +++ b/pkg/ipfs/processor.go @@ -72,8 +72,8 @@ func (i *Processor) Process(wg *sync.WaitGroup) error { if err != nil { return err } + wg.Add(1) go func() { - wg.Add(1) for { select { case payload := <-i.PayloadChan: @@ -98,6 +98,7 @@ func (i *Processor) Process(wg *sync.WaitGroup) error { case err = <-sub.Err(): log.Error(err) case <-i.QuitChan: + println("quiting") log.Info("quiting IPFSProcessor") wg.Done() return diff --git a/pkg/ipfs/processor_test.go b/pkg/ipfs/processor_test.go index bf23d00d..a5664078 100644 --- a/pkg/ipfs/processor_test.go +++ b/pkg/ipfs/processor_test.go @@ -18,6 +18,7 @@ package ipfs_test import ( "sync" + "time" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/statediff" @@ -34,36 +35,43 @@ var _ = Describe("Processor", func() { Describe("Process", func() { It("Streams StatediffPayloads, converts them to IPLDPayloads, publishes IPLDPayloads, and indexes CIDPayloads", func() { wg := new(sync.WaitGroup) - payloadChan := make(chan statediff.Payload, 800) - processor := ipfs.Processor{ - Repository: &mocks.CIDRepository{ - ReturnErr: nil, - }, - Publisher: &mocks.IPLDPublisher{ - ReturnCIDPayload: &test_helpers.MockCIDPayload, - ReturnErr: nil, - }, - Streamer: &mocks.StateDiffStreamer{ - ReturnSub: &rpc.ClientSubscription{}, - StreamPayloads: []statediff.Payload{ - test_helpers.MockStatediffPayload, - }, - ReturnErr: nil, - WaitGroup: wg, - }, - Converter: &mocks.PayloadConverter{ - ReturnIPLDPayload: &test_helpers.MockIPLDPayload, - ReturnErr: nil, + payloadChan := make(chan statediff.Payload, 1) + quitChan := make(chan bool, 1) + mockCidRepo := &mocks.CIDRepository{ + ReturnErr: nil, + } + mockPublisher := &mocks.IPLDPublisher{ + ReturnCIDPayload: &test_helpers.MockCIDPayload, + ReturnErr: nil, + } + mockStreamer := &mocks.StateDiffStreamer{ + ReturnSub: &rpc.ClientSubscription{}, + StreamPayloads: []statediff.Payload{ + test_helpers.MockStatediffPayload, }, + ReturnErr: nil, + } + mockConverter := &mocks.PayloadConverter{ + ReturnIPLDPayload: &test_helpers.MockIPLDPayload, + ReturnErr: nil, + } + processor := &ipfs.Processor{ + Repository: mockCidRepo, + Publisher: mockPublisher, + Streamer: mockStreamer, + Converter: mockConverter, PayloadChan: payloadChan, + QuitChan: quitChan, } err := processor.Process(wg) Expect(err).ToNot(HaveOccurred()) + time.Sleep(2 * time.Second) + quitChan <- true wg.Wait() - }) - - It("Fails if", func() { - + Expect(mockConverter.PassedStatediffPayload).To(Equal(test_helpers.MockStatediffPayload)) + Expect(mockCidRepo.PassedCIDPayload).To(Equal(&test_helpers.MockCIDPayload)) + Expect(mockPublisher.PassedIPLDPayload).To(Equal(&test_helpers.MockIPLDPayload)) + Expect(mockStreamer.PassedPayloadChan).To(Equal(payloadChan)) }) }) }) diff --git a/pkg/ipfs/publisher.go b/pkg/ipfs/publisher.go index a8d85c72..b99df6b8 100644 --- a/pkg/ipfs/publisher.go +++ b/pkg/ipfs/publisher.go @@ -98,27 +98,27 @@ func (pub *Publisher) Publish(payload *IPLDPayload) (*CIDPayload, error) { } // Process and publish state leafs - stateLeafCids, err := pub.publishStateLeafs(payload.StateLeafs) + stateLeafCids, err := pub.publishStateNodes(payload.StateNodes) if err != nil { return nil, err } // Process and publish storage leafs - storageLeafCids, err := pub.publishStorageLeafs(payload.StorageLeafs) + storageLeafCids, err := pub.publishStorageNodes(payload.StorageNodes) if err != nil { return nil, err } // Package CIDs into a single struct return &CIDPayload{ - BlockHash: payload.BlockHash.Hex(), + BlockHash: payload.BlockHash, BlockNumber: payload.BlockNumber.String(), HeaderCID: headerCid, UncleCIDS: uncleCids, TransactionCIDs: transactionCids, ReceiptCIDs: receiptsCids, - StateLeafCIDs: stateLeafCids, - StorageLeafCIDs: storageLeafCids, + StateNodeCIDs: stateLeafCids, + StorageNodeCIDs: storageLeafCids, }, nil } @@ -165,34 +165,41 @@ func (pub *Publisher) publishReceipts(receipts types.Receipts, receiptMeta []*Re return mappedRctCids, nil } -func (pub *Publisher) publishStateLeafs(stateLeafs map[common.Hash][]byte) (map[common.Hash]string, error) { - stateLeafCids := make(map[common.Hash]string) - for addr, leaf := range stateLeafs { - stateLeafCid, err := pub.StatePutter.DagPut(leaf) +func (pub *Publisher) publishStateNodes(stateNodes map[common.Hash]StateNode) (map[common.Hash]StateNodeCID, error) { + stateNodeCids := make(map[common.Hash]StateNodeCID) + for addr, node := range stateNodes { + stateNodeCid, err := pub.StatePutter.DagPut(node.Value) if err != nil { return nil, err } - if len(stateLeafCid) != 1 { + if len(stateNodeCid) != 1 { return nil, errors.New("single CID expected to be returned for state leaf") } - stateLeafCids[addr] = stateLeafCid[0] + stateNodeCids[addr] = StateNodeCID{ + CID: stateNodeCid[0], + Leaf: node.Leaf, + } } - return stateLeafCids, nil + return stateNodeCids, nil } -func (pub *Publisher) publishStorageLeafs(storageLeafs map[common.Hash]map[common.Hash][]byte) (map[common.Hash]map[common.Hash]string, error) { - storageLeafCids := make(map[common.Hash]map[common.Hash]string) - for addr, storageTrie := range storageLeafs { - storageLeafCids[addr] = make(map[common.Hash]string) - for key, leaf := range storageTrie { - storageLeafCid, err := pub.StoragePutter.DagPut(leaf) +func (pub *Publisher) publishStorageNodes(storageNodes map[common.Hash][]StorageNode) (map[common.Hash][]StorageNodeCID, error) { + storageLeafCids := make(map[common.Hash][]StorageNodeCID) + for addr, storageTrie := range storageNodes { + storageLeafCids[addr] = make([]StorageNodeCID, 0) + for _, node := range storageTrie { + storageNodeCid, err := pub.StoragePutter.DagPut(node.Value) if err != nil { return nil, err } - if len(storageLeafCid) != 1 { + if len(storageNodeCid) != 1 { return nil, errors.New("single CID expected to be returned for storage leaf") } - storageLeafCids[addr][key] = storageLeafCid[0] + storageLeafCids[addr] = append(storageLeafCids[addr], StorageNodeCID{ + Key: node.Key, + CID: storageNodeCid[0], + Leaf: node.Leaf, + }) } } return storageLeafCids, nil diff --git a/pkg/ipfs/publisher_test.go b/pkg/ipfs/publisher_test.go index 339474de..744f886e 100644 --- a/pkg/ipfs/publisher_test.go +++ b/pkg/ipfs/publisher_test.go @@ -34,9 +34,5 @@ var _ = Describe("Publisher", func() { Expect(cidPayload).To(Equal(&test_helpers.MockCIDPayload)) Expect(mockPublisher.PassedIPLDPayload).To(Equal(&test_helpers.MockIPLDPayload)) }) - - It("Fails if", func() { - - }) }) }) diff --git a/pkg/ipfs/repository.go b/pkg/ipfs/repository.go index 99451b28..d1f0bdb8 100644 --- a/pkg/ipfs/repository.go +++ b/pkg/ipfs/repository.go @@ -43,11 +43,14 @@ func NewCIDRepository(db *postgres.DB) *Repository { // Index indexes a cidPayload in Postgres func (repo *Repository) Index(cidPayload *CIDPayload) error { tx, _ := repo.db.Beginx() - headerID, err := repo.indexHeaderCID(tx, cidPayload.HeaderCID, cidPayload.BlockNumber, cidPayload.BlockHash) + headerID, err := repo.indexHeaderCID(tx, cidPayload.HeaderCID, cidPayload.BlockNumber, cidPayload.BlockHash.Hex()) if err != nil { tx.Rollback() return err } + for uncleHash, cid := range cidPayload.UncleCIDS { + err = repo.indexUncleCID(tx, cid, cidPayload.BlockNumber, uncleHash.Hex()) + } err = repo.indexTransactionAndReceiptCIDs(tx, cidPayload, headerID) if err != nil { tx.Rollback() @@ -63,13 +66,20 @@ func (repo *Repository) Index(cidPayload *CIDPayload) error { func (repo *Repository) indexHeaderCID(tx *sqlx.Tx, cid, blockNumber, hash string) (int64, error) { var headerID int64 - err := tx.QueryRowx(`INSERT INTO public.header_cids (block_number, block_hash, cid) VALUES ($1, $2, $3) - ON CONFLICT DO UPDATE SET cid = $3 + err := tx.QueryRowx(`INSERT INTO public.header_cids (block_number, block_hash, cid, uncle) VALUES ($1, $2, $3, $4) + ON CONFLICT DO UPDATE SET (cid, uncle) = ($3, $4) RETURNING id`, - blockNumber, hash, cid).Scan(&headerID) + blockNumber, hash, cid, false).Scan(&headerID) return headerID, err } +func (repo *Repository) indexUncleCID(tx *sqlx.Tx, cid, blockNumber, hash string) error { + _, err := tx.Queryx(`INSERT INTO public.header_cids (block_number, block_hash, cid, uncle) VALUES ($1, $2, $3, $4) + ON CONFLICT DO UPDATE SET (cid, uncle) = ($3, $4)`, + blockNumber, hash, cid, true) + return err +} + func (repo *Repository) indexTransactionAndReceiptCIDs(tx *sqlx.Tx, payload *CIDPayload, headerID int64) error { for hash, trxCidMeta := range payload.TransactionCIDs { var txID int64 @@ -98,17 +108,17 @@ func (repo *Repository) indexReceiptCID(tx *sqlx.Tx, cidMeta *ReceiptMetaData, t } func (repo *Repository) indexStateAndStorageCIDs(tx *sqlx.Tx, payload *CIDPayload, headerID int64) error { - for accountKey, stateCID := range payload.StateLeafCIDs { + for accountKey, stateCID := range payload.StateNodeCIDs { var stateID int64 - err := tx.QueryRowx(`INSERT INTO public.state_cids (header_id, account_key, cid) VALUES ($1, $2, $3) - ON CONFLICT DO UPDATE SET cid = $3 + err := tx.QueryRowx(`INSERT INTO public.state_cids (header_id, state_key, cid, leaf) VALUES ($1, $2, $3, $4) + ON CONFLICT DO UPDATE SET (cid, leaf) = ($3, $4) RETURNING id`, - headerID, accountKey.Hex(), stateCID).Scan(&stateID) + headerID, accountKey.Hex(), stateCID.CID, stateCID.Leaf).Scan(&stateID) if err != nil { return err } - for storageKey, storageCID := range payload.StorageLeafCIDs[accountKey] { - err = repo.indexStorageCID(tx, storageKey.Hex(), storageCID, stateID) + for _, storageCID := range payload.StorageNodeCIDs[accountKey] { + err = repo.indexStorageCID(tx, storageCID, stateID) if err != nil { return err } @@ -117,8 +127,9 @@ func (repo *Repository) indexStateAndStorageCIDs(tx *sqlx.Tx, payload *CIDPayloa return nil } -func (repo *Repository) indexStorageCID(tx *sqlx.Tx, key, cid string, stateID int64) error { - _, err := repo.db.Exec(`INSERT INTO public.storage_cids (state_id, storage_key, cid) VALUES ($1, $2, $3) - ON CONFLICT DO UPDATE SET cid = $3`, stateID, key, cid) +func (repo *Repository) indexStorageCID(tx *sqlx.Tx, storageCID StorageNodeCID, stateID int64) error { + _, err := repo.db.Exec(`INSERT INTO public.storage_cids (state_id, storage_key, cid, leaf) VALUES ($1, $2, $3, $4) + ON CONFLICT DO UPDATE SET (cid, leaf) = ($3, $4)`, + stateID, storageCID.Key, storageCID.CID, storageCID.Leaf) return err } diff --git a/pkg/ipfs/repository_test.go b/pkg/ipfs/repository_test.go index 7543551a..bc5829a1 100644 --- a/pkg/ipfs/repository_test.go +++ b/pkg/ipfs/repository_test.go @@ -32,9 +32,5 @@ var _ = Describe("Repository", func() { Expect(err).ToNot(HaveOccurred()) Expect(mockRepo.PassedCIDPayload).To(Equal(&test_helpers.MockCIDPayload)) }) - - It("Fails if", func() { - - }) }) }) diff --git a/pkg/ipfs/streamer_test.go b/pkg/ipfs/streamer_test.go index 192be897..1378fa3e 100644 --- a/pkg/ipfs/streamer_test.go +++ b/pkg/ipfs/streamer_test.go @@ -17,8 +17,6 @@ package ipfs_test import ( - "sync" - "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/statediff" . "github.com/onsi/ginkgo" @@ -31,25 +29,18 @@ import ( var _ = Describe("Streamer", func() { Describe("Stream", func() { It("Streams StatediffPayloads from a Geth RPC subscription", func() { - wg := new(sync.WaitGroup) mockStreamer := mocks.StateDiffStreamer{} mockStreamer.ReturnSub = &rpc.ClientSubscription{} - mockStreamer.WaitGroup = wg mockStreamer.StreamPayloads = []statediff.Payload{ test_helpers.MockStatediffPayload, } payloadChan := make(chan statediff.Payload, 1) sub, err := mockStreamer.Stream(payloadChan) - wg.Wait() Expect(err).ToNot(HaveOccurred()) Expect(sub).To(Equal(&rpc.ClientSubscription{})) Expect(mockStreamer.PassedPayloadChan).To(Equal(payloadChan)) streamedPayload := <-payloadChan Expect(streamedPayload).To(Equal(test_helpers.MockStatediffPayload)) }) - - It("Fails if", func() { - - }) }) }) diff --git a/pkg/ipfs/test_helpers/mocks/streamer.go b/pkg/ipfs/test_helpers/mocks/streamer.go index e3ec7a4e..cd387ee6 100644 --- a/pkg/ipfs/test_helpers/mocks/streamer.go +++ b/pkg/ipfs/test_helpers/mocks/streamer.go @@ -17,8 +17,6 @@ package mocks import ( - "sync" - "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/statediff" ) @@ -29,7 +27,6 @@ type StateDiffStreamer struct { ReturnSub *rpc.ClientSubscription ReturnErr error StreamPayloads []statediff.Payload - WaitGroup *sync.WaitGroup } // Stream is the main loop for subscribing to data from the Geth state diff process @@ -37,11 +34,9 @@ func (sds *StateDiffStreamer) Stream(payloadChan chan statediff.Payload) (*rpc.C sds.PassedPayloadChan = payloadChan go func() { - sds.WaitGroup.Add(1) for _, payload := range sds.StreamPayloads { sds.PassedPayloadChan <- payload } - sds.WaitGroup.Done() }() return sds.ReturnSub, sds.ReturnErr diff --git a/pkg/ipfs/test_helpers/test_data.go b/pkg/ipfs/test_helpers/test_data.go index de6e2f70..83fc7f94 100644 --- a/pkg/ipfs/test_helpers/test_data.go +++ b/pkg/ipfs/test_helpers/test_data.go @@ -38,7 +38,7 @@ func AddressToLeafKey(address common.Address) common.Hash { // Test variables var ( - BlockNumber = rand.Int63() + BlockNumber = big.NewInt(rand.Int63()) BlockHash = "0xfa40fbe2d98d98b3363a778d52f2bcd29d6790b9b3f3cab2b167fd12d3550f73" CodeHash = common.Hex2Bytes("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470") NewNonceValue = rand.Uint64() @@ -65,26 +65,26 @@ var ( CodeHash: CodeHash, } valueBytes, _ = rlp.EncodeToBytes(testAccount) - CreatedAccountDiffs = statediff.AccountDiffsMap{ - ContractLeafKey: { + CreatedAccountDiffs = []statediff.AccountDiff{ + { Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, }, - AnotherContractLeafKey: { + { Key: AnotherContractLeafKey.Bytes(), Value: valueBytes, Storage: emptyStorage, }, } - UpdatedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: { + UpdatedAccountDiffs = []statediff.AccountDiff{{ Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, }} - DeletedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: { + DeletedAccountDiffs = []statediff.AccountDiff{{ Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, @@ -97,7 +97,7 @@ var ( DeletedAccounts: DeletedAccountDiffs, UpdatedAccounts: UpdatedAccountDiffs, } - MockStateDiffRlp, _ = rlp.EncodeToBytes(MockStateDiff) + MockStateDiffBytes, _ = rlp.EncodeToBytes(MockStateDiff) mockTransaction1 = types.NewTransaction(0, common.HexToAddress("0x0"), big.NewInt(1000), 50, big.NewInt(100), nil) mockTransaction2 = types.NewTransaction(1, common.HexToAddress("0x1"), big.NewInt(2000), 100, big.NewInt(200), nil) @@ -119,7 +119,7 @@ var ( MockStatediffPayload = statediff.Payload{ BlockRlp: MockBlockRlp, - StateDiffRlp: MockStateDiffRlp, + StateDiffRlp: MockStateDiffBytes, Err: nil, } @@ -139,7 +139,7 @@ var ( MockCIDPayload = ipfs.CIDPayload{ BlockNumber: "1", - BlockHash: "0x0", + BlockHash: common.HexToHash("0x0"), HeaderCID: "mockHeaderCID", TransactionCIDs: map[common.Hash]*ipfs.TrxMetaData{ common.HexToHash("0x0"): { @@ -163,16 +163,30 @@ var ( Topic0s: []string{"mockTopic1", "mockTopic2"}, }, }, - StateLeafCIDs: map[common.Hash]string{ - common.HexToHash("0x0"): "mockStateCID1", - common.HexToHash("0x1"): "mockStateCID2", - }, - StorageLeafCIDs: map[common.Hash]map[common.Hash]string{ + StateNodeCIDs: map[common.Hash]ipfs.StateNodeCID{ common.HexToHash("0x0"): { - common.HexToHash("0x0"): "mockStorageCID1", + CID: "mockStateCID1", + Leaf: true, }, common.HexToHash("0x1"): { - common.HexToHash("0x1"): "mockStorageCID2", + CID: "mockStateCID2", + Leaf: true, + }, + }, + StorageNodeCIDs: map[common.Hash][]ipfs.StorageNodeCID{ + common.HexToHash("0x0"): { + { + CID: "mockStorageCID1", + Key: common.HexToHash("0x0"), + Leaf: true, + }, + }, + common.HexToHash("0x1"): { + { + CID: "mockStorageCID2", + Key: common.HexToHash("0x1"), + Leaf: true, + }, }, }, } diff --git a/pkg/ipfs/types.go b/pkg/ipfs/types.go index 0d99460d..ef53d8ec 100644 --- a/pkg/ipfs/types.go +++ b/pkg/ipfs/types.go @@ -17,9 +17,10 @@ package ipfs import ( + "math/big" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" - "math/big" ) // IPLDPayload is a custom type which packages ETH data for the IPFS publisher @@ -31,20 +32,42 @@ type IPLDPayload struct { TrxMetaData []*TrxMetaData Receipts types.Receipts ReceiptMetaData []*ReceiptMetaData - StateLeafs map[common.Hash][]byte - StorageLeafs map[common.Hash]map[common.Hash][]byte + StateNodes map[common.Hash]StateNode + StorageNodes map[common.Hash][]StorageNode +} + +type StateNode struct { + Value []byte + Leaf bool +} + +type StorageNode struct { + Key common.Hash + Value []byte + Leaf bool } // CIDPayload is a struct to hold all the CIDs and their meta data type CIDPayload struct { BlockNumber string - BlockHash string + BlockHash common.Hash HeaderCID string UncleCIDS map[common.Hash]string TransactionCIDs map[common.Hash]*TrxMetaData ReceiptCIDs map[common.Hash]*ReceiptMetaData - StateLeafCIDs map[common.Hash]string - StorageLeafCIDs map[common.Hash]map[common.Hash]string + StateNodeCIDs map[common.Hash]StateNodeCID + StorageNodeCIDs map[common.Hash][]StorageNodeCID +} + +type StateNodeCID struct { + CID string + Leaf bool +} + +type StorageNodeCID struct { + Key common.Hash + CID string + Leaf bool } // ReceiptMetaData wraps some additional data around our receipt CIDs for indexing diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/api.go b/vendor/github.com/ethereum/go-ethereum/statediff/api.go index 19ad7cfc..db9573d4 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/api.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/api.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -19,7 +19,6 @@ package statediff import ( "context" "sync" - "time" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rpc" @@ -37,16 +36,14 @@ const APIVersion = "0.0.1" type PublicStateDiffAPI struct { sds IService - mu sync.Mutex - lastUsed map[string]time.Time // keeps track when a filter was polled for the last time. + mu sync.Mutex } // NewPublicStateDiffAPI create a new state diff websocket streaming service. func NewPublicStateDiffAPI(sds IService) *PublicStateDiffAPI { return &PublicStateDiffAPI{ - sds: sds, - lastUsed: make(map[string]time.Time), - mu: sync.Mutex{}, + sds: sds, + mu: sync.Mutex{}, } } @@ -80,12 +77,6 @@ func (api *PublicStateDiffAPI) Subscribe(ctx context.Context) (*rpc.Subscription log.Error("Failed to unsubscribe from the state diff service", err) } return - case <-notifier.Closed(): - err := api.sds.Unsubscribe(rpcSub.ID) - if err != nil { - log.Error("Failed to unsubscribe from the state diff service", err) - } - return case <-quitChan: // don't need to unsubscribe, statediff service does so before sending the quit signal return diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/builder.go b/vendor/github.com/ethereum/go-ethereum/statediff/builder.go index 4e8ab0e2..7de9d8be 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/builder.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/builder.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -20,47 +20,55 @@ package statediff import ( + "bytes" + "fmt" + "math/big" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/state" + "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" ) +var nullNode = common.Hex2Bytes("0000000000000000000000000000000000000000000000000000000000000000") + // Builder interface exposes the method for building a state diff between two blocks type Builder interface { - BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (StateDiff, error) + BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber *big.Int, blockHash common.Hash) (StateDiff, error) } type builder struct { chainDB ethdb.Database + config Config blockChain *core.BlockChain + stateCache state.Database } -// NewBuilder is used to create a builder -func NewBuilder(db ethdb.Database, blockChain *core.BlockChain) Builder { +// NewBuilder is used to create a state diff builder +func NewBuilder(db ethdb.Database, blockChain *core.BlockChain, config Config) Builder { return &builder{ chainDB: db, + config: config, blockChain: blockChain, } } // BuildStateDiff builds a StateDiff object from two blocks -func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (StateDiff, error) { +func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber *big.Int, blockHash common.Hash) (StateDiff, error) { // Generate tries for old and new states - stateCache := sdb.blockChain.StateCache() - oldTrie, err := stateCache.OpenTrie(oldStateRoot) + sdb.stateCache = sdb.blockChain.StateCache() + oldTrie, err := sdb.stateCache.OpenTrie(oldStateRoot) if err != nil { - log.Error("Error creating trie for oldStateRoot", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error creating trie for oldStateRoot: %v", err) } - newTrie, err := stateCache.OpenTrie(newStateRoot) + newTrie, err := sdb.stateCache.OpenTrie(newStateRoot) if err != nil { - log.Error("Error creating trie for newStateRoot", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error creating trie for newStateRoot: %v", err) } // Find created accounts @@ -68,8 +76,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block newIt := newTrie.NodeIterator([]byte{}) creations, err := sdb.collectDiffNodes(oldIt, newIt) if err != nil { - log.Error("Error collecting creation diff nodes", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error collecting creation diff nodes: %v", err) } // Find deleted accounts @@ -77,8 +84,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block newIt = newTrie.NodeIterator([]byte{}) deletions, err := sdb.collectDiffNodes(newIt, oldIt) if err != nil { - log.Error("Error collecting deletion diff nodes", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error collecting deletion diff nodes: %v", err) } // Find all the diffed keys @@ -89,18 +95,15 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block // Build and return the statediff updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, updatedKeys) if err != nil { - log.Error("Error building diff for updated accounts", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error building diff for updated accounts: %v", err) } createdAccounts, err := sdb.buildDiffEventual(creations) if err != nil { - log.Error("Error building diff for created accounts", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error building diff for created accounts: %v", err) } deletedAccounts, err := sdb.buildDiffEventual(deletions) if err != nil { - log.Error("Error building diff for deleted accounts", "error", err) - return StateDiff{}, err + return StateDiff{}, fmt.Errorf("error building diff for deleted accounts: %v", err) } return StateDiff{ @@ -112,17 +115,27 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block }, nil } +func (sdb *builder) isWatchedAddress(hashKey []byte) bool { + // If we aren't watching any addresses, we are watching everything + if len(sdb.config.WatchedAddresses) == 0 { + return true + } + for _, addrStr := range sdb.config.WatchedAddresses { + addr := common.HexToAddress(addrStr) + addrHashKey := crypto.Keccak256(addr[:]) + if bytes.Equal(addrHashKey, hashKey) { + return true + } + } + return false +} + func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (AccountsMap, error) { var diffAccounts = make(AccountsMap) it, _ := trie.NewDifferenceIterator(a, b) - for { - log.Debug("Current Path and Hash", "path", pathToStr(it), "hashold", it.Hash()) - if it.Leaf() { - leafProof := make([][]byte, len(it.LeafProof())) - copy(leafProof, it.LeafProof()) - leafPath := make([]byte, len(it.Path())) - copy(leafPath, it.Path()) + log.Debug("Current Path and Hash", "path", pathToStr(it), "old hash", it.Hash()) + if it.Leaf() && sdb.isWatchedAddress(it.LeafKey()) { leafKey := make([]byte, len(it.LeafKey())) copy(leafKey, it.LeafKey()) leafKeyHash := common.BytesToHash(leafKey) @@ -131,19 +144,38 @@ func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (AccountsMap, error // lookup account state var account state.Account if err := rlp.DecodeBytes(leafValue, &account); err != nil { - log.Error("Error looking up account via address", "address", leafKeyHash, "error", err) - return nil, err + return nil, fmt.Errorf("error looking up account via address %s\r\nerror: %v", leafKeyHash.Hex(), err) } aw := accountWrapper{ - Account: account, + Leaf: true, + Account: &account, RawKey: leafKey, RawValue: leafValue, - Proof: leafProof, - Path: leafPath, + } + if sdb.config.PathsAndProofs { + leafProof := make([][]byte, len(it.LeafProof())) + copy(leafProof, it.LeafProof()) + leafPath := make([]byte, len(it.Path())) + copy(leafPath, it.Path()) + aw.Proof = leafProof + aw.Path = leafPath } // record account to diffs (creation if we are looking at new - old; deletion if old - new) log.Debug("Account lookup successful", "address", leafKeyHash, "account", account) diffAccounts[leafKeyHash] = aw + } else if sdb.config.AllNodes && !bytes.Equal(nullNode, it.Hash().Bytes()) { + nodeKey := it.Hash() + node, err := sdb.stateCache.TrieDB().Node(nodeKey) + if err != nil { + return nil, fmt.Errorf("error looking up intermediate state trie node %s\r\nerror: %v", nodeKey.Hex(), err) + } + aw := accountWrapper{ + Leaf: false, + RawKey: nodeKey.Bytes(), + RawValue: node, + } + log.Debug("intermediate state trie node lookup successful", "key", nodeKey.Hex(), "value", node) + diffAccounts[nodeKey] = aw } cont := it.Next(true) if !cont { @@ -154,45 +186,55 @@ func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (AccountsMap, error return diffAccounts, nil } -func (sdb *builder) buildDiffEventual(accounts AccountsMap) (AccountDiffsMap, error) { - accountDiffs := make(AccountDiffsMap) +func (sdb *builder) buildDiffEventual(accounts AccountsMap) ([]AccountDiff, error) { + accountDiffs := make([]AccountDiff, 0) + var err error for _, val := range accounts { - storageDiffs, err := sdb.buildStorageDiffsEventual(val.Account.Root) - if err != nil { - log.Error("Failed building eventual storage diffs", "Address", common.BytesToHash(val.RawKey), "error", err) - return nil, err + // If account is not nil, we need to process storage diffs + var storageDiffs []StorageDiff + if val.Account != nil { + storageDiffs, err = sdb.buildStorageDiffsEventual(val.Account.Root) + if err != nil { + return nil, fmt.Errorf("failed building eventual storage diffs for %s\r\nerror: %v", common.BytesToHash(val.RawKey), err) + } } - accountDiffs[common.BytesToHash(val.RawKey)] = AccountDiff{ + accountDiffs = append(accountDiffs, AccountDiff{ + Leaf: val.Leaf, Key: val.RawKey, Value: val.RawValue, Proof: val.Proof, Path: val.Path, Storage: storageDiffs, - } + }) } return accountDiffs, nil } -func (sdb *builder) buildDiffIncremental(creations AccountsMap, deletions AccountsMap, updatedKeys []string) (AccountDiffsMap, error) { - updatedAccounts := make(AccountDiffsMap) +func (sdb *builder) buildDiffIncremental(creations AccountsMap, deletions AccountsMap, updatedKeys []string) ([]AccountDiff, error) { + updatedAccounts := make([]AccountDiff, 0) + var err error for _, val := range updatedKeys { - createdAcc := creations[common.HexToHash(val)] - deletedAcc := deletions[common.HexToHash(val)] - oldSR := deletedAcc.Account.Root - newSR := createdAcc.Account.Root - storageDiffs, err := sdb.buildStorageDiffsIncremental(oldSR, newSR) - if err != nil { - log.Error("Failed building storage diffs", "Address", val, "error", err) - return nil, err + hashKey := common.HexToHash(val) + createdAcc := creations[hashKey] + deletedAcc := deletions[hashKey] + var storageDiffs []StorageDiff + if deletedAcc.Account != nil && createdAcc.Account != nil { + oldSR := deletedAcc.Account.Root + newSR := createdAcc.Account.Root + storageDiffs, err = sdb.buildStorageDiffsIncremental(oldSR, newSR) + if err != nil { + return nil, fmt.Errorf("failed building incremental storage diffs for %s\r\nerror: %v", hashKey.Hex(), err) + } } - updatedAccounts[common.HexToHash(val)] = AccountDiff{ + updatedAccounts = append(updatedAccounts, AccountDiff{ + Leaf: createdAcc.Leaf, Key: createdAcc.RawKey, Value: createdAcc.RawValue, Proof: createdAcc.Proof, Path: createdAcc.Path, Storage: storageDiffs, - } + }) delete(creations, common.HexToHash(val)) delete(deletions, common.HexToHash(val)) } @@ -209,8 +251,7 @@ func (sdb *builder) buildStorageDiffsEventual(sr common.Hash) ([]StorageDiff, er return nil, err } it := sTrie.NodeIterator(make([]byte, 0)) - storageDiffs := buildStorageDiffsFromTrie(it) - return storageDiffs, nil + return sdb.buildStorageDiffsFromTrie(it) } func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) ([]StorageDiff, error) { @@ -229,31 +270,45 @@ func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common oldIt := oldTrie.NodeIterator(make([]byte, 0)) newIt := newTrie.NodeIterator(make([]byte, 0)) it, _ := trie.NewDifferenceIterator(oldIt, newIt) - storageDiffs := buildStorageDiffsFromTrie(it) - - return storageDiffs, nil + return sdb.buildStorageDiffsFromTrie(it) } -func buildStorageDiffsFromTrie(it trie.NodeIterator) []StorageDiff { +func (sdb *builder) buildStorageDiffsFromTrie(it trie.NodeIterator) ([]StorageDiff, error) { storageDiffs := make([]StorageDiff, 0) for { log.Debug("Iterating over state at path ", "path", pathToStr(it)) if it.Leaf() { log.Debug("Found leaf in storage", "path", pathToStr(it)) - leafProof := make([][]byte, len(it.LeafProof())) - copy(leafProof, it.LeafProof()) - leafPath := make([]byte, len(it.Path())) - copy(leafPath, it.Path()) leafKey := make([]byte, len(it.LeafKey())) copy(leafKey, it.LeafKey()) leafValue := make([]byte, len(it.LeafBlob())) copy(leafValue, it.LeafBlob()) - storageDiffs = append(storageDiffs, StorageDiff{ + sd := StorageDiff{ + Leaf: true, Key: leafKey, Value: leafValue, - Path: leafPath, - Proof: leafProof, + } + if sdb.config.PathsAndProofs { + leafProof := make([][]byte, len(it.LeafProof())) + copy(leafProof, it.LeafProof()) + leafPath := make([]byte, len(it.Path())) + copy(leafPath, it.Path()) + sd.Proof = leafProof + sd.Path = leafPath + } + storageDiffs = append(storageDiffs, sd) + } else if sdb.config.AllNodes && !bytes.Equal(nullNode, it.Hash().Bytes()) { + nodeKey := it.Hash() + node, err := sdb.stateCache.TrieDB().Node(nodeKey) + if err != nil { + return nil, fmt.Errorf("error looking up intermediate storage trie node %s\r\nerror: %v", nodeKey.Hex(), err) + } + storageDiffs = append(storageDiffs, StorageDiff{ + Leaf: false, + Key: nodeKey.Bytes(), + Value: node, }) + log.Debug("intermediate storage trie node lookup successful", "key", nodeKey.Hex(), "value", node) } cont := it.Next(true) if !cont { @@ -261,7 +316,7 @@ func buildStorageDiffsFromTrie(it trie.NodeIterator) []StorageDiff { } } - return storageDiffs + return storageDiffs, nil } func (sdb *builder) addressByPath(path []byte) (*common.Address, error) { diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/builder_test.go b/vendor/github.com/ethereum/go-ethereum/statediff/builder_test.go new file mode 100644 index 00000000..7575b060 --- /dev/null +++ b/vendor/github.com/ethereum/go-ethereum/statediff/builder_test.go @@ -0,0 +1,564 @@ +// 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 . + +package statediff_test + +import ( + "bytes" + "math/big" + "sort" + "testing" + + "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/rlp" + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/testhelpers" +) + +var ( + contractLeafKey common.Hash + emptyAccountDiffEventualMap = make([]statediff.AccountDiff, 0) + emptyAccountDiffIncrementalMap = make([]statediff.AccountDiff, 0) + block0, block1, block2, block3 *types.Block + builder statediff.Builder + miningReward = int64(2000000000000000000) + burnAddress = common.HexToAddress("0x0") + burnLeafKey = testhelpers.AddressToLeafKey(burnAddress) + + block0Hash = common.HexToHash("0xd1721cfd0b29c36fd7a68f25c128e86413fb666a6e1d68e89b875bd299262661") + block1Hash = common.HexToHash("0xbbe88de60ba33a3f18c0caa37d827bfb70252e19e40a07cd34041696c35ecb1a") + block2Hash = common.HexToHash("0x34ad0fd9bb2911986b75d518c822641079dea823bc6952343ebf05da1062b6f5") + block3Hash = common.HexToHash("0x9872058136c560a6ebed0c0522b8d3016fc21f4fb0fb6585ddd8fd4c54f9909a") + balanceChange10000 = int64(10000) + balanceChange1000 = int64(1000) + block1BankBalance = int64(99990000) + block1Account1Balance = int64(10000) + block2Account2Balance = int64(1000) + nonce0 = uint64(0) + nonce1 = uint64(1) + nonce2 = uint64(2) + nonce3 = uint64(3) + originalContractRoot = "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421" + contractContractRoot = "0x821e2556a290c86405f8160a2d662042a431ba456b9db265c79bb837c04be5f0" + newContractRoot = "0x71e0d14b2b93e5c7f9748e69e1fe5f17498a1c3ac3cec29f96af13d7f8a4e070" + originalStorageLocation = common.HexToHash("0") + originalStorageKey = crypto.Keccak256Hash(originalStorageLocation[:]).Bytes() + updatedStorageLocation = common.HexToHash("2") + updatedStorageKey = crypto.Keccak256Hash(updatedStorageLocation[:]).Bytes() + originalStorageValue = common.Hex2Bytes("01") + updatedStorageValue = common.Hex2Bytes("03") + + account1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce0, + Balance: big.NewInt(balanceChange10000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + burnAccount1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce0, + Balance: big.NewInt(miningReward), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + bankAccount1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce1, + Balance: big.NewInt(testhelpers.TestBankFunds.Int64() - balanceChange10000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + account2, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce0, + Balance: big.NewInt(balanceChange1000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + contractAccount, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce1, + Balance: big.NewInt(0), + CodeHash: common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea").Bytes(), + Root: common.HexToHash(contractContractRoot), + }) + bankAccount2, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce2, + Balance: big.NewInt(block1BankBalance - balanceChange1000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + account3, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce2, + Balance: big.NewInt(block1Account1Balance - balanceChange1000 + balanceChange1000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + burnAccount2, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce0, + Balance: big.NewInt(miningReward + miningReward), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + account4, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce0, + Balance: big.NewInt(block2Account2Balance + miningReward), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) + contractAccount2, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce1, + Balance: big.NewInt(0), + CodeHash: common.HexToHash("0x753f98a8d4328b15636e46f66f2cb4bc860100aa17967cc145fcd17d1d4710ea").Bytes(), + Root: common.HexToHash(newContractRoot), + }) + bankAccount3, _ = rlp.EncodeToBytes(state.Account{ + Nonce: nonce3, + Balance: big.NewInt(99989000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash(originalContractRoot), + }) +) + +type arguments struct { + oldStateRoot common.Hash + newStateRoot common.Hash + blockNumber *big.Int + blockHash common.Hash +} + +func TestBuilder(t *testing.T) { + _, blockMap, chain := testhelpers.MakeChain(3, testhelpers.Genesis) + contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + defer chain.Stop() + block0 = blockMap[block0Hash] + block1 = blockMap[block1Hash] + block2 = blockMap[block2Hash] + block3 = blockMap[block3Hash] + config := statediff.Config{ + PathsAndProofs: true, + AllNodes: false, + } + builder = statediff.NewBuilder(testhelpers.Testdb, chain, config) + + var tests = []struct { + name string + startingArguments arguments + expected *statediff.StateDiff + }{ + { + "testEmptyDiff", + arguments{ + oldStateRoot: block0.Root(), + newStateRoot: block0.Root(), + blockNumber: block0.Number(), + blockHash: block0Hash, + }, + &statediff.StateDiff{ + BlockNumber: block0.Number(), + BlockHash: block0Hash, + CreatedAccounts: emptyAccountDiffEventualMap, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: emptyAccountDiffIncrementalMap, + }, + }, + { + "testBlock1", + //10000 transferred from testBankAddress to account1Addr + arguments{ + oldStateRoot: block0.Root(), + newStateRoot: block1.Root(), + blockNumber: block1.Number(), + blockHash: block1Hash, + }, + &statediff.StateDiff{ + BlockNumber: block1.Number(), + BlockHash: block1.Hash(), + CreatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: burnLeafKey.Bytes(), + Value: burnAccount1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + { + Leaf: true, + Key: testhelpers.Account1LeafKey.Bytes(), + Value: account1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 128, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.BankLeafKey.Bytes(), + Value: bankAccount1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 1, 132, 5, 245, 185, 240, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + }, + }, + { + "testBlock2", + //1000 transferred from testBankAddress to account1Addr + //1000 transferred from account1Addr to account2Addr + arguments{ + oldStateRoot: block1.Root(), + newStateRoot: block2.Root(), + blockNumber: block2.Number(), + blockHash: block2Hash, + }, + &statediff.StateDiff{ + BlockNumber: block2.Number(), + BlockHash: block2.Hash(), + CreatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: contractLeafKey.Bytes(), + Value: contractAccount, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 130, 30, 37, 86, 162, 144, 200, 100, 5, 248, 22, 10, 45, 102, 32, 66, 164, 49, 186, 69, 107, 157, 178, 101, 199, 155, 184, 55, 192, 75, 229, 240, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}}, + Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16}, + Storage: []statediff.StorageDiff{ + { + Leaf: true, + Key: originalStorageKey, + Value: originalStorageValue, + Proof: [][]byte{{227, 161, 32, 41, 13, 236, 217, 84, 139, 98, 168, 214, 3, 69, 169, 136, 56, 111, 200, 75, 166, 188, 149, 72, 64, 8, 246, 54, 47, 147, 22, 14, 243, 229, 99, 1}}, + Path: []byte{2, 9, 0, 13, 14, 12, 13, 9, 5, 4, 8, 11, 6, 2, 10, 8, 13, 6, 0, 3, 4, 5, 10, 9, 8, 8, 3, 8, 6, 15, 12, 8, 4, 11, 10, 6, 11, 12, 9, 5, 4, 8, 4, 0, 0, 8, 15, 6, 3, 6, 2, 15, 9, 3, 1, 6, 0, 14, 15, 3, 14, 5, 6, 3, 16}, + }, + }, + }, + { + Leaf: true, + Key: testhelpers.Account2LeafKey.Bytes(), + Value: account2, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 107, 160, 57, 87, 243, 226, 240, 74, 7, 100, 195, 160, 73, 27, 23, 95, 105, 146, 109, 166, 30, 251, 204, 143, 97, 250, 20, 85, 253, 45, 43, 76, 221, 69, 184, 72, 248, 70, 128, 130, 3, 232, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{12, 9, 5, 7, 15, 3, 14, 2, 15, 0, 4, 10, 0, 7, 6, 4, 12, 3, 10, 0, 4, 9, 1, 11, 1, 7, 5, 15, 6, 9, 9, 2, 6, 13, 10, 6, 1, 14, 15, 11, 12, 12, 8, 15, 6, 1, 15, 10, 1, 4, 5, 5, 15, 13, 2, 13, 2, 11, 4, 12, 13, 13, 4, 5, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.BankLeafKey.Bytes(), + Value: bankAccount2, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 2, 132, 5, 245, 182, 8, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + { + Leaf: true, + Key: burnLeafKey.Bytes(), + Value: burnAccount2, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 55, 130, 218, 206, 157, 144, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + { + Leaf: true, + Key: testhelpers.Account1LeafKey.Bytes(), + Value: account3, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 2, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + }, + }, + { + "testBlock3", + //the contract's storage is changed + //and the block is mined by account 2 + arguments{ + oldStateRoot: block2.Root(), + newStateRoot: block3.Root(), + blockNumber: block3.Number(), + blockHash: block3.Hash(), + }, + &statediff.StateDiff{ + BlockNumber: block3.Number(), + BlockHash: block3.Hash(), + CreatedAccounts: []statediff.AccountDiff{}, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.BankLeafKey.Bytes(), + Value: bankAccount3, + Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 3, 132, 5, 245, 182, 8, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + { + Leaf: true, + Key: contractLeafKey.Bytes(), + Value: contractAccount2, + Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 113, 224, 209, 75, 43, 147, 229, 199, 249, 116, 142, 105, 225, 254, 95, 23, 73, 138, 28, 58, 195, 206, 194, 159, 150, 175, 19, 215, 248, 164, 224, 112, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}}, + Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16}, + Storage: []statediff.StorageDiff{ + { + Leaf: true, + Key: updatedStorageKey, + Value: updatedStorageValue, + Proof: [][]byte{{248, 81, 128, 128, 160, 79, 197, 241, 58, 178, 249, 186, 12, 45, 168, 139, 1, 81, 171, 14, 124, 244, 216, 93, 8, 204, 164, 92, 205, 146, 60, 106, 183, 99, 35, 235, 40, 128, 160, 205, 69, 114, 89, 105, 97, 21, 35, 94, 100, 199, 130, 35, 52, 214, 33, 41, 226, 241, 96, 68, 37, 167, 218, 100, 148, 243, 95, 196, 91, 229, 24, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128}, + {226, 160, 48, 87, 135, 250, 18, 168, 35, 224, 242, 183, 99, 28, 196, 27, 59, 168, 130, 139, 51, 33, 202, 129, 17, 17, 250, 117, 205, 58, 163, 187, 90, 206, 3}}, + Path: []byte{4, 0, 5, 7, 8, 7, 15, 10, 1, 2, 10, 8, 2, 3, 14, 0, 15, 2, 11, 7, 6, 3, 1, 12, 12, 4, 1, 11, 3, 11, 10, 8, 8, 2, 8, 11, 3, 3, 2, 1, 12, 10, 8, 1, 1, 1, 1, 1, 15, 10, 7, 5, 12, 13, 3, 10, 10, 3, 11, 11, 5, 10, 12, 14, 16}, + }, + }, + }, + { + Leaf: true, + Key: testhelpers.Account2LeafKey.Bytes(), + Value: account4, + Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 113, 160, 57, 87, 243, 226, 240, 74, 7, 100, 195, 160, 73, 27, 23, 95, 105, 146, 109, 166, 30, 251, 204, 143, 97, 250, 20, 85, 253, 45, 43, 76, 221, 69, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 3, 232, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{12, 9, 5, 7, 15, 3, 14, 2, 15, 0, 4, 10, 0, 7, 6, 4, 12, 3, 10, 0, 4, 9, 1, 11, 1, 7, 5, 15, 6, 9, 9, 2, 6, 13, 10, 6, 1, 14, 15, 11, 12, 12, 8, 15, 6, 1, 15, 10, 1, 4, 5, 5, 15, 13, 2, 13, 2, 11, 4, 12, 13, 13, 4, 5, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + }, + }, + } + + for _, test := range tests { + arguments := test.startingArguments + diff, err := builder.BuildStateDiff(arguments.oldStateRoot, arguments.newStateRoot, arguments.blockNumber, arguments.blockHash) + 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) { + t.Logf("Test failed: %s", test.name) + t.Errorf("actual state diff rlp: %+v\nexpected state diff rlp: %+v", receivedStateDiffRlp, expectedStateDiffRlp) + } + } +} + +func TestBuilderWithWatchedAddressList(t *testing.T) { + _, blockMap, chain := testhelpers.MakeChain(3, testhelpers.Genesis) + contractLeafKey = testhelpers.AddressToLeafKey(testhelpers.ContractAddr) + defer chain.Stop() + block0 = blockMap[block0Hash] + block1 = blockMap[block1Hash] + block2 = blockMap[block2Hash] + block3 = blockMap[block3Hash] + config := statediff.Config{ + PathsAndProofs: true, + AllNodes: false, + WatchedAddresses: []string{testhelpers.Account1Addr.Hex(), testhelpers.ContractAddr.Hex()}, + } + builder = statediff.NewBuilder(testhelpers.Testdb, chain, config) + + var tests = []struct { + name string + startingArguments arguments + expected *statediff.StateDiff + }{ + { + "testEmptyDiff", + arguments{ + oldStateRoot: block0.Root(), + newStateRoot: block0.Root(), + blockNumber: block0.Number(), + blockHash: block0Hash, + }, + &statediff.StateDiff{ + BlockNumber: block0.Number(), + BlockHash: block0Hash, + CreatedAccounts: emptyAccountDiffEventualMap, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: emptyAccountDiffIncrementalMap, + }, + }, + { + "testBlock1", + //10000 transferred from testBankAddress to account1Addr + arguments{ + oldStateRoot: block0.Root(), + newStateRoot: block1.Root(), + blockNumber: block1.Number(), + blockHash: block1Hash, + }, + &statediff.StateDiff{ + BlockNumber: block1.Number(), + BlockHash: block1.Hash(), + CreatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.Account1LeafKey.Bytes(), + Value: account1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 128, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{}, + }, + }, + { + "testBlock2", + //1000 transferred from testBankAddress to account1Addr + //1000 transferred from account1Addr to account2Addr + arguments{ + oldStateRoot: block1.Root(), + newStateRoot: block2.Root(), + blockNumber: block2.Number(), + blockHash: block2Hash, + }, + &statediff.StateDiff{ + BlockNumber: block2.Number(), + BlockHash: block2.Hash(), + CreatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: contractLeafKey.Bytes(), + Value: contractAccount, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 130, 30, 37, 86, 162, 144, 200, 100, 5, 248, 22, 10, 45, 102, 32, 66, 164, 49, 186, 69, 107, 157, 178, 101, 199, 155, 184, 55, 192, 75, 229, 240, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}}, + Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16}, + Storage: []statediff.StorageDiff{ + { + Leaf: true, + Key: originalStorageKey, + Value: originalStorageValue, + Proof: [][]byte{{227, 161, 32, 41, 13, 236, 217, 84, 139, 98, 168, 214, 3, 69, 169, 136, 56, 111, 200, 75, 166, 188, 149, 72, 64, 8, 246, 54, 47, 147, 22, 14, 243, 229, 99, 1}}, + Path: []byte{2, 9, 0, 13, 14, 12, 13, 9, 5, 4, 8, 11, 6, 2, 10, 8, 13, 6, 0, 3, 4, 5, 10, 9, 8, 8, 3, 8, 6, 15, 12, 8, 4, 11, 10, 6, 11, 12, 9, 5, 4, 8, 4, 0, 0, 8, 15, 6, 3, 6, 2, 15, 9, 3, 1, 6, 0, 14, 15, 3, 14, 5, 6, 3, 16}, + }, + }, + }, + }, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.Account1LeafKey.Bytes(), + Value: account3, + Proof: [][]byte{{248, 177, 160, 177, 155, 238, 178, 242, 47, 83, 2, 49, 141, 155, 92, 149, 175, 245, 120, 233, 177, 101, 67, 46, 200, 23, 250, 41, 74, 135, 94, 61, 133, 51, 162, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 114, 57, 32, 11, 115, 232, 140, 238, 165, 222, 121, 226, 208, 2, 192, 216, 67, 198, 179, 31, 181, 27, 208, 243, 99, 202, 48, 148, 207, 107, 106, 177, 128, 128, 128, 128, 128, 160, 10, 173, 165, 125, 110, 240, 77, 112, 149, 100, 135, 237, 25, 228, 116, 7, 195, 9, 210, 166, 208, 148, 101, 23, 244, 238, 84, 84, 211, 249, 138, 137, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 2, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + }, + }, + { + "testBlock3", + //the contract's storage is changed + //and the block is mined by account 2 + arguments{ + oldStateRoot: block2.Root(), + newStateRoot: block3.Root(), + blockNumber: block3.Number(), + blockHash: block3.Hash(), + }, + &statediff.StateDiff{ + BlockNumber: block3.Number(), + BlockHash: block3.Hash(), + CreatedAccounts: []statediff.AccountDiff{}, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: contractLeafKey.Bytes(), + Value: contractAccount2, + Proof: [][]byte{{248, 177, 160, 101, 223, 138, 81, 34, 40, 229, 170, 198, 188, 136, 99, 7, 55, 33, 112, 160, 111, 181, 131, 167, 201, 131, 24, 201, 211, 177, 30, 159, 229, 246, 6, 128, 128, 128, 128, 160, 179, 86, 53, 29, 96, 188, 152, 148, 207, 31, 29, 108, 182, 140, 129, 95, 1, 49, 213, 15, 29, 168, 60, 64, 35, 160, 158, 200, 85, 207, 255, 145, 160, 32, 135, 108, 213, 150, 150, 110, 44, 170, 65, 75, 154, 74, 249, 94, 65, 74, 107, 100, 115, 39, 5, 3, 26, 22, 238, 138, 114, 254, 21, 6, 171, 128, 128, 128, 128, 128, 160, 4, 228, 121, 222, 255, 218, 60, 247, 15, 0, 34, 198, 28, 229, 180, 129, 109, 157, 68, 181, 248, 229, 200, 123, 29, 81, 145, 114, 90, 209, 205, 210, 128, 160, 255, 115, 147, 190, 57, 135, 174, 188, 86, 51, 227, 70, 22, 253, 237, 49, 24, 19, 149, 199, 142, 195, 186, 244, 70, 51, 138, 0, 146, 148, 117, 60, 128, 128}, + {248, 105, 160, 49, 20, 101, 138, 116, 217, 204, 159, 122, 207, 44, 92, 214, 150, 195, 73, 77, 124, 52, 77, 120, 191, 236, 58, 221, 13, 145, 236, 78, 141, 28, 69, 184, 70, 248, 68, 1, 128, 160, 113, 224, 209, 75, 43, 147, 229, 199, 249, 116, 142, 105, 225, 254, 95, 23, 73, 138, 28, 58, 195, 206, 194, 159, 150, 175, 19, 215, 248, 164, 224, 112, 160, 117, 63, 152, 168, 212, 50, 139, 21, 99, 110, 70, 246, 111, 44, 180, 188, 134, 1, 0, 170, 23, 150, 124, 193, 69, 252, 209, 125, 29, 71, 16, 234}}, + Path: []byte{6, 1, 1, 4, 6, 5, 8, 10, 7, 4, 13, 9, 12, 12, 9, 15, 7, 10, 12, 15, 2, 12, 5, 12, 13, 6, 9, 6, 12, 3, 4, 9, 4, 13, 7, 12, 3, 4, 4, 13, 7, 8, 11, 15, 14, 12, 3, 10, 13, 13, 0, 13, 9, 1, 14, 12, 4, 14, 8, 13, 1, 12, 4, 5, 16}, + Storage: []statediff.StorageDiff{ + { + Leaf: true, + Key: updatedStorageKey, + Value: updatedStorageValue, + Proof: [][]byte{{248, 81, 128, 128, 160, 79, 197, 241, 58, 178, 249, 186, 12, 45, 168, 139, 1, 81, 171, 14, 124, 244, 216, 93, 8, 204, 164, 92, 205, 146, 60, 106, 183, 99, 35, 235, 40, 128, 160, 205, 69, 114, 89, 105, 97, 21, 35, 94, 100, 199, 130, 35, 52, 214, 33, 41, 226, 241, 96, 68, 37, 167, 218, 100, 148, 243, 95, 196, 91, 229, 24, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128, 128}, + {226, 160, 48, 87, 135, 250, 18, 168, 35, 224, 242, 183, 99, 28, 196, 27, 59, 168, 130, 139, 51, 33, 202, 129, 17, 17, 250, 117, 205, 58, 163, 187, 90, 206, 3}}, + Path: []byte{4, 0, 5, 7, 8, 7, 15, 10, 1, 2, 10, 8, 2, 3, 14, 0, 15, 2, 11, 7, 6, 3, 1, 12, 12, 4, 1, 11, 3, 11, 10, 8, 8, 2, 8, 11, 3, 3, 2, 1, 12, 10, 8, 1, 1, 1, 1, 1, 15, 10, 7, 5, 12, 13, 3, 10, 10, 3, 11, 11, 5, 10, 12, 14, 16}, + }, + }, + }, + }, + }, + }, + } + + for _, test := range tests { + arguments := test.startingArguments + diff, err := builder.BuildStateDiff(arguments.oldStateRoot, arguments.newStateRoot, arguments.blockNumber, arguments.blockHash) + 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) { + t.Logf("Test failed: %s", test.name) + t.Errorf("actual state diff rlp: %+v\nexpected state diff rlp: %+v", receivedStateDiffRlp, expectedStateDiffRlp) + } + } +} + +/* +contract test { + + uint256[100] data; + + constructor() public { + data = [1]; + } + + function Put(uint256 addr, uint256 value) { + data[addr] = value; + } + + function Get(uint256 addr) constant returns (uint256 value) { + return data[addr]; + } +} +*/ diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/config.go b/vendor/github.com/ethereum/go-ethereum/statediff/config.go new file mode 100644 index 00000000..c246cfc8 --- /dev/null +++ b/vendor/github.com/ethereum/go-ethereum/statediff/config.go @@ -0,0 +1,25 @@ +// 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 . + +package statediff + +// Config is used to carry in parameters from CLI configuration +type Config struct { + StreamBlock bool + PathsAndProofs bool + AllNodes bool + WatchedAddresses []string +} diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/helpers.go b/vendor/github.com/ethereum/go-ethereum/statediff/helpers.go index 96f9ddf3..89852b55 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/helpers.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/helpers.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/service.go b/vendor/github.com/ethereum/go-ethereum/statediff/service.go index 314a8ab1..5e3f3e59 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/service.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/service.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -21,7 +21,6 @@ import ( "fmt" "sync" - "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/common" @@ -30,6 +29,7 @@ import ( "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/event" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/node" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/rpc" ) @@ -64,29 +64,21 @@ type Service struct { QuitChan chan bool // A mapping of rpc.IDs to their subscription channels Subscriptions map[rpc.ID]Subscription -} - -// Subscription struct holds our subscription channels -type Subscription struct { - PayloadChan chan<- Payload - QuitChan chan<- bool -} - -// Payload packages the data to send to StateDiffingService subscriptions -type Payload struct { - BlockRlp []byte `json:"blockRlp" gencodec:"required"` - StateDiffRlp []byte `json:"stateDiffRlp" gencodec:"required"` - Err error `json:"error"` + // Cache the last block so that we can avoid having to lookup the next block's parent + lastBlock *types.Block + // Whether or not the block data is streamed alongside the state diff data in the subscription payload + streamBlock bool } // NewStateDiffService creates a new StateDiffingService -func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*Service, error) { +func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain, config Config) (*Service, error) { return &Service{ Mutex: sync.Mutex{}, BlockChain: blockChain, - Builder: NewBuilder(db, blockChain), + Builder: NewBuilder(db, blockChain, config), QuitChan: make(chan bool), Subscriptions: make(map[rpc.ID]Subscription), + streamBlock: config.StreamBlock, }, nil } @@ -109,70 +101,72 @@ func (sds *Service) APIs() []rpc.API { // Loop is the main processing method func (sds *Service) Loop(chainEventCh chan core.ChainEvent) { - chainEventSub := sds.BlockChain.SubscribeChainEvent(chainEventCh) defer chainEventSub.Unsubscribe() - - blocksCh := make(chan *types.Block, 10) errCh := chainEventSub.Err() - go func() { - HandleChainEventChLoop: - for { - select { - //Notify chain event channel of events - case chainEvent := <-chainEventCh: - log.Debug("Event received from chainEventCh", "event", chainEvent) - blocksCh <- chainEvent.Block - //if node stopped - case err := <-errCh: - log.Warn("Error from chain event subscription, breaking loop.", "error", err) - close(sds.QuitChan) - break HandleChainEventChLoop - case <-sds.QuitChan: - break HandleChainEventChLoop - } - } - }() - - //loop through chain events until no more -HandleBlockChLoop: for { select { - case block := <-blocksCh: - currentBlock := block + //Notify chain event channel of events + case chainEvent := <-chainEventCh: + log.Debug("Event received from chainEventCh", "event", chainEvent) + currentBlock := chainEvent.Block parentHash := currentBlock.ParentHash() - parentBlock := sds.BlockChain.GetBlockByHash(parentHash) + var parentBlock *types.Block + if sds.lastBlock != nil && bytes.Equal(sds.lastBlock.Hash().Bytes(), currentBlock.ParentHash().Bytes()) { + parentBlock = sds.lastBlock + } else { + parentBlock = sds.BlockChain.GetBlockByHash(parentHash) + } + sds.lastBlock = currentBlock if parentBlock == nil { log.Error("Parent block is nil, skipping this block", "parent block hash", parentHash.String(), "current block number", currentBlock.Number()) - break HandleBlockChLoop + continue } - - stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number().Int64(), currentBlock.Hash()) - if err != nil { + if err := sds.process(currentBlock, parentBlock); err != nil { log.Error("Error building statediff", "block number", currentBlock.Number(), "error", err) } - rlpBuff := new(bytes.Buffer) - currentBlock.EncodeRLP(rlpBuff) - blockRlp := rlpBuff.Bytes() - stateDiffRlp, _ := rlp.EncodeToBytes(stateDiff) - payload := Payload{ - BlockRlp: blockRlp, - StateDiffRlp: stateDiffRlp, - Err: err, - } - // If we have any websocket subscription listening in, send the data to them - sds.send(payload) + case err := <-errCh: + log.Warn("Error from chain event subscription, breaking loop.", "error", err) + sds.close() + return case <-sds.QuitChan: - log.Debug("Quitting the statediff block channel") + log.Info("Quitting the statediff block channel") sds.close() return } } } +// process method builds the state diff payload from the current and parent block and streams it to listening subscriptions +func (sds *Service) process(currentBlock, parentBlock *types.Block) error { + stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number(), currentBlock.Hash()) + if err != nil { + return err + } + stateDiffRlp, err := rlp.EncodeToBytes(stateDiff) + if err != nil { + return err + } + payload := Payload{ + StateDiffRlp: stateDiffRlp, + Err: err, + } + if sds.streamBlock { + rlpBuff := new(bytes.Buffer) + if err = currentBlock.EncodeRLP(rlpBuff); err != nil { + return err + } + payload.BlockRlp = rlpBuff.Bytes() + } + + // If we have any websocket subscriptions listening in, send the data to them + sds.send(payload) + return nil +} + // Subscribe is used by the API to subscribe to the StateDiffingService loop func (sds *Service) Subscribe(id rpc.ID, sub chan<- Payload, quitChan chan<- bool) { log.Info("Subscribing to the statediff service") diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/service_test.go b/vendor/github.com/ethereum/go-ethereum/statediff/service_test.go new file mode 100644 index 00000000..d5edee04 --- /dev/null +++ b/vendor/github.com/ethereum/go-ethereum/statediff/service_test.go @@ -0,0 +1,130 @@ +// 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 . + +package statediff_test + +import ( + "bytes" + "math/big" + "math/rand" + "reflect" + "testing" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/testhelpers/mocks" +) + +func TestServiceLoop(t *testing.T) { + testErrorInChainEventLoop(t) + testErrorInBlockLoop(t) +} + +var ( + eventsChannel = make(chan core.ChainEvent, 1) + + parentRoot1 = common.HexToHash("0x01") + parentRoot2 = common.HexToHash("0x02") + parentHeader1 = types.Header{Number: big.NewInt(rand.Int63()), Root: parentRoot1} + parentHeader2 = types.Header{Number: big.NewInt(rand.Int63()), Root: parentRoot2} + + parentBlock1 = types.NewBlock(&parentHeader1, nil, nil, nil) + parentBlock2 = types.NewBlock(&parentHeader2, nil, nil, nil) + + parentHash1 = parentBlock1.Hash() + parentHash2 = parentBlock2.Hash() + + testRoot1 = common.HexToHash("0x03") + testRoot2 = common.HexToHash("0x04") + testRoot3 = common.HexToHash("0x04") + header1 = types.Header{ParentHash: parentHash1, Root: testRoot1} + header2 = types.Header{ParentHash: parentHash2, Root: testRoot2} + header3 = types.Header{ParentHash: common.HexToHash("parent hash"), Root: testRoot3} + + testBlock1 = types.NewBlock(&header1, nil, nil, nil) + testBlock2 = types.NewBlock(&header2, nil, nil, nil) + testBlock3 = types.NewBlock(&header3, nil, nil, nil) + + event1 = core.ChainEvent{Block: testBlock1} + event2 = core.ChainEvent{Block: testBlock2} + event3 = core.ChainEvent{Block: testBlock3} +) + +func testErrorInChainEventLoop(t *testing.T) { + //the first chain event causes and error (in blockchain mock) + builder := mocks.Builder{} + blockChain := mocks.BlockChain{} + service := statediff.Service{ + Builder: &builder, + BlockChain: &blockChain, + QuitChan: make(chan bool), + Subscriptions: make(map[rpc.ID]statediff.Subscription), + } + testRoot2 = common.HexToHash("0xTestRoot2") + blockChain.SetParentBlocksToReturn([]*types.Block{parentBlock1, parentBlock2}) + blockChain.SetChainEvents([]core.ChainEvent{event1, event2, event3}) + service.Loop(eventsChannel) + if !reflect.DeepEqual(builder.BlockHash, testBlock2.Hash()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.BlockHash, testBlock2.Hash()) + } + if !bytes.Equal(builder.OldStateRoot.Bytes(), parentBlock2.Root().Bytes()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.OldStateRoot, parentBlock2.Root()) + } + if !bytes.Equal(builder.NewStateRoot.Bytes(), testBlock2.Root().Bytes()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.NewStateRoot, testBlock2.Root()) + } + //look up the parent block from its hash + expectedHashes := []common.Hash{testBlock1.ParentHash(), testBlock2.ParentHash()} + if !reflect.DeepEqual(blockChain.ParentHashesLookedUp, expectedHashes) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", blockChain.ParentHashesLookedUp, expectedHashes) + } +} + +func testErrorInBlockLoop(t *testing.T) { + //second block's parent block can't be found + builder := mocks.Builder{} + blockChain := mocks.BlockChain{} + service := statediff.Service{ + Builder: &builder, + BlockChain: &blockChain, + QuitChan: make(chan bool), + Subscriptions: make(map[rpc.ID]statediff.Subscription), + } + + blockChain.SetParentBlocksToReturn([]*types.Block{parentBlock1, nil}) + blockChain.SetChainEvents([]core.ChainEvent{event1, event2}) + service.Loop(eventsChannel) + + if !bytes.Equal(builder.BlockHash.Bytes(), testBlock1.Hash().Bytes()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.BlockHash, testBlock1.Hash()) + } + if !bytes.Equal(builder.OldStateRoot.Bytes(), parentBlock1.Root().Bytes()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.OldStateRoot, parentBlock1.Root()) + } + if !bytes.Equal(builder.NewStateRoot.Bytes(), testBlock1.Root().Bytes()) { + t.Error("Test failure:", t.Name()) + t.Logf("Actual does not equal expected.\nactual:%+v\nexpected: %+v", builder.NewStateRoot, testBlock1.Root()) + } +} diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/helpers.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/helpers.go index 5126c655..8f52bc8c 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/helpers.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/helpers.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/blockchain.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/blockchain.go index cececde6..f2c097d3 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/blockchain.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/blockchain.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/builder.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/builder.go index e9668629..034af041 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/builder.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/builder.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -17,6 +17,8 @@ package mocks import ( + "math/big" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/statediff" ) @@ -25,14 +27,14 @@ import ( type Builder struct { OldStateRoot common.Hash NewStateRoot common.Hash - BlockNumber int64 + BlockNumber *big.Int BlockHash common.Hash stateDiff statediff.StateDiff builderError error } // BuildStateDiff mock method -func (builder *Builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (statediff.StateDiff, error) { +func (builder *Builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber *big.Int, blockHash common.Hash) (statediff.StateDiff, error) { builder.OldStateRoot = oldStateRoot builder.NewStateRoot = newStateRoot builder.BlockNumber = blockNumber diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/publisher.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/publisher.go index 3ea18abf..6a601874 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/publisher.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/publisher.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service.go index 5882f7e9..687a7c77 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -22,11 +22,12 @@ import ( "fmt" "sync" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/statediff" ) @@ -41,6 +42,7 @@ type MockStateDiffService struct { ParentBlockChan chan *types.Block QuitChan chan bool Subscriptions map[rpc.ID]statediff.Subscription + streamBlock bool } // Protocols mock method @@ -63,7 +65,6 @@ func (sds *MockStateDiffService) APIs() []rpc.API { // Loop mock method func (sds *MockStateDiffService) Loop(chan core.ChainEvent) { //loop through chain events until no more -HandleBlockChLoop: for { select { case block := <-sds.BlockChan: @@ -74,24 +75,12 @@ HandleBlockChLoop: log.Error("Parent block is nil, skipping this block", "parent block hash", parentHash.String(), "current block number", currentBlock.Number()) - break HandleBlockChLoop + continue } - - stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number().Int64(), currentBlock.Hash()) - if err != nil { + if err := sds.process(currentBlock, parentBlock); err != nil { + println(err.Error()) log.Error("Error building statediff", "block number", currentBlock.Number(), "error", err) } - rlpBuff := new(bytes.Buffer) - currentBlock.EncodeRLP(rlpBuff) - blockRlp := rlpBuff.Bytes() - stateDiffRlp, _ := rlp.EncodeToBytes(stateDiff) - payload := statediff.Payload{ - BlockRlp: blockRlp, - StateDiffRlp: stateDiffRlp, - Err: err, - } - // If we have any websocket subscription listening in, send the data to them - sds.send(payload) case <-sds.QuitChan: log.Debug("Quitting the statediff block channel") sds.close() @@ -100,6 +89,34 @@ HandleBlockChLoop: } } +// process method builds the state diff payload from the current and parent block and streams it to listening subscriptions +func (sds *MockStateDiffService) process(currentBlock, parentBlock *types.Block) error { + stateDiff, err := sds.Builder.BuildStateDiff(parentBlock.Root(), currentBlock.Root(), currentBlock.Number(), currentBlock.Hash()) + if err != nil { + return err + } + + stateDiffRlp, err := rlp.EncodeToBytes(stateDiff) + if err != nil { + return err + } + payload := statediff.Payload{ + StateDiffRlp: stateDiffRlp, + Err: err, + } + if sds.streamBlock { + rlpBuff := new(bytes.Buffer) + if err = currentBlock.EncodeRLP(rlpBuff); err != nil { + return err + } + payload.BlockRlp = rlpBuff.Bytes() + } + + // If we have any websocket subscription listening in, send the data to them + sds.send(payload) + return nil +} + // Subscribe mock method func (sds *MockStateDiffService) Subscribe(id rpc.ID, sub chan<- statediff.Payload, quitChan chan<- bool) { log.Info("Subscribing to the statediff service") diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service_test.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service_test.go new file mode 100644 index 00000000..4b4ac95a --- /dev/null +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/mocks/service_test.go @@ -0,0 +1,145 @@ +// 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 . + +package mocks + +import ( + "bytes" + "math/big" + "sort" + "sync" + "testing" + + "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/rlp" + "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/testhelpers" +) + +var block0, block1 *types.Block +var burnLeafKey = testhelpers.AddressToLeafKey(common.HexToAddress("0x0")) +var emptyAccountDiffEventualMap = make([]statediff.AccountDiff, 0) +var account1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: uint64(0), + Balance: big.NewInt(10000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"), +}) +var burnAccount1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: uint64(0), + Balance: big.NewInt(2000000000000000000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"), +}) +var bankAccount1, _ = rlp.EncodeToBytes(state.Account{ + Nonce: uint64(1), + Balance: big.NewInt(testhelpers.TestBankFunds.Int64() - 10000), + CodeHash: common.HexToHash("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").Bytes(), + Root: common.HexToHash("0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"), +}) + +func TestAPI(t *testing.T) { + _, blockMap, chain := testhelpers.MakeChain(3, testhelpers.Genesis) + defer chain.Stop() + block0Hash := common.HexToHash("0xd1721cfd0b29c36fd7a68f25c128e86413fb666a6e1d68e89b875bd299262661") + block1Hash := common.HexToHash("0xbbe88de60ba33a3f18c0caa37d827bfb70252e19e40a07cd34041696c35ecb1a") + block0 = blockMap[block0Hash] + block1 = blockMap[block1Hash] + blockChan := make(chan *types.Block) + parentBlockChain := make(chan *types.Block) + serviceQuitChan := make(chan bool) + config := statediff.Config{ + PathsAndProofs: true, + AllNodes: false, + } + mockService := MockStateDiffService{ + Mutex: sync.Mutex{}, + Builder: statediff.NewBuilder(testhelpers.Testdb, chain, config), + BlockChan: blockChan, + ParentBlockChan: parentBlockChain, + QuitChan: serviceQuitChan, + Subscriptions: make(map[rpc.ID]statediff.Subscription), + streamBlock: true, + } + mockService.Start(nil) + id := rpc.NewID() + payloadChan := make(chan statediff.Payload) + quitChan := make(chan bool) + mockService.Subscribe(id, payloadChan, quitChan) + blockChan <- block1 + parentBlockChain <- block0 + expectedBlockRlp, _ := rlp.EncodeToBytes(block1) + expectedStateDiff := statediff.StateDiff{ + BlockNumber: block1.Number(), + BlockHash: block1.Hash(), + CreatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: burnLeafKey.Bytes(), + Value: burnAccount1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 113, 160, 51, 128, 199, 183, 174, 129, 165, 142, 185, 141, 156, 120, 222, 74, 31, 215, 253, 149, 53, 252, 149, 62, 210, 190, 96, 45, 170, 164, 23, 103, 49, 42, 184, 78, 248, 76, 128, 136, 27, 193, 109, 103, 78, 200, 0, 0, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{5, 3, 8, 0, 12, 7, 11, 7, 10, 14, 8, 1, 10, 5, 8, 14, 11, 9, 8, 13, 9, 12, 7, 8, 13, 14, 4, 10, 1, 15, 13, 7, 15, 13, 9, 5, 3, 5, 15, 12, 9, 5, 3, 14, 13, 2, 11, 14, 6, 0, 2, 13, 10, 10, 10, 4, 1, 7, 6, 7, 3, 1, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + { + Leaf: true, + Key: testhelpers.Account1LeafKey.Bytes(), + Value: account1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 107, 160, 57, 38, 219, 105, 170, 206, 213, 24, 233, 185, 240, 244, 52, 164, 115, 231, 23, 65, 9, 201, 67, 84, 139, 184, 242, 59, 228, 28, 167, 109, 154, 210, 184, 72, 248, 70, 128, 130, 39, 16, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{14, 9, 2, 6, 13, 11, 6, 9, 10, 10, 12, 14, 13, 5, 1, 8, 14, 9, 11, 9, 15, 0, 15, 4, 3, 4, 10, 4, 7, 3, 14, 7, 1, 7, 4, 1, 0, 9, 12, 9, 4, 3, 5, 4, 8, 11, 11, 8, 15, 2, 3, 11, 14, 4, 1, 12, 10, 7, 6, 13, 9, 10, 13, 2, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: []statediff.AccountDiff{ + { + Leaf: true, + Key: testhelpers.BankLeafKey.Bytes(), + Value: bankAccount1, + Proof: [][]byte{{248, 113, 160, 87, 118, 82, 182, 37, 183, 123, 219, 91, 247, 123, 196, 63, 49, 37, 202, 215, 70, 77, 103, 157, 21, 117, 86, 82, 119, 211, 97, 27, 128, 83, 231, 128, 128, 128, 128, 160, 254, 136, 159, 16, 229, 219, 143, 44, 43, 243, 85, 146, 129, 82, 161, 127, 110, 59, 185, 154, 146, 65, 172, 109, 132, 199, 126, 98, 100, 80, 156, 121, 128, 128, 128, 128, 128, 128, 128, 128, 160, 17, 219, 12, 218, 52, 168, 150, 218, 190, 182, 131, 155, 176, 106, 56, 244, 149, 20, 207, 164, 134, 67, 89, 132, 235, 1, 59, 125, 249, 238, 133, 197, 128, 128}, + {248, 109, 160, 48, 191, 73, 244, 64, 161, 205, 5, 39, 228, 208, 110, 39, 101, 101, 76, 15, 86, 69, 34, 87, 81, 109, 121, 58, 155, 141, 96, 77, 207, 223, 42, 184, 74, 248, 72, 1, 132, 5, 245, 185, 240, 160, 86, 232, 31, 23, 27, 204, 85, 166, 255, 131, 69, 230, 146, 192, 248, 110, 91, 72, 224, 27, 153, 108, 173, 192, 1, 98, 47, 181, 227, 99, 180, 33, 160, 197, 210, 70, 1, 134, 247, 35, 60, 146, 126, 125, 178, 220, 199, 3, 192, 229, 0, 182, 83, 202, 130, 39, 59, 123, 250, 216, 4, 93, 133, 164, 112}}, + Path: []byte{0, 0, 11, 15, 4, 9, 15, 4, 4, 0, 10, 1, 12, 13, 0, 5, 2, 7, 14, 4, 13, 0, 6, 14, 2, 7, 6, 5, 6, 5, 4, 12, 0, 15, 5, 6, 4, 5, 2, 2, 5, 7, 5, 1, 6, 13, 7, 9, 3, 10, 9, 11, 8, 13, 6, 0, 4, 13, 12, 15, 13, 15, 2, 10, 16}, + Storage: []statediff.StorageDiff{}, + }, + }, + } + expectedStateDiffBytes, err := rlp.EncodeToBytes(expectedStateDiff) + if err != nil { + t.Error(err) + } + sort.Slice(expectedStateDiffBytes, func(i, j int) bool { return expectedStateDiffBytes[i] < expectedStateDiffBytes[j] }) + + select { + case payload := <-payloadChan: + if !bytes.Equal(payload.BlockRlp, expectedBlockRlp) { + t.Errorf("payload does not have expected block\r\actual block rlp: %v\r\nexpected block rlp: %v", payload.BlockRlp, expectedBlockRlp) + } + sort.Slice(payload.StateDiffRlp, func(i, j int) bool { return payload.StateDiffRlp[i] < payload.StateDiffRlp[j] }) + if !bytes.Equal(payload.StateDiffRlp, expectedStateDiffBytes) { + t.Errorf("payload does not have expected state diff\r\actual state diff rlp: %v\r\nexpected state diff rlp: %v", payload.StateDiffRlp, expectedStateDiffBytes) + } + if payload.Err != nil { + t.Errorf("payload should not contain an error, but does: %v", payload.Err) + } + case <-quitChan: + t.Errorf("channel quit before delivering payload") + } +} diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/test_data.go b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/test_data.go index 17dac847..2f6088f8 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/test_data.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/testhelpers/test_data.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -36,7 +36,7 @@ func AddressToLeafKey(address common.Address) common.Hash { // Test variables var ( - BlockNumber = rand.Int63() + BlockNumber = big.NewInt(rand.Int63()) BlockHash = "0xfa40fbe2d98d98b3363a778d52f2bcd29d6790b9b3f3cab2b167fd12d3550f73" CodeHash = common.Hex2Bytes("0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470") NewNonceValue = rand.Uint64() @@ -63,26 +63,26 @@ var ( CodeHash: CodeHash, } valueBytes, _ = rlp.EncodeToBytes(testAccount) - CreatedAccountDiffs = statediff.AccountDiffsMap{ - ContractLeafKey: { + CreatedAccountDiffs = []statediff.AccountDiff{ + { Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, }, - AnotherContractLeafKey: { + { Key: AnotherContractLeafKey.Bytes(), Value: valueBytes, Storage: emptyStorage, }, } - UpdatedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: { + UpdatedAccountDiffs = []statediff.AccountDiff{{ Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, }} - DeletedAccountDiffs = statediff.AccountDiffsMap{ContractLeafKey: { + DeletedAccountDiffs = []statediff.AccountDiff{{ Key: ContractLeafKey.Bytes(), Value: valueBytes, Storage: storage, diff --git a/vendor/github.com/ethereum/go-ethereum/statediff/types.go b/vendor/github.com/ethereum/go-ethereum/statediff/types.go index 41e83ddd..6df398a1 100644 --- a/vendor/github.com/ethereum/go-ethereum/statediff/types.go +++ b/vendor/github.com/ethereum/go-ethereum/statediff/types.go @@ -1,4 +1,4 @@ -// Copyright 2015 The go-ethereum Authors +// 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 @@ -21,30 +21,33 @@ package statediff import ( "encoding/json" + "math/big" + + "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/state" ) -// AccountsMap is a mapping of keccak256(address) => accountWrapper -type AccountsMap map[common.Hash]accountWrapper +// Subscription struct holds our subscription channels +type Subscription struct { + PayloadChan chan<- Payload + QuitChan chan<- bool +} -// AccountWrapper is used to temporary associate the unpacked account with its raw values -type accountWrapper struct { - Account state.Account - RawKey []byte - RawValue []byte - Proof [][]byte - Path []byte +// Payload packages the data to send to StateDiffingService subscriptions +type Payload struct { + BlockRlp []byte `json:"blockRlp" gencodec:"required"` + StateDiffRlp []byte `json:"stateDiff" gencodec:"required"` + Err error `json:"error"` } // StateDiff is the final output structure from the builder type StateDiff struct { - BlockNumber int64 `json:"blockNumber" gencodec:"required"` - BlockHash common.Hash `json:"blockHash" gencodec:"required"` - CreatedAccounts AccountDiffsMap `json:"createdAccounts" gencodec:"required"` - DeletedAccounts AccountDiffsMap `json:"deletedAccounts" gencodec:"required"` - UpdatedAccounts AccountDiffsMap `json:"updatedAccounts" gencodec:"required"` + BlockNumber *big.Int `json:"blockNumber" gencodec:"required"` + BlockHash common.Hash `json:"blockHash" gencodec:"required"` + CreatedAccounts []AccountDiff `json:"createdAccounts" gencodec:"required"` + DeletedAccounts []AccountDiff `json:"deletedAccounts" gencodec:"required"` + UpdatedAccounts []AccountDiff `json:"updatedAccounts" gencodec:"required"` encoded []byte err error @@ -68,46 +71,34 @@ func (sd *StateDiff) Encode() ([]byte, error) { return sd.encoded, sd.err } -// AccountDiffsMap is a mapping of keccak256(address) => AccountDiff -type AccountDiffsMap map[common.Hash]AccountDiff - -// AccountDiff holds the data for a single state diff leaf node +// AccountDiff holds the data for a single state diff node type AccountDiff struct { + Leaf bool `json:"leaf" gencodec:"required"` Key []byte `json:"key" gencodec:"required"` Value []byte `json:"value" gencodec:"required"` Proof [][]byte `json:"proof" gencodec:"required"` - Storage []StorageDiff `json:"storage" gencodec:"required"` Path []byte `json:"path" gencodec:"required"` + Storage []StorageDiff `json:"storage" gencodec:"required"` } -// StorageDiff holds the data for a single storage diff leaf node +// StorageDiff holds the data for a single storage diff node type StorageDiff struct { + Leaf bool `json:"leaf" gencodec:"required"` Key []byte `json:"key" gencodec:"required"` Value []byte `json:"value" gencodec:"required"` Proof [][]byte `json:"proof" gencodec:"required"` Path []byte `json:"path" gencodec:"required"` } -/* -// State trie leaf is just a short node, below -// that has an rlp encoded account as the value +// AccountsMap is a mapping of keccak256(address) => accountWrapper +type AccountsMap map[common.Hash]accountWrapper - -// SO each account diffs map is reall a map of shortnode keys to values -// Flatten to a slice of short nodes? - -// Need to coerce into: - -type TrieNode struct { - // leaf, extension or branch - nodeKind string - - // If leaf or extension: [0] is key, [1] is val. - // If branch: [0] - [16] are children. - elements []interface{} - - // IPLD block information - cid *cid.Cid - rawdata []byte +// AccountWrapper is used to temporary associate the unpacked account with its raw values +type accountWrapper struct { + Account *state.Account + Leaf bool + RawKey []byte + RawValue []byte + Proof [][]byte + Path []byte } -*/