core, eth: receipt chain reconstruction

This commit is contained in:
Péter Szilágyi
2015-09-30 19:23:31 +03:00
parent 42c8afd440
commit 832b37c822
22 changed files with 613 additions and 230 deletions

View File

@ -45,9 +45,9 @@ var (
// the returned hash chain is ordered head->parent. In addition, every 3rd block
// contains a transaction and every 5th an uncle to allow testing correct block
// reassembly.
func makeChain(n int, seed byte, parent *types.Block) ([]common.Hash, map[common.Hash]*types.Header, map[common.Hash]*types.Block) {
func makeChain(n int, seed byte, parent *types.Block, parentReceipts types.Receipts) ([]common.Hash, map[common.Hash]*types.Header, map[common.Hash]*types.Block, map[common.Hash]types.Receipts) {
// Generate the block chain
blocks := core.GenerateChain(parent, testdb, n, func(i int, block *core.BlockGen) {
blocks, receipts := core.GenerateChain(parent, testdb, n, func(i int, block *core.BlockGen) {
block.SetCoinbase(common.Address{seed})
// If the block number is multiple of 3, send a bonus transaction to the miner
@ -73,25 +73,29 @@ func makeChain(n int, seed byte, parent *types.Block) ([]common.Hash, map[common
blockm := make(map[common.Hash]*types.Block, n+1)
blockm[parent.Hash()] = parent
receiptm := make(map[common.Hash]types.Receipts, n+1)
receiptm[parent.Hash()] = parentReceipts
for i, b := range blocks {
hashes[len(hashes)-i-2] = b.Hash()
headerm[b.Hash()] = b.Header()
blockm[b.Hash()] = b
receiptm[b.Hash()] = receipts[i]
}
return hashes, headerm, blockm
return hashes, headerm, blockm, receiptm
}
// makeChainFork creates two chains of length n, such that h1[:f] and
// h2[:f] are different but have a common suffix of length n-f.
func makeChainFork(n, f int, parent *types.Block) ([]common.Hash, []common.Hash, map[common.Hash]*types.Header, map[common.Hash]*types.Header, map[common.Hash]*types.Block, map[common.Hash]*types.Block) {
func makeChainFork(n, f int, parent *types.Block, parentReceipts types.Receipts) ([]common.Hash, []common.Hash, map[common.Hash]*types.Header, map[common.Hash]*types.Header, map[common.Hash]*types.Block, map[common.Hash]*types.Block, map[common.Hash]types.Receipts, map[common.Hash]types.Receipts) {
// Create the common suffix
hashes, headers, blocks := makeChain(n-f, 0, parent)
hashes, headers, blocks, receipts := makeChain(n-f, 0, parent, parentReceipts)
// Create the forks
hashes1, headers1, blocks1 := makeChain(f, 1, blocks[hashes[0]])
hashes1, headers1, blocks1, receipts1 := makeChain(f, 1, blocks[hashes[0]], receipts[hashes[0]])
hashes1 = append(hashes1, hashes[1:]...)
hashes2, headers2, blocks2 := makeChain(f, 2, blocks[hashes[0]])
hashes2, headers2, blocks2, receipts2 := makeChain(f, 2, blocks[hashes[0]], receipts[hashes[0]])
hashes2 = append(hashes2, hashes[1:]...)
for hash, header := range headers {
@ -102,22 +106,28 @@ func makeChainFork(n, f int, parent *types.Block) ([]common.Hash, []common.Hash,
blocks1[hash] = block
blocks2[hash] = block
}
return hashes1, hashes2, headers1, headers2, blocks1, blocks2
for hash, receipt := range receipts {
receipts1[hash] = receipt
receipts2[hash] = receipt
}
return hashes1, hashes2, headers1, headers2, blocks1, blocks2, receipts1, receipts2
}
// downloadTester is a test simulator for mocking out local block chain.
type downloadTester struct {
downloader *Downloader
ownHashes []common.Hash // Hash chain belonging to the tester
ownHeaders map[common.Hash]*types.Header // Headers belonging to the tester
ownBlocks map[common.Hash]*types.Block // Blocks belonging to the tester
ownReceipts map[common.Hash]types.Receipts // Receipts belonging to the tester
ownChainTd map[common.Hash]*big.Int // Total difficulties of the blocks in the local chain
peerHashes map[string][]common.Hash // Hash chain belonging to different test peers
peerHeaders map[string]map[common.Hash]*types.Header // Headers belonging to different test peers
peerBlocks map[string]map[common.Hash]*types.Block // Blocks belonging to different test peers
peerChainTds map[string]map[common.Hash]*big.Int // Total difficulties of the blocks in the peer chains
ownHashes []common.Hash // Hash chain belonging to the tester
ownHeaders map[common.Hash]*types.Header // Headers belonging to the tester
ownBlocks map[common.Hash]*types.Block // Blocks belonging to the tester
ownReceipts map[common.Hash]types.Receipts // Receipts belonging to the tester
ownChainTd map[common.Hash]*big.Int // Total difficulties of the blocks in the local chain
peerHashes map[string][]common.Hash // Hash chain belonging to different test peers
peerHeaders map[string]map[common.Hash]*types.Header // Headers belonging to different test peers
peerBlocks map[string]map[common.Hash]*types.Block // Blocks belonging to different test peers
peerReceipts map[string]map[common.Hash]types.Receipts // Receipts belonging to different test peers
peerChainTds map[string]map[common.Hash]*big.Int // Total difficulties of the blocks in the peer chains
lock sync.RWMutex
}
@ -128,15 +138,16 @@ func newTester(mode SyncMode) *downloadTester {
ownHashes: []common.Hash{genesis.Hash()},
ownHeaders: map[common.Hash]*types.Header{genesis.Hash(): genesis.Header()},
ownBlocks: map[common.Hash]*types.Block{genesis.Hash(): genesis},
ownReceipts: map[common.Hash]types.Receipts{genesis.Hash(): genesis.Receipts()},
ownReceipts: map[common.Hash]types.Receipts{genesis.Hash(): nil},
ownChainTd: map[common.Hash]*big.Int{genesis.Hash(): genesis.Difficulty()},
peerHashes: make(map[string][]common.Hash),
peerHeaders: make(map[string]map[common.Hash]*types.Header),
peerBlocks: make(map[string]map[common.Hash]*types.Block),
peerReceipts: make(map[string]map[common.Hash]types.Receipts),
peerChainTds: make(map[string]map[common.Hash]*big.Int),
}
tester.downloader = New(mode, new(event.TypeMux), tester.hasHeader, tester.hasBlock, tester.getHeader, tester.getBlock,
tester.headHeader, tester.headBlock, tester.getTd, tester.insertHeaders, tester.insertBlocks, tester.insertConfirmedBlocks, tester.dropPeer)
tester.headHeader, tester.headBlock, tester.headFastBlock, tester.getTd, tester.insertHeaders, tester.insertBlocks, tester.insertReceipts, tester.dropPeer)
return tester
}
@ -197,7 +208,12 @@ func (dl *downloadTester) headHeader() *types.Header {
dl.lock.RLock()
defer dl.lock.RUnlock()
return dl.getHeader(dl.ownHashes[len(dl.ownHashes)-1])
for i := len(dl.ownHashes) - 1; i >= 0; i-- {
if header := dl.getHeader(dl.ownHashes[i]); header != nil {
return header
}
}
return nil
}
// headBlock retrieves the current head block from the canonical chain.
@ -213,6 +229,21 @@ func (dl *downloadTester) headBlock() *types.Block {
return nil
}
// headFastBlock retrieves the current head fast-sync block from the canonical chain.
func (dl *downloadTester) headFastBlock() *types.Block {
dl.lock.RLock()
defer dl.lock.RUnlock()
for i := len(dl.ownHashes) - 1; i >= 0; i-- {
if block := dl.getBlock(dl.ownHashes[i]); block != nil {
if _, ok := dl.ownReceipts[block.Hash()]; ok {
return block
}
}
}
return nil
}
// getTd retrieves the block's total difficulty from the canonical chain.
func (dl *downloadTester) getTd(hash common.Hash) *big.Int {
dl.lock.RLock()
@ -227,6 +258,9 @@ func (dl *downloadTester) insertHeaders(headers []*types.Header, verify bool) (i
defer dl.lock.Unlock()
for i, header := range headers {
if _, ok := dl.ownHeaders[header.Hash()]; ok {
continue
}
if _, ok := dl.ownHeaders[header.ParentHash]; !ok {
return i, errors.New("unknown parent")
}
@ -254,33 +288,33 @@ func (dl *downloadTester) insertBlocks(blocks types.Blocks) (int, error) {
return len(blocks), nil
}
// insertBlocks injects a new batch of blocks into the simulated chain.
func (dl *downloadTester) insertConfirmedBlocks(blocks types.Blocks, receipts []types.Receipts) (int, error) {
// insertReceipts injects a new batch of blocks into the simulated chain.
func (dl *downloadTester) insertReceipts(blocks types.Blocks, receipts []types.Receipts) (int, error) {
dl.lock.Lock()
defer dl.lock.Unlock()
for i := 0; i < len(blocks) && i < len(receipts); i++ {
if _, ok := dl.ownHeaders[blocks[i].Hash()]; !ok {
return i, errors.New("unknown owner")
}
if _, ok := dl.ownBlocks[blocks[i].ParentHash()]; !ok {
return i, errors.New("unknown parent")
}
dl.ownHashes = append(dl.ownHashes, blocks[i].Hash())
dl.ownHeaders[blocks[i].Hash()] = blocks[i].Header()
dl.ownBlocks[blocks[i].Hash()] = blocks[i]
dl.ownReceipts[blocks[i].Hash()] = blocks[i].Receipts()
dl.ownChainTd[blocks[i].Hash()] = dl.ownChainTd[blocks[i].ParentHash()]
dl.ownReceipts[blocks[i].Hash()] = receipts[i]
}
return len(blocks), nil
}
// newPeer registers a new block download source into the downloader.
func (dl *downloadTester) newPeer(id string, version int, hashes []common.Hash, headers map[common.Hash]*types.Header, blocks map[common.Hash]*types.Block) error {
return dl.newSlowPeer(id, version, hashes, headers, blocks, 0)
func (dl *downloadTester) newPeer(id string, version int, hashes []common.Hash, headers map[common.Hash]*types.Header, blocks map[common.Hash]*types.Block, receipts map[common.Hash]types.Receipts) error {
return dl.newSlowPeer(id, version, hashes, headers, blocks, receipts, 0)
}
// newSlowPeer registers a new block download source into the downloader, with a
// specific delay time on processing the network packets sent to it, simulating
// potentially slow network IO.
func (dl *downloadTester) newSlowPeer(id string, version int, hashes []common.Hash, headers map[common.Hash]*types.Header, blocks map[common.Hash]*types.Block, delay time.Duration) error {
func (dl *downloadTester) newSlowPeer(id string, version int, hashes []common.Hash, headers map[common.Hash]*types.Header, blocks map[common.Hash]*types.Block, receipts map[common.Hash]types.Receipts, delay time.Duration) error {
dl.lock.Lock()
defer dl.lock.Unlock()
@ -302,6 +336,7 @@ func (dl *downloadTester) newSlowPeer(id string, version int, hashes []common.Ha
dl.peerHeaders[id] = make(map[common.Hash]*types.Header)
dl.peerBlocks[id] = make(map[common.Hash]*types.Block)
dl.peerReceipts[id] = make(map[common.Hash]types.Receipts)
dl.peerChainTds[id] = make(map[common.Hash]*big.Int)
for _, hash := range hashes {
@ -317,6 +352,9 @@ func (dl *downloadTester) newSlowPeer(id string, version int, hashes []common.Ha
dl.peerChainTds[id][hash] = new(big.Int).Add(block.Difficulty(), dl.peerChainTds[id][block.ParentHash()])
}
}
if receipt, ok := receipts[hash]; ok {
dl.peerReceipts[id][hash] = receipt
}
}
}
return err
@ -501,15 +539,15 @@ func (dl *downloadTester) peerGetReceiptsFn(id string, delay time.Duration) func
dl.lock.RLock()
defer dl.lock.RUnlock()
blocks := dl.peerBlocks[id]
receipts := dl.peerReceipts[id]
receipts := make([][]*types.Receipt, 0, len(hashes))
results := make([][]*types.Receipt, 0, len(hashes))
for _, hash := range hashes {
if block, ok := blocks[hash]; ok {
receipts = append(receipts, block.Receipts())
if receipt, ok := receipts[hash]; ok {
results = append(results, receipt)
}
}
go dl.downloader.DeliverReceipts(id, receipts)
go dl.downloader.DeliverReceipts(id, results)
return nil
}
@ -551,10 +589,10 @@ func TestCanonicalSynchronisation64Light(t *testing.T) { testCanonicalSynchronis
func testCanonicalSynchronisation(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
tester.newPeer("peer", protocol, hashes, headers, blocks)
tester.newPeer("peer", protocol, hashes, headers, blocks, receipts)
// Synchronise with the peer and make sure all relevant data was retrieved
if err := tester.sync("peer", nil); err != nil {
@ -575,10 +613,10 @@ func TestThrottling64Fast(t *testing.T) { testThrottling(t, 64, FastSync) }
func testThrottling(t *testing.T, protocol int, mode SyncMode) {
// Create a long block chain to download and the tester
targetBlocks := 8 * blockCacheLimit
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
tester.newPeer("peer", protocol, hashes, headers, blocks)
tester.newPeer("peer", protocol, hashes, headers, blocks, receipts)
// Wrap the importer to allow stepping
blocked, proceed := uint32(0), make(chan struct{})
@ -650,11 +688,11 @@ func TestForkedSynchronisation64Light(t *testing.T) { testForkedSynchronisation(
func testForkedSynchronisation(t *testing.T, protocol int, mode SyncMode) {
// Create a long enough forked chain
common, fork := MaxHashFetch, 2*MaxHashFetch
hashesA, hashesB, headersA, headersB, blocksA, blocksB := makeChainFork(common+fork, fork, genesis)
hashesA, hashesB, headersA, headersB, blocksA, blocksB, receiptsA, receiptsB := makeChainFork(common+fork, fork, genesis, nil)
tester := newTester(mode)
tester.newPeer("fork A", protocol, hashesA, headersA, blocksA)
tester.newPeer("fork B", protocol, hashesB, headersB, blocksB)
tester.newPeer("fork A", protocol, hashesA, headersA, blocksA, receiptsA)
tester.newPeer("fork B", protocol, hashesB, headersB, blocksB, receiptsB)
// Synchronise with the peer and make sure all blocks were retrieved
if err := tester.sync("fork A", nil); err != nil {
@ -731,10 +769,10 @@ func testCancel(t *testing.T, protocol int, mode SyncMode) {
if targetBlocks >= MaxHeaderFetch {
targetBlocks = MaxHeaderFetch - 15
}
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
tester.newPeer("peer", protocol, hashes, headers, blocks)
tester.newPeer("peer", protocol, hashes, headers, blocks, receipts)
// Make sure canceling works with a pristine downloader
tester.downloader.cancel()
@ -764,12 +802,12 @@ func testMultiSynchronisation(t *testing.T, protocol int, mode SyncMode) {
// Create various peers with various parts of the chain
targetPeers := 8
targetBlocks := targetPeers*blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
for i := 0; i < targetPeers; i++ {
id := fmt.Sprintf("peer #%d", i)
tester.newPeer(id, protocol, hashes[i*blockCacheLimit:], headers, blocks)
tester.newPeer(id, protocol, hashes[i*blockCacheLimit:], headers, blocks, receipts)
}
// Synchronise with the middle peer and make sure half of the blocks were retrieved
id := fmt.Sprintf("peer #%d", targetPeers/2)
@ -798,22 +836,21 @@ func TestMultiProtoSynchronisation64Light(t *testing.T) { testMultiProtoSync(t,
func testMultiProtoSync(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
// Create peers of every type
tester := newTester(mode)
tester.newPeer("peer 61", 61, hashes, headers, blocks)
tester.newPeer("peer 62", 62, hashes, headers, blocks)
tester.newPeer("peer 63", 63, hashes, headers, blocks)
tester.newPeer("peer 64", 64, hashes, headers, blocks)
tester.newPeer("peer 61", 61, hashes, headers, blocks, receipts)
tester.newPeer("peer 62", 62, hashes, headers, blocks, receipts)
tester.newPeer("peer 63", 63, hashes, headers, blocks, receipts)
tester.newPeer("peer 64", 64, hashes, headers, blocks, receipts)
// Synchronise with the requestd peer and make sure all blocks were retrieved
if err := tester.sync(fmt.Sprintf("peer %d", protocol), nil); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
if imported := len(tester.ownBlocks); imported != targetBlocks+1 {
t.Fatalf("synchronised block mismatch: have %v, want %v", imported, targetBlocks+1)
}
assertOwnChain(t, tester, targetBlocks+1)
// Check that no peers have been dropped off
for _, version := range []int{61, 62, 63, 64} {
peer := fmt.Sprintf("peer %d", version)
@ -835,18 +872,18 @@ func TestEmptyShortCircuit64Light(t *testing.T) { testEmptyShortCircuit(t, 64, L
func testEmptyShortCircuit(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
tester.newPeer("peer", protocol, hashes, headers, blocks)
tester.newPeer("peer", protocol, hashes, headers, blocks, receipts)
// Instrument the downloader to signal body requests
bodies, receipts := int32(0), int32(0)
bodiesHave, receiptsHave := int32(0), int32(0)
tester.downloader.bodyFetchHook = func(headers []*types.Header) {
atomic.AddInt32(&bodies, int32(len(headers)))
atomic.AddInt32(&bodiesHave, int32(len(headers)))
}
tester.downloader.receiptFetchHook = func(headers []*types.Header) {
atomic.AddInt32(&receipts, int32(len(headers)))
atomic.AddInt32(&receiptsHave, int32(len(headers)))
}
// Synchronise with the peer and make sure all blocks were retrieved
if err := tester.sync("peer", nil); err != nil {
@ -860,15 +897,17 @@ func testEmptyShortCircuit(t *testing.T, protocol int, mode SyncMode) {
if mode != LightSync && block != genesis && (len(block.Transactions()) > 0 || len(block.Uncles()) > 0) {
bodiesNeeded++
}
if mode == FastSync && block != genesis && len(block.Receipts()) > 0 {
}
for _, receipt := range receipts {
if mode == FastSync && len(receipt) > 0 {
receiptsNeeded++
}
}
if int(bodies) != bodiesNeeded {
t.Errorf("body retrieval count mismatch: have %v, want %v", bodies, bodiesNeeded)
if int(bodiesHave) != bodiesNeeded {
t.Errorf("body retrieval count mismatch: have %v, want %v", bodiesHave, bodiesNeeded)
}
if int(receipts) != receiptsNeeded {
t.Errorf("receipt retrieval count mismatch: have %v, want %v", receipts, receiptsNeeded)
if int(receiptsHave) != receiptsNeeded {
t.Errorf("receipt retrieval count mismatch: have %v, want %v", receiptsHave, receiptsNeeded)
}
}
@ -884,21 +923,20 @@ func TestMissingHeaderAttack64Light(t *testing.T) { testMissingHeaderAttack(t, 6
func testMissingHeaderAttack(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
// Attempt a full sync with an attacker feeding gapped headers
tester.newPeer("attack", protocol, hashes, headers, blocks)
tester.newPeer("attack", protocol, hashes, headers, blocks, receipts)
missing := targetBlocks / 2
delete(tester.peerHeaders["attack"], hashes[missing])
delete(tester.peerBlocks["attack"], hashes[missing])
if err := tester.sync("attack", nil); err == nil {
t.Fatalf("succeeded attacker synchronisation")
}
// Synchronise with the valid peer and make sure sync succeeds
tester.newPeer("valid", protocol, hashes, headers, blocks)
tester.newPeer("valid", protocol, hashes, headers, blocks, receipts)
if err := tester.sync("valid", nil); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
@ -917,20 +955,21 @@ func TestShiftedHeaderAttack64Light(t *testing.T) { testShiftedHeaderAttack(t, 6
func testShiftedHeaderAttack(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
tester := newTester(mode)
// Attempt a full sync with an attacker feeding shifted headers
tester.newPeer("attack", protocol, hashes, headers, blocks)
tester.newPeer("attack", protocol, hashes, headers, blocks, receipts)
delete(tester.peerHeaders["attack"], hashes[len(hashes)-2])
delete(tester.peerBlocks["attack"], hashes[len(hashes)-2])
delete(tester.peerReceipts["attack"], hashes[len(hashes)-2])
if err := tester.sync("attack", nil); err == nil {
t.Fatalf("succeeded attacker synchronisation")
}
// Synchronise with the valid peer and make sure sync succeeds
tester.newPeer("valid", protocol, hashes, headers, blocks)
tester.newPeer("valid", protocol, hashes, headers, blocks, receipts)
if err := tester.sync("valid", nil); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
@ -949,24 +988,24 @@ func TestInvalidContentAttack64Light(t *testing.T) { testInvalidContentAttack(t,
func testInvalidContentAttack(t *testing.T, protocol int, mode SyncMode) {
// Create two peers, one feeding invalid block bodies
targetBlocks := 4*blockCacheLimit - 15
hashes, headers, validBlocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, validBlocks, validReceipts := makeChain(targetBlocks, 0, genesis, nil)
invalidBlocks := make(map[common.Hash]*types.Block)
for hash, block := range validBlocks {
invalidBlocks[hash] = types.NewBlockWithHeader(block.Header())
}
invalidReceipts := make(map[common.Hash]*types.Block)
for hash, block := range validBlocks {
invalidReceipts[hash] = types.NewBlockWithHeader(block.Header()).WithBody(block.Transactions(), block.Uncles())
invalidReceipts := make(map[common.Hash]types.Receipts)
for hash, _ := range validReceipts {
invalidReceipts[hash] = types.Receipts{&types.Receipt{}}
}
tester := newTester(mode)
tester.newPeer("valid", protocol, hashes, headers, validBlocks)
tester.newPeer("valid", protocol, hashes, headers, validBlocks, validReceipts)
if mode != LightSync {
tester.newPeer("body attack", protocol, hashes, headers, invalidBlocks)
tester.newPeer("body attack", protocol, hashes, headers, invalidBlocks, validReceipts)
}
if mode == FastSync {
tester.newPeer("receipt attack", protocol, hashes, headers, invalidReceipts)
tester.newPeer("receipt attack", protocol, hashes, headers, validBlocks, invalidReceipts)
}
// Synchronise with the valid peer (will pull contents from the attacker too)
if err := tester.sync("valid", nil); err != nil {
@ -995,9 +1034,9 @@ func TestHighTDStarvationAttack64Light(t *testing.T) { testHighTDStarvationAttac
func testHighTDStarvationAttack(t *testing.T, protocol int, mode SyncMode) {
tester := newTester(mode)
hashes, headers, blocks := makeChain(0, 0, genesis)
hashes, headers, blocks, receipts := makeChain(0, 0, genesis, nil)
tester.newPeer("attack", protocol, []common.Hash{hashes[0]}, headers, blocks)
tester.newPeer("attack", protocol, []common.Hash{hashes[0]}, headers, blocks, receipts)
if err := tester.sync("attack", big.NewInt(1000000)); err != errStallingPeer {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errStallingPeer)
}
@ -1040,7 +1079,7 @@ func testBlockHeaderAttackerDropping(t *testing.T, protocol int) {
for i, tt := range tests {
// Register a new peer and ensure it's presence
id := fmt.Sprintf("test %d", i)
if err := tester.newPeer(id, protocol, []common.Hash{genesis.Hash()}, nil, nil); err != nil {
if err := tester.newPeer(id, protocol, []common.Hash{genesis.Hash()}, nil, nil, nil); err != nil {
t.Fatalf("test %d: failed to register new peer: %v", i, err)
}
if _, ok := tester.peerHashes[id]; !ok {
@ -1069,7 +1108,7 @@ func TestSyncBoundaries64Light(t *testing.T) { testSyncBoundaries(t, 64, LightSy
func testSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
// Set a sync init hook to catch boundary changes
starting := make(chan struct{})
@ -1085,7 +1124,7 @@ func testSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
t.Fatalf("Pristine boundary mismatch: have %v/%v, want %v/%v", origin, latest, 0, 0)
}
// Synchronise half the blocks and check initial boundaries
tester.newPeer("peer-half", protocol, hashes[targetBlocks/2:], headers, blocks)
tester.newPeer("peer-half", protocol, hashes[targetBlocks/2:], headers, blocks, receipts)
pending := new(sync.WaitGroup)
pending.Add(1)
@ -1103,7 +1142,7 @@ func testSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
pending.Wait()
// Synchronise all the blocks and check continuation boundaries
tester.newPeer("peer-full", protocol, hashes, headers, blocks)
tester.newPeer("peer-full", protocol, hashes, headers, blocks, receipts)
pending.Add(1)
go func() {
@ -1134,7 +1173,7 @@ func TestForkedSyncBoundaries64Light(t *testing.T) { testForkedSyncBoundaries(t,
func testForkedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
// Create a forked chain to simulate origin revertal
common, fork := MaxHashFetch, 2*MaxHashFetch
hashesA, hashesB, headersA, headersB, blocksA, blocksB := makeChainFork(common+fork, fork, genesis)
hashesA, hashesB, headersA, headersB, blocksA, blocksB, receiptsA, receiptsB := makeChainFork(common+fork, fork, genesis, nil)
// Set a sync init hook to catch boundary changes
starting := make(chan struct{})
@ -1150,7 +1189,7 @@ func testForkedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
t.Fatalf("Pristine boundary mismatch: have %v/%v, want %v/%v", origin, latest, 0, 0)
}
// Synchronise with one of the forks and check boundaries
tester.newPeer("fork A", protocol, hashesA, headersA, blocksA)
tester.newPeer("fork A", protocol, hashesA, headersA, blocksA, receiptsA)
pending := new(sync.WaitGroup)
pending.Add(1)
@ -1171,7 +1210,7 @@ func testForkedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
tester.downloader.syncStatsOrigin = tester.downloader.syncStatsHeight
// Synchronise with the second fork and check boundary resets
tester.newPeer("fork B", protocol, hashesB, headersB, blocksB)
tester.newPeer("fork B", protocol, hashesB, headersB, blocksB, receiptsB)
pending.Add(1)
go func() {
@ -1202,7 +1241,7 @@ func TestFailedSyncBoundaries64Light(t *testing.T) { testFailedSyncBoundaries(t,
func testFailedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
// Create a small enough block chain to download
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks, 0, genesis, nil)
// Set a sync init hook to catch boundary changes
starting := make(chan struct{})
@ -1218,10 +1257,11 @@ func testFailedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
t.Fatalf("Pristine boundary mismatch: have %v/%v, want %v/%v", origin, latest, 0, 0)
}
// Attempt a full sync with a faulty peer
tester.newPeer("faulty", protocol, hashes, headers, blocks)
tester.newPeer("faulty", protocol, hashes, headers, blocks, receipts)
missing := targetBlocks / 2
delete(tester.peerHeaders["faulty"], hashes[missing])
delete(tester.peerBlocks["faulty"], hashes[missing])
delete(tester.peerReceipts["faulty"], hashes[missing])
pending := new(sync.WaitGroup)
pending.Add(1)
@ -1240,7 +1280,7 @@ func testFailedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
pending.Wait()
// Synchronise with a good peer and check that the boundary origin remind the same after a failure
tester.newPeer("valid", protocol, hashes, headers, blocks)
tester.newPeer("valid", protocol, hashes, headers, blocks, receipts)
pending.Add(1)
go func() {
@ -1270,7 +1310,7 @@ func TestFakedSyncBoundaries64Light(t *testing.T) { testFakedSyncBoundaries(t, 6
func testFakedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
// Create a small block chain
targetBlocks := blockCacheLimit - 15
hashes, headers, blocks := makeChain(targetBlocks+3, 0, genesis)
hashes, headers, blocks, receipts := makeChain(targetBlocks+3, 0, genesis, nil)
// Set a sync init hook to catch boundary changes
starting := make(chan struct{})
@ -1286,10 +1326,11 @@ func testFakedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
t.Fatalf("Pristine boundary mismatch: have %v/%v, want %v/%v", origin, latest, 0, 0)
}
// Create and sync with an attacker that promises a higher chain than available
tester.newPeer("attack", protocol, hashes, headers, blocks)
tester.newPeer("attack", protocol, hashes, headers, blocks, receipts)
for i := 1; i < 3; i++ {
delete(tester.peerHeaders["attack"], hashes[i])
delete(tester.peerBlocks["attack"], hashes[i])
delete(tester.peerReceipts["attack"], hashes[i])
}
pending := new(sync.WaitGroup)
@ -1309,7 +1350,7 @@ func testFakedSyncBoundaries(t *testing.T, protocol int, mode SyncMode) {
pending.Wait()
// Synchronise with a good peer and check that the boundary height has been reduced to the true value
tester.newPeer("valid", protocol, hashes[3:], headers, blocks)
tester.newPeer("valid", protocol, hashes[3:], headers, blocks, receipts)
pending.Add(1)
go func() {