aboutsummaryrefslogtreecommitdiffstats
path: root/eth/downloader/downloader_test.go
diff options
context:
space:
mode:
authorPéter Szilágyi <peterke@gmail.com>2015-06-12 18:35:29 +0800
committerPéter Szilágyi <peterke@gmail.com>2015-06-15 14:22:37 +0800
commitfc7abd98865f3bdc6cc36258026db98a649cd577 (patch)
tree0bca7e6e40445df3f21304e84b6df4dd5d3e7bf5 /eth/downloader/downloader_test.go
parent0fc71877a7d7a46f35147f753cba0de7b937c77a (diff)
downloaddexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar.gz
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar.bz2
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar.lz
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar.xz
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.tar.zst
dexon-fc7abd98865f3bdc6cc36258026db98a649cd577.zip
eth, eth/downloader: move block processing into the downlaoder
Diffstat (limited to 'eth/downloader/downloader_test.go')
-rw-r--r--eth/downloader/downloader_test.go252
1 files changed, 130 insertions, 122 deletions
diff --git a/eth/downloader/downloader_test.go b/eth/downloader/downloader_test.go
index 5b85f01fb..6cd141ef7 100644
--- a/eth/downloader/downloader_test.go
+++ b/eth/downloader/downloader_test.go
@@ -2,8 +2,10 @@ package downloader
import (
"encoding/binary"
+ "errors"
"fmt"
"math/big"
+ "sync/atomic"
"testing"
"time"
@@ -81,7 +83,7 @@ func newTester() *downloadTester {
peerBlocks: make(map[string]map[common.Hash]*types.Block),
}
var mux event.TypeMux
- downloader := New(&mux, tester.hasBlock, tester.getBlock, tester.dropPeer)
+ downloader := New(&mux, tester.hasBlock, tester.getBlock, tester.insertChain, tester.dropPeer)
tester.downloader = downloader
return tester
@@ -89,44 +91,14 @@ func newTester() *downloadTester {
// sync starts synchronizing with a remote peer, blocking until it completes.
func (dl *downloadTester) sync(id string) error {
- return dl.downloader.synchronise(id, dl.peerHashes[id][0])
-}
-
-// syncTake is starts synchronising with a remote peer, but concurrently it also
-// starts fetching blocks that the downloader retrieved. IT blocks until both go
-// routines terminate.
-func (dl *downloadTester) syncTake(id string) ([]*Block, error) {
- // Start a block collector to take blocks as they become available
- done := make(chan struct{})
- took := []*Block{}
- go func() {
- for running := true; running; {
- select {
- case <-done:
- running = false
- default:
- time.Sleep(time.Millisecond)
- }
- // Take a batch of blocks and accumulate
- blocks := dl.downloader.TakeBlocks()
- for _, block := range blocks {
- dl.ownHashes = append(dl.ownHashes, block.RawBlock.Hash())
- dl.ownBlocks[block.RawBlock.Hash()] = block.RawBlock
- }
- took = append(took, blocks...)
- }
- done <- struct{}{}
- }()
- // Start the downloading, sync the taker and return
- err := dl.sync(id)
-
- done <- struct{}{}
- <-done
-
- return took, err
+ err := dl.downloader.synchronise(id, dl.peerHashes[id][0])
+ for atomic.LoadInt32(&dl.downloader.processing) == 1 {
+ time.Sleep(time.Millisecond)
+ }
+ return err
}
-// hasBlock checks if a block is present in the testers canonical chain.
+// hasBlock checks if a block is pres ent in the testers canonical chain.
func (dl *downloadTester) hasBlock(hash common.Hash) bool {
return dl.getBlock(hash) != nil
}
@@ -136,6 +108,18 @@ func (dl *downloadTester) getBlock(hash common.Hash) *types.Block {
return dl.ownBlocks[hash]
}
+// insertChain injects a new batch of blocks into the simulated chain.
+func (dl *downloadTester) insertChain(blocks types.Blocks) (int, error) {
+ for i, block := range blocks {
+ if _, ok := dl.ownBlocks[block.ParentHash()]; !ok {
+ return i, errors.New("unknown parent")
+ }
+ dl.ownHashes = append(dl.ownHashes, block.Hash())
+ dl.ownBlocks[block.Hash()] = block
+ }
+ return len(blocks), nil
+}
+
// newPeer registers a new block download source into the downloader.
func (dl *downloadTester) newPeer(id string, hashes []common.Hash, blocks map[common.Hash]*types.Block) error {
err := dl.downloader.RegisterPeer(id, hashes[0], dl.peerGetHashesFn(id), dl.peerGetBlocksFn(id))
@@ -223,27 +207,8 @@ func TestSynchronisation(t *testing.T) {
if err := tester.sync("peer"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
- if queued := len(tester.downloader.queue.blockPool); queued != targetBlocks {
- t.Fatalf("synchronised block mismatch: have %v, want %v", queued, targetBlocks)
- }
-}
-
-// Tests that the synchronized blocks can be correctly retrieved.
-func TestBlockTaking(t *testing.T) {
- // Create a small enough block chain to download and the tester
- targetBlocks := blockCacheLimit - 15
- hashes := createHashes(targetBlocks, knownHash)
- blocks := createBlocksFromHashes(hashes)
-
- tester := newTester()
- tester.newPeer("peer", hashes, blocks)
-
- // Synchronise with the peer and test block retrieval
- if err := tester.sync("peer"); err != nil {
- t.Fatalf("failed to synchronise blocks: %v", err)
- }
- if took := tester.downloader.TakeBlocks(); len(took) != targetBlocks {
- t.Fatalf("took block mismatch: have %v, want %v", len(took), targetBlocks)
+ if imported := len(tester.ownBlocks); imported != targetBlocks+1 {
+ t.Fatalf("synchronised block mismatch: have %v, want %v", imported, targetBlocks+1)
}
}
@@ -270,21 +235,21 @@ func TestCancel(t *testing.T) {
tester := newTester()
tester.newPeer("peer", hashes, blocks)
+ // Make sure canceling works with a pristine downloader
+ tester.downloader.Cancel()
+ hashCount, blockCount := tester.downloader.queue.Size()
+ if hashCount > 0 || blockCount > 0 {
+ t.Errorf("block or hash count mismatch: %d hashes, %d blocks, want 0", hashCount, blockCount)
+ }
// Synchronise with the peer, but cancel afterwards
if err := tester.sync("peer"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
- if !tester.downloader.Cancel() {
- t.Fatalf("cancel operation failed")
- }
- // Make sure the queue reports empty and no blocks can be taken
- hashCount, blockCount := tester.downloader.queue.Size()
+ tester.downloader.Cancel()
+ hashCount, blockCount = tester.downloader.queue.Size()
if hashCount > 0 || blockCount > 0 {
t.Errorf("block or hash count mismatch: %d hashes, %d blocks, want 0", hashCount, blockCount)
}
- if took := tester.downloader.TakeBlocks(); len(took) != 0 {
- t.Errorf("taken blocks mismatch: have %d, want %d", len(took), 0)
- }
}
// Tests that if a large batch of blocks are being downloaded, it is throttled
@@ -298,29 +263,46 @@ func TestThrottling(t *testing.T) {
tester := newTester()
tester.newPeer("peer", hashes, blocks)
+ // Wrap the importer to allow stepping
+ done := make(chan int)
+ tester.downloader.insertChain = func(blocks types.Blocks) (int, error) {
+ n, err := tester.insertChain(blocks)
+ done <- n
+ return n, err
+ }
// Start a synchronisation concurrently
errc := make(chan error)
go func() {
errc <- tester.sync("peer")
}()
// Iteratively take some blocks, always checking the retrieval count
- for total := 0; total < targetBlocks; {
- // Wait a bit for sync to complete
+ for len(tester.ownBlocks) < targetBlocks+1 {
+ // Wait a bit for sync to throttle itself
+ var cached int
for start := time.Now(); time.Since(start) < 3*time.Second; {
time.Sleep(25 * time.Millisecond)
- if len(tester.downloader.queue.blockPool) == blockCacheLimit {
+
+ cached = len(tester.downloader.queue.blockPool)
+ if cached == blockCacheLimit || len(tester.ownBlocks)+cached == targetBlocks+1 {
break
}
}
- // Fetch the next batch of blocks
- took := tester.downloader.TakeBlocks()
- if len(took) != blockCacheLimit {
- t.Fatalf("block count mismatch: have %v, want %v", len(took), blockCacheLimit)
+ // Make sure we filled up the cache, then exhaust it
+ time.Sleep(25 * time.Millisecond) // give it a chance to screw up
+ if cached != blockCacheLimit && len(tester.ownBlocks)+cached < targetBlocks+1 {
+ t.Fatalf("block count mismatch: have %v, want %v", cached, blockCacheLimit)
}
- total += len(took)
- if total > targetBlocks {
- t.Fatalf("target block count mismatch: have %v, want %v", total, targetBlocks)
+ <-done // finish previous blocking import
+ for cached > maxBlockProcess {
+ cached -= <-done
}
+ time.Sleep(25 * time.Millisecond) // yield to the insertion
+ }
+ <-done // finish the last blocking import
+
+ // Check that we haven't pulled more blocks than available
+ if len(tester.ownBlocks) > targetBlocks+1 {
+ t.Fatalf("target block count mismatch: have %v, want %v", len(tester.ownBlocks), targetBlocks+1)
}
if err := <-errc; err != nil {
t.Fatalf("block synchronization failed: %v", err)
@@ -343,28 +325,18 @@ func TestNonExistingParentAttack(t *testing.T) {
tester.newPeer("attack", hashes, blocks)
// Try and sync with the malicious node and check that it fails
- if err := tester.sync("attack"); err != nil {
- t.Fatalf("failed to synchronise blocks: %v", err)
+ if err := tester.sync("attack"); err == nil {
+ t.Fatalf("block synchronization succeeded")
}
- bs := tester.downloader.TakeBlocks()
- if len(bs) != 1 {
- t.Fatalf("retrieved block mismatch: have %v, want %v", len(bs), 1)
+ if tester.hasBlock(hashes[0]) {
+ t.Fatalf("tester accepted unknown-parent block: %v", blocks[hashes[0]])
}
- if tester.hasBlock(bs[0].RawBlock.ParentHash()) {
- t.Fatalf("tester knows about the unknown hash")
- }
- tester.downloader.Cancel()
-
// Try to synchronize with the valid chain and make sure it succeeds
if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
- bs = tester.downloader.TakeBlocks()
- if len(bs) != 1 {
- t.Fatalf("retrieved block mismatch: have %v, want %v", len(bs), 1)
- }
- if !tester.hasBlock(bs[0].RawBlock.ParentHash()) {
- t.Fatalf("tester doesn't know about the origin hash")
+ if !tester.hasBlock(tester.peerHashes["valid"][0]) {
+ t.Fatalf("tester didn't accept known-parent block: %v", tester.peerBlocks["valid"][hashes[0]])
}
}
@@ -442,11 +414,11 @@ func TestInvalidHashOrderAttack(t *testing.T) {
tester.newPeer("attack", hashes, blocks)
// Try and sync with the malicious node and check that it fails
- if _, err := tester.syncTake("attack"); err != errInvalidChain {
+ if err := tester.sync("attack"); err != errInvalidChain {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errInvalidChain)
}
// Ensure that a valid chain can still pass sync
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
@@ -466,11 +438,11 @@ func TestMadeupHashChainAttack(t *testing.T) {
tester.newPeer("attack", createHashes(1024*blockCacheLimit, knownHash), nil)
// Try and sync with the malicious node and check that it fails
- if _, err := tester.syncTake("attack"); err != errCrossCheckFailed {
+ if err := tester.sync("attack"); err != errCrossCheckFailed {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errCrossCheckFailed)
}
// Ensure that a valid chain can still pass sync
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
@@ -487,7 +459,7 @@ func TestMadeupHashChainDrippingAttack(t *testing.T) {
// Try and sync with the attacker, one hash at a time
tester.maxHashFetch = 1
tester.newPeer("attack", hashes, nil)
- if _, err := tester.syncTake("attack"); err != errStallingPeer {
+ if err := tester.sync("attack"); err != errStallingPeer {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errStallingPeer)
}
}
@@ -512,7 +484,7 @@ func TestMadeupBlockChainAttack(t *testing.T) {
// Try and sync with the malicious node and check that it fails
tester := newTester()
tester.newPeer("attack", gapped, blocks)
- if _, err := tester.syncTake("attack"); err != errCrossCheckFailed {
+ if err := tester.sync("attack"); err != errCrossCheckFailed {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errCrossCheckFailed)
}
// Ensure that a valid chain can still pass sync
@@ -520,7 +492,7 @@ func TestMadeupBlockChainAttack(t *testing.T) {
crossCheckCycle = defaultCrossCheckCycle
tester.newPeer("valid", hashes, blocks)
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
@@ -548,14 +520,14 @@ func TestMadeupParentBlockChainAttack(t *testing.T) {
tester.newPeer("attack", hashes, blocks)
// Try and sync with the malicious node and check that it fails
- if _, err := tester.syncTake("attack"); err != errCrossCheckFailed {
+ if err := tester.sync("attack"); err != errCrossCheckFailed {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errCrossCheckFailed)
}
// Ensure that a valid chain can still pass sync
blockSoftTTL = defaultBlockTTL
crossCheckCycle = defaultCrossCheckCycle
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
@@ -582,7 +554,7 @@ func TestBannedChainStarvationAttack(t *testing.T) {
// the head of the invalid chain is blocked too.
for banned := tester.downloader.banned.Size(); ; {
// Try to sync with the attacker, check hash chain failure
- if _, err := tester.syncTake("attack"); err != errInvalidChain {
+ if err := tester.sync("attack"); err != errInvalidChain {
if tester.downloader.banned.Has(hashes[0]) && err == errBannedHead {
break
}
@@ -603,7 +575,7 @@ func TestBannedChainStarvationAttack(t *testing.T) {
t.Fatalf("banned attacker registered: %v", peer)
}
// Ensure that a valid chain can still pass sync
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
@@ -637,7 +609,7 @@ func TestBannedChainMemoryExhaustionAttack(t *testing.T) {
// the head of the invalid chain is blocked too.
for {
// Try to sync with the attacker, check hash chain failure
- if _, err := tester.syncTake("attack"); err != errInvalidChain {
+ if err := tester.sync("attack"); err != errInvalidChain {
t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errInvalidChain)
}
// Short circuit if the entire chain was banned
@@ -658,33 +630,34 @@ func TestBannedChainMemoryExhaustionAttack(t *testing.T) {
MaxBlockFetch = defaultMaxBlockFetch
maxBannedHashes = defaultMaxBannedHashes
- if _, err := tester.syncTake("valid"); err != nil {
+ if err := tester.sync("valid"); err != nil {
t.Fatalf("failed to synchronise blocks: %v", err)
}
}
// Tests that misbehaving peers are disconnected, whilst behaving ones are not.
-func TestAttackerDropping(t *testing.T) {
- // Define the disconnection requirement for individual errors
+func TestHashAttackerDropping(t *testing.T) {
+ // Define the disconnection requirement for individual hash fetch errors
tests := []struct {
result error
drop bool
}{
- {nil, false}, // Sync succeeded, all is well
- {errBusy, false}, // Sync is already in progress, no problem
- {errUnknownPeer, false}, // Peer is unknown, was already dropped, don't double drop
- {errBadPeer, true}, // Peer was deemed bad for some reason, drop it
- {errStallingPeer, true}, // Peer was detected to be stalling, drop it
- {errBannedHead, true}, // Peer's head hash is a known bad hash, drop it
- {errNoPeers, false}, // No peers to download from, soft race, no issue
- {errPendingQueue, false}, // There are blocks still cached, wait to exhaust, no issue
- {errTimeout, true}, // No hashes received in due time, drop the peer
- {errEmptyHashSet, true}, // No hashes were returned as a response, drop as it's a dead end
- {errPeersUnavailable, true}, // Nobody had the advertised blocks, drop the advertiser
- {errInvalidChain, true}, // Hash chain was detected as invalid, definitely drop
- {errCrossCheckFailed, true}, // Hash-origin failed to pass a block cross check, drop
- {errCancelHashFetch, false}, // Synchronisation was canceled, origin may be innocent, don't drop
- {errCancelBlockFetch, false}, // Synchronisation was canceled, origin may be innocent, don't drop
+ {nil, false}, // Sync succeeded, all is well
+ {errBusy, false}, // Sync is already in progress, no problem
+ {errUnknownPeer, false}, // Peer is unknown, was already dropped, don't double drop
+ {errBadPeer, true}, // Peer was deemed bad for some reason, drop it
+ {errStallingPeer, true}, // Peer was detected to be stalling, drop it
+ {errBannedHead, true}, // Peer's head hash is a known bad hash, drop it
+ {errNoPeers, false}, // No peers to download from, soft race, no issue
+ {errPendingQueue, false}, // There are blocks still cached, wait to exhaust, no issue
+ {errTimeout, true}, // No hashes received in due time, drop the peer
+ {errEmptyHashSet, true}, // No hashes were returned as a response, drop as it's a dead end
+ {errPeersUnavailable, true}, // Nobody had the advertised blocks, drop the advertiser
+ {errInvalidChain, true}, // Hash chain was detected as invalid, definitely drop
+ {errCrossCheckFailed, true}, // Hash-origin failed to pass a block cross check, drop
+ {errCancelHashFetch, false}, // Synchronisation was canceled, origin may be innocent, don't drop
+ {errCancelBlockFetch, false}, // Synchronisation was canceled, origin may be innocent, don't drop
+ {errCancelChainImport, false}, // Synchronisation was canceled, origin may be innocent, don't drop
}
// Run the tests and check disconnection status
tester := newTester()
@@ -706,3 +679,38 @@ func TestAttackerDropping(t *testing.T) {
}
}
}
+
+// Tests that feeding bad blocks will result in a peer drop.
+func TestBlockAttackerDropping(t *testing.T) {
+ // Define the disconnection requirement for individual block import errors
+ tests := []struct {
+ failure bool
+ drop bool
+ }{{true, true}, {false, false}}
+
+ // Run the tests and check disconnection status
+ tester := newTester()
+ 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, []common.Hash{common.Hash{}}, nil); err != nil {
+ t.Fatalf("test %d: failed to register new peer: %v", i, err)
+ }
+ if _, ok := tester.peerHashes[id]; !ok {
+ t.Fatalf("test %d: registered peer not found", i)
+ }
+ // Assemble a good or bad block, depending of the test
+ raw := createBlock(1, knownHash, common.Hash{})
+ if tt.failure {
+ raw = createBlock(1, unknownHash, common.Hash{})
+ }
+ block := &Block{OriginPeer: id, RawBlock: raw}
+
+ // Simulate block processing and check the result
+ tester.downloader.queue.blockCache[0] = block
+ tester.downloader.process()
+ if _, ok := tester.peerHashes[id]; !ok != tt.drop {
+ t.Errorf("test %d: peer drop mismatch for %v: have %v, want %v", i, tt.failure, !ok, tt.drop)
+ }
+ }
+}