Merge pull request #1701 from karalabe/eth62-sync-rebase
eth: implement eth/62 synchronization logic
This commit is contained in:
		
						commit
						6ec13e7e2b
					
				| @ -283,6 +283,7 @@ JavaScript API. See https://github.com/ethereum/go-ethereum/wiki/Javascipt-Conso | ||||
| 		utils.DataDirFlag, | ||||
| 		utils.BlockchainVersionFlag, | ||||
| 		utils.OlympicFlag, | ||||
| 		utils.EthVersionFlag, | ||||
| 		utils.CacheFlag, | ||||
| 		utils.JSpathFlag, | ||||
| 		utils.ListenPortFlag, | ||||
| @ -333,6 +334,7 @@ JavaScript API. See https://github.com/ethereum/go-ethereum/wiki/Javascipt-Conso | ||||
| 	app.Before = func(ctx *cli.Context) error { | ||||
| 		utils.SetupLogger(ctx) | ||||
| 		utils.SetupVM(ctx) | ||||
| 		utils.SetupEth(ctx) | ||||
| 		if ctx.GlobalBool(utils.PProfEanbledFlag.Name) { | ||||
| 			utils.StartPProf(ctx) | ||||
| 		} | ||||
|  | ||||
| @ -289,7 +289,7 @@ func updateChart(metric string, data []float64, base *int, chart *termui.LineCha | ||||
| 		} | ||||
| 	} | ||||
| 	unit, scale := 0, 1.0 | ||||
| 	for high >= 1000 { | ||||
| 	for high >= 1000 && unit+1 < len(dataUnits) { | ||||
| 		high, unit, scale = high/1000, unit+1, scale*1000 | ||||
| 	} | ||||
| 	// If the unit changes, re-create the chart (hack to set max height...)
 | ||||
|  | ||||
| @ -138,6 +138,11 @@ var ( | ||||
| 		Name:  "olympic", | ||||
| 		Usage: "Use olympic style protocol", | ||||
| 	} | ||||
| 	EthVersionFlag = cli.IntFlag{ | ||||
| 		Name:  "eth", | ||||
| 		Value: 62, | ||||
| 		Usage: "Highest eth protocol to advertise (temporary, dev option)", | ||||
| 	} | ||||
| 
 | ||||
| 	// miner settings
 | ||||
| 	MinerThreadsFlag = cli.IntFlag{ | ||||
| @ -459,6 +464,18 @@ func SetupVM(ctx *cli.Context) { | ||||
| 	vm.SetJITCacheSize(ctx.GlobalInt(VMJitCacheFlag.Name)) | ||||
| } | ||||
| 
 | ||||
| // SetupEth configures the eth packages global settings
 | ||||
| func SetupEth(ctx *cli.Context) { | ||||
| 	version := ctx.GlobalInt(EthVersionFlag.Name) | ||||
| 	for len(eth.ProtocolVersions) > 0 && eth.ProtocolVersions[0] > uint(version) { | ||||
| 		eth.ProtocolVersions = eth.ProtocolVersions[1:] | ||||
| 		eth.ProtocolLengths = eth.ProtocolLengths[1:] | ||||
| 	} | ||||
| 	if len(eth.ProtocolVersions) == 0 { | ||||
| 		Fatalf("No valid eth protocols remaining") | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // MakeChain creates a chain manager from set command line flags.
 | ||||
| func MakeChain(ctx *cli.Context) (chain *core.ChainManager, chainDb common.Database) { | ||||
| 	datadir := ctx.GlobalString(DataDirFlag.Name) | ||||
|  | ||||
| @ -360,6 +360,20 @@ func (b *Block) WithMiningResult(nonce uint64, mixDigest common.Hash) *Block { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // WithBody returns a new block with the given transaction and uncle contents.
 | ||||
| func (b *Block) WithBody(transactions []*Transaction, uncles []*Header) *Block { | ||||
| 	block := &Block{ | ||||
| 		header:       copyHeader(b.header), | ||||
| 		transactions: make([]*Transaction, len(transactions)), | ||||
| 		uncles:       make([]*Header, len(uncles)), | ||||
| 	} | ||||
| 	copy(block.transactions, transactions) | ||||
| 	for i := range uncles { | ||||
| 		block.uncles[i] = copyHeader(uncles[i]) | ||||
| 	} | ||||
| 	return block | ||||
| } | ||||
| 
 | ||||
| // Implement pow.Block
 | ||||
| 
 | ||||
| func (b *Block) Hash() common.Hash { | ||||
|  | ||||
| @ -373,7 +373,7 @@ func New(config *Config) (*Ethereum, error) { | ||||
| 
 | ||||
| 	eth.blockProcessor = core.NewBlockProcessor(chainDb, eth.pow, eth.chainManager, eth.EventMux()) | ||||
| 	eth.chainManager.SetProcessor(eth.blockProcessor) | ||||
| 	eth.protocolManager = NewProtocolManager(config.NetworkId, eth.eventMux, eth.txPool, eth.pow, eth.chainManager) | ||||
| 	eth.protocolManager = NewProtocolManager(config.NetworkId, eth.eventMux, eth.txPool, eth.pow, eth.chainManager, chainDb) | ||||
| 
 | ||||
| 	eth.miner = miner.New(eth, eth.EventMux(), eth.pow) | ||||
| 	eth.miner.SetGasPrice(config.GasPrice) | ||||
|  | ||||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							| @ -17,7 +17,6 @@ | ||||
| package downloader | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| 	"errors" | ||||
| 	"fmt" | ||||
| 	"math/big" | ||||
| @ -28,20 +27,39 @@ import ( | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/core" | ||||
| 	"github.com/ethereum/go-ethereum/core/types" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/ethdb" | ||||
| 	"github.com/ethereum/go-ethereum/event" | ||||
| 	"github.com/ethereum/go-ethereum/params" | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	testdb, _ = ethdb.NewMemDatabase() | ||||
| 	genesis   = core.GenesisBlockForTesting(testdb, common.Address{}, big.NewInt(0)) | ||||
| 	testdb, _   = ethdb.NewMemDatabase() | ||||
| 	testKey, _  = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") | ||||
| 	testAddress = crypto.PubkeyToAddress(testKey.PublicKey) | ||||
| 	genesis     = core.GenesisBlockForTesting(testdb, testAddress, big.NewInt(1000000000)) | ||||
| ) | ||||
| 
 | ||||
| // makeChain creates a chain of n blocks starting at but not including
 | ||||
| // parent. the returned hash chain is ordered head->parent.
 | ||||
| // makeChain creates a chain of n blocks starting at and including parent.
 | ||||
| // 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.Block) { | ||||
| 	blocks := core.GenerateChain(parent, testdb, n, func(i int, gen *core.BlockGen) { | ||||
| 		gen.SetCoinbase(common.Address{seed}) | ||||
| 	blocks := 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
 | ||||
| 		if parent == genesis && i%3 == 0 { | ||||
| 			tx, err := types.NewTransaction(block.TxNonce(testAddress), common.Address{seed}, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(testKey) | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 			block.AddTx(tx) | ||||
| 		} | ||||
| 		// If the block number is a multiple of 5, add a bonus uncle to the block
 | ||||
| 		if i%5 == 0 { | ||||
| 			block.AddUncle(&types.Header{ParentHash: block.PrevBlock(i - 1).Hash(), Number: big.NewInt(int64(i - 1))}) | ||||
| 		} | ||||
| 	}) | ||||
| 	hashes := make([]common.Hash, n+1) | ||||
| 	hashes[len(hashes)-1] = parent.Hash() | ||||
| @ -79,8 +97,6 @@ type downloadTester struct { | ||||
| 	ownBlocks  map[common.Hash]*types.Block            // Blocks belonging to the tester
 | ||||
| 	peerHashes map[string][]common.Hash                // Hash chain belonging to different test peers
 | ||||
| 	peerBlocks map[string]map[common.Hash]*types.Block // Blocks belonging to different test peers
 | ||||
| 
 | ||||
| 	maxHashFetch int // Overrides the maximum number of retrieved hashes
 | ||||
| } | ||||
| 
 | ||||
| // newTester creates a new downloader test mocker.
 | ||||
| @ -157,7 +173,9 @@ func (dl *downloadTester) newPeer(id string, version int, hashes []common.Hash, | ||||
| // 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, blocks map[common.Hash]*types.Block, delay time.Duration) error { | ||||
| 	err := dl.downloader.RegisterPeer(id, version, hashes[0], dl.peerGetRelHashesFn(id, delay), dl.peerGetAbsHashesFn(id, version, delay), dl.peerGetBlocksFn(id, delay)) | ||||
| 	err := dl.downloader.RegisterPeer(id, version, hashes[0], | ||||
| 		dl.peerGetRelHashesFn(id, delay), dl.peerGetAbsHashesFn(id, delay), dl.peerGetBlocksFn(id, delay), | ||||
| 		nil, dl.peerGetAbsHeadersFn(id, delay), dl.peerGetBodiesFn(id, delay)) | ||||
| 	if err == nil { | ||||
| 		// Assign the owned hashes and blocks to the peer (deep copy)
 | ||||
| 		dl.peerHashes[id] = make([]common.Hash, len(hashes)) | ||||
| @ -185,13 +203,9 @@ func (dl *downloadTester) peerGetRelHashesFn(id string, delay time.Duration) fun | ||||
| 	return func(head common.Hash) error { | ||||
| 		time.Sleep(delay) | ||||
| 
 | ||||
| 		limit := MaxHashFetch | ||||
| 		if dl.maxHashFetch > 0 { | ||||
| 			limit = dl.maxHashFetch | ||||
| 		} | ||||
| 		// Gather the next batch of hashes
 | ||||
| 		hashes := dl.peerHashes[id] | ||||
| 		result := make([]common.Hash, 0, limit) | ||||
| 		result := make([]common.Hash, 0, MaxHashFetch) | ||||
| 		for i, hash := range hashes { | ||||
| 			if hash == head { | ||||
| 				i++ | ||||
| @ -205,7 +219,7 @@ func (dl *downloadTester) peerGetRelHashesFn(id string, delay time.Duration) fun | ||||
| 		// Delay delivery a bit to allow attacks to unfold
 | ||||
| 		go func() { | ||||
| 			time.Sleep(time.Millisecond) | ||||
| 			dl.downloader.DeliverHashes(id, result) | ||||
| 			dl.downloader.DeliverHashes61(id, result) | ||||
| 		}() | ||||
| 		return nil | ||||
| 	} | ||||
| @ -214,29 +228,20 @@ func (dl *downloadTester) peerGetRelHashesFn(id string, delay time.Duration) fun | ||||
| // peerGetAbsHashesFn constructs a GetHashesFromNumber function associated with
 | ||||
| // a particular peer in the download tester. The returned function can be used to
 | ||||
| // retrieve batches of hashes from the particularly requested peer.
 | ||||
| func (dl *downloadTester) peerGetAbsHashesFn(id string, version int, delay time.Duration) func(uint64, int) error { | ||||
| 	// If the simulated peer runs eth/60, this message is not supported
 | ||||
| 	if version == eth60 { | ||||
| 		return func(uint64, int) error { return nil } | ||||
| 	} | ||||
| 	// Otherwise create a method to request the blocks by number
 | ||||
| func (dl *downloadTester) peerGetAbsHashesFn(id string, delay time.Duration) func(uint64, int) error { | ||||
| 	return func(head uint64, count int) error { | ||||
| 		time.Sleep(delay) | ||||
| 
 | ||||
| 		limit := count | ||||
| 		if dl.maxHashFetch > 0 { | ||||
| 			limit = dl.maxHashFetch | ||||
| 		} | ||||
| 		// Gather the next batch of hashes
 | ||||
| 		hashes := dl.peerHashes[id] | ||||
| 		result := make([]common.Hash, 0, limit) | ||||
| 		for i := 0; i < limit && len(hashes)-int(head)-1-i >= 0; i++ { | ||||
| 		result := make([]common.Hash, 0, count) | ||||
| 		for i := 0; i < count && len(hashes)-int(head)-1-i >= 0; i++ { | ||||
| 			result = append(result, hashes[len(hashes)-int(head)-1-i]) | ||||
| 		} | ||||
| 		// Delay delivery a bit to allow attacks to unfold
 | ||||
| 		go func() { | ||||
| 			time.Sleep(time.Millisecond) | ||||
| 			dl.downloader.DeliverHashes(id, result) | ||||
| 			dl.downloader.DeliverHashes61(id, result) | ||||
| 		}() | ||||
| 		return nil | ||||
| 	} | ||||
| @ -255,40 +260,75 @@ func (dl *downloadTester) peerGetBlocksFn(id string, delay time.Duration) func([ | ||||
| 				result = append(result, block) | ||||
| 			} | ||||
| 		} | ||||
| 		go dl.downloader.DeliverBlocks(id, result) | ||||
| 		go dl.downloader.DeliverBlocks61(id, result) | ||||
| 
 | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that simple synchronization, without throttling from a good peer works.
 | ||||
| func TestSynchronisation60(t *testing.T) { | ||||
| 	// Create a small enough block chain to download and the tester
 | ||||
| 	targetBlocks := blockCacheLimit - 15 | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| // peerGetAbsHeadersFn constructs a GetBlockHeaders function based on a numbered
 | ||||
| // origin; associated with a particular peer in the download tester. The returned
 | ||||
| // function can be used to retrieve batches of headers from the particular peer.
 | ||||
| func (dl *downloadTester) peerGetAbsHeadersFn(id string, delay time.Duration) func(uint64, int, int, bool) error { | ||||
| 	return func(origin uint64, amount int, skip int, reverse bool) error { | ||||
| 		time.Sleep(delay) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("peer", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	// Synchronise with the peer and make sure all blocks were retrieved
 | ||||
| 	if err := tester.sync("peer", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 		// Gather the next batch of hashes
 | ||||
| 		hashes := dl.peerHashes[id] | ||||
| 		blocks := dl.peerBlocks[id] | ||||
| 		result := make([]*types.Header, 0, amount) | ||||
| 		for i := 0; i < amount && len(hashes)-int(origin)-1-i >= 0; i++ { | ||||
| 			if block, ok := blocks[hashes[len(hashes)-int(origin)-1-i]]; ok { | ||||
| 				result = append(result, block.Header()) | ||||
| 			} | ||||
| 		} | ||||
| 		// Delay delivery a bit to allow attacks to unfold
 | ||||
| 		go func() { | ||||
| 			time.Sleep(time.Millisecond) | ||||
| 			dl.downloader.DeliverHeaders(id, result) | ||||
| 		}() | ||||
| 		return nil | ||||
| 	} | ||||
| 	if imported := len(tester.ownBlocks); imported != targetBlocks+1 { | ||||
| 		t.Fatalf("synchronised block mismatch: have %v, want %v", imported, targetBlocks+1) | ||||
| } | ||||
| 
 | ||||
| // peerGetBodiesFn constructs a getBlockBodies method associated with a particular
 | ||||
| // peer in the download tester. The returned function can be used to retrieve
 | ||||
| // batches of block bodies from the particularly requested peer.
 | ||||
| func (dl *downloadTester) peerGetBodiesFn(id string, delay time.Duration) func([]common.Hash) error { | ||||
| 	return func(hashes []common.Hash) error { | ||||
| 		time.Sleep(delay) | ||||
| 		blocks := dl.peerBlocks[id] | ||||
| 
 | ||||
| 		transactions := make([][]*types.Transaction, 0, len(hashes)) | ||||
| 		uncles := make([][]*types.Header, 0, len(hashes)) | ||||
| 
 | ||||
| 		for _, hash := range hashes { | ||||
| 			if block, ok := blocks[hash]; ok { | ||||
| 				transactions = append(transactions, block.Transactions()) | ||||
| 				uncles = append(uncles, block.Uncles()) | ||||
| 			} | ||||
| 		} | ||||
| 		go dl.downloader.DeliverBodies(id, transactions, uncles) | ||||
| 
 | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that simple synchronization against a canonical chain works correctly.
 | ||||
| // In this test common ancestor lookup should be short circuited and not require
 | ||||
| // binary searching.
 | ||||
| func TestCanonicalSynchronisation61(t *testing.T) { | ||||
| func TestCanonicalSynchronisation61(t *testing.T) { testCanonicalSynchronisation(t, 61) } | ||||
| func TestCanonicalSynchronisation62(t *testing.T) { testCanonicalSynchronisation(t, 62) } | ||||
| func TestCanonicalSynchronisation63(t *testing.T) { testCanonicalSynchronisation(t, 63) } | ||||
| func TestCanonicalSynchronisation64(t *testing.T) { testCanonicalSynchronisation(t, 64) } | ||||
| 
 | ||||
| func testCanonicalSynchronisation(t *testing.T, protocol int) { | ||||
| 	// Create a small enough block chain to download
 | ||||
| 	targetBlocks := blockCacheLimit - 15 | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("peer", eth61, hashes, blocks) | ||||
| 	tester.newPeer("peer", protocol, hashes, blocks) | ||||
| 
 | ||||
| 	// Synchronise with the peer and make sure all blocks were retrieved
 | ||||
| 	if err := tester.sync("peer", nil); err != nil { | ||||
| @ -301,8 +341,10 @@ func TestCanonicalSynchronisation61(t *testing.T) { | ||||
| 
 | ||||
| // Tests that if a large batch of blocks are being downloaded, it is throttled
 | ||||
| // until the cached blocks are retrieved.
 | ||||
| func TestThrottling60(t *testing.T) { testThrottling(t, eth60) } | ||||
| func TestThrottling61(t *testing.T) { testThrottling(t, eth61) } | ||||
| func TestThrottling61(t *testing.T) { testThrottling(t, 61) } | ||||
| func TestThrottling62(t *testing.T) { testThrottling(t, 62) } | ||||
| func TestThrottling63(t *testing.T) { testThrottling(t, 63) } | ||||
| func TestThrottling64(t *testing.T) { testThrottling(t, 64) } | ||||
| 
 | ||||
| func testThrottling(t *testing.T, protocol int) { | ||||
| 	// Create a long block chain to download and the tester
 | ||||
| @ -313,11 +355,10 @@ func testThrottling(t *testing.T, protocol int) { | ||||
| 	tester.newPeer("peer", protocol, 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 | ||||
| 	blocked, proceed := uint32(0), make(chan struct{}) | ||||
| 	tester.downloader.chainInsertHook = func(blocks []*Block) { | ||||
| 		atomic.StoreUint32(&blocked, uint32(len(blocks))) | ||||
| 		<-proceed | ||||
| 	} | ||||
| 	// Start a synchronisation concurrently
 | ||||
| 	errc := make(chan error) | ||||
| @ -328,27 +369,25 @@ func testThrottling(t *testing.T, protocol int) { | ||||
| 	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; { | ||||
| 		for start := time.Now(); time.Since(start) < time.Second; { | ||||
| 			time.Sleep(25 * time.Millisecond) | ||||
| 
 | ||||
| 			cached = len(tester.downloader.queue.blockPool) | ||||
| 			if cached == blockCacheLimit || len(tester.ownBlocks)+cached == targetBlocks+1 { | ||||
| 			if cached == blockCacheLimit || len(tester.ownBlocks)+cached+int(atomic.LoadUint32(&blocked)) == targetBlocks+1 { | ||||
| 				break | ||||
| 			} | ||||
| 		} | ||||
| 		// 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) | ||||
| 		if cached != blockCacheLimit && len(tester.ownBlocks)+cached+int(atomic.LoadUint32(&blocked)) != targetBlocks+1 { | ||||
| 			t.Fatalf("block count mismatch: have %v, want %v (owned %v, target %v)", cached, blockCacheLimit, len(tester.ownBlocks), targetBlocks+1) | ||||
| 		} | ||||
| 		<-done // finish previous blocking import
 | ||||
| 		for cached > maxBlockProcess { | ||||
| 			cached -= <-done | ||||
| 		// Permit the blocked blocks to import
 | ||||
| 		if atomic.LoadUint32(&blocked) > 0 { | ||||
| 			atomic.StoreUint32(&blocked, uint32(0)) | ||||
| 			proceed <- struct{}{} | ||||
| 		} | ||||
| 		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) | ||||
| @ -361,14 +400,19 @@ func testThrottling(t *testing.T, protocol int) { | ||||
| // Tests that simple synchronization against a forked chain works correctly. In
 | ||||
| // this test common ancestor lookup should *not* be short circuited, and a full
 | ||||
| // binary search should be executed.
 | ||||
| func TestForkedSynchronisation61(t *testing.T) { | ||||
| func TestForkedSynchronisation61(t *testing.T) { testForkedSynchronisation(t, 61) } | ||||
| func TestForkedSynchronisation62(t *testing.T) { testForkedSynchronisation(t, 62) } | ||||
| func TestForkedSynchronisation63(t *testing.T) { testForkedSynchronisation(t, 63) } | ||||
| func TestForkedSynchronisation64(t *testing.T) { testForkedSynchronisation(t, 64) } | ||||
| 
 | ||||
| func testForkedSynchronisation(t *testing.T, protocol int) { | ||||
| 	// Create a long enough forked chain
 | ||||
| 	common, fork := MaxHashFetch, 2*MaxHashFetch | ||||
| 	hashesA, hashesB, blocksA, blocksB := makeChainFork(common+fork, fork, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("fork A", eth61, hashesA, blocksA) | ||||
| 	tester.newPeer("fork B", eth61, hashesB, blocksB) | ||||
| 	tester.newPeer("fork A", protocol, hashesA, blocksA) | ||||
| 	tester.newPeer("fork B", protocol, hashesB, blocksB) | ||||
| 
 | ||||
| 	// Synchronise with the peer and make sure all blocks were retrieved
 | ||||
| 	if err := tester.sync("fork A", nil); err != nil { | ||||
| @ -387,21 +431,36 @@ func TestForkedSynchronisation61(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| // Tests that an inactive downloader will not accept incoming hashes and blocks.
 | ||||
| func TestInactiveDownloader(t *testing.T) { | ||||
| func TestInactiveDownloader61(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Check that neither hashes nor blocks are accepted
 | ||||
| 	if err := tester.downloader.DeliverHashes("bad peer", []common.Hash{}); err != errNoSyncActive { | ||||
| 	if err := tester.downloader.DeliverHashes61("bad peer", []common.Hash{}); err != errNoSyncActive { | ||||
| 		t.Errorf("error mismatch: have %v, want %v", err, errNoSyncActive) | ||||
| 	} | ||||
| 	if err := tester.downloader.DeliverBlocks("bad peer", []*types.Block{}); err != errNoSyncActive { | ||||
| 	if err := tester.downloader.DeliverBlocks61("bad peer", []*types.Block{}); err != errNoSyncActive { | ||||
| 		t.Errorf("error mismatch: have %v, want %v", err, errNoSyncActive) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that an inactive downloader will not accept incoming block headers and bodies.
 | ||||
| func TestInactiveDownloader62(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Check that neither block headers nor bodies are accepted
 | ||||
| 	if err := tester.downloader.DeliverHeaders("bad peer", []*types.Header{}); err != errNoSyncActive { | ||||
| 		t.Errorf("error mismatch: have %v, want %v", err, errNoSyncActive) | ||||
| 	} | ||||
| 	if err := tester.downloader.DeliverBodies("bad peer", [][]*types.Transaction{}, [][]*types.Header{}); err != errNoSyncActive { | ||||
| 		t.Errorf("error mismatch: have %v, want %v", err, errNoSyncActive) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that a canceled download wipes all previously accumulated state.
 | ||||
| func TestCancel60(t *testing.T) { testCancel(t, eth60) } | ||||
| func TestCancel61(t *testing.T) { testCancel(t, eth61) } | ||||
| func TestCancel61(t *testing.T) { testCancel(t, 61) } | ||||
| func TestCancel62(t *testing.T) { testCancel(t, 62) } | ||||
| func TestCancel63(t *testing.T) { testCancel(t, 63) } | ||||
| func TestCancel64(t *testing.T) { testCancel(t, 64) } | ||||
| 
 | ||||
| func testCancel(t *testing.T, protocol int) { | ||||
| 	// Create a small enough block chain to download and the tester
 | ||||
| @ -409,6 +468,9 @@ func testCancel(t *testing.T, protocol int) { | ||||
| 	if targetBlocks >= MaxHashFetch { | ||||
| 		targetBlocks = MaxHashFetch - 15 | ||||
| 	} | ||||
| 	if targetBlocks >= MaxHeaderFetch { | ||||
| 		targetBlocks = MaxHeaderFetch - 15 | ||||
| 	} | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| @ -416,28 +478,30 @@ func testCancel(t *testing.T, protocol int) { | ||||
| 
 | ||||
| 	// 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) | ||||
| 	downloading, importing := tester.downloader.queue.Size() | ||||
| 	if downloading > 0 || importing > 0 { | ||||
| 		t.Errorf("download or import count mismatch: %d downloading, %d importing, want 0", downloading, importing) | ||||
| 	} | ||||
| 	// Synchronise with the peer, but cancel afterwards
 | ||||
| 	if err := tester.sync("peer", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| 	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) | ||||
| 	downloading, importing = tester.downloader.queue.Size() | ||||
| 	if downloading > 0 || importing > 0 { | ||||
| 		t.Errorf("download or import count mismatch: %d downloading, %d importing, want 0", downloading, importing) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that synchronisation from multiple peers works as intended (multi thread sanity test).
 | ||||
| func TestMultiSynchronisation60(t *testing.T) { testMultiSynchronisation(t, eth60) } | ||||
| func TestMultiSynchronisation61(t *testing.T) { testMultiSynchronisation(t, eth61) } | ||||
| func TestMultiSynchronisation61(t *testing.T) { testMultiSynchronisation(t, 61) } | ||||
| func TestMultiSynchronisation62(t *testing.T) { testMultiSynchronisation(t, 62) } | ||||
| func TestMultiSynchronisation63(t *testing.T) { testMultiSynchronisation(t, 63) } | ||||
| func TestMultiSynchronisation64(t *testing.T) { testMultiSynchronisation(t, 64) } | ||||
| 
 | ||||
| func testMultiSynchronisation(t *testing.T, protocol int) { | ||||
| 	// Create various peers with various parts of the chain
 | ||||
| 	targetPeers := 16 | ||||
| 	targetPeers := 8 | ||||
| 	targetBlocks := targetPeers*blockCacheLimit - 15 | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| @ -463,394 +527,130 @@ func testMultiSynchronisation(t *testing.T, protocol int) { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that synchronising with a peer who's very slow at network IO does not
 | ||||
| // stall the other peers in the system.
 | ||||
| func TestSlowSynchronisation60(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| // Tests that if a block is empty (i.e. header only), no body request should be
 | ||||
| // made, and instead the header should be assembled into a whole block in itself.
 | ||||
| func TestEmptyBlockShortCircuit62(t *testing.T) { testEmptyBlockShortCircuit(t, 62) } | ||||
| func TestEmptyBlockShortCircuit63(t *testing.T) { testEmptyBlockShortCircuit(t, 63) } | ||||
| func TestEmptyBlockShortCircuit64(t *testing.T) { testEmptyBlockShortCircuit(t, 64) } | ||||
| 
 | ||||
| 	// Create a batch of blocks, with a slow and a full speed peer
 | ||||
| 	targetCycles := 2 | ||||
| 	targetBlocks := targetCycles*blockCacheLimit - 15 | ||||
| 	targetIODelay := time.Second | ||||
| func testEmptyBlockShortCircuit(t *testing.T, protocol int) { | ||||
| 	// Create a small enough block chain to download
 | ||||
| 	targetBlocks := blockCacheLimit - 15 | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	tester.newSlowPeer("fast", eth60, hashes, blocks, 0) | ||||
| 	tester.newSlowPeer("slow", eth60, hashes, blocks, targetIODelay) | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("peer", protocol, hashes, blocks) | ||||
| 
 | ||||
| 	// Try to sync with the peers (pull hashes from fast)
 | ||||
| 	start := time.Now() | ||||
| 	if err := tester.sync("fast", nil); err != nil { | ||||
| 	// Instrument the downloader to signal body requests
 | ||||
| 	requested := int32(0) | ||||
| 	tester.downloader.bodyFetchHook = func(headers []*types.Header) { | ||||
| 		atomic.AddInt32(&requested, int32(len(headers))) | ||||
| 	} | ||||
| 	// Synchronise with the peer and make sure all blocks were retrieved
 | ||||
| 	if err := tester.sync("peer", 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) | ||||
| 	} | ||||
| 	// Check that the slow peer got hit at most once per block-cache-size import
 | ||||
| 	limit := time.Duration(targetCycles+1) * targetIODelay | ||||
| 	if delay := time.Since(start); delay >= limit { | ||||
| 		t.Fatalf("synchronisation exceeded delay limit: have %v, want %v", delay, limit) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a peer returns an invalid chain with a block pointing to a non-
 | ||||
| // existing parent, it is correctly detected and handled.
 | ||||
| func TestNonExistingParentAttack60(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Forge a single-link chain with a forged header
 | ||||
| 	hashes, blocks := makeChain(1, 0, genesis) | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	wrongblock := types.NewBlock(&types.Header{}, nil, nil, nil) | ||||
| 	wrongblock.Td = blocks[hashes[0]].Td | ||||
| 	hashes, blocks = makeChain(1, 0, wrongblock) | ||||
| 	tester.newPeer("attack", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	// Try and sync with the malicious node and check that it fails
 | ||||
| 	if err := tester.sync("attack", nil); err == nil { | ||||
| 		t.Fatalf("block synchronization succeeded") | ||||
| 	} | ||||
| 	if tester.hasBlock(hashes[0]) { | ||||
| 		t.Fatalf("tester accepted unknown-parent block: %v", blocks[hashes[0]]) | ||||
| 	} | ||||
| 	// Try to synchronize with the valid chain and make sure it succeeds
 | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| 	if !tester.hasBlock(tester.peerHashes["valid"][0]) { | ||||
| 		t.Fatalf("tester didn't accept known-parent block: %v", tester.peerBlocks["valid"][hashes[0]]) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peers keeps sending us repeating hashes, we don't
 | ||||
| // loop indefinitely.
 | ||||
| func TestRepeatingHashAttack60(t *testing.T) { // TODO: Is this thing valid??
 | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Create a valid chain, but drop the last link
 | ||||
| 	hashes, blocks := makeChain(blockCacheLimit, 0, genesis) | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 	tester.newPeer("attack", eth60, hashes[:len(hashes)-1], blocks) | ||||
| 
 | ||||
| 	// Try and sync with the malicious node
 | ||||
| 	errc := make(chan error) | ||||
| 	go func() { | ||||
| 		errc <- tester.sync("attack", nil) | ||||
| 	}() | ||||
| 	// Make sure that syncing returns and does so with a failure
 | ||||
| 	select { | ||||
| 	case <-time.After(time.Second): | ||||
| 		t.Fatalf("synchronisation blocked") | ||||
| 	case err := <-errc: | ||||
| 		if err == nil { | ||||
| 			t.Fatalf("synchronisation succeeded") | ||||
| 	// Validate the number of block bodies that should have been requested
 | ||||
| 	needed := 0 | ||||
| 	for _, block := range blocks { | ||||
| 		if block != genesis && (len(block.Transactions()) > 0 || len(block.Uncles()) > 0) { | ||||
| 			needed++ | ||||
| 		} | ||||
| 	} | ||||
| 	// Ensure that a valid chain can still pass sync
 | ||||
| 	if int(requested) != needed { | ||||
| 		t.Fatalf("block body retrieval count mismatch: have %v, want %v", requested, needed) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a peer sends an invalid body for a requested block, it gets
 | ||||
| // dropped immediately by the downloader.
 | ||||
| func TestInvalidBlockBodyAttack62(t *testing.T) { testInvalidBlockBodyAttack(t, 62) } | ||||
| func TestInvalidBlockBodyAttack63(t *testing.T) { testInvalidBlockBodyAttack(t, 63) } | ||||
| func TestInvalidBlockBodyAttack64(t *testing.T) { testInvalidBlockBodyAttack(t, 64) } | ||||
| 
 | ||||
| func testInvalidBlockBodyAttack(t *testing.T, protocol int) { | ||||
| 	// Create two peers, one feeding invalid block bodies
 | ||||
| 	targetBlocks := 4*blockCacheLimit - 15 | ||||
| 	hashes, validBlocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	invalidBlocks := make(map[common.Hash]*types.Block) | ||||
| 	for hash, block := range validBlocks { | ||||
| 		invalidBlocks[hash] = types.NewBlockWithHeader(block.Header()) | ||||
| 	} | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("valid", protocol, hashes, validBlocks) | ||||
| 	tester.newPeer("attack", protocol, hashes, invalidBlocks) | ||||
| 
 | ||||
| 	// Synchronise with the valid peer (will pull contents from the attacker too)
 | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peers returns a non-existent block hash, it should
 | ||||
| // eventually time out and the sync reattempted.
 | ||||
| func TestNonExistingBlockAttack60(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Create a valid chain, but forge the last link
 | ||||
| 	hashes, blocks := makeChain(blockCacheLimit, 0, genesis) | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	hashes[len(hashes)/2] = common.Hash{} | ||||
| 	tester.newPeer("attack", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	// Try and sync with the malicious node and check that it fails
 | ||||
| 	if err := tester.sync("attack", nil); err != errPeersUnavailable { | ||||
| 		t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errPeersUnavailable) | ||||
| 	if imported := len(tester.ownBlocks); imported != len(hashes) { | ||||
| 		t.Fatalf("synchronised block mismatch: have %v, want %v", imported, len(hashes)) | ||||
| 	} | ||||
| 	// Ensure that a valid chain can still pass sync
 | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peer is returning hashes in a weird order, that the
 | ||||
| // sync throttler doesn't choke on them waiting for the valid blocks.
 | ||||
| func TestInvalidHashOrderAttack60(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Create a valid long chain, but reverse some hashes within
 | ||||
| 	hashes, blocks := makeChain(4*blockCacheLimit, 0, genesis) | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	chunk1 := make([]common.Hash, blockCacheLimit) | ||||
| 	chunk2 := make([]common.Hash, blockCacheLimit) | ||||
| 	copy(chunk1, hashes[blockCacheLimit:2*blockCacheLimit]) | ||||
| 	copy(chunk2, hashes[2*blockCacheLimit:3*blockCacheLimit]) | ||||
| 
 | ||||
| 	copy(hashes[2*blockCacheLimit:], chunk1) | ||||
| 	copy(hashes[blockCacheLimit:], chunk2) | ||||
| 	tester.newPeer("attack", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	// Try and sync with the malicious node and check that it fails
 | ||||
| 	if err := tester.sync("attack", nil); 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.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peer makes up a random hash chain and tries to push
 | ||||
| // indefinitely, it actually gets caught with it.
 | ||||
| func TestMadeupHashChainAttack60(t *testing.T) { | ||||
| 	tester := newTester() | ||||
| 	blockSoftTTL = 100 * time.Millisecond | ||||
| 	crossCheckCycle = 25 * time.Millisecond | ||||
| 
 | ||||
| 	// Create a long chain of hashes without backing blocks
 | ||||
| 	hashes, blocks := makeChain(4*blockCacheLimit, 0, genesis) | ||||
| 
 | ||||
| 	randomHashes := make([]common.Hash, 1024*blockCacheLimit) | ||||
| 	for i := range randomHashes { | ||||
| 		rand.Read(randomHashes[i][:]) | ||||
| 	} | ||||
| 
 | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 	tester.newPeer("attack", eth60, randomHashes, nil) | ||||
| 
 | ||||
| 	// Try and sync with the malicious node and check that it fails
 | ||||
| 	if err := tester.sync("attack", nil); 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.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peer makes up a random hash chain, and tries to push
 | ||||
| // indefinitely, one hash at a time, it actually gets caught with it. The reason
 | ||||
| // this is separate from the classical made up chain attack is that sending hashes
 | ||||
| // one by one prevents reliable block/parent verification.
 | ||||
| func TestMadeupHashChainDrippingAttack60(t *testing.T) { | ||||
| 	// Create a random chain of hashes to drip
 | ||||
| 	randomHashes := make([]common.Hash, 16*blockCacheLimit) | ||||
| 	for i := range randomHashes { | ||||
| 		rand.Read(randomHashes[i][:]) | ||||
| 	} | ||||
| 	randomHashes[len(randomHashes)-1] = genesis.Hash() | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| 	// Try and sync with the attacker, one hash at a time
 | ||||
| 	tester.maxHashFetch = 1 | ||||
| 	tester.newPeer("attack", eth60, randomHashes, nil) | ||||
| 	if err := tester.sync("attack", nil); err != errStallingPeer { | ||||
| 		t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errStallingPeer) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a malicious peer makes up a random block chain, and tried to
 | ||||
| // push indefinitely, it actually gets caught with it.
 | ||||
| func TestMadeupBlockChainAttack60(t *testing.T) { | ||||
| 	defaultBlockTTL := blockSoftTTL | ||||
| 	defaultCrossCheckCycle := crossCheckCycle | ||||
| 
 | ||||
| 	blockSoftTTL = 100 * time.Millisecond | ||||
| 	crossCheckCycle = 25 * time.Millisecond | ||||
| 
 | ||||
| 	// Create a long chain of blocks and simulate an invalid chain by dropping every second
 | ||||
| 	hashes, blocks := makeChain(16*blockCacheLimit, 0, genesis) | ||||
| 	gapped := make([]common.Hash, len(hashes)/2) | ||||
| 	for i := 0; i < len(gapped); i++ { | ||||
| 		gapped[i] = hashes[2*i] | ||||
| 	} | ||||
| 	// Try and sync with the malicious node and check that it fails
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("attack", eth60, gapped, blocks) | ||||
| 	if err := tester.sync("attack", nil); 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 | ||||
| 
 | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if one/multiple malicious peers try to feed a banned blockchain to
 | ||||
| // the downloader, it will not keep refetching the same chain indefinitely, but
 | ||||
| // gradually block pieces of it, until its head is also blocked.
 | ||||
| func TestBannedChainStarvationAttack60(t *testing.T) { | ||||
| 	n := 8 * blockCacheLimit | ||||
| 	fork := n/2 - 23 | ||||
| 	hashes, forkHashes, blocks, forkBlocks := makeChainFork(n, fork, genesis) | ||||
| 
 | ||||
| 	// Create the tester and ban the selected hash.
 | ||||
| 	tester := newTester() | ||||
| 	tester.downloader.banned.Add(forkHashes[fork-1]) | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 	tester.newPeer("attack", eth60, forkHashes, forkBlocks) | ||||
| 
 | ||||
| 	// Iteratively try to sync, and verify that the banned hash list grows until
 | ||||
| 	// 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.sync("attack", nil); err != errInvalidChain { | ||||
| 			if tester.downloader.banned.Has(forkHashes[0]) && err == errBannedHead { | ||||
| 				break | ||||
| 			} | ||||
| 			t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errInvalidChain) | ||||
| 		} | ||||
| 		// Check that the ban list grew with at least 1 new item, or all banned
 | ||||
| 		bans := tester.downloader.banned.Size() | ||||
| 		if bans < banned+1 { | ||||
| 			t.Fatalf("ban count mismatch: have %v, want %v+", bans, banned+1) | ||||
| 		} | ||||
| 		banned = bans | ||||
| 	} | ||||
| 	// Check that after banning an entire chain, bad peers get dropped
 | ||||
| 	if err := tester.newPeer("new attacker", eth60, forkHashes, forkBlocks); err != errBannedHead { | ||||
| 		t.Fatalf("peer registration mismatch: have %v, want %v", err, errBannedHead) | ||||
| 	} | ||||
| 	if peer := tester.downloader.peers.Peer("new attacker"); peer != nil { | ||||
| 		t.Fatalf("banned attacker registered: %v", peer) | ||||
| 	} | ||||
| 	// Ensure that a valid chain can still pass sync
 | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that if a peer sends excessively many/large invalid chains that are
 | ||||
| // gradually banned, it will have an upper limit on the consumed memory and also
 | ||||
| // the origin bad hashes will not be evacuated.
 | ||||
| func TestBannedChainMemoryExhaustionAttack60(t *testing.T) { | ||||
| 	// Construct a banned chain with more chunks than the ban limit
 | ||||
| 	n := 8 * blockCacheLimit | ||||
| 	fork := n/2 - 23 | ||||
| 	hashes, forkHashes, blocks, forkBlocks := makeChainFork(n, fork, genesis) | ||||
| 
 | ||||
| 	// Create the tester and ban the root hash of the fork.
 | ||||
| 	tester := newTester() | ||||
| 	tester.downloader.banned.Add(forkHashes[fork-1]) | ||||
| 
 | ||||
| 	// Reduce the test size a bit
 | ||||
| 	defaultMaxBlockFetch := MaxBlockFetch | ||||
| 	defaultMaxBannedHashes := maxBannedHashes | ||||
| 
 | ||||
| 	MaxBlockFetch = 4 | ||||
| 	maxBannedHashes = 256 | ||||
| 
 | ||||
| 	tester.newPeer("valid", eth60, hashes, blocks) | ||||
| 	tester.newPeer("attack", eth60, forkHashes, forkBlocks) | ||||
| 
 | ||||
| 	// Iteratively try to sync, and verify that the banned hash list grows until
 | ||||
| 	// the head of the invalid chain is blocked too.
 | ||||
| 	for { | ||||
| 		// Try to sync with the attacker, check hash chain failure
 | ||||
| 		if err := tester.sync("attack", nil); err != errInvalidChain { | ||||
| 			t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errInvalidChain) | ||||
| 		} | ||||
| 		// Short circuit if the entire chain was banned.
 | ||||
| 		if tester.downloader.banned.Has(forkHashes[0]) { | ||||
| 			break | ||||
| 		} | ||||
| 		// Otherwise ensure we never exceed the memory allowance and the hard coded bans are untouched
 | ||||
| 		if bans := tester.downloader.banned.Size(); bans > maxBannedHashes { | ||||
| 			t.Fatalf("ban cap exceeded: have %v, want max %v", bans, maxBannedHashes) | ||||
| 		} | ||||
| 		for hash := range core.BadHashes { | ||||
| 			if !tester.downloader.banned.Has(hash) { | ||||
| 				t.Fatalf("hard coded ban evacuated: %x", hash) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| 	// Ensure that a valid chain can still pass sync
 | ||||
| 	MaxBlockFetch = defaultMaxBlockFetch | ||||
| 	maxBannedHashes = defaultMaxBannedHashes | ||||
| 
 | ||||
| 	if err := tester.sync("valid", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests a corner case (potential attack) where a peer delivers both good as well
 | ||||
| // as unrequested blocks to a hash request. This may trigger a different code
 | ||||
| // path than the fully correct or fully invalid delivery, potentially causing
 | ||||
| // internal state problems
 | ||||
| //
 | ||||
| // No, don't delete this test, it actually did happen!
 | ||||
| func TestOverlappingDeliveryAttack60(t *testing.T) { | ||||
| 	// Create an arbitrary batch of blocks ( < cache-size not to block)
 | ||||
| 	targetBlocks := blockCacheLimit - 23 | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	// Register an attacker that always returns non-requested blocks too
 | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("attack", eth60, hashes, blocks) | ||||
| 
 | ||||
| 	rawGetBlocks := tester.downloader.peers.Peer("attack").getBlocks | ||||
| 	tester.downloader.peers.Peer("attack").getBlocks = func(request []common.Hash) error { | ||||
| 		// Add a non requested hash the screw the delivery (genesis should be fine)
 | ||||
| 		return rawGetBlocks(append(request, hashes[0])) | ||||
| 	} | ||||
| 	// Test that synchronisation can complete, check for import success
 | ||||
| 	if err := tester.sync("attack", nil); err != nil { | ||||
| 		t.Fatalf("failed to synchronise blocks: %v", err) | ||||
| 	} | ||||
| 	start := time.Now() | ||||
| 	for len(tester.ownHashes) != len(hashes) && time.Since(start) < time.Second { | ||||
| 		time.Sleep(50 * time.Millisecond) | ||||
| 	} | ||||
| 	if len(tester.ownHashes) != len(hashes) { | ||||
| 		t.Fatalf("chain length mismatch: have %v, want %v", len(tester.ownHashes), len(hashes)) | ||||
| 	// Make sure the attacker was detected and dropped in the mean time
 | ||||
| 	if _, ok := tester.peerHashes["attack"]; ok { | ||||
| 		t.Fatalf("block body attacker not detected/dropped") | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that a peer advertising an high TD doesn't get to stall the downloader
 | ||||
| // afterwards by not sending any useful hashes.
 | ||||
| func TestHighTDStarvationAttack61(t *testing.T) { | ||||
| func TestHighTDStarvationAttack61(t *testing.T) { testHighTDStarvationAttack(t, 61) } | ||||
| func TestHighTDStarvationAttack62(t *testing.T) { testHighTDStarvationAttack(t, 62) } | ||||
| func TestHighTDStarvationAttack63(t *testing.T) { testHighTDStarvationAttack(t, 63) } | ||||
| func TestHighTDStarvationAttack64(t *testing.T) { testHighTDStarvationAttack(t, 64) } | ||||
| 
 | ||||
| func testHighTDStarvationAttack(t *testing.T, protocol int) { | ||||
| 	tester := newTester() | ||||
| 	tester.newPeer("attack", eth61, []common.Hash{genesis.Hash()}, nil) | ||||
| 	hashes, blocks := makeChain(0, 0, genesis) | ||||
| 
 | ||||
| 	tester.newPeer("attack", protocol, []common.Hash{hashes[0]}, blocks) | ||||
| 	if err := tester.sync("attack", big.NewInt(1000000)); err != errStallingPeer { | ||||
| 		t.Fatalf("synchronisation error mismatch: have %v, want %v", err, errStallingPeer) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that misbehaving peers are disconnected, whilst behaving ones are not.
 | ||||
| func TestHashAttackerDropping(t *testing.T) { | ||||
| func TestBlockHeaderAttackerDropping61(t *testing.T) { testBlockHeaderAttackerDropping(t, 61) } | ||||
| func TestBlockHeaderAttackerDropping62(t *testing.T) { testBlockHeaderAttackerDropping(t, 62) } | ||||
| func TestBlockHeaderAttackerDropping63(t *testing.T) { testBlockHeaderAttackerDropping(t, 63) } | ||||
| func TestBlockHeaderAttackerDropping64(t *testing.T) { testBlockHeaderAttackerDropping(t, 64) } | ||||
| 
 | ||||
| func testBlockHeaderAttackerDropping(t *testing.T, protocol int) { | ||||
| 	// 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
 | ||||
| 		{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
 | ||||
| 		{errEmptyHeaderSet, true},     // No headers 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
 | ||||
| 		{errInvalidBody, false},       // A bad peer was detected, but not the sync origin
 | ||||
| 		{errCancelHashFetch, false},   // Synchronisation was canceled, origin may be innocent, don't drop
 | ||||
| 		{errCancelBlockFetch, false},  // Synchronisation was canceled, origin may be innocent, don't drop
 | ||||
| 		{errCancelHeaderFetch, false}, // Synchronisation was canceled, origin may be innocent, don't drop
 | ||||
| 		{errCancelBodyFetch, false},   // Synchronisation was canceled, origin may be innocent, don't drop
 | ||||
| 	} | ||||
| 	// 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, eth60, []common.Hash{genesis.Hash()}, nil); err != nil { | ||||
| 		if err := tester.newPeer(id, protocol, []common.Hash{genesis.Hash()}, nil); err != nil { | ||||
| 			t.Fatalf("test %d: failed to register new peer: %v", i, err) | ||||
| 		} | ||||
| 		if _, ok := tester.peerHashes[id]; !ok { | ||||
| @ -867,7 +667,12 @@ func TestHashAttackerDropping(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| // Tests that feeding bad blocks will result in a peer drop.
 | ||||
| func TestBlockAttackerDropping(t *testing.T) { | ||||
| func TestBlockBodyAttackerDropping61(t *testing.T) { testBlockBodyAttackerDropping(t, 61) } | ||||
| func TestBlockBodyAttackerDropping62(t *testing.T) { testBlockBodyAttackerDropping(t, 62) } | ||||
| func TestBlockBodyAttackerDropping63(t *testing.T) { testBlockBodyAttackerDropping(t, 63) } | ||||
| func TestBlockBodyAttackerDropping64(t *testing.T) { testBlockBodyAttackerDropping(t, 64) } | ||||
| 
 | ||||
| func testBlockBodyAttackerDropping(t *testing.T, protocol int) { | ||||
| 	// Define the disconnection requirement for individual block import errors
 | ||||
| 	tests := []struct { | ||||
| 		failure bool | ||||
| @ -882,7 +687,7 @@ func TestBlockAttackerDropping(t *testing.T) { | ||||
| 	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, eth60, []common.Hash{common.Hash{}}, nil); err != nil { | ||||
| 		if err := tester.newPeer(id, protocol, []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 { | ||||
|  | ||||
							
								
								
									
										45
									
								
								eth/downloader/metrics.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										45
									
								
								eth/downloader/metrics.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,45 @@ | ||||
| // Copyright 2015 The go-ethereum Authors
 | ||||
| // This file is part of the go-ethereum library.
 | ||||
| //
 | ||||
| // The go-ethereum library is free software: you can redistribute it and/or modify
 | ||||
| // it under the terms of the GNU Lesser General Public License as published by
 | ||||
| // the Free Software Foundation, either version 3 of the License, or
 | ||||
| // (at your option) any later version.
 | ||||
| //
 | ||||
| // The go-ethereum library is distributed in the hope that it will be useful,
 | ||||
| // but WITHOUT ANY WARRANTY; without even the implied warranty of
 | ||||
| // MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
 | ||||
| // GNU Lesser General Public License for more details.
 | ||||
| //
 | ||||
| // You should have received a copy of the GNU Lesser General Public License
 | ||||
| // along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
 | ||||
| 
 | ||||
| // Contains the metrics collected by the downloader.
 | ||||
| 
 | ||||
| package downloader | ||||
| 
 | ||||
| import ( | ||||
| 	"github.com/ethereum/go-ethereum/metrics" | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	hashInMeter      = metrics.NewMeter("eth/downloader/hashes/in") | ||||
| 	hashReqTimer     = metrics.NewTimer("eth/downloader/hashes/req") | ||||
| 	hashDropMeter    = metrics.NewMeter("eth/downloader/hashes/drop") | ||||
| 	hashTimeoutMeter = metrics.NewMeter("eth/downloader/hashes/timeout") | ||||
| 
 | ||||
| 	blockInMeter      = metrics.NewMeter("eth/downloader/blocks/in") | ||||
| 	blockReqTimer     = metrics.NewTimer("eth/downloader/blocks/req") | ||||
| 	blockDropMeter    = metrics.NewMeter("eth/downloader/blocks/drop") | ||||
| 	blockTimeoutMeter = metrics.NewMeter("eth/downloader/blocks/timeout") | ||||
| 
 | ||||
| 	headerInMeter      = metrics.NewMeter("eth/downloader/headers/in") | ||||
| 	headerReqTimer     = metrics.NewTimer("eth/downloader/headers/req") | ||||
| 	headerDropMeter    = metrics.NewMeter("eth/downloader/headers/drop") | ||||
| 	headerTimeoutMeter = metrics.NewMeter("eth/downloader/headers/timeout") | ||||
| 
 | ||||
| 	bodyInMeter      = metrics.NewMeter("eth/downloader/bodies/in") | ||||
| 	bodyReqTimer     = metrics.NewTimer("eth/downloader/bodies/req") | ||||
| 	bodyDropMeter    = metrics.NewMeter("eth/downloader/bodies/drop") | ||||
| 	bodyTimeoutMeter = metrics.NewMeter("eth/downloader/bodies/timeout") | ||||
| ) | ||||
| @ -31,10 +31,16 @@ import ( | ||||
| 	"gopkg.in/fatih/set.v0" | ||||
| ) | ||||
| 
 | ||||
| // Hash and block fetchers belonging to eth/61 and below
 | ||||
| type relativeHashFetcherFn func(common.Hash) error | ||||
| type absoluteHashFetcherFn func(uint64, int) error | ||||
| type blockFetcherFn func([]common.Hash) error | ||||
| 
 | ||||
| // Block header and body fethers belonging to eth/62 and above
 | ||||
| type relativeHeaderFetcherFn func(common.Hash, int, int, bool) error | ||||
| type absoluteHeaderFetcherFn func(uint64, int, int, bool) error | ||||
| type blockBodyFetcherFn func([]common.Hash) error | ||||
| 
 | ||||
| var ( | ||||
| 	errAlreadyFetching   = errors.New("already fetching blocks from peer") | ||||
| 	errAlreadyRegistered = errors.New("peer is already registered") | ||||
| @ -54,25 +60,37 @@ type peer struct { | ||||
| 
 | ||||
| 	ignored *set.Set // Set of hashes not to request (didn't have previously)
 | ||||
| 
 | ||||
| 	getRelHashes relativeHashFetcherFn // Method to retrieve a batch of hashes from an origin hash
 | ||||
| 	getAbsHashes absoluteHashFetcherFn // Method to retrieve a batch of hashes from an absolute position
 | ||||
| 	getBlocks    blockFetcherFn        // Method to retrieve a batch of blocks
 | ||||
| 	getRelHashes relativeHashFetcherFn // [eth/61] Method to retrieve a batch of hashes from an origin hash
 | ||||
| 	getAbsHashes absoluteHashFetcherFn // [eth/61] Method to retrieve a batch of hashes from an absolute position
 | ||||
| 	getBlocks    blockFetcherFn        // [eth/61] Method to retrieve a batch of blocks
 | ||||
| 
 | ||||
| 	getRelHeaders  relativeHeaderFetcherFn // [eth/62] Method to retrieve a batch of headers from an origin hash
 | ||||
| 	getAbsHeaders  absoluteHeaderFetcherFn // [eth/62] Method to retrieve a batch of headers from an absolute position
 | ||||
| 	getBlockBodies blockBodyFetcherFn      // [eth/62] Method to retrieve a batch of block bodies
 | ||||
| 
 | ||||
| 	version int // Eth protocol version number to switch strategies
 | ||||
| } | ||||
| 
 | ||||
| // newPeer create a new downloader peer, with specific hash and block retrieval
 | ||||
| // mechanisms.
 | ||||
| func newPeer(id string, version int, head common.Hash, getRelHashes relativeHashFetcherFn, getAbsHashes absoluteHashFetcherFn, getBlocks blockFetcherFn) *peer { | ||||
| func newPeer(id string, version int, head common.Hash, | ||||
| 	getRelHashes relativeHashFetcherFn, getAbsHashes absoluteHashFetcherFn, getBlocks blockFetcherFn, // eth/61 callbacks, remove when upgrading
 | ||||
| 	getRelHeaders relativeHeaderFetcherFn, getAbsHeaders absoluteHeaderFetcherFn, getBlockBodies blockBodyFetcherFn) *peer { | ||||
| 	return &peer{ | ||||
| 		id:           id, | ||||
| 		head:         head, | ||||
| 		capacity:     1, | ||||
| 		id:       id, | ||||
| 		head:     head, | ||||
| 		capacity: 1, | ||||
| 		ignored:  set.New(), | ||||
| 
 | ||||
| 		getRelHashes: getRelHashes, | ||||
| 		getAbsHashes: getAbsHashes, | ||||
| 		getBlocks:    getBlocks, | ||||
| 		ignored:      set.New(), | ||||
| 		version:      version, | ||||
| 
 | ||||
| 		getRelHeaders:  getRelHeaders, | ||||
| 		getAbsHeaders:  getAbsHeaders, | ||||
| 		getBlockBodies: getBlockBodies, | ||||
| 
 | ||||
| 		version: version, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| @ -83,8 +101,8 @@ func (p *peer) Reset() { | ||||
| 	p.ignored.Clear() | ||||
| } | ||||
| 
 | ||||
| // Fetch sends a block retrieval request to the remote peer.
 | ||||
| func (p *peer) Fetch(request *fetchRequest) error { | ||||
| // Fetch61 sends a block retrieval request to the remote peer.
 | ||||
| func (p *peer) Fetch61(request *fetchRequest) error { | ||||
| 	// Short circuit if the peer is already fetching
 | ||||
| 	if !atomic.CompareAndSwapInt32(&p.idle, 0, 1) { | ||||
| 		return errAlreadyFetching | ||||
| @ -101,10 +119,28 @@ func (p *peer) Fetch(request *fetchRequest) error { | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // SetIdle sets the peer to idle, allowing it to execute new retrieval requests.
 | ||||
| // Fetch sends a block body retrieval request to the remote peer.
 | ||||
| func (p *peer) Fetch(request *fetchRequest) error { | ||||
| 	// Short circuit if the peer is already fetching
 | ||||
| 	if !atomic.CompareAndSwapInt32(&p.idle, 0, 1) { | ||||
| 		return errAlreadyFetching | ||||
| 	} | ||||
| 	p.started = time.Now() | ||||
| 
 | ||||
| 	// Convert the header set to a retrievable slice
 | ||||
| 	hashes := make([]common.Hash, 0, len(request.Headers)) | ||||
| 	for _, header := range request.Headers { | ||||
| 		hashes = append(hashes, header.Hash()) | ||||
| 	} | ||||
| 	go p.getBlockBodies(hashes) | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // SetIdle61 sets the peer to idle, allowing it to execute new retrieval requests.
 | ||||
| // Its block retrieval allowance will also be updated either up- or downwards,
 | ||||
| // depending on whether the previous fetch completed in time or not.
 | ||||
| func (p *peer) SetIdle() { | ||||
| func (p *peer) SetIdle61() { | ||||
| 	// Update the peer's download allowance based on previous performance
 | ||||
| 	scale := 2.0 | ||||
| 	if time.Since(p.started) > blockSoftTTL { | ||||
| @ -131,6 +167,36 @@ func (p *peer) SetIdle() { | ||||
| 	atomic.StoreInt32(&p.idle, 0) | ||||
| } | ||||
| 
 | ||||
| // SetIdle sets the peer to idle, allowing it to execute new retrieval requests.
 | ||||
| // Its block body retrieval allowance will also be updated either up- or downwards,
 | ||||
| // depending on whether the previous fetch completed in time or not.
 | ||||
| func (p *peer) SetIdle() { | ||||
| 	// Update the peer's download allowance based on previous performance
 | ||||
| 	scale := 2.0 | ||||
| 	if time.Since(p.started) > bodySoftTTL { | ||||
| 		scale = 0.5 | ||||
| 		if time.Since(p.started) > bodyHardTTL { | ||||
| 			scale = 1 / float64(MaxBodyFetch) // reduces capacity to 1
 | ||||
| 		} | ||||
| 	} | ||||
| 	for { | ||||
| 		// Calculate the new download bandwidth allowance
 | ||||
| 		prev := atomic.LoadInt32(&p.capacity) | ||||
| 		next := int32(math.Max(1, math.Min(float64(MaxBodyFetch), float64(prev)*scale))) | ||||
| 
 | ||||
| 		// Try to update the old value
 | ||||
| 		if atomic.CompareAndSwapInt32(&p.capacity, prev, next) { | ||||
| 			// If we're having problems at 1 capacity, try to find better peers
 | ||||
| 			if next == 1 { | ||||
| 				p.Demote() | ||||
| 			} | ||||
| 			break | ||||
| 		} | ||||
| 	} | ||||
| 	// Set the peer to idle to allow further block requests
 | ||||
| 	atomic.StoreInt32(&p.idle, 0) | ||||
| } | ||||
| 
 | ||||
| // Capacity retrieves the peers block download allowance based on its previously
 | ||||
| // discovered bandwidth capacity.
 | ||||
| func (p *peer) Capacity() int { | ||||
|  | ||||
| @ -43,16 +43,20 @@ var ( | ||||
| 
 | ||||
| // fetchRequest is a currently running block retrieval operation.
 | ||||
| type fetchRequest struct { | ||||
| 	Peer   *peer               // Peer to which the request was sent
 | ||||
| 	Hashes map[common.Hash]int // Requested hashes with their insertion index (priority)
 | ||||
| 	Time   time.Time           // Time when the request was made
 | ||||
| 	Peer    *peer               // Peer to which the request was sent
 | ||||
| 	Hashes  map[common.Hash]int // [eth/61] Requested hashes with their insertion index (priority)
 | ||||
| 	Headers []*types.Header     // [eth/62] Requested headers, sorted by request order
 | ||||
| 	Time    time.Time           // Time when the request was made
 | ||||
| } | ||||
| 
 | ||||
| // queue represents hashes that are either need fetching or are being fetched
 | ||||
| type queue struct { | ||||
| 	hashPool    map[common.Hash]int // Pending hashes, mapping to their insertion index (priority)
 | ||||
| 	hashQueue   *prque.Prque        // Priority queue of the block hashes to fetch
 | ||||
| 	hashCounter int                 // Counter indexing the added hashes to ensure retrieval order
 | ||||
| 	hashPool    map[common.Hash]int // [eth/61] Pending hashes, mapping to their insertion index (priority)
 | ||||
| 	hashQueue   *prque.Prque        // [eth/61] Priority queue of the block hashes to fetch
 | ||||
| 	hashCounter int                 // [eth/61] Counter indexing the added hashes to ensure retrieval order
 | ||||
| 
 | ||||
| 	headerPool  map[common.Hash]*types.Header // [eth/62] Pending headers, mapping from their hashes
 | ||||
| 	headerQueue *prque.Prque                  // [eth/62] Priority queue of the headers to fetch the bodies for
 | ||||
| 
 | ||||
| 	pendPool map[string]*fetchRequest // Currently pending block retrieval operations
 | ||||
| 
 | ||||
| @ -66,11 +70,13 @@ type queue struct { | ||||
| // newQueue creates a new download queue for scheduling block retrieval.
 | ||||
| func newQueue() *queue { | ||||
| 	return &queue{ | ||||
| 		hashPool:   make(map[common.Hash]int), | ||||
| 		hashQueue:  prque.New(), | ||||
| 		pendPool:   make(map[string]*fetchRequest), | ||||
| 		blockPool:  make(map[common.Hash]uint64), | ||||
| 		blockCache: make([]*Block, blockCacheLimit), | ||||
| 		hashPool:    make(map[common.Hash]int), | ||||
| 		hashQueue:   prque.New(), | ||||
| 		headerPool:  make(map[common.Hash]*types.Header), | ||||
| 		headerQueue: prque.New(), | ||||
| 		pendPool:    make(map[string]*fetchRequest), | ||||
| 		blockPool:   make(map[common.Hash]uint64), | ||||
| 		blockCache:  make([]*Block, blockCacheLimit), | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| @ -83,6 +89,9 @@ func (q *queue) Reset() { | ||||
| 	q.hashQueue.Reset() | ||||
| 	q.hashCounter = 0 | ||||
| 
 | ||||
| 	q.headerPool = make(map[common.Hash]*types.Header) | ||||
| 	q.headerQueue.Reset() | ||||
| 
 | ||||
| 	q.pendPool = make(map[string]*fetchRequest) | ||||
| 
 | ||||
| 	q.blockPool = make(map[common.Hash]uint64) | ||||
| @ -90,21 +99,21 @@ func (q *queue) Reset() { | ||||
| 	q.blockCache = make([]*Block, blockCacheLimit) | ||||
| } | ||||
| 
 | ||||
| // Size retrieves the number of hashes in the queue, returning separately for
 | ||||
| // Size retrieves the number of blocks in the queue, returning separately for
 | ||||
| // pending and already downloaded.
 | ||||
| func (q *queue) Size() (int, int) { | ||||
| 	q.lock.RLock() | ||||
| 	defer q.lock.RUnlock() | ||||
| 
 | ||||
| 	return len(q.hashPool), len(q.blockPool) | ||||
| 	return len(q.hashPool) + len(q.headerPool), len(q.blockPool) | ||||
| } | ||||
| 
 | ||||
| // Pending retrieves the number of hashes pending for retrieval.
 | ||||
| // Pending retrieves the number of blocks pending for retrieval.
 | ||||
| func (q *queue) Pending() int { | ||||
| 	q.lock.RLock() | ||||
| 	defer q.lock.RUnlock() | ||||
| 
 | ||||
| 	return q.hashQueue.Size() | ||||
| 	return q.hashQueue.Size() + q.headerQueue.Size() | ||||
| } | ||||
| 
 | ||||
| // InFlight retrieves the number of fetch requests currently in flight.
 | ||||
| @ -124,7 +133,7 @@ func (q *queue) Throttle() bool { | ||||
| 	// Calculate the currently in-flight block requests
 | ||||
| 	pending := 0 | ||||
| 	for _, request := range q.pendPool { | ||||
| 		pending += len(request.Hashes) | ||||
| 		pending += len(request.Hashes) + len(request.Headers) | ||||
| 	} | ||||
| 	// Throttle if more blocks are in-flight than free space in the cache
 | ||||
| 	return pending >= len(q.blockCache)-len(q.blockPool) | ||||
| @ -138,15 +147,18 @@ func (q *queue) Has(hash common.Hash) bool { | ||||
| 	if _, ok := q.hashPool[hash]; ok { | ||||
| 		return true | ||||
| 	} | ||||
| 	if _, ok := q.headerPool[hash]; ok { | ||||
| 		return true | ||||
| 	} | ||||
| 	if _, ok := q.blockPool[hash]; ok { | ||||
| 		return true | ||||
| 	} | ||||
| 	return false | ||||
| } | ||||
| 
 | ||||
| // Insert adds a set of hashes for the download queue for scheduling, returning
 | ||||
| // Insert61 adds a set of hashes for the download queue for scheduling, returning
 | ||||
| // the new hashes encountered.
 | ||||
| func (q *queue) Insert(hashes []common.Hash, fifo bool) []common.Hash { | ||||
| func (q *queue) Insert61(hashes []common.Hash, fifo bool) []common.Hash { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| @ -172,6 +184,29 @@ func (q *queue) Insert(hashes []common.Hash, fifo bool) []common.Hash { | ||||
| 	return inserts | ||||
| } | ||||
| 
 | ||||
| // Insert adds a set of headers for the download queue for scheduling, returning
 | ||||
| // the new headers encountered.
 | ||||
| func (q *queue) Insert(headers []*types.Header) []*types.Header { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| 	// Insert all the headers prioritized by the contained block number
 | ||||
| 	inserts := make([]*types.Header, 0, len(headers)) | ||||
| 	for _, header := range headers { | ||||
| 		// Make sure no duplicate requests are executed
 | ||||
| 		hash := header.Hash() | ||||
| 		if _, ok := q.headerPool[hash]; ok { | ||||
| 			glog.V(logger.Warn).Infof("Header %x already scheduled", hash) | ||||
| 			continue | ||||
| 		} | ||||
| 		// Queue the header for body retrieval
 | ||||
| 		inserts = append(inserts, header) | ||||
| 		q.headerPool[hash] = header | ||||
| 		q.headerQueue.Push(header, -float32(header.Number.Uint64())) | ||||
| 	} | ||||
| 	return inserts | ||||
| } | ||||
| 
 | ||||
| // GetHeadBlock retrieves the first block from the cache, or nil if it hasn't
 | ||||
| // been downloaded yet (or simply non existent).
 | ||||
| func (q *queue) GetHeadBlock() *Block { | ||||
| @ -227,9 +262,9 @@ func (q *queue) TakeBlocks() []*Block { | ||||
| 	return blocks | ||||
| } | ||||
| 
 | ||||
| // Reserve reserves a set of hashes for the given peer, skipping any previously
 | ||||
| // Reserve61 reserves a set of hashes for the given peer, skipping any previously
 | ||||
| // failed download.
 | ||||
| func (q *queue) Reserve(p *peer, count int) *fetchRequest { | ||||
| func (q *queue) Reserve61(p *peer, count int) *fetchRequest { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| @ -276,6 +311,68 @@ func (q *queue) Reserve(p *peer, count int) *fetchRequest { | ||||
| 	return request | ||||
| } | ||||
| 
 | ||||
| // Reserve reserves a set of headers for the given peer, skipping any previously
 | ||||
| // failed download. Beside the next batch of needed fetches, it also returns a
 | ||||
| // flag whether empty blocks were queued requiring processing.
 | ||||
| func (q *queue) Reserve(p *peer, count int) (*fetchRequest, bool, error) { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| 	// Short circuit if the pool has been depleted, or if the peer's already
 | ||||
| 	// downloading something (sanity check not to corrupt state)
 | ||||
| 	if q.headerQueue.Empty() { | ||||
| 		return nil, false, nil | ||||
| 	} | ||||
| 	if _, ok := q.pendPool[p.id]; ok { | ||||
| 		return nil, false, nil | ||||
| 	} | ||||
| 	// Calculate an upper limit on the bodies we might fetch (i.e. throttling)
 | ||||
| 	space := len(q.blockCache) - len(q.blockPool) | ||||
| 	for _, request := range q.pendPool { | ||||
| 		space -= len(request.Headers) | ||||
| 	} | ||||
| 	// Retrieve a batch of headers, skipping previously failed ones
 | ||||
| 	send := make([]*types.Header, 0, count) | ||||
| 	skip := make([]*types.Header, 0) | ||||
| 
 | ||||
| 	process := false | ||||
| 	for proc := 0; proc < space && len(send) < count && !q.headerQueue.Empty(); proc++ { | ||||
| 		header := q.headerQueue.PopItem().(*types.Header) | ||||
| 
 | ||||
| 		// If the header defines an empty block, deliver straight
 | ||||
| 		if header.TxHash == types.DeriveSha(types.Transactions{}) && header.UncleHash == types.CalcUncleHash([]*types.Header{}) { | ||||
| 			if err := q.enqueue("", types.NewBlockWithHeader(header)); err != nil { | ||||
| 				return nil, false, errInvalidChain | ||||
| 			} | ||||
| 			delete(q.headerPool, header.Hash()) | ||||
| 			process, space, proc = true, space-1, proc-1 | ||||
| 			continue | ||||
| 		} | ||||
| 		// If it's a content block, add to the body fetch request
 | ||||
| 		if p.ignored.Has(header.Hash()) { | ||||
| 			skip = append(skip, header) | ||||
| 		} else { | ||||
| 			send = append(send, header) | ||||
| 		} | ||||
| 	} | ||||
| 	// Merge all the skipped headers back
 | ||||
| 	for _, header := range skip { | ||||
| 		q.headerQueue.Push(header, -float32(header.Number.Uint64())) | ||||
| 	} | ||||
| 	// Assemble and return the block download request
 | ||||
| 	if len(send) == 0 { | ||||
| 		return nil, process, nil | ||||
| 	} | ||||
| 	request := &fetchRequest{ | ||||
| 		Peer:    p, | ||||
| 		Headers: send, | ||||
| 		Time:    time.Now(), | ||||
| 	} | ||||
| 	q.pendPool[p.id] = request | ||||
| 
 | ||||
| 	return request, process, nil | ||||
| } | ||||
| 
 | ||||
| // Cancel aborts a fetch request, returning all pending hashes to the queue.
 | ||||
| func (q *queue) Cancel(request *fetchRequest) { | ||||
| 	q.lock.Lock() | ||||
| @ -284,6 +381,9 @@ func (q *queue) Cancel(request *fetchRequest) { | ||||
| 	for hash, index := range request.Hashes { | ||||
| 		q.hashQueue.Push(hash, float32(index)) | ||||
| 	} | ||||
| 	for _, header := range request.Headers { | ||||
| 		q.headerQueue.Push(header, -float32(header.Number.Uint64())) | ||||
| 	} | ||||
| 	delete(q.pendPool, request.Peer.id) | ||||
| } | ||||
| 
 | ||||
| @ -297,9 +397,19 @@ func (q *queue) Expire(timeout time.Duration) []string { | ||||
| 	peers := []string{} | ||||
| 	for id, request := range q.pendPool { | ||||
| 		if time.Since(request.Time) > timeout { | ||||
| 			// Update the metrics with the timeout
 | ||||
| 			if len(request.Hashes) > 0 { | ||||
| 				blockTimeoutMeter.Mark(1) | ||||
| 			} else { | ||||
| 				bodyTimeoutMeter.Mark(1) | ||||
| 			} | ||||
| 			// Return any non satisfied requests to the pool
 | ||||
| 			for hash, index := range request.Hashes { | ||||
| 				q.hashQueue.Push(hash, float32(index)) | ||||
| 			} | ||||
| 			for _, header := range request.Headers { | ||||
| 				q.headerQueue.Push(header, -float32(header.Number.Uint64())) | ||||
| 			} | ||||
| 			peers = append(peers, id) | ||||
| 		} | ||||
| 	} | ||||
| @ -310,8 +420,8 @@ func (q *queue) Expire(timeout time.Duration) []string { | ||||
| 	return peers | ||||
| } | ||||
| 
 | ||||
| // Deliver injects a block retrieval response into the download queue.
 | ||||
| func (q *queue) Deliver(id string, blocks []*types.Block) (err error) { | ||||
| // Deliver61 injects a block retrieval response into the download queue.
 | ||||
| func (q *queue) Deliver61(id string, blocks []*types.Block) (err error) { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| @ -320,6 +430,7 @@ func (q *queue) Deliver(id string, blocks []*types.Block) (err error) { | ||||
| 	if request == nil { | ||||
| 		return errNoFetchesPending | ||||
| 	} | ||||
| 	blockReqTimer.UpdateSince(request.Time) | ||||
| 	delete(q.pendPool, id) | ||||
| 
 | ||||
| 	// If no blocks were retrieved, mark them as unavailable for the origin peer
 | ||||
| @ -337,19 +448,12 @@ func (q *queue) Deliver(id string, blocks []*types.Block) (err error) { | ||||
| 			errs = append(errs, fmt.Errorf("non-requested block %x", hash)) | ||||
| 			continue | ||||
| 		} | ||||
| 		// If a requested block falls out of the range, the hash chain is invalid
 | ||||
| 		index := int(int64(block.NumberU64()) - int64(q.blockOffset)) | ||||
| 		if index >= len(q.blockCache) || index < 0 { | ||||
| 			return errInvalidChain | ||||
| 		} | ||||
| 		// Otherwise merge the block and mark the hash block
 | ||||
| 		q.blockCache[index] = &Block{ | ||||
| 			RawBlock:   block, | ||||
| 			OriginPeer: id, | ||||
| 		// Queue the block up for processing
 | ||||
| 		if err := q.enqueue(id, block); err != nil { | ||||
| 			return err | ||||
| 		} | ||||
| 		delete(request.Hashes, hash) | ||||
| 		delete(q.hashPool, hash) | ||||
| 		q.blockPool[hash] = block.NumberU64() | ||||
| 	} | ||||
| 	// Return all failed or missing fetches to the queue
 | ||||
| 	for hash, index := range request.Hashes { | ||||
| @ -365,6 +469,89 @@ func (q *queue) Deliver(id string, blocks []*types.Block) (err error) { | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // Deliver injects a block body retrieval response into the download queue.
 | ||||
| func (q *queue) Deliver(id string, txLists [][]*types.Transaction, uncleLists [][]*types.Header) error { | ||||
| 	q.lock.Lock() | ||||
| 	defer q.lock.Unlock() | ||||
| 
 | ||||
| 	// Short circuit if the block bodies were never requested
 | ||||
| 	request := q.pendPool[id] | ||||
| 	if request == nil { | ||||
| 		return errNoFetchesPending | ||||
| 	} | ||||
| 	bodyReqTimer.UpdateSince(request.Time) | ||||
| 	delete(q.pendPool, id) | ||||
| 
 | ||||
| 	// If no block bodies were retrieved, mark them as unavailable for the origin peer
 | ||||
| 	if len(txLists) == 0 || len(uncleLists) == 0 { | ||||
| 		for hash, _ := range request.Headers { | ||||
| 			request.Peer.ignored.Add(hash) | ||||
| 		} | ||||
| 	} | ||||
| 	// Assemble each of the block bodies with their headers and queue for processing
 | ||||
| 	errs := make([]error, 0) | ||||
| 	for i, header := range request.Headers { | ||||
| 		// Short circuit block assembly if no more bodies are found
 | ||||
| 		if i >= len(txLists) || i >= len(uncleLists) { | ||||
| 			break | ||||
| 		} | ||||
| 		// Reconstruct the next block if contents match up
 | ||||
| 		if types.DeriveSha(types.Transactions(txLists[i])) != header.TxHash || types.CalcUncleHash(uncleLists[i]) != header.UncleHash { | ||||
| 			errs = []error{errInvalidBody} | ||||
| 			break | ||||
| 		} | ||||
| 		block := types.NewBlockWithHeader(header).WithBody(txLists[i], uncleLists[i]) | ||||
| 
 | ||||
| 		// Queue the block up for processing
 | ||||
| 		if err := q.enqueue(id, block); err != nil { | ||||
| 			errs = []error{err} | ||||
| 			break | ||||
| 		} | ||||
| 		request.Headers[i] = nil | ||||
| 		delete(q.headerPool, header.Hash()) | ||||
| 	} | ||||
| 	// Return all failed or missing fetches to the queue
 | ||||
| 	for _, header := range request.Headers { | ||||
| 		if header != nil { | ||||
| 			q.headerQueue.Push(header, -float32(header.Number.Uint64())) | ||||
| 		} | ||||
| 	} | ||||
| 	// If none of the blocks were good, it's a stale delivery
 | ||||
| 	switch { | ||||
| 	case len(errs) == 0: | ||||
| 		return nil | ||||
| 
 | ||||
| 	case len(errs) == 1 && errs[0] == errInvalidBody: | ||||
| 		return errInvalidBody | ||||
| 
 | ||||
| 	case len(errs) == 1 && errs[0] == errInvalidChain: | ||||
| 		return errInvalidChain | ||||
| 
 | ||||
| 	case len(errs) == len(request.Headers): | ||||
| 		return errStaleDelivery | ||||
| 
 | ||||
| 	default: | ||||
| 		return fmt.Errorf("multiple failures: %v", errs) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // enqueue inserts a new block into the final delivery queue, waiting for pickup
 | ||||
| // by the processor.
 | ||||
| func (q *queue) enqueue(origin string, block *types.Block) error { | ||||
| 	// If a requested block falls out of the range, the hash chain is invalid
 | ||||
| 	index := int(int64(block.NumberU64()) - int64(q.blockOffset)) | ||||
| 	if index >= len(q.blockCache) || index < 0 { | ||||
| 		return errInvalidChain | ||||
| 	} | ||||
| 	// Otherwise merge the block and mark the hash done
 | ||||
| 	q.blockCache[index] = &Block{ | ||||
| 		RawBlock:   block, | ||||
| 		OriginPeer: origin, | ||||
| 	} | ||||
| 	q.blockPool[block.Header().Hash()] = block.NumberU64() | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // Prepare configures the block cache offset to allow accepting inbound blocks.
 | ||||
| func (q *queue) Prepare(offset uint64) { | ||||
| 	q.lock.Lock() | ||||
|  | ||||
| @ -51,6 +51,12 @@ type blockRetrievalFn func(common.Hash) *types.Block | ||||
| // blockRequesterFn is a callback type for sending a block retrieval request.
 | ||||
| type blockRequesterFn func([]common.Hash) error | ||||
| 
 | ||||
| // headerRequesterFn is a callback type for sending a header retrieval request.
 | ||||
| type headerRequesterFn func(common.Hash) error | ||||
| 
 | ||||
| // bodyRequesterFn is a callback type for sending a body retrieval request.
 | ||||
| type bodyRequesterFn func([]common.Hash) error | ||||
| 
 | ||||
| // blockValidatorFn is a callback type to verify a block's header for fast propagation.
 | ||||
| type blockValidatorFn func(block *types.Block, parent *types.Block) error | ||||
| 
 | ||||
| @ -69,11 +75,30 @@ type peerDropFn func(id string) | ||||
| // announce is the hash notification of the availability of a new block in the
 | ||||
| // network.
 | ||||
| type announce struct { | ||||
| 	hash common.Hash // Hash of the block being announced
 | ||||
| 	time time.Time   // Timestamp of the announcement
 | ||||
| 	hash   common.Hash   // Hash of the block being announced
 | ||||
| 	number uint64        // Number of the block being announced (0 = unknown | old protocol)
 | ||||
| 	header *types.Header // Header of the block partially reassembled (new protocol)
 | ||||
| 	time   time.Time     // Timestamp of the announcement
 | ||||
| 
 | ||||
| 	origin string           // Identifier of the peer originating the notification
 | ||||
| 	fetch  blockRequesterFn // Fetcher function to retrieve
 | ||||
| 	origin string // Identifier of the peer originating the notification
 | ||||
| 
 | ||||
| 	fetch61     blockRequesterFn  // [eth/61] Fetcher function to retrieve an announced block
 | ||||
| 	fetchHeader headerRequesterFn // [eth/62] Fetcher function to retrieve the header of an announced block
 | ||||
| 	fetchBodies bodyRequesterFn   // [eth/62] Fetcher function to retrieve the body of an announced block
 | ||||
| } | ||||
| 
 | ||||
| // headerFilterTask represents a batch of headers needing fetcher filtering.
 | ||||
| type headerFilterTask struct { | ||||
| 	headers []*types.Header // Collection of headers to filter
 | ||||
| 	time    time.Time       // Arrival time of the headers
 | ||||
| } | ||||
| 
 | ||||
| // headerFilterTask represents a batch of block bodies (transactions and uncles)
 | ||||
| // needing fetcher filtering.
 | ||||
| type bodyFilterTask struct { | ||||
| 	transactions [][]*types.Transaction // Collection of transactions per block bodies
 | ||||
| 	uncles       [][]*types.Header      // Collection of uncles per block bodies
 | ||||
| 	time         time.Time              // Arrival time of the blocks' contents
 | ||||
| } | ||||
| 
 | ||||
| // inject represents a schedules import operation.
 | ||||
| @ -88,14 +113,20 @@ type Fetcher struct { | ||||
| 	// Various event channels
 | ||||
| 	notify chan *announce | ||||
| 	inject chan *inject | ||||
| 	filter chan chan []*types.Block | ||||
| 	done   chan common.Hash | ||||
| 	quit   chan struct{} | ||||
| 
 | ||||
| 	blockFilter  chan chan []*types.Block | ||||
| 	headerFilter chan chan *headerFilterTask | ||||
| 	bodyFilter   chan chan *bodyFilterTask | ||||
| 
 | ||||
| 	done chan common.Hash | ||||
| 	quit chan struct{} | ||||
| 
 | ||||
| 	// Announce states
 | ||||
| 	announces map[string]int              // Per peer announce counts to prevent memory exhaustion
 | ||||
| 	announced map[common.Hash][]*announce // Announced blocks, scheduled for fetching
 | ||||
| 	fetching  map[common.Hash]*announce   // Announced blocks, currently fetching
 | ||||
| 	announces  map[string]int              // Per peer announce counts to prevent memory exhaustion
 | ||||
| 	announced  map[common.Hash][]*announce // Announced blocks, scheduled for fetching
 | ||||
| 	fetching   map[common.Hash]*announce   // Announced blocks, currently fetching
 | ||||
| 	fetched    map[common.Hash][]*announce // Blocks with headers fetched, scheduled for body retrieval
 | ||||
| 	completing map[common.Hash]*announce   // Blocks with headers, currently body-completing
 | ||||
| 
 | ||||
| 	// Block cache
 | ||||
| 	queue  *prque.Prque            // Queue containing the import operations (block number sorted)
 | ||||
| @ -111,8 +142,9 @@ type Fetcher struct { | ||||
| 	dropPeer       peerDropFn         // Drops a peer for misbehaving
 | ||||
| 
 | ||||
| 	// Testing hooks
 | ||||
| 	fetchingHook func([]common.Hash) // Method to call upon starting a block fetch
 | ||||
| 	importedHook func(*types.Block)  // Method to call upon successful block import
 | ||||
| 	fetchingHook   func([]common.Hash) // Method to call upon starting a block (eth/61) or header (eth/62) fetch
 | ||||
| 	completingHook func([]common.Hash) // Method to call upon starting a block body fetch (eth/62)
 | ||||
| 	importedHook   func(*types.Block)  // Method to call upon successful block import (both eth/61 and eth/62)
 | ||||
| } | ||||
| 
 | ||||
| // New creates a block fetcher to retrieve blocks based on hash announcements.
 | ||||
| @ -120,12 +152,16 @@ func New(getBlock blockRetrievalFn, validateBlock blockValidatorFn, broadcastBlo | ||||
| 	return &Fetcher{ | ||||
| 		notify:         make(chan *announce), | ||||
| 		inject:         make(chan *inject), | ||||
| 		filter:         make(chan chan []*types.Block), | ||||
| 		blockFilter:    make(chan chan []*types.Block), | ||||
| 		headerFilter:   make(chan chan *headerFilterTask), | ||||
| 		bodyFilter:     make(chan chan *bodyFilterTask), | ||||
| 		done:           make(chan common.Hash), | ||||
| 		quit:           make(chan struct{}), | ||||
| 		announces:      make(map[string]int), | ||||
| 		announced:      make(map[common.Hash][]*announce), | ||||
| 		fetching:       make(map[common.Hash]*announce), | ||||
| 		fetched:        make(map[common.Hash][]*announce), | ||||
| 		completing:     make(map[common.Hash]*announce), | ||||
| 		queue:          prque.New(), | ||||
| 		queues:         make(map[string]int), | ||||
| 		queued:         make(map[common.Hash]*inject), | ||||
| @ -152,12 +188,17 @@ func (f *Fetcher) Stop() { | ||||
| 
 | ||||
| // Notify announces the fetcher of the potential availability of a new block in
 | ||||
| // the network.
 | ||||
| func (f *Fetcher) Notify(peer string, hash common.Hash, time time.Time, fetcher blockRequesterFn) error { | ||||
| func (f *Fetcher) Notify(peer string, hash common.Hash, number uint64, time time.Time, | ||||
| 	blockFetcher blockRequesterFn, // eth/61 specific whole block fetcher
 | ||||
| 	headerFetcher headerRequesterFn, bodyFetcher bodyRequesterFn) error { | ||||
| 	block := &announce{ | ||||
| 		hash:   hash, | ||||
| 		time:   time, | ||||
| 		origin: peer, | ||||
| 		fetch:  fetcher, | ||||
| 		hash:        hash, | ||||
| 		number:      number, | ||||
| 		time:        time, | ||||
| 		origin:      peer, | ||||
| 		fetch61:     blockFetcher, | ||||
| 		fetchHeader: headerFetcher, | ||||
| 		fetchBodies: bodyFetcher, | ||||
| 	} | ||||
| 	select { | ||||
| 	case f.notify <- block: | ||||
| @ -181,14 +222,16 @@ func (f *Fetcher) Enqueue(peer string, block *types.Block) error { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Filter extracts all the blocks that were explicitly requested by the fetcher,
 | ||||
| // FilterBlocks extracts all the blocks that were explicitly requested by the fetcher,
 | ||||
| // returning those that should be handled differently.
 | ||||
| func (f *Fetcher) Filter(blocks types.Blocks) types.Blocks { | ||||
| func (f *Fetcher) FilterBlocks(blocks types.Blocks) types.Blocks { | ||||
| 	glog.V(logger.Detail).Infof("[eth/61] filtering %d blocks", len(blocks)) | ||||
| 
 | ||||
| 	// Send the filter channel to the fetcher
 | ||||
| 	filter := make(chan []*types.Block) | ||||
| 
 | ||||
| 	select { | ||||
| 	case f.filter <- filter: | ||||
| 	case f.blockFilter <- filter: | ||||
| 	case <-f.quit: | ||||
| 		return nil | ||||
| 	} | ||||
| @ -207,11 +250,69 @@ func (f *Fetcher) Filter(blocks types.Blocks) types.Blocks { | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // FilterHeaders extracts all the headers that were explicitly requested by the fetcher,
 | ||||
| // returning those that should be handled differently.
 | ||||
| func (f *Fetcher) FilterHeaders(headers []*types.Header, time time.Time) []*types.Header { | ||||
| 	glog.V(logger.Detail).Infof("[eth/62] filtering %d headers", len(headers)) | ||||
| 
 | ||||
| 	// Send the filter channel to the fetcher
 | ||||
| 	filter := make(chan *headerFilterTask) | ||||
| 
 | ||||
| 	select { | ||||
| 	case f.headerFilter <- filter: | ||||
| 	case <-f.quit: | ||||
| 		return nil | ||||
| 	} | ||||
| 	// Request the filtering of the header list
 | ||||
| 	select { | ||||
| 	case filter <- &headerFilterTask{headers: headers, time: time}: | ||||
| 	case <-f.quit: | ||||
| 		return nil | ||||
| 	} | ||||
| 	// Retrieve the headers remaining after filtering
 | ||||
| 	select { | ||||
| 	case task := <-filter: | ||||
| 		return task.headers | ||||
| 	case <-f.quit: | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // FilterBodies extracts all the block bodies that were explicitly requested by
 | ||||
| // the fetcher, returning those that should be handled differently.
 | ||||
| func (f *Fetcher) FilterBodies(transactions [][]*types.Transaction, uncles [][]*types.Header, time time.Time) ([][]*types.Transaction, [][]*types.Header) { | ||||
| 	glog.V(logger.Detail).Infof("[eth/62] filtering %d:%d bodies", len(transactions), len(uncles)) | ||||
| 
 | ||||
| 	// Send the filter channel to the fetcher
 | ||||
| 	filter := make(chan *bodyFilterTask) | ||||
| 
 | ||||
| 	select { | ||||
| 	case f.bodyFilter <- filter: | ||||
| 	case <-f.quit: | ||||
| 		return nil, nil | ||||
| 	} | ||||
| 	// Request the filtering of the body list
 | ||||
| 	select { | ||||
| 	case filter <- &bodyFilterTask{transactions: transactions, uncles: uncles, time: time}: | ||||
| 	case <-f.quit: | ||||
| 		return nil, nil | ||||
| 	} | ||||
| 	// Retrieve the bodies remaining after filtering
 | ||||
| 	select { | ||||
| 	case task := <-filter: | ||||
| 		return task.transactions, task.uncles | ||||
| 	case <-f.quit: | ||||
| 		return nil, nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Loop is the main fetcher loop, checking and processing various notification
 | ||||
| // events.
 | ||||
| func (f *Fetcher) loop() { | ||||
| 	// Iterate the block fetching until a quit is requested
 | ||||
| 	fetch := time.NewTimer(0) | ||||
| 	fetchTimer := time.NewTimer(0) | ||||
| 	completeTimer := time.NewTimer(0) | ||||
| 
 | ||||
| 	for { | ||||
| 		// Clean up any expired block fetches
 | ||||
| 		for hash, announce := range f.fetching { | ||||
| @ -246,26 +347,38 @@ func (f *Fetcher) loop() { | ||||
| 
 | ||||
| 		case notification := <-f.notify: | ||||
| 			// A block was announced, make sure the peer isn't DOSing us
 | ||||
| 			announceMeter.Mark(1) | ||||
| 			propAnnounceInMeter.Mark(1) | ||||
| 
 | ||||
| 			count := f.announces[notification.origin] + 1 | ||||
| 			if count > hashLimit { | ||||
| 				glog.V(logger.Debug).Infof("Peer %s: exceeded outstanding announces (%d)", notification.origin, hashLimit) | ||||
| 				propAnnounceDOSMeter.Mark(1) | ||||
| 				break | ||||
| 			} | ||||
| 			// If we have a valid block number, check that it's potentially useful
 | ||||
| 			if notification.number > 0 { | ||||
| 				if dist := int64(notification.number) - int64(f.chainHeight()); dist < -maxUncleDist || dist > maxQueueDist { | ||||
| 					glog.V(logger.Debug).Infof("[eth/62] Peer %s: discarded announcement #%d [%x…], distance %d", notification.origin, notification.number, notification.hash[:4], dist) | ||||
| 					propAnnounceDropMeter.Mark(1) | ||||
| 					break | ||||
| 				} | ||||
| 			} | ||||
| 			// All is well, schedule the announce if block's not yet downloading
 | ||||
| 			if _, ok := f.fetching[notification.hash]; ok { | ||||
| 				break | ||||
| 			} | ||||
| 			if _, ok := f.completing[notification.hash]; ok { | ||||
| 				break | ||||
| 			} | ||||
| 			f.announces[notification.origin] = count | ||||
| 			f.announced[notification.hash] = append(f.announced[notification.hash], notification) | ||||
| 			if len(f.announced) == 1 { | ||||
| 				f.reschedule(fetch) | ||||
| 				f.rescheduleFetch(fetchTimer) | ||||
| 			} | ||||
| 
 | ||||
| 		case op := <-f.inject: | ||||
| 			// A direct block insertion was requested, try and fill any pending gaps
 | ||||
| 			broadcastMeter.Mark(1) | ||||
| 			propBroadcastInMeter.Mark(1) | ||||
| 			f.enqueue(op.origin, op.block) | ||||
| 
 | ||||
| 		case hash := <-f.done: | ||||
| @ -273,7 +386,7 @@ func (f *Fetcher) loop() { | ||||
| 			f.forgetHash(hash) | ||||
| 			f.forgetBlock(hash) | ||||
| 
 | ||||
| 		case <-fetch.C: | ||||
| 		case <-fetchTimer.C: | ||||
| 			// At least one block's timer ran out, check for needing retrieval
 | ||||
| 			request := make(map[string][]common.Hash) | ||||
| 
 | ||||
| @ -290,30 +403,80 @@ func (f *Fetcher) loop() { | ||||
| 					} | ||||
| 				} | ||||
| 			} | ||||
| 			// Send out all block requests
 | ||||
| 			// Send out all block (eth/61) or header (eth/62) requests
 | ||||
| 			for peer, hashes := range request { | ||||
| 				if glog.V(logger.Detail) && len(hashes) > 0 { | ||||
| 					list := "[" | ||||
| 					for _, hash := range hashes { | ||||
| 						list += fmt.Sprintf("%x, ", hash[:4]) | ||||
| 						list += fmt.Sprintf("%x…, ", hash[:4]) | ||||
| 					} | ||||
| 					list = list[:len(list)-2] + "]" | ||||
| 
 | ||||
| 					glog.V(logger.Detail).Infof("Peer %s: fetching %s", peer, list) | ||||
| 					if f.fetching[hashes[0]].fetch61 != nil { | ||||
| 						glog.V(logger.Detail).Infof("[eth/61] Peer %s: fetching blocks %s", peer, list) | ||||
| 					} else { | ||||
| 						glog.V(logger.Detail).Infof("[eth/62] Peer %s: fetching headers %s", peer, list) | ||||
| 					} | ||||
| 				} | ||||
| 				// Create a closure of the fetch and schedule in on a new thread
 | ||||
| 				fetcher, hashes := f.fetching[hashes[0]].fetch, hashes | ||||
| 				fetchBlocks, fetchHeader, hashes := f.fetching[hashes[0]].fetch61, f.fetching[hashes[0]].fetchHeader, hashes | ||||
| 				go func() { | ||||
| 					if f.fetchingHook != nil { | ||||
| 						f.fetchingHook(hashes) | ||||
| 					} | ||||
| 					fetcher(hashes) | ||||
| 					if fetchBlocks != nil { | ||||
| 						// Use old eth/61 protocol to retrieve whole blocks
 | ||||
| 						blockFetchMeter.Mark(int64(len(hashes))) | ||||
| 						fetchBlocks(hashes) | ||||
| 					} else { | ||||
| 						// Use new eth/62 protocol to retrieve headers first
 | ||||
| 						for _, hash := range hashes { | ||||
| 							headerFetchMeter.Mark(1) | ||||
| 							fetchHeader(hash) // Suboptimal, but protocol doesn't allow batch header retrievals
 | ||||
| 						} | ||||
| 					} | ||||
| 				}() | ||||
| 			} | ||||
| 			// Schedule the next fetch if blocks are still pending
 | ||||
| 			f.reschedule(fetch) | ||||
| 			f.rescheduleFetch(fetchTimer) | ||||
| 
 | ||||
| 		case filter := <-f.filter: | ||||
| 		case <-completeTimer.C: | ||||
| 			// At least one header's timer ran out, retrieve everything
 | ||||
| 			request := make(map[string][]common.Hash) | ||||
| 
 | ||||
| 			for hash, announces := range f.fetched { | ||||
| 				// Pick a random peer to retrieve from, reset all others
 | ||||
| 				announce := announces[rand.Intn(len(announces))] | ||||
| 				f.forgetHash(hash) | ||||
| 
 | ||||
| 				// If the block still didn't arrive, queue for completion
 | ||||
| 				if f.getBlock(hash) == nil { | ||||
| 					request[announce.origin] = append(request[announce.origin], hash) | ||||
| 					f.completing[hash] = announce | ||||
| 				} | ||||
| 			} | ||||
| 			// Send out all block body requests
 | ||||
| 			for peer, hashes := range request { | ||||
| 				if glog.V(logger.Detail) && len(hashes) > 0 { | ||||
| 					list := "[" | ||||
| 					for _, hash := range hashes { | ||||
| 						list += fmt.Sprintf("%x…, ", hash[:4]) | ||||
| 					} | ||||
| 					list = list[:len(list)-2] + "]" | ||||
| 
 | ||||
| 					glog.V(logger.Detail).Infof("[eth/62] Peer %s: fetching bodies %s", peer, list) | ||||
| 				} | ||||
| 				// Create a closure of the fetch and schedule in on a new thread
 | ||||
| 				if f.completingHook != nil { | ||||
| 					f.completingHook(hashes) | ||||
| 				} | ||||
| 				bodyFetchMeter.Mark(int64(len(hashes))) | ||||
| 				go f.completing[hashes[0]].fetchBodies(hashes) | ||||
| 			} | ||||
| 			// Schedule the next fetch if blocks are still pending
 | ||||
| 			f.rescheduleComplete(completeTimer) | ||||
| 
 | ||||
| 		case filter := <-f.blockFilter: | ||||
| 			// Blocks arrived, extract any explicit fetches, return all else
 | ||||
| 			var blocks types.Blocks | ||||
| 			select { | ||||
| @ -321,6 +484,7 @@ func (f *Fetcher) loop() { | ||||
| 			case <-f.quit: | ||||
| 				return | ||||
| 			} | ||||
| 			blockFilterInMeter.Mark(int64(len(blocks))) | ||||
| 
 | ||||
| 			explicit, download := []*types.Block{}, []*types.Block{} | ||||
| 			for _, block := range blocks { | ||||
| @ -339,6 +503,7 @@ func (f *Fetcher) loop() { | ||||
| 				} | ||||
| 			} | ||||
| 
 | ||||
| 			blockFilterOutMeter.Mark(int64(len(download))) | ||||
| 			select { | ||||
| 			case filter <- download: | ||||
| 			case <-f.quit: | ||||
| @ -350,12 +515,146 @@ func (f *Fetcher) loop() { | ||||
| 					f.enqueue(announce.origin, block) | ||||
| 				} | ||||
| 			} | ||||
| 
 | ||||
| 		case filter := <-f.headerFilter: | ||||
| 			// Headers arrived from a remote peer. Extract those that were explicitly
 | ||||
| 			// requested by the fetcher, and return everything else so it's delivered
 | ||||
| 			// to other parts of the system.
 | ||||
| 			var task *headerFilterTask | ||||
| 			select { | ||||
| 			case task = <-filter: | ||||
| 			case <-f.quit: | ||||
| 				return | ||||
| 			} | ||||
| 			headerFilterInMeter.Mark(int64(len(task.headers))) | ||||
| 
 | ||||
| 			// Split the batch of headers into unknown ones (to return to the caller),
 | ||||
| 			// known incomplete ones (requiring body retrievals) and completed blocks.
 | ||||
| 			unknown, incomplete, complete := []*types.Header{}, []*announce{}, []*types.Block{} | ||||
| 			for _, header := range task.headers { | ||||
| 				hash := header.Hash() | ||||
| 
 | ||||
| 				// Filter fetcher-requested headers from other synchronisation algorithms
 | ||||
| 				if announce := f.fetching[hash]; announce != nil && f.fetched[hash] == nil && f.completing[hash] == nil && f.queued[hash] == nil { | ||||
| 					// If the delivered header does not match the promised number, drop the announcer
 | ||||
| 					if header.Number.Uint64() != announce.number { | ||||
| 						glog.V(logger.Detail).Infof("[eth/62] Peer %s: invalid block number for [%x…]: announced %d, provided %d", announce.origin, header.Hash().Bytes()[:4], announce.number, header.Number.Uint64()) | ||||
| 						f.dropPeer(announce.origin) | ||||
| 						f.forgetHash(hash) | ||||
| 						continue | ||||
| 					} | ||||
| 					// Only keep if not imported by other means
 | ||||
| 					if f.getBlock(hash) == nil { | ||||
| 						announce.header = header | ||||
| 						announce.time = task.time | ||||
| 
 | ||||
| 						// If the block is empty (header only), short circuit into the final import queue
 | ||||
| 						if header.TxHash == types.DeriveSha(types.Transactions{}) && header.UncleHash == types.CalcUncleHash([]*types.Header{}) { | ||||
| 							glog.V(logger.Detail).Infof("[eth/62] Peer %s: block #%d [%x…] empty, skipping body retrieval", announce.origin, header.Number.Uint64(), header.Hash().Bytes()[:4]) | ||||
| 
 | ||||
| 							block := types.NewBlockWithHeader(header) | ||||
| 							block.ReceivedAt = task.time | ||||
| 
 | ||||
| 							complete = append(complete, block) | ||||
| 							f.completing[hash] = announce | ||||
| 							continue | ||||
| 						} | ||||
| 						// Otherwise add to the list of blocks needing completion
 | ||||
| 						incomplete = append(incomplete, announce) | ||||
| 					} else { | ||||
| 						glog.V(logger.Detail).Infof("[eth/62] Peer %s: block #%d [%x…] already imported, discarding header", announce.origin, header.Number.Uint64(), header.Hash().Bytes()[:4]) | ||||
| 						f.forgetHash(hash) | ||||
| 					} | ||||
| 				} else { | ||||
| 					// Fetcher doesn't know about it, add to the return list
 | ||||
| 					unknown = append(unknown, header) | ||||
| 				} | ||||
| 			} | ||||
| 			headerFilterOutMeter.Mark(int64(len(unknown))) | ||||
| 			select { | ||||
| 			case filter <- &headerFilterTask{headers: unknown, time: task.time}: | ||||
| 			case <-f.quit: | ||||
| 				return | ||||
| 			} | ||||
| 			// Schedule the retrieved headers for body completion
 | ||||
| 			for _, announce := range incomplete { | ||||
| 				hash := announce.header.Hash() | ||||
| 				if _, ok := f.completing[hash]; ok { | ||||
| 					continue | ||||
| 				} | ||||
| 				f.fetched[hash] = append(f.fetched[hash], announce) | ||||
| 				if len(f.fetched) == 1 { | ||||
| 					f.rescheduleComplete(completeTimer) | ||||
| 				} | ||||
| 			} | ||||
| 			// Schedule the header-only blocks for import
 | ||||
| 			for _, block := range complete { | ||||
| 				if announce := f.completing[block.Hash()]; announce != nil { | ||||
| 					f.enqueue(announce.origin, block) | ||||
| 				} | ||||
| 			} | ||||
| 
 | ||||
| 		case filter := <-f.bodyFilter: | ||||
| 			// Block bodies arrived, extract any explicitly requested blocks, return the rest
 | ||||
| 			var task *bodyFilterTask | ||||
| 			select { | ||||
| 			case task = <-filter: | ||||
| 			case <-f.quit: | ||||
| 				return | ||||
| 			} | ||||
| 			bodyFilterInMeter.Mark(int64(len(task.transactions))) | ||||
| 
 | ||||
| 			blocks := []*types.Block{} | ||||
| 			for i := 0; i < len(task.transactions) && i < len(task.uncles); i++ { | ||||
| 				// Match up a body to any possible completion request
 | ||||
| 				matched := false | ||||
| 
 | ||||
| 				for hash, announce := range f.completing { | ||||
| 					if f.queued[hash] == nil { | ||||
| 						txnHash := types.DeriveSha(types.Transactions(task.transactions[i])) | ||||
| 						uncleHash := types.CalcUncleHash(task.uncles[i]) | ||||
| 
 | ||||
| 						if txnHash == announce.header.TxHash && uncleHash == announce.header.UncleHash { | ||||
| 							// Mark the body matched, reassemble if still unknown
 | ||||
| 							matched = true | ||||
| 
 | ||||
| 							if f.getBlock(hash) == nil { | ||||
| 								block := types.NewBlockWithHeader(announce.header).WithBody(task.transactions[i], task.uncles[i]) | ||||
| 								block.ReceivedAt = task.time | ||||
| 
 | ||||
| 								blocks = append(blocks, block) | ||||
| 							} else { | ||||
| 								f.forgetHash(hash) | ||||
| 							} | ||||
| 						} | ||||
| 					} | ||||
| 				} | ||||
| 				if matched { | ||||
| 					task.transactions = append(task.transactions[:i], task.transactions[i+1:]...) | ||||
| 					task.uncles = append(task.uncles[:i], task.uncles[i+1:]...) | ||||
| 					i-- | ||||
| 					continue | ||||
| 				} | ||||
| 			} | ||||
| 
 | ||||
| 			bodyFilterOutMeter.Mark(int64(len(task.transactions))) | ||||
| 			select { | ||||
| 			case filter <- task: | ||||
| 			case <-f.quit: | ||||
| 				return | ||||
| 			} | ||||
| 			// Schedule the retrieved blocks for ordered import
 | ||||
| 			for _, block := range blocks { | ||||
| 				if announce := f.completing[block.Hash()]; announce != nil { | ||||
| 					f.enqueue(announce.origin, block) | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // reschedule resets the specified fetch timer to the next announce timeout.
 | ||||
| func (f *Fetcher) reschedule(fetch *time.Timer) { | ||||
| // rescheduleFetch resets the specified fetch timer to the next announce timeout.
 | ||||
| func (f *Fetcher) rescheduleFetch(fetch *time.Timer) { | ||||
| 	// Short circuit if no blocks are announced
 | ||||
| 	if len(f.announced) == 0 { | ||||
| 		return | ||||
| @ -370,6 +669,22 @@ func (f *Fetcher) reschedule(fetch *time.Timer) { | ||||
| 	fetch.Reset(arriveTimeout - time.Since(earliest)) | ||||
| } | ||||
| 
 | ||||
| // rescheduleComplete resets the specified completion timer to the next fetch timeout.
 | ||||
| func (f *Fetcher) rescheduleComplete(complete *time.Timer) { | ||||
| 	// Short circuit if no headers are fetched
 | ||||
| 	if len(f.fetched) == 0 { | ||||
| 		return | ||||
| 	} | ||||
| 	// Otherwise find the earliest expiring announcement
 | ||||
| 	earliest := time.Now() | ||||
| 	for _, announces := range f.fetched { | ||||
| 		if earliest.After(announces[0].time) { | ||||
| 			earliest = announces[0].time | ||||
| 		} | ||||
| 	} | ||||
| 	complete.Reset(gatherSlack - time.Since(earliest)) | ||||
| } | ||||
| 
 | ||||
| // enqueue schedules a new future import operation, if the block to be imported
 | ||||
| // has not yet been seen.
 | ||||
| func (f *Fetcher) enqueue(peer string, block *types.Block) { | ||||
| @ -378,13 +693,16 @@ func (f *Fetcher) enqueue(peer string, block *types.Block) { | ||||
| 	// Ensure the peer isn't DOSing us
 | ||||
| 	count := f.queues[peer] + 1 | ||||
| 	if count > blockLimit { | ||||
| 		glog.V(logger.Debug).Infof("Peer %s: discarded block #%d [%x], exceeded allowance (%d)", peer, block.NumberU64(), hash.Bytes()[:4], blockLimit) | ||||
| 		glog.V(logger.Debug).Infof("Peer %s: discarded block #%d [%x…], exceeded allowance (%d)", peer, block.NumberU64(), hash.Bytes()[:4], blockLimit) | ||||
| 		propBroadcastDOSMeter.Mark(1) | ||||
| 		f.forgetHash(hash) | ||||
| 		return | ||||
| 	} | ||||
| 	// Discard any past or too distant blocks
 | ||||
| 	if dist := int64(block.NumberU64()) - int64(f.chainHeight()); dist < -maxUncleDist || dist > maxQueueDist { | ||||
| 		glog.V(logger.Debug).Infof("Peer %s: discarded block #%d [%x], distance %d", peer, block.NumberU64(), hash.Bytes()[:4], dist) | ||||
| 		discardMeter.Mark(1) | ||||
| 		glog.V(logger.Debug).Infof("Peer %s: discarded block #%d [%x…], distance %d", peer, block.NumberU64(), hash.Bytes()[:4], dist) | ||||
| 		propBroadcastDropMeter.Mark(1) | ||||
| 		f.forgetHash(hash) | ||||
| 		return | ||||
| 	} | ||||
| 	// Schedule the block for future importing
 | ||||
| @ -398,7 +716,7 @@ func (f *Fetcher) enqueue(peer string, block *types.Block) { | ||||
| 		f.queue.Push(op, -float32(block.NumberU64())) | ||||
| 
 | ||||
| 		if glog.V(logger.Debug) { | ||||
| 			glog.Infof("Peer %s: queued block #%d [%x], total %v", peer, block.NumberU64(), hash.Bytes()[:4], f.queue.Size()) | ||||
| 			glog.Infof("Peer %s: queued block #%d [%x…], total %v", peer, block.NumberU64(), hash.Bytes()[:4], f.queue.Size()) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| @ -410,39 +728,39 @@ func (f *Fetcher) insert(peer string, block *types.Block) { | ||||
| 	hash := block.Hash() | ||||
| 
 | ||||
| 	// Run the import on a new thread
 | ||||
| 	glog.V(logger.Debug).Infof("Peer %s: importing block #%d [%x]", peer, block.NumberU64(), hash[:4]) | ||||
| 	glog.V(logger.Debug).Infof("Peer %s: importing block #%d [%x…]", peer, block.NumberU64(), hash[:4]) | ||||
| 	go func() { | ||||
| 		defer func() { f.done <- hash }() | ||||
| 
 | ||||
| 		// If the parent's unknown, abort insertion
 | ||||
| 		parent := f.getBlock(block.ParentHash()) | ||||
| 		if parent == nil { | ||||
| 			glog.V(logger.Debug).Infof("Peer %s: parent []%x] of block #%d [%x…] unknown", block.ParentHash().Bytes()[:4], peer, block.NumberU64(), hash[:4]) | ||||
| 			return | ||||
| 		} | ||||
| 		// Quickly validate the header and propagate the block if it passes
 | ||||
| 		switch err := f.validateBlock(block, parent); err { | ||||
| 		case nil: | ||||
| 			// All ok, quickly propagate to our peers
 | ||||
| 			broadcastTimer.UpdateSince(block.ReceivedAt) | ||||
| 			propBroadcastOutTimer.UpdateSince(block.ReceivedAt) | ||||
| 			go f.broadcastBlock(block, true) | ||||
| 
 | ||||
| 		case core.BlockFutureErr: | ||||
| 			futureMeter.Mark(1) | ||||
| 			// Weird future block, don't fail, but neither propagate
 | ||||
| 
 | ||||
| 		default: | ||||
| 			// Something went very wrong, drop the peer
 | ||||
| 			glog.V(logger.Debug).Infof("Peer %s: block #%d [%x] verification failed: %v", peer, block.NumberU64(), hash[:4], err) | ||||
| 			glog.V(logger.Debug).Infof("Peer %s: block #%d [%x…] verification failed: %v", peer, block.NumberU64(), hash[:4], err) | ||||
| 			f.dropPeer(peer) | ||||
| 			return | ||||
| 		} | ||||
| 		// Run the actual import and log any issues
 | ||||
| 		if _, err := f.insertChain(types.Blocks{block}); err != nil { | ||||
| 			glog.V(logger.Warn).Infof("Peer %s: block #%d [%x] import failed: %v", peer, block.NumberU64(), hash[:4], err) | ||||
| 			glog.V(logger.Warn).Infof("Peer %s: block #%d [%x…] import failed: %v", peer, block.NumberU64(), hash[:4], err) | ||||
| 			return | ||||
| 		} | ||||
| 		// If import succeeded, broadcast the block
 | ||||
| 		announceTimer.UpdateSince(block.ReceivedAt) | ||||
| 		propAnnounceOutTimer.UpdateSince(block.ReceivedAt) | ||||
| 		go f.broadcastBlock(block, false) | ||||
| 
 | ||||
| 		// Invoke the testing hook if needed
 | ||||
| @ -472,9 +790,27 @@ func (f *Fetcher) forgetHash(hash common.Hash) { | ||||
| 		} | ||||
| 		delete(f.fetching, hash) | ||||
| 	} | ||||
| 
 | ||||
| 	// Remove any pending completion requests and decrement the DOS counters
 | ||||
| 	for _, announce := range f.fetched[hash] { | ||||
| 		f.announces[announce.origin]-- | ||||
| 		if f.announces[announce.origin] == 0 { | ||||
| 			delete(f.announces, announce.origin) | ||||
| 		} | ||||
| 	} | ||||
| 	delete(f.fetched, hash) | ||||
| 
 | ||||
| 	// Remove any pending completions and decrement the DOS counters
 | ||||
| 	if announce := f.completing[hash]; announce != nil { | ||||
| 		f.announces[announce.origin]-- | ||||
| 		if f.announces[announce.origin] == 0 { | ||||
| 			delete(f.announces, announce.origin) | ||||
| 		} | ||||
| 		delete(f.completing, hash) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // forgetBlock removes all traces of a queued block frmo the fetcher's internal
 | ||||
| // forgetBlock removes all traces of a queued block from the fetcher's internal
 | ||||
| // state.
 | ||||
| func (f *Fetcher) forgetBlock(hash common.Hash) { | ||||
| 	if insert := f.queued[hash]; insert != nil { | ||||
|  | ||||
| @ -27,21 +27,39 @@ import ( | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/core" | ||||
| 	"github.com/ethereum/go-ethereum/core/types" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/ethdb" | ||||
| 	"github.com/ethereum/go-ethereum/params" | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	testdb, _    = ethdb.NewMemDatabase() | ||||
| 	genesis      = core.GenesisBlockForTesting(testdb, common.Address{}, big.NewInt(0)) | ||||
| 	testKey, _   = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") | ||||
| 	testAddress  = crypto.PubkeyToAddress(testKey.PublicKey) | ||||
| 	genesis      = core.GenesisBlockForTesting(testdb, testAddress, big.NewInt(1000000000)) | ||||
| 	unknownBlock = types.NewBlock(&types.Header{GasLimit: params.GenesisGasLimit}, nil, nil, nil) | ||||
| ) | ||||
| 
 | ||||
| // makeChain creates a chain of n blocks starting at and including parent.
 | ||||
| // the returned hash chain is ordered head->parent.
 | ||||
| // 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.Block) { | ||||
| 	blocks := core.GenerateChain(parent, testdb, n, func(i int, gen *core.BlockGen) { | ||||
| 		gen.SetCoinbase(common.Address{seed}) | ||||
| 	blocks := 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
 | ||||
| 		if parent == genesis && i%3 == 0 { | ||||
| 			tx, err := types.NewTransaction(block.TxNonce(testAddress), common.Address{seed}, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(testKey) | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 			block.AddTx(tx) | ||||
| 		} | ||||
| 		// If the block number is a multiple of 5, add a bonus uncle to the block
 | ||||
| 		if i%5 == 0 { | ||||
| 			block.AddUncle(&types.Header{ParentHash: block.PrevBlock(i - 1).Hash(), Number: big.NewInt(int64(i - 1))}) | ||||
| 		} | ||||
| 	}) | ||||
| 	hashes := make([]common.Hash, n+1) | ||||
| 	hashes[len(hashes)-1] = parent.Hash() | ||||
| @ -60,6 +78,7 @@ type fetcherTester struct { | ||||
| 
 | ||||
| 	hashes []common.Hash                // Hash chain belonging to the tester
 | ||||
| 	blocks map[common.Hash]*types.Block // Blocks belonging to the tester
 | ||||
| 	drops  map[string]bool              // Map of peers dropped by the fetcher
 | ||||
| 
 | ||||
| 	lock sync.RWMutex | ||||
| } | ||||
| @ -69,6 +88,7 @@ func newTester() *fetcherTester { | ||||
| 	tester := &fetcherTester{ | ||||
| 		hashes: []common.Hash{genesis.Hash()}, | ||||
| 		blocks: map[common.Hash]*types.Block{genesis.Hash(): genesis}, | ||||
| 		drops:  make(map[string]bool), | ||||
| 	} | ||||
| 	tester.fetcher = New(tester.getBlock, tester.verifyBlock, tester.broadcastBlock, tester.chainHeight, tester.insertChain, tester.dropPeer) | ||||
| 	tester.fetcher.Start() | ||||
| @ -122,12 +142,14 @@ func (f *fetcherTester) insertChain(blocks types.Blocks) (int, error) { | ||||
| 	return 0, nil | ||||
| } | ||||
| 
 | ||||
| // dropPeer is a nop placeholder for the peer removal.
 | ||||
| // dropPeer is an emulator for the peer removal, simply accumulating the various
 | ||||
| // peers dropped by the fetcher.
 | ||||
| func (f *fetcherTester) dropPeer(peer string) { | ||||
| 	f.drops[peer] = true | ||||
| } | ||||
| 
 | ||||
| // peerFetcher retrieves a fetcher associated with a simulated peer.
 | ||||
| func (f *fetcherTester) makeFetcher(blocks map[common.Hash]*types.Block) blockRequesterFn { | ||||
| // makeBlockFetcher retrieves a block fetcher associated with a simulated peer.
 | ||||
| func (f *fetcherTester) makeBlockFetcher(blocks map[common.Hash]*types.Block) blockRequesterFn { | ||||
| 	closure := make(map[common.Hash]*types.Block) | ||||
| 	for hash, block := range blocks { | ||||
| 		closure[hash] = block | ||||
| @ -142,18 +164,105 @@ func (f *fetcherTester) makeFetcher(blocks map[common.Hash]*types.Block) blockRe | ||||
| 			} | ||||
| 		} | ||||
| 		// Return on a new thread
 | ||||
| 		go f.fetcher.Filter(blocks) | ||||
| 		go f.fetcher.FilterBlocks(blocks) | ||||
| 
 | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // makeHeaderFetcher retrieves a block header fetcher associated with a simulated peer.
 | ||||
| func (f *fetcherTester) makeHeaderFetcher(blocks map[common.Hash]*types.Block, drift time.Duration) headerRequesterFn { | ||||
| 	closure := make(map[common.Hash]*types.Block) | ||||
| 	for hash, block := range blocks { | ||||
| 		closure[hash] = block | ||||
| 	} | ||||
| 	// Create a function that return a header from the closure
 | ||||
| 	return func(hash common.Hash) error { | ||||
| 		// Gather the blocks to return
 | ||||
| 		headers := make([]*types.Header, 0, 1) | ||||
| 		if block, ok := closure[hash]; ok { | ||||
| 			headers = append(headers, block.Header()) | ||||
| 		} | ||||
| 		// Return on a new thread
 | ||||
| 		go f.fetcher.FilterHeaders(headers, time.Now().Add(drift)) | ||||
| 
 | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // makeBodyFetcher retrieves a block body fetcher associated with a simulated peer.
 | ||||
| func (f *fetcherTester) makeBodyFetcher(blocks map[common.Hash]*types.Block, drift time.Duration) bodyRequesterFn { | ||||
| 	closure := make(map[common.Hash]*types.Block) | ||||
| 	for hash, block := range blocks { | ||||
| 		closure[hash] = block | ||||
| 	} | ||||
| 	// Create a function that returns blocks from the closure
 | ||||
| 	return func(hashes []common.Hash) error { | ||||
| 		// Gather the block bodies to return
 | ||||
| 		transactions := make([][]*types.Transaction, 0, len(hashes)) | ||||
| 		uncles := make([][]*types.Header, 0, len(hashes)) | ||||
| 
 | ||||
| 		for _, hash := range hashes { | ||||
| 			if block, ok := closure[hash]; ok { | ||||
| 				transactions = append(transactions, block.Transactions()) | ||||
| 				uncles = append(uncles, block.Uncles()) | ||||
| 			} | ||||
| 		} | ||||
| 		// Return on a new thread
 | ||||
| 		go f.fetcher.FilterBodies(transactions, uncles, time.Now().Add(drift)) | ||||
| 
 | ||||
| 		return nil | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // verifyFetchingEvent verifies that one single event arrive on an fetching channel.
 | ||||
| func verifyFetchingEvent(t *testing.T, fetching chan []common.Hash, arrive bool) { | ||||
| 	if arrive { | ||||
| 		select { | ||||
| 		case <-fetching: | ||||
| 		case <-time.After(time.Second): | ||||
| 			t.Fatalf("fetching timeout") | ||||
| 		} | ||||
| 	} else { | ||||
| 		select { | ||||
| 		case <-fetching: | ||||
| 			t.Fatalf("fetching invoked") | ||||
| 		case <-time.After(10 * time.Millisecond): | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // verifyCompletingEvent verifies that one single event arrive on an completing channel.
 | ||||
| func verifyCompletingEvent(t *testing.T, completing chan []common.Hash, arrive bool) { | ||||
| 	if arrive { | ||||
| 		select { | ||||
| 		case <-completing: | ||||
| 		case <-time.After(time.Second): | ||||
| 			t.Fatalf("completing timeout") | ||||
| 		} | ||||
| 	} else { | ||||
| 		select { | ||||
| 		case <-completing: | ||||
| 			t.Fatalf("completing invoked") | ||||
| 		case <-time.After(10 * time.Millisecond): | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // verifyImportEvent verifies that one single event arrive on an import channel.
 | ||||
| func verifyImportEvent(t *testing.T, imported chan *types.Block) { | ||||
| 	select { | ||||
| 	case <-imported: | ||||
| 	case <-time.After(time.Second): | ||||
| 		t.Fatalf("import timeout") | ||||
| func verifyImportEvent(t *testing.T, imported chan *types.Block, arrive bool) { | ||||
| 	if arrive { | ||||
| 		select { | ||||
| 		case <-imported: | ||||
| 		case <-time.After(time.Second): | ||||
| 			t.Fatalf("import timeout") | ||||
| 		} | ||||
| 	} else { | ||||
| 		select { | ||||
| 		case <-imported: | ||||
| 			t.Fatalf("import invoked") | ||||
| 		case <-time.After(10 * time.Millisecond): | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| @ -164,7 +273,7 @@ func verifyImportCount(t *testing.T, imported chan *types.Block, count int) { | ||||
| 		select { | ||||
| 		case <-imported: | ||||
| 		case <-time.After(time.Second): | ||||
| 			t.Fatalf("block %d: import timeout", i) | ||||
| 			t.Fatalf("block %d: import timeout", i+1) | ||||
| 		} | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| @ -181,51 +290,78 @@ func verifyImportDone(t *testing.T, imported chan *types.Block) { | ||||
| 
 | ||||
| // Tests that a fetcher accepts block announcements and initiates retrievals for
 | ||||
| // them, successfully importing into the local chain.
 | ||||
| func TestSequentialAnnouncements(t *testing.T) { | ||||
| func TestSequentialAnnouncements61(t *testing.T) { testSequentialAnnouncements(t, 61) } | ||||
| func TestSequentialAnnouncements62(t *testing.T) { testSequentialAnnouncements(t, 62) } | ||||
| func TestSequentialAnnouncements63(t *testing.T) { testSequentialAnnouncements(t, 63) } | ||||
| func TestSequentialAnnouncements64(t *testing.T) { testSequentialAnnouncements(t, 64) } | ||||
| 
 | ||||
| func testSequentialAnnouncements(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import
 | ||||
| 	targetBlocks := 4 * hashLimit | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	// Iteratively announce blocks until all are imported
 | ||||
| 	imported := make(chan *types.Block) | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	for i := len(hashes) - 2; i >= 0; i-- { | ||||
| 		tester.fetcher.Notify("valid", hashes[i], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 		verifyImportEvent(t, imported) | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 		} | ||||
| 		verifyImportEvent(t, imported, true) | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| } | ||||
| 
 | ||||
| // Tests that if blocks are announced by multiple peers (or even the same buggy
 | ||||
| // peer), they will only get downloaded at most once.
 | ||||
| func TestConcurrentAnnouncements(t *testing.T) { | ||||
| func TestConcurrentAnnouncements61(t *testing.T) { testConcurrentAnnouncements(t, 61) } | ||||
| func TestConcurrentAnnouncements62(t *testing.T) { testConcurrentAnnouncements(t, 62) } | ||||
| func TestConcurrentAnnouncements63(t *testing.T) { testConcurrentAnnouncements(t, 63) } | ||||
| func TestConcurrentAnnouncements64(t *testing.T) { testConcurrentAnnouncements(t, 64) } | ||||
| 
 | ||||
| func testConcurrentAnnouncements(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import
 | ||||
| 	targetBlocks := 4 * hashLimit | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	// Assemble a tester with a built in counter for the requests
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	counter := uint32(0) | ||||
| 	wrapper := func(hashes []common.Hash) error { | ||||
| 	blockWrapper := func(hashes []common.Hash) error { | ||||
| 		atomic.AddUint32(&counter, uint32(len(hashes))) | ||||
| 		return fetcher(hashes) | ||||
| 		return blockFetcher(hashes) | ||||
| 	} | ||||
| 	headerWrapper := func(hash common.Hash) error { | ||||
| 		atomic.AddUint32(&counter, 1) | ||||
| 		return headerFetcher(hash) | ||||
| 	} | ||||
| 	// Iteratively announce blocks until all are imported
 | ||||
| 	imported := make(chan *types.Block) | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	for i := len(hashes) - 2; i >= 0; i-- { | ||||
| 		tester.fetcher.Notify("first", hashes[i], time.Now().Add(-arriveTimeout), wrapper) | ||||
| 		tester.fetcher.Notify("second", hashes[i], time.Now().Add(-arriveTimeout+time.Millisecond), wrapper) | ||||
| 		tester.fetcher.Notify("second", hashes[i], time.Now().Add(-arriveTimeout-time.Millisecond), wrapper) | ||||
| 
 | ||||
| 		verifyImportEvent(t, imported) | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("first", hashes[i], 0, time.Now().Add(-arriveTimeout), blockWrapper, nil, nil) | ||||
| 			tester.fetcher.Notify("second", hashes[i], 0, time.Now().Add(-arriveTimeout+time.Millisecond), blockWrapper, nil, nil) | ||||
| 			tester.fetcher.Notify("second", hashes[i], 0, time.Now().Add(-arriveTimeout-time.Millisecond), blockWrapper, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("first", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerWrapper, bodyFetcher) | ||||
| 			tester.fetcher.Notify("second", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout+time.Millisecond), nil, headerWrapper, bodyFetcher) | ||||
| 			tester.fetcher.Notify("second", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout-time.Millisecond), nil, headerWrapper, bodyFetcher) | ||||
| 		} | ||||
| 		verifyImportEvent(t, imported, true) | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| 
 | ||||
| @ -237,56 +373,90 @@ func TestConcurrentAnnouncements(t *testing.T) { | ||||
| 
 | ||||
| // Tests that announcements arriving while a previous is being fetched still
 | ||||
| // results in a valid import.
 | ||||
| func TestOverlappingAnnouncements(t *testing.T) { | ||||
| func TestOverlappingAnnouncements61(t *testing.T) { testOverlappingAnnouncements(t, 61) } | ||||
| func TestOverlappingAnnouncements62(t *testing.T) { testOverlappingAnnouncements(t, 62) } | ||||
| func TestOverlappingAnnouncements63(t *testing.T) { testOverlappingAnnouncements(t, 63) } | ||||
| func TestOverlappingAnnouncements64(t *testing.T) { testOverlappingAnnouncements(t, 64) } | ||||
| 
 | ||||
| func testOverlappingAnnouncements(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import
 | ||||
| 	targetBlocks := 4 * hashLimit | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	// Iteratively announce blocks, but overlap them continuously
 | ||||
| 	fetching := make(chan []common.Hash) | ||||
| 	overlap := 16 | ||||
| 	imported := make(chan *types.Block, len(hashes)-1) | ||||
| 	tester.fetcher.fetchingHook = func(hashes []common.Hash) { fetching <- hashes } | ||||
| 	for i := 0; i < overlap; i++ { | ||||
| 		imported <- nil | ||||
| 	} | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	for i := len(hashes) - 2; i >= 0; i-- { | ||||
| 		tester.fetcher.Notify("valid", hashes[i], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 		} | ||||
| 		select { | ||||
| 		case <-fetching: | ||||
| 		case <-imported: | ||||
| 		case <-time.After(time.Second): | ||||
| 			t.Fatalf("hash %d: announce timeout", len(hashes)-i) | ||||
| 			t.Fatalf("block %d: import timeout", len(hashes)-i) | ||||
| 		} | ||||
| 	} | ||||
| 	// Wait for all the imports to complete and check count
 | ||||
| 	verifyImportCount(t, imported, len(hashes)-1) | ||||
| 	verifyImportCount(t, imported, overlap) | ||||
| } | ||||
| 
 | ||||
| // Tests that announces already being retrieved will not be duplicated.
 | ||||
| func TestPendingDeduplication(t *testing.T) { | ||||
| func TestPendingDeduplication61(t *testing.T) { testPendingDeduplication(t, 61) } | ||||
| func TestPendingDeduplication62(t *testing.T) { testPendingDeduplication(t, 62) } | ||||
| func TestPendingDeduplication63(t *testing.T) { testPendingDeduplication(t, 63) } | ||||
| func TestPendingDeduplication64(t *testing.T) { testPendingDeduplication(t, 64) } | ||||
| 
 | ||||
| func testPendingDeduplication(t *testing.T, protocol int) { | ||||
| 	// Create a hash and corresponding block
 | ||||
| 	hashes, blocks := makeChain(1, 0, genesis) | ||||
| 
 | ||||
| 	// Assemble a tester with a built in counter and delayed fetcher
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	delay := 50 * time.Millisecond | ||||
| 	counter := uint32(0) | ||||
| 	wrapper := func(hashes []common.Hash) error { | ||||
| 	blockWrapper := func(hashes []common.Hash) error { | ||||
| 		atomic.AddUint32(&counter, uint32(len(hashes))) | ||||
| 
 | ||||
| 		// Simulate a long running fetch
 | ||||
| 		go func() { | ||||
| 			time.Sleep(delay) | ||||
| 			fetcher(hashes) | ||||
| 			blockFetcher(hashes) | ||||
| 		}() | ||||
| 		return nil | ||||
| 	} | ||||
| 	headerWrapper := func(hash common.Hash) error { | ||||
| 		atomic.AddUint32(&counter, 1) | ||||
| 
 | ||||
| 		// Simulate a long running fetch
 | ||||
| 		go func() { | ||||
| 			time.Sleep(delay) | ||||
| 			headerFetcher(hash) | ||||
| 		}() | ||||
| 		return nil | ||||
| 	} | ||||
| 	// Announce the same block many times until it's fetched (wait for any pending ops)
 | ||||
| 	for tester.getBlock(hashes[0]) == nil { | ||||
| 		tester.fetcher.Notify("repeater", hashes[0], time.Now().Add(-arriveTimeout), wrapper) | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("repeater", hashes[0], 0, time.Now().Add(-arriveTimeout), blockWrapper, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("repeater", hashes[0], 1, time.Now().Add(-arriveTimeout), nil, headerWrapper, bodyFetcher) | ||||
| 		} | ||||
| 		time.Sleep(time.Millisecond) | ||||
| 	} | ||||
| 	time.Sleep(delay) | ||||
| @ -302,14 +472,21 @@ func TestPendingDeduplication(t *testing.T) { | ||||
| 
 | ||||
| // Tests that announcements retrieved in a random order are cached and eventually
 | ||||
| // imported when all the gaps are filled in.
 | ||||
| func TestRandomArrivalImport(t *testing.T) { | ||||
| func TestRandomArrivalImport61(t *testing.T) { testRandomArrivalImport(t, 61) } | ||||
| func TestRandomArrivalImport62(t *testing.T) { testRandomArrivalImport(t, 62) } | ||||
| func TestRandomArrivalImport63(t *testing.T) { testRandomArrivalImport(t, 63) } | ||||
| func TestRandomArrivalImport64(t *testing.T) { testRandomArrivalImport(t, 64) } | ||||
| 
 | ||||
| func testRandomArrivalImport(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import, and choose one to delay
 | ||||
| 	targetBlocks := maxQueueDist | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 	skip := targetBlocks / 2 | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	// Iteratively announce blocks, skipping one entry
 | ||||
| 	imported := make(chan *types.Block, len(hashes)-1) | ||||
| @ -317,25 +494,40 @@ func TestRandomArrivalImport(t *testing.T) { | ||||
| 
 | ||||
| 	for i := len(hashes) - 1; i >= 0; i-- { | ||||
| 		if i != skip { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 			if protocol < 62 { | ||||
| 				tester.fetcher.Notify("valid", hashes[i], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 			} else { | ||||
| 				tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 			} | ||||
| 			time.Sleep(time.Millisecond) | ||||
| 		} | ||||
| 	} | ||||
| 	// Finally announce the skipped entry and check full import
 | ||||
| 	tester.fetcher.Notify("valid", hashes[skip], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 	if protocol < 62 { | ||||
| 		tester.fetcher.Notify("valid", hashes[skip], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 	} else { | ||||
| 		tester.fetcher.Notify("valid", hashes[skip], uint64(len(hashes)-skip-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	} | ||||
| 	verifyImportCount(t, imported, len(hashes)-1) | ||||
| } | ||||
| 
 | ||||
| // Tests that direct block enqueues (due to block propagation vs. hash announce)
 | ||||
| // are correctly schedule, filling and import queue gaps.
 | ||||
| func TestQueueGapFill(t *testing.T) { | ||||
| func TestQueueGapFill61(t *testing.T) { testQueueGapFill(t, 61) } | ||||
| func TestQueueGapFill62(t *testing.T) { testQueueGapFill(t, 62) } | ||||
| func TestQueueGapFill63(t *testing.T) { testQueueGapFill(t, 63) } | ||||
| func TestQueueGapFill64(t *testing.T) { testQueueGapFill(t, 64) } | ||||
| 
 | ||||
| func testQueueGapFill(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import, and choose one to not announce at all
 | ||||
| 	targetBlocks := maxQueueDist | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 	skip := targetBlocks / 2 | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	// Iteratively announce blocks, skipping one entry
 | ||||
| 	imported := make(chan *types.Block, len(hashes)-1) | ||||
| @ -343,7 +535,11 @@ func TestQueueGapFill(t *testing.T) { | ||||
| 
 | ||||
| 	for i := len(hashes) - 1; i >= 0; i-- { | ||||
| 		if i != skip { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 			if protocol < 62 { | ||||
| 				tester.fetcher.Notify("valid", hashes[i], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 			} else { | ||||
| 				tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 			} | ||||
| 			time.Sleep(time.Millisecond) | ||||
| 		} | ||||
| 	} | ||||
| @ -354,13 +550,20 @@ func TestQueueGapFill(t *testing.T) { | ||||
| 
 | ||||
| // Tests that blocks arriving from various sources (multiple propagations, hash
 | ||||
| // announces, etc) do not get scheduled for import multiple times.
 | ||||
| func TestImportDeduplication(t *testing.T) { | ||||
| func TestImportDeduplication61(t *testing.T) { testImportDeduplication(t, 61) } | ||||
| func TestImportDeduplication62(t *testing.T) { testImportDeduplication(t, 62) } | ||||
| func TestImportDeduplication63(t *testing.T) { testImportDeduplication(t, 63) } | ||||
| func TestImportDeduplication64(t *testing.T) { testImportDeduplication(t, 64) } | ||||
| 
 | ||||
| func testImportDeduplication(t *testing.T, protocol int) { | ||||
| 	// Create two blocks to import (one for duplication, the other for stalling)
 | ||||
| 	hashes, blocks := makeChain(2, 0, genesis) | ||||
| 
 | ||||
| 	// Create the tester and wrap the importer with a counter
 | ||||
| 	tester := newTester() | ||||
| 	fetcher := tester.makeFetcher(blocks) | ||||
| 	blockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	counter := uint32(0) | ||||
| 	tester.fetcher.insertChain = func(blocks types.Blocks) (int, error) { | ||||
| @ -374,7 +577,11 @@ func TestImportDeduplication(t *testing.T) { | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	// Announce the duplicating block, wait for retrieval, and also propagate directly
 | ||||
| 	tester.fetcher.Notify("valid", hashes[0], time.Now().Add(-arriveTimeout), fetcher) | ||||
| 	if protocol < 62 { | ||||
| 		tester.fetcher.Notify("valid", hashes[0], 0, time.Now().Add(-arriveTimeout), blockFetcher, nil, nil) | ||||
| 	} else { | ||||
| 		tester.fetcher.Notify("valid", hashes[0], 1, time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	} | ||||
| 	<-fetching | ||||
| 
 | ||||
| 	tester.fetcher.Enqueue("valid", blocks[hashes[0]]) | ||||
| @ -391,35 +598,157 @@ func TestImportDeduplication(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| // Tests that blocks with numbers much lower or higher than out current head get
 | ||||
| // discarded no prevent wasting resources on useless blocks from faulty peers.
 | ||||
| func TestDistantDiscarding(t *testing.T) { | ||||
| 	// Create a long chain to import
 | ||||
| // discarded to prevent wasting resources on useless blocks from faulty peers.
 | ||||
| func TestDistantPropagationDiscarding(t *testing.T) { | ||||
| 	// Create a long chain to import and define the discard boundaries
 | ||||
| 	hashes, blocks := makeChain(3*maxQueueDist, 0, genesis) | ||||
| 	head := hashes[len(hashes)/2] | ||||
| 
 | ||||
| 	low, high := len(hashes)/2+maxUncleDist+1, len(hashes)/2-maxQueueDist-1 | ||||
| 
 | ||||
| 	// Create a tester and simulate a head block being the middle of the above chain
 | ||||
| 	tester := newTester() | ||||
| 	tester.hashes = []common.Hash{head} | ||||
| 	tester.blocks = map[common.Hash]*types.Block{head: blocks[head]} | ||||
| 
 | ||||
| 	// Ensure that a block with a lower number than the threshold is discarded
 | ||||
| 	tester.fetcher.Enqueue("lower", blocks[hashes[0]]) | ||||
| 	tester.fetcher.Enqueue("lower", blocks[hashes[low]]) | ||||
| 	time.Sleep(10 * time.Millisecond) | ||||
| 	if !tester.fetcher.queue.Empty() { | ||||
| 		t.Fatalf("fetcher queued stale block") | ||||
| 	} | ||||
| 	// Ensure that a block with a higher number than the threshold is discarded
 | ||||
| 	tester.fetcher.Enqueue("higher", blocks[hashes[len(hashes)-1]]) | ||||
| 	tester.fetcher.Enqueue("higher", blocks[hashes[high]]) | ||||
| 	time.Sleep(10 * time.Millisecond) | ||||
| 	if !tester.fetcher.queue.Empty() { | ||||
| 		t.Fatalf("fetcher queued future block") | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that announcements with numbers much lower or higher than out current
 | ||||
| // head get discarded to prevent wasting resources on useless blocks from faulty
 | ||||
| // peers.
 | ||||
| func TestDistantAnnouncementDiscarding62(t *testing.T) { testDistantAnnouncementDiscarding(t, 62) } | ||||
| func TestDistantAnnouncementDiscarding63(t *testing.T) { testDistantAnnouncementDiscarding(t, 63) } | ||||
| func TestDistantAnnouncementDiscarding64(t *testing.T) { testDistantAnnouncementDiscarding(t, 64) } | ||||
| 
 | ||||
| func testDistantAnnouncementDiscarding(t *testing.T, protocol int) { | ||||
| 	// Create a long chain to import and define the discard boundaries
 | ||||
| 	hashes, blocks := makeChain(3*maxQueueDist, 0, genesis) | ||||
| 	head := hashes[len(hashes)/2] | ||||
| 
 | ||||
| 	low, high := len(hashes)/2+maxUncleDist+1, len(hashes)/2-maxQueueDist-1 | ||||
| 
 | ||||
| 	// Create a tester and simulate a head block being the middle of the above chain
 | ||||
| 	tester := newTester() | ||||
| 	tester.hashes = []common.Hash{head} | ||||
| 	tester.blocks = map[common.Hash]*types.Block{head: blocks[head]} | ||||
| 
 | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	fetching := make(chan struct{}, 2) | ||||
| 	tester.fetcher.fetchingHook = func(hashes []common.Hash) { fetching <- struct{}{} } | ||||
| 
 | ||||
| 	// Ensure that a block with a lower number than the threshold is discarded
 | ||||
| 	tester.fetcher.Notify("lower", hashes[low], blocks[hashes[low]].NumberU64(), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	select { | ||||
| 	case <-time.After(50 * time.Millisecond): | ||||
| 	case <-fetching: | ||||
| 		t.Fatalf("fetcher requested stale header") | ||||
| 	} | ||||
| 	// Ensure that a block with a higher number than the threshold is discarded
 | ||||
| 	tester.fetcher.Notify("higher", hashes[high], blocks[hashes[high]].NumberU64(), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	select { | ||||
| 	case <-time.After(50 * time.Millisecond): | ||||
| 	case <-fetching: | ||||
| 		t.Fatalf("fetcher requested future header") | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that peers announcing blocks with invalid numbers (i.e. not matching
 | ||||
| // the headers provided afterwards) get dropped as malicious.
 | ||||
| func TestInvalidNumberAnnouncement62(t *testing.T) { testInvalidNumberAnnouncement(t, 62) } | ||||
| func TestInvalidNumberAnnouncement63(t *testing.T) { testInvalidNumberAnnouncement(t, 63) } | ||||
| func TestInvalidNumberAnnouncement64(t *testing.T) { testInvalidNumberAnnouncement(t, 64) } | ||||
| 
 | ||||
| func testInvalidNumberAnnouncement(t *testing.T, protocol int) { | ||||
| 	// Create a single block to import and check numbers against
 | ||||
| 	hashes, blocks := makeChain(1, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	imported := make(chan *types.Block) | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	// Announce a block with a bad number, check for immediate drop
 | ||||
| 	tester.fetcher.Notify("bad", hashes[0], 2, time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	verifyImportEvent(t, imported, false) | ||||
| 
 | ||||
| 	if !tester.drops["bad"] { | ||||
| 		t.Fatalf("peer with invalid numbered announcement not dropped") | ||||
| 	} | ||||
| 	// Make sure a good announcement passes without a drop
 | ||||
| 	tester.fetcher.Notify("good", hashes[0], 1, time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 	verifyImportEvent(t, imported, true) | ||||
| 
 | ||||
| 	if tester.drops["good"] { | ||||
| 		t.Fatalf("peer with valid numbered announcement dropped") | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| } | ||||
| 
 | ||||
| // Tests that if a block is empty (i.e. header only), no body request should be
 | ||||
| // made, and instead the header should be assembled into a whole block in itself.
 | ||||
| func TestEmptyBlockShortCircuit62(t *testing.T) { testEmptyBlockShortCircuit(t, 62) } | ||||
| func TestEmptyBlockShortCircuit63(t *testing.T) { testEmptyBlockShortCircuit(t, 63) } | ||||
| func TestEmptyBlockShortCircuit64(t *testing.T) { testEmptyBlockShortCircuit(t, 64) } | ||||
| 
 | ||||
| func testEmptyBlockShortCircuit(t *testing.T, protocol int) { | ||||
| 	// Create a chain of blocks to import
 | ||||
| 	hashes, blocks := makeChain(32, 0, genesis) | ||||
| 
 | ||||
| 	tester := newTester() | ||||
| 	headerFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	bodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	// Add a monitoring hook for all internal events
 | ||||
| 	fetching := make(chan []common.Hash) | ||||
| 	tester.fetcher.fetchingHook = func(hashes []common.Hash) { fetching <- hashes } | ||||
| 
 | ||||
| 	completing := make(chan []common.Hash) | ||||
| 	tester.fetcher.completingHook = func(hashes []common.Hash) { completing <- hashes } | ||||
| 
 | ||||
| 	imported := make(chan *types.Block) | ||||
| 	tester.fetcher.importedHook = func(block *types.Block) { imported <- block } | ||||
| 
 | ||||
| 	// Iteratively announce blocks until all are imported
 | ||||
| 	for i := len(hashes) - 2; i >= 0; i-- { | ||||
| 		tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, headerFetcher, bodyFetcher) | ||||
| 
 | ||||
| 		// All announces should fetch the header
 | ||||
| 		verifyFetchingEvent(t, fetching, true) | ||||
| 
 | ||||
| 		// Only blocks with data contents should request bodies
 | ||||
| 		verifyCompletingEvent(t, completing, len(blocks[hashes[i]].Transactions()) > 0 || len(blocks[hashes[i]].Uncles()) > 0) | ||||
| 
 | ||||
| 		// Irrelevant of the construct, import should succeed
 | ||||
| 		verifyImportEvent(t, imported, true) | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| } | ||||
| 
 | ||||
| // Tests that a peer is unable to use unbounded memory with sending infinite
 | ||||
| // block announcements to a node, but that even in the face of such an attack,
 | ||||
| // the fetcher remains operational.
 | ||||
| func TestHashMemoryExhaustionAttack(t *testing.T) { | ||||
| func TestHashMemoryExhaustionAttack61(t *testing.T) { testHashMemoryExhaustionAttack(t, 61) } | ||||
| func TestHashMemoryExhaustionAttack62(t *testing.T) { testHashMemoryExhaustionAttack(t, 62) } | ||||
| func TestHashMemoryExhaustionAttack63(t *testing.T) { testHashMemoryExhaustionAttack(t, 63) } | ||||
| func TestHashMemoryExhaustionAttack64(t *testing.T) { testHashMemoryExhaustionAttack(t, 64) } | ||||
| 
 | ||||
| func testHashMemoryExhaustionAttack(t *testing.T, protocol int) { | ||||
| 	// Create a tester with instrumented import hooks
 | ||||
| 	tester := newTester() | ||||
| 
 | ||||
| @ -429,17 +758,29 @@ func TestHashMemoryExhaustionAttack(t *testing.T) { | ||||
| 	// Create a valid chain and an infinite junk chain
 | ||||
| 	targetBlocks := hashLimit + 2*maxQueueDist | ||||
| 	hashes, blocks := makeChain(targetBlocks, 0, genesis) | ||||
| 	valid := tester.makeFetcher(blocks) | ||||
| 	validBlockFetcher := tester.makeBlockFetcher(blocks) | ||||
| 	validHeaderFetcher := tester.makeHeaderFetcher(blocks, -gatherSlack) | ||||
| 	validBodyFetcher := tester.makeBodyFetcher(blocks, 0) | ||||
| 
 | ||||
| 	attack, _ := makeChain(targetBlocks, 0, unknownBlock) | ||||
| 	attacker := tester.makeFetcher(nil) | ||||
| 	attackerBlockFetcher := tester.makeBlockFetcher(nil) | ||||
| 	attackerHeaderFetcher := tester.makeHeaderFetcher(nil, -gatherSlack) | ||||
| 	attackerBodyFetcher := tester.makeBodyFetcher(nil, 0) | ||||
| 
 | ||||
| 	// Feed the tester a huge hashset from the attacker, and a limited from the valid peer
 | ||||
| 	for i := 0; i < len(attack); i++ { | ||||
| 		if i < maxQueueDist { | ||||
| 			tester.fetcher.Notify("valid", hashes[len(hashes)-2-i], time.Now(), valid) | ||||
| 			if protocol < 62 { | ||||
| 				tester.fetcher.Notify("valid", hashes[len(hashes)-2-i], 0, time.Now(), validBlockFetcher, nil, nil) | ||||
| 			} else { | ||||
| 				tester.fetcher.Notify("valid", hashes[len(hashes)-2-i], uint64(i+1), time.Now(), nil, validHeaderFetcher, validBodyFetcher) | ||||
| 			} | ||||
| 		} | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("attacker", attack[i], 0, time.Now(), attackerBlockFetcher, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("attacker", attack[i], 1 /* don't distance drop */, time.Now(), nil, attackerHeaderFetcher, attackerBodyFetcher) | ||||
| 		} | ||||
| 		tester.fetcher.Notify("attacker", attack[i], time.Now(), attacker) | ||||
| 	} | ||||
| 	if len(tester.fetcher.announced) != hashLimit+maxQueueDist { | ||||
| 		t.Fatalf("queued announce count mismatch: have %d, want %d", len(tester.fetcher.announced), hashLimit+maxQueueDist) | ||||
| @ -449,8 +790,12 @@ func TestHashMemoryExhaustionAttack(t *testing.T) { | ||||
| 
 | ||||
| 	// Feed the remaining valid hashes to ensure DOS protection state remains clean
 | ||||
| 	for i := len(hashes) - maxQueueDist - 2; i >= 0; i-- { | ||||
| 		tester.fetcher.Notify("valid", hashes[i], time.Now().Add(-arriveTimeout), valid) | ||||
| 		verifyImportEvent(t, imported) | ||||
| 		if protocol < 62 { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], 0, time.Now().Add(-arriveTimeout), validBlockFetcher, nil, nil) | ||||
| 		} else { | ||||
| 			tester.fetcher.Notify("valid", hashes[i], uint64(len(hashes)-i-1), time.Now().Add(-arriveTimeout), nil, validHeaderFetcher, validBodyFetcher) | ||||
| 		} | ||||
| 		verifyImportEvent(t, imported, true) | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| } | ||||
| @ -498,7 +843,7 @@ func TestBlockMemoryExhaustionAttack(t *testing.T) { | ||||
| 	// Insert the remaining blocks in chunks to ensure clean DOS protection
 | ||||
| 	for i := maxQueueDist; i < len(hashes)-1; i++ { | ||||
| 		tester.fetcher.Enqueue("valid", blocks[hashes[len(hashes)-2-i]]) | ||||
| 		verifyImportEvent(t, imported) | ||||
| 		verifyImportEvent(t, imported, true) | ||||
| 	} | ||||
| 	verifyImportDone(t, imported) | ||||
| } | ||||
|  | ||||
| @ -23,10 +23,24 @@ import ( | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	announceMeter  = metrics.NewMeter("eth/sync/RemoteAnnounces") | ||||
| 	announceTimer  = metrics.NewTimer("eth/sync/LocalAnnounces") | ||||
| 	broadcastMeter = metrics.NewMeter("eth/sync/RemoteBroadcasts") | ||||
| 	broadcastTimer = metrics.NewTimer("eth/sync/LocalBroadcasts") | ||||
| 	discardMeter   = metrics.NewMeter("eth/sync/DiscardedBlocks") | ||||
| 	futureMeter    = metrics.NewMeter("eth/sync/FutureBlocks") | ||||
| 	propAnnounceInMeter   = metrics.NewMeter("eth/fetcher/prop/announces/in") | ||||
| 	propAnnounceOutTimer  = metrics.NewTimer("eth/fetcher/prop/announces/out") | ||||
| 	propAnnounceDropMeter = metrics.NewMeter("eth/fetcher/prop/announces/drop") | ||||
| 	propAnnounceDOSMeter  = metrics.NewMeter("eth/fetcher/prop/announces/dos") | ||||
| 
 | ||||
| 	propBroadcastInMeter   = metrics.NewMeter("eth/fetcher/prop/broadcasts/in") | ||||
| 	propBroadcastOutTimer  = metrics.NewTimer("eth/fetcher/prop/broadcasts/out") | ||||
| 	propBroadcastDropMeter = metrics.NewMeter("eth/fetcher/prop/broadcasts/drop") | ||||
| 	propBroadcastDOSMeter  = metrics.NewMeter("eth/fetcher/prop/broadcasts/dos") | ||||
| 
 | ||||
| 	blockFetchMeter  = metrics.NewMeter("eth/fetcher/fetch/blocks") | ||||
| 	headerFetchMeter = metrics.NewMeter("eth/fetcher/fetch/headers") | ||||
| 	bodyFetchMeter   = metrics.NewMeter("eth/fetcher/fetch/bodies") | ||||
| 
 | ||||
| 	blockFilterInMeter   = metrics.NewMeter("eth/fetcher/filter/blocks/in") | ||||
| 	blockFilterOutMeter  = metrics.NewMeter("eth/fetcher/filter/blocks/out") | ||||
| 	headerFilterInMeter  = metrics.NewMeter("eth/fetcher/filter/headers/in") | ||||
| 	headerFilterOutMeter = metrics.NewMeter("eth/fetcher/filter/headers/out") | ||||
| 	bodyFilterInMeter    = metrics.NewMeter("eth/fetcher/filter/bodies/in") | ||||
| 	bodyFilterOutMeter   = metrics.NewMeter("eth/fetcher/filter/bodies/out") | ||||
| ) | ||||
|  | ||||
							
								
								
									
										346
									
								
								eth/handler.go
									
									
									
									
									
								
							
							
						
						
									
										346
									
								
								eth/handler.go
									
									
									
									
									
								
							| @ -36,10 +36,8 @@ import ( | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // This is the target maximum size of returned blocks for the
 | ||||
| // getBlocks message. The reply message may exceed it
 | ||||
| // if a single block is larger than the limit.
 | ||||
| const maxBlockRespSize = 2 * 1024 * 1024 | ||||
| // This is the target maximum size of returned blocks, headers or node data.
 | ||||
| const softResponseLimit = 2 * 1024 * 1024 | ||||
| 
 | ||||
| func errResp(code errCode, format string, v ...interface{}) error { | ||||
| 	return fmt.Errorf("%v - %v", code, fmt.Sprintf(format, v...)) | ||||
| @ -59,12 +57,13 @@ func (ep extProt) GetHashes(hash common.Hash) error    { return ep.getHashes(has | ||||
| func (ep extProt) GetBlock(hashes []common.Hash) error { return ep.getBlocks(hashes) } | ||||
| 
 | ||||
| type ProtocolManager struct { | ||||
| 	protVer, netId int | ||||
| 	txpool         txPool | ||||
| 	chainman       *core.ChainManager | ||||
| 	downloader     *downloader.Downloader | ||||
| 	fetcher        *fetcher.Fetcher | ||||
| 	peers          *peerSet | ||||
| 	txpool   txPool | ||||
| 	chainman *core.ChainManager | ||||
| 	chaindb  common.Database | ||||
| 
 | ||||
| 	downloader *downloader.Downloader | ||||
| 	fetcher    *fetcher.Fetcher | ||||
| 	peers      *peerSet | ||||
| 
 | ||||
| 	SubProtocols []p2p.Protocol | ||||
| 
 | ||||
| @ -85,17 +84,17 @@ type ProtocolManager struct { | ||||
| 
 | ||||
| // NewProtocolManager returns a new ethereum sub protocol manager. The Ethereum sub protocol manages peers capable
 | ||||
| // with the ethereum network.
 | ||||
| func NewProtocolManager(networkId int, mux *event.TypeMux, txpool txPool, pow pow.PoW, chainman *core.ChainManager) *ProtocolManager { | ||||
| func NewProtocolManager(networkId int, mux *event.TypeMux, txpool txPool, pow pow.PoW, chainman *core.ChainManager, chaindb common.Database) *ProtocolManager { | ||||
| 	// Create the protocol manager with the base fields
 | ||||
| 	manager := &ProtocolManager{ | ||||
| 		eventMux:  mux, | ||||
| 		txpool:    txpool, | ||||
| 		chainman:  chainman, | ||||
| 		chaindb:   chaindb, | ||||
| 		peers:     newPeerSet(), | ||||
| 		newPeerCh: make(chan *peer, 1), | ||||
| 		txsyncCh:  make(chan *txsync), | ||||
| 		quitSync:  make(chan struct{}), | ||||
| 		netId:     networkId, | ||||
| 	} | ||||
| 	// Initiate a sub-protocol for every implemented version we can handle
 | ||||
| 	manager.SubProtocols = make([]p2p.Protocol, len(ProtocolVersions)) | ||||
| @ -176,7 +175,7 @@ func (pm *ProtocolManager) Stop() { | ||||
| } | ||||
| 
 | ||||
| func (pm *ProtocolManager) newPeer(pv, nv int, p *p2p.Peer, rw p2p.MsgReadWriter) *peer { | ||||
| 	return newPeer(pv, nv, p, rw) | ||||
| 	return newPeer(pv, nv, p, newMeteredMsgWriter(rw)) | ||||
| } | ||||
| 
 | ||||
| // handle is the callback invoked to manage the life cycle of an eth peer. When
 | ||||
| @ -190,6 +189,9 @@ func (pm *ProtocolManager) handle(p *peer) error { | ||||
| 		glog.V(logger.Debug).Infof("%v: handshake failed: %v", p, err) | ||||
| 		return err | ||||
| 	} | ||||
| 	if rw, ok := p.rw.(*meteredMsgReadWriter); ok { | ||||
| 		rw.Init(p.version) | ||||
| 	} | ||||
| 	// Register the peer locally
 | ||||
| 	glog.V(logger.Detail).Infof("%v: adding peer", p) | ||||
| 	if err := pm.peers.Register(p); err != nil { | ||||
| @ -199,7 +201,9 @@ func (pm *ProtocolManager) handle(p *peer) error { | ||||
| 	defer pm.removePeer(p.id) | ||||
| 
 | ||||
| 	// Register the peer in the downloader. If the downloader considers it banned, we disconnect
 | ||||
| 	if err := pm.downloader.RegisterPeer(p.id, p.version, p.Head(), p.RequestHashes, p.RequestHashesFromNumber, p.RequestBlocks); err != nil { | ||||
| 	if err := pm.downloader.RegisterPeer(p.id, p.version, p.Head(), | ||||
| 		p.RequestHashes, p.RequestHashesFromNumber, p.RequestBlocks, | ||||
| 		p.RequestHeadersByHash, p.RequestHeadersByNumber, p.RequestBodies); err != nil { | ||||
| 		return err | ||||
| 	} | ||||
| 	// Propagate existing transactions. new transactions appearing
 | ||||
| @ -230,12 +234,12 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 	defer msg.Discard() | ||||
| 
 | ||||
| 	// Handle the message depending on its contents
 | ||||
| 	switch msg.Code { | ||||
| 	case StatusMsg: | ||||
| 	switch { | ||||
| 	case msg.Code == StatusMsg: | ||||
| 		// Status messages should never arrive after the handshake
 | ||||
| 		return errResp(ErrExtraStatusMsg, "uncontrolled status message") | ||||
| 
 | ||||
| 	case GetBlockHashesMsg: | ||||
| 	case p.version < eth62 && msg.Code == GetBlockHashesMsg: | ||||
| 		// Retrieve the number of hashes to return and from which origin hash
 | ||||
| 		var request getBlockHashesData | ||||
| 		if err := msg.Decode(&request); err != nil { | ||||
| @ -251,7 +255,7 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 		} | ||||
| 		return p.SendBlockHashes(hashes) | ||||
| 
 | ||||
| 	case GetBlockHashesFromNumberMsg: | ||||
| 	case p.version < eth62 && msg.Code == GetBlockHashesFromNumberMsg: | ||||
| 		// Retrieve and decode the number of hashes to return and from which origin number
 | ||||
| 		var request getBlockHashesFromNumberData | ||||
| 		if err := msg.Decode(&request); err != nil { | ||||
| @ -278,24 +282,19 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 		} | ||||
| 		return p.SendBlockHashes(hashes) | ||||
| 
 | ||||
| 	case BlockHashesMsg: | ||||
| 	case p.version < eth62 && msg.Code == BlockHashesMsg: | ||||
| 		// A batch of hashes arrived to one of our previous requests
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		reqHashInPacketsMeter.Mark(1) | ||||
| 
 | ||||
| 		var hashes []common.Hash | ||||
| 		if err := msgStream.Decode(&hashes); err != nil { | ||||
| 		if err := msg.Decode(&hashes); err != nil { | ||||
| 			break | ||||
| 		} | ||||
| 		reqHashInTrafficMeter.Mark(int64(32 * len(hashes))) | ||||
| 
 | ||||
| 		// Deliver them all to the downloader for queuing
 | ||||
| 		err := pm.downloader.DeliverHashes(p.id, hashes) | ||||
| 		err := pm.downloader.DeliverHashes61(p.id, hashes) | ||||
| 		if err != nil { | ||||
| 			glog.V(logger.Debug).Infoln(err) | ||||
| 		} | ||||
| 
 | ||||
| 	case GetBlocksMsg: | ||||
| 	case p.version < eth62 && msg.Code == GetBlocksMsg: | ||||
| 		// Decode the retrieval message
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		if _, err := msgStream.List(); err != nil { | ||||
| @ -305,94 +304,279 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 		var ( | ||||
| 			hash   common.Hash | ||||
| 			bytes  common.StorageSize | ||||
| 			hashes []common.Hash | ||||
| 			blocks []*types.Block | ||||
| 		) | ||||
| 		for { | ||||
| 		for len(blocks) < downloader.MaxBlockFetch && bytes < softResponseLimit { | ||||
| 			//Retrieve the hash of the next block
 | ||||
| 			err := msgStream.Decode(&hash) | ||||
| 			if err == rlp.EOL { | ||||
| 				break | ||||
| 			} else if err != nil { | ||||
| 				return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 			} | ||||
| 			hashes = append(hashes, hash) | ||||
| 
 | ||||
| 			// Retrieve the requested block, stopping if enough was found
 | ||||
| 			if block := pm.chainman.GetBlock(hash); block != nil { | ||||
| 				blocks = append(blocks, block) | ||||
| 				bytes += block.Size() | ||||
| 				if len(blocks) >= downloader.MaxBlockFetch || bytes > maxBlockRespSize { | ||||
| 					break | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 		if glog.V(logger.Detail) && len(blocks) == 0 && len(hashes) > 0 { | ||||
| 			list := "[" | ||||
| 			for _, hash := range hashes { | ||||
| 				list += fmt.Sprintf("%x, ", hash[:4]) | ||||
| 			} | ||||
| 			list = list[:len(list)-2] + "]" | ||||
| 
 | ||||
| 			glog.Infof("%v: no blocks found for requested hashes %s", p, list) | ||||
| 		} | ||||
| 		return p.SendBlocks(blocks) | ||||
| 
 | ||||
| 	case BlocksMsg: | ||||
| 	case p.version < eth62 && msg.Code == BlocksMsg: | ||||
| 		// Decode the arrived block message
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		reqBlockInPacketsMeter.Mark(1) | ||||
| 
 | ||||
| 		var blocks []*types.Block | ||||
| 		if err := msgStream.Decode(&blocks); err != nil { | ||||
| 		if err := msg.Decode(&blocks); err != nil { | ||||
| 			glog.V(logger.Detail).Infoln("Decode error", err) | ||||
| 			blocks = nil | ||||
| 		} | ||||
| 		// Update the receive timestamp of each block
 | ||||
| 		for _, block := range blocks { | ||||
| 			reqBlockInTrafficMeter.Mark(block.Size().Int64()) | ||||
| 			block.ReceivedAt = msg.ReceivedAt | ||||
| 		} | ||||
| 		// Filter out any explicitly requested blocks, deliver the rest to the downloader
 | ||||
| 		if blocks := pm.fetcher.Filter(blocks); len(blocks) > 0 { | ||||
| 			pm.downloader.DeliverBlocks(p.id, blocks) | ||||
| 		if blocks := pm.fetcher.FilterBlocks(blocks); len(blocks) > 0 { | ||||
| 			pm.downloader.DeliverBlocks61(p.id, blocks) | ||||
| 		} | ||||
| 
 | ||||
| 	case NewBlockHashesMsg: | ||||
| 		// Retrieve and deseralize the remote new block hashes notification
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 
 | ||||
| 		var hashes []common.Hash | ||||
| 		if err := msgStream.Decode(&hashes); err != nil { | ||||
| 			break | ||||
| 	// Block header query, collect the requested headers and reply
 | ||||
| 	case p.version >= eth62 && msg.Code == GetBlockHeadersMsg: | ||||
| 		// Decode the complex header query
 | ||||
| 		var query getBlockHeadersData | ||||
| 		if err := msg.Decode(&query); err != nil { | ||||
| 			return errResp(ErrDecode, "%v: %v", msg, err) | ||||
| 		} | ||||
| 		propHashInPacketsMeter.Mark(1) | ||||
| 		propHashInTrafficMeter.Mark(int64(32 * len(hashes))) | ||||
| 		// Gather blocks until the fetch or network limits is reached
 | ||||
| 		var ( | ||||
| 			bytes   common.StorageSize | ||||
| 			headers []*types.Header | ||||
| 			unknown bool | ||||
| 		) | ||||
| 		for !unknown && len(headers) < int(query.Amount) && bytes < softResponseLimit && len(headers) < downloader.MaxHeaderFetch { | ||||
| 			// Retrieve the next block satisfying the query
 | ||||
| 			var origin *types.Block | ||||
| 			if query.Origin.Hash != (common.Hash{}) { | ||||
| 				origin = pm.chainman.GetBlock(query.Origin.Hash) | ||||
| 			} else { | ||||
| 				origin = pm.chainman.GetBlockByNumber(query.Origin.Number) | ||||
| 			} | ||||
| 			if origin == nil { | ||||
| 				break | ||||
| 			} | ||||
| 			headers = append(headers, origin.Header()) | ||||
| 			bytes += origin.Size() | ||||
| 
 | ||||
| 		// Mark the hashes as present at the remote node
 | ||||
| 		for _, hash := range hashes { | ||||
| 			p.MarkBlock(hash) | ||||
| 			p.SetHead(hash) | ||||
| 		} | ||||
| 		// Schedule all the unknown hashes for retrieval
 | ||||
| 		unknown := make([]common.Hash, 0, len(hashes)) | ||||
| 		for _, hash := range hashes { | ||||
| 			if !pm.chainman.HasBlock(hash) { | ||||
| 				unknown = append(unknown, hash) | ||||
| 			// Advance to the next block of the query
 | ||||
| 			switch { | ||||
| 			case query.Origin.Hash != (common.Hash{}) && query.Reverse: | ||||
| 				// Hash based traversal towards the genesis block
 | ||||
| 				for i := 0; i < int(query.Skip)+1; i++ { | ||||
| 					if block := pm.chainman.GetBlock(query.Origin.Hash); block != nil { | ||||
| 						query.Origin.Hash = block.ParentHash() | ||||
| 					} else { | ||||
| 						unknown = true | ||||
| 						break | ||||
| 					} | ||||
| 				} | ||||
| 			case query.Origin.Hash != (common.Hash{}) && !query.Reverse: | ||||
| 				// Hash based traversal towards the leaf block
 | ||||
| 				if block := pm.chainman.GetBlockByNumber(origin.NumberU64() + query.Skip + 1); block != nil { | ||||
| 					if pm.chainman.GetBlockHashesFromHash(block.Hash(), query.Skip+1)[query.Skip] == query.Origin.Hash { | ||||
| 						query.Origin.Hash = block.Hash() | ||||
| 					} else { | ||||
| 						unknown = true | ||||
| 					} | ||||
| 				} else { | ||||
| 					unknown = true | ||||
| 				} | ||||
| 			case query.Reverse: | ||||
| 				// Number based traversal towards the genesis block
 | ||||
| 				if query.Origin.Number >= query.Skip+1 { | ||||
| 					query.Origin.Number -= (query.Skip + 1) | ||||
| 				} else { | ||||
| 					unknown = true | ||||
| 				} | ||||
| 
 | ||||
| 			case !query.Reverse: | ||||
| 				// Number based traversal towards the leaf block
 | ||||
| 				query.Origin.Number += (query.Skip + 1) | ||||
| 			} | ||||
| 		} | ||||
| 		for _, hash := range unknown { | ||||
| 			pm.fetcher.Notify(p.id, hash, time.Now(), p.RequestBlocks) | ||||
| 		return p.SendBlockHeaders(headers) | ||||
| 
 | ||||
| 	case p.version >= eth62 && msg.Code == BlockHeadersMsg: | ||||
| 		// A batch of headers arrived to one of our previous requests
 | ||||
| 		var headers []*types.Header | ||||
| 		if err := msg.Decode(&headers); err != nil { | ||||
| 			return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 		} | ||||
| 		// Filter out any explicitly requested headers, deliver the rest to the downloader
 | ||||
| 		filter := len(headers) == 1 | ||||
| 		if filter { | ||||
| 			headers = pm.fetcher.FilterHeaders(headers, time.Now()) | ||||
| 		} | ||||
| 		if len(headers) > 0 || !filter { | ||||
| 			err := pm.downloader.DeliverHeaders(p.id, headers) | ||||
| 			if err != nil { | ||||
| 				glog.V(logger.Debug).Infoln(err) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 	case NewBlockMsg: | ||||
| 	case p.version >= eth62 && msg.Code == BlockBodiesMsg: | ||||
| 		// A batch of block bodies arrived to one of our previous requests
 | ||||
| 		var request blockBodiesData | ||||
| 		if err := msg.Decode(&request); err != nil { | ||||
| 			return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 		} | ||||
| 		// Deliver them all to the downloader for queuing
 | ||||
| 		trasactions := make([][]*types.Transaction, len(request)) | ||||
| 		uncles := make([][]*types.Header, len(request)) | ||||
| 
 | ||||
| 		for i, body := range request { | ||||
| 			trasactions[i] = body.Transactions | ||||
| 			uncles[i] = body.Uncles | ||||
| 		} | ||||
| 		// Filter out any explicitly requested bodies, deliver the rest to the downloader
 | ||||
| 		if trasactions, uncles := pm.fetcher.FilterBodies(trasactions, uncles, time.Now()); len(trasactions) > 0 || len(uncles) > 0 { | ||||
| 			err := pm.downloader.DeliverBodies(p.id, trasactions, uncles) | ||||
| 			if err != nil { | ||||
| 				glog.V(logger.Debug).Infoln(err) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 	case p.version >= eth62 && msg.Code == GetBlockBodiesMsg: | ||||
| 		// Decode the retrieval message
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		if _, err := msgStream.List(); err != nil { | ||||
| 			return err | ||||
| 		} | ||||
| 		// Gather blocks until the fetch or network limits is reached
 | ||||
| 		var ( | ||||
| 			hash   common.Hash | ||||
| 			bytes  common.StorageSize | ||||
| 			bodies []*blockBody | ||||
| 		) | ||||
| 		for bytes < softResponseLimit && len(bodies) < downloader.MaxBlockFetch { | ||||
| 			//Retrieve the hash of the next block
 | ||||
| 			if err := msgStream.Decode(&hash); err == rlp.EOL { | ||||
| 				break | ||||
| 			} else if err != nil { | ||||
| 				return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 			} | ||||
| 			// Retrieve the requested block, stopping if enough was found
 | ||||
| 			if block := pm.chainman.GetBlock(hash); block != nil { | ||||
| 				bodies = append(bodies, &blockBody{Transactions: block.Transactions(), Uncles: block.Uncles()}) | ||||
| 				bytes += block.Size() | ||||
| 			} | ||||
| 		} | ||||
| 		return p.SendBlockBodies(bodies) | ||||
| 
 | ||||
| 	case p.version >= eth63 && msg.Code == GetNodeDataMsg: | ||||
| 		// Decode the retrieval message
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		if _, err := msgStream.List(); err != nil { | ||||
| 			return err | ||||
| 		} | ||||
| 		// Gather state data until the fetch or network limits is reached
 | ||||
| 		var ( | ||||
| 			hash  common.Hash | ||||
| 			bytes int | ||||
| 			data  [][]byte | ||||
| 		) | ||||
| 		for bytes < softResponseLimit && len(data) < downloader.MaxStateFetch { | ||||
| 			// Retrieve the hash of the next state entry
 | ||||
| 			if err := msgStream.Decode(&hash); err == rlp.EOL { | ||||
| 				break | ||||
| 			} else if err != nil { | ||||
| 				return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 			} | ||||
| 			// Retrieve the requested state entry, stopping if enough was found
 | ||||
| 			if entry, err := pm.chaindb.Get(hash.Bytes()); err == nil { | ||||
| 				data = append(data, entry) | ||||
| 				bytes += len(entry) | ||||
| 			} | ||||
| 		} | ||||
| 		return p.SendNodeData(data) | ||||
| 
 | ||||
| 	case p.version >= eth63 && msg.Code == GetReceiptsMsg: | ||||
| 		// Decode the retrieval message
 | ||||
| 		msgStream := rlp.NewStream(msg.Payload, uint64(msg.Size)) | ||||
| 		if _, err := msgStream.List(); err != nil { | ||||
| 			return err | ||||
| 		} | ||||
| 		// Gather state data until the fetch or network limits is reached
 | ||||
| 		var ( | ||||
| 			hash     common.Hash | ||||
| 			bytes    int | ||||
| 			receipts []*types.Receipt | ||||
| 		) | ||||
| 		for bytes < softResponseLimit && len(receipts) < downloader.MaxReceiptsFetch { | ||||
| 			// Retrieve the hash of the next transaction receipt
 | ||||
| 			if err := msgStream.Decode(&hash); err == rlp.EOL { | ||||
| 				break | ||||
| 			} else if err != nil { | ||||
| 				return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 			} | ||||
| 			// Retrieve the requested receipt, stopping if enough was found
 | ||||
| 			if receipt := core.GetReceipt(pm.chaindb, hash); receipt != nil { | ||||
| 				receipts = append(receipts, receipt) | ||||
| 				bytes += len(receipt.RlpEncode()) | ||||
| 			} | ||||
| 		} | ||||
| 		return p.SendReceipts(receipts) | ||||
| 
 | ||||
| 	case msg.Code == NewBlockHashesMsg: | ||||
| 		// Retrieve and deseralize the remote new block hashes notification
 | ||||
| 		type announce struct { | ||||
| 			Hash   common.Hash | ||||
| 			Number uint64 | ||||
| 		} | ||||
| 		var announces = []announce{} | ||||
| 
 | ||||
| 		if p.version < eth62 { | ||||
| 			// We're running the old protocol, make block number unknown (0)
 | ||||
| 			var hashes []common.Hash | ||||
| 			if err := msg.Decode(&hashes); err != nil { | ||||
| 				return errResp(ErrDecode, "%v: %v", msg, err) | ||||
| 			} | ||||
| 			for _, hash := range hashes { | ||||
| 				announces = append(announces, announce{hash, 0}) | ||||
| 			} | ||||
| 		} else { | ||||
| 			// Otherwise extract both block hash and number
 | ||||
| 			var request newBlockHashesData | ||||
| 			if err := msg.Decode(&request); err != nil { | ||||
| 				return errResp(ErrDecode, "%v: %v", msg, err) | ||||
| 			} | ||||
| 			for _, block := range request { | ||||
| 				announces = append(announces, announce{block.Hash, block.Number}) | ||||
| 			} | ||||
| 		} | ||||
| 		// Mark the hashes as present at the remote node
 | ||||
| 		for _, block := range announces { | ||||
| 			p.MarkBlock(block.Hash) | ||||
| 			p.SetHead(block.Hash) | ||||
| 		} | ||||
| 		// Schedule all the unknown hashes for retrieval
 | ||||
| 		unknown := make([]announce, 0, len(announces)) | ||||
| 		for _, block := range announces { | ||||
| 			if !pm.chainman.HasBlock(block.Hash) { | ||||
| 				unknown = append(unknown, block) | ||||
| 			} | ||||
| 		} | ||||
| 		for _, block := range unknown { | ||||
| 			if p.version < eth62 { | ||||
| 				pm.fetcher.Notify(p.id, block.Hash, block.Number, time.Now(), p.RequestBlocks, nil, nil) | ||||
| 			} else { | ||||
| 				pm.fetcher.Notify(p.id, block.Hash, block.Number, time.Now(), nil, p.RequestOneHeader, p.RequestBodies) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 	case msg.Code == NewBlockMsg: | ||||
| 		// Retrieve and decode the propagated block
 | ||||
| 		var request newBlockData | ||||
| 		if err := msg.Decode(&request); err != nil { | ||||
| 			return errResp(ErrDecode, "%v: %v", msg, err) | ||||
| 		} | ||||
| 		propBlockInPacketsMeter.Mark(1) | ||||
| 		propBlockInTrafficMeter.Mark(request.Block.Size().Int64()) | ||||
| 
 | ||||
| 		if err := request.Block.ValidateFields(); err != nil { | ||||
| 			return errResp(ErrDecode, "block validation %v: %v", msg, err) | ||||
| 		} | ||||
| @ -421,13 +605,12 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 	case TxMsg: | ||||
| 	case msg.Code == TxMsg: | ||||
| 		// Transactions arrived, parse all of them and deliver to the pool
 | ||||
| 		var txs []*types.Transaction | ||||
| 		if err := msg.Decode(&txs); err != nil { | ||||
| 			return errResp(ErrDecode, "msg %v: %v", msg, err) | ||||
| 		} | ||||
| 		propTxnInPacketsMeter.Mark(1) | ||||
| 		for i, tx := range txs { | ||||
| 			// Validate and mark the remote transaction
 | ||||
| 			if tx == nil { | ||||
| @ -436,7 +619,6 @@ func (pm *ProtocolManager) handleMsg(p *peer) error { | ||||
| 			p.MarkTransaction(tx.Hash()) | ||||
| 
 | ||||
| 			// Log it's arrival for later analysis
 | ||||
| 			propTxnInTrafficMeter.Mark(tx.Size().Int64()) | ||||
| 			jsonlogger.LogJson(&logger.EthTxReceived{ | ||||
| 				TxHash:   tx.Hash().Hex(), | ||||
| 				RemoteId: p.ID().String(), | ||||
| @ -476,7 +658,11 @@ func (pm *ProtocolManager) BroadcastBlock(block *types.Block, propagate bool) { | ||||
| 	// Otherwise if the block is indeed in out own chain, announce it
 | ||||
| 	if pm.chainman.HasBlock(hash) { | ||||
| 		for _, peer := range peers { | ||||
| 			peer.SendNewBlockHashes([]common.Hash{hash}) | ||||
| 			if peer.version < eth62 { | ||||
| 				peer.SendNewBlockHashes61([]common.Hash{hash}) | ||||
| 			} else { | ||||
| 				peer.SendNewBlockHashes([]common.Hash{hash}, []uint64{block.NumberU64()}) | ||||
| 			} | ||||
| 		} | ||||
| 		glog.V(logger.Detail).Infof("announced block %x to %d peers in %v", hash[:4], len(peers), time.Since(block.ReceivedAt)) | ||||
| 	} | ||||
|  | ||||
							
								
								
									
										522
									
								
								eth/handler_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										522
									
								
								eth/handler_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,522 @@ | ||||
| package eth | ||||
| 
 | ||||
| import ( | ||||
| 	"fmt" | ||||
| 	"math/big" | ||||
| 	"math/rand" | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/core" | ||||
| 	"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/eth/downloader" | ||||
| 	"github.com/ethereum/go-ethereum/ethdb" | ||||
| 	"github.com/ethereum/go-ethereum/p2p" | ||||
| 	"github.com/ethereum/go-ethereum/params" | ||||
| ) | ||||
| 
 | ||||
| // Tests that hashes can be retrieved from a remote chain by hashes in reverse
 | ||||
| // order.
 | ||||
| func TestGetBlockHashes61(t *testing.T) { testGetBlockHashes(t, 61) } | ||||
| 
 | ||||
| func testGetBlockHashes(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(downloader.MaxHashFetch+15, nil, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Create a batch of tests for various scenarios
 | ||||
| 	limit := downloader.MaxHashFetch | ||||
| 	tests := []struct { | ||||
| 		origin common.Hash | ||||
| 		number int | ||||
| 		result int | ||||
| 	}{ | ||||
| 		{common.Hash{}, 1, 0},                                 // Make sure non existent hashes don't return results
 | ||||
| 		{pm.chainman.Genesis().Hash(), 1, 0},                  // There are no hashes to retrieve up from the genesis
 | ||||
| 		{pm.chainman.GetBlockByNumber(5).Hash(), 5, 5},        // All the hashes including the genesis requested
 | ||||
| 		{pm.chainman.GetBlockByNumber(5).Hash(), 10, 5},       // More hashes than available till the genesis requested
 | ||||
| 		{pm.chainman.GetBlockByNumber(100).Hash(), 10, 10},    // All hashes available from the middle of the chain
 | ||||
| 		{pm.chainman.CurrentBlock().Hash(), 10, 10},           // All hashes available from the head of the chain
 | ||||
| 		{pm.chainman.CurrentBlock().Hash(), limit, limit},     // Request the maximum allowed hash count
 | ||||
| 		{pm.chainman.CurrentBlock().Hash(), limit + 1, limit}, // Request more than the maximum allowed hash count
 | ||||
| 	} | ||||
| 	// Run each of the tests and verify the results against the chain
 | ||||
| 	for i, tt := range tests { | ||||
| 		// Assemble the hash response we would like to receive
 | ||||
| 		resp := make([]common.Hash, tt.result) | ||||
| 		if len(resp) > 0 { | ||||
| 			from := pm.chainman.GetBlock(tt.origin).NumberU64() - 1 | ||||
| 			for j := 0; j < len(resp); j++ { | ||||
| 				resp[j] = pm.chainman.GetBlockByNumber(uint64(int(from) - j)).Hash() | ||||
| 			} | ||||
| 		} | ||||
| 		// Send the hash request and verify the response
 | ||||
| 		p2p.Send(peer.app, 0x03, getBlockHashesData{tt.origin, uint64(tt.number)}) | ||||
| 		if err := p2p.ExpectMsg(peer.app, 0x04, resp); err != nil { | ||||
| 			t.Errorf("test %d: block hashes mismatch: %v", i, err) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that hashes can be retrieved from a remote chain by numbers in forward
 | ||||
| // order.
 | ||||
| func TestGetBlockHashesFromNumber61(t *testing.T) { testGetBlockHashesFromNumber(t, 61) } | ||||
| 
 | ||||
| func testGetBlockHashesFromNumber(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(downloader.MaxHashFetch+15, nil, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Create a batch of tests for various scenarios
 | ||||
| 	limit := downloader.MaxHashFetch | ||||
| 	tests := []struct { | ||||
| 		origin uint64 | ||||
| 		number int | ||||
| 		result int | ||||
| 	}{ | ||||
| 		{pm.chainman.CurrentBlock().NumberU64() + 1, 1, 0},     // Out of bounds requests should return empty
 | ||||
| 		{pm.chainman.CurrentBlock().NumberU64(), 1, 1},         // Make sure the head hash can be retrieved
 | ||||
| 		{pm.chainman.CurrentBlock().NumberU64() - 4, 5, 5},     // All hashes, including the head hash requested
 | ||||
| 		{pm.chainman.CurrentBlock().NumberU64() - 4, 10, 5},    // More hashes requested than available till the head
 | ||||
| 		{pm.chainman.CurrentBlock().NumberU64() - 100, 10, 10}, // All hashes available from the middle of the chain
 | ||||
| 		{0, 10, 10},           // All hashes available from the root of the chain
 | ||||
| 		{0, limit, limit},     // Request the maximum allowed hash count
 | ||||
| 		{0, limit + 1, limit}, // Request more than the maximum allowed hash count
 | ||||
| 		{0, 1, 1},             // Make sure the genesis hash can be retrieved
 | ||||
| 	} | ||||
| 	// Run each of the tests and verify the results against the chain
 | ||||
| 	for i, tt := range tests { | ||||
| 		// Assemble the hash response we would like to receive
 | ||||
| 		resp := make([]common.Hash, tt.result) | ||||
| 		for j := 0; j < len(resp); j++ { | ||||
| 			resp[j] = pm.chainman.GetBlockByNumber(tt.origin + uint64(j)).Hash() | ||||
| 		} | ||||
| 		// Send the hash request and verify the response
 | ||||
| 		p2p.Send(peer.app, 0x08, getBlockHashesFromNumberData{tt.origin, uint64(tt.number)}) | ||||
| 		if err := p2p.ExpectMsg(peer.app, 0x04, resp); err != nil { | ||||
| 			t.Errorf("test %d: block hashes mismatch: %v", i, err) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that blocks can be retrieved from a remote chain based on their hashes.
 | ||||
| func TestGetBlocks61(t *testing.T) { testGetBlocks(t, 61) } | ||||
| 
 | ||||
| func testGetBlocks(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(downloader.MaxHashFetch+15, nil, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Create a batch of tests for various scenarios
 | ||||
| 	limit := downloader.MaxBlockFetch | ||||
| 	tests := []struct { | ||||
| 		random    int           // Number of blocks to fetch randomly from the chain
 | ||||
| 		explicit  []common.Hash // Explicitly requested blocks
 | ||||
| 		available []bool        // Availability of explicitly requested blocks
 | ||||
| 		expected  int           // Total number of existing blocks to expect
 | ||||
| 	}{ | ||||
| 		{1, nil, nil, 1},                                                       // A single random block should be retrievable
 | ||||
| 		{10, nil, nil, 10},                                                     // Multiple random blocks should be retrievable
 | ||||
| 		{limit, nil, nil, limit},                                               // The maximum possible blocks should be retrievable
 | ||||
| 		{limit + 1, nil, nil, limit},                                           // No more that the possible block count should be returned
 | ||||
| 		{0, []common.Hash{pm.chainman.Genesis().Hash()}, []bool{true}, 1},      // The genesis block should be retrievable
 | ||||
| 		{0, []common.Hash{pm.chainman.CurrentBlock().Hash()}, []bool{true}, 1}, // The chains head block should be retrievable
 | ||||
| 		{0, []common.Hash{common.Hash{}}, []bool{false}, 0},                    // A non existent block should not be returned
 | ||||
| 
 | ||||
| 		// Existing and non-existing blocks interleaved should not cause problems
 | ||||
| 		{0, []common.Hash{ | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(1).Hash(), | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(10).Hash(), | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(100).Hash(), | ||||
| 			common.Hash{}, | ||||
| 		}, []bool{false, true, false, true, false, true, false}, 3}, | ||||
| 	} | ||||
| 	// Run each of the tests and verify the results against the chain
 | ||||
| 	for i, tt := range tests { | ||||
| 		// Collect the hashes to request, and the response to expect
 | ||||
| 		hashes, seen := []common.Hash{}, make(map[int64]bool) | ||||
| 		blocks := []*types.Block{} | ||||
| 
 | ||||
| 		for j := 0; j < tt.random; j++ { | ||||
| 			for { | ||||
| 				num := rand.Int63n(int64(pm.chainman.CurrentBlock().NumberU64())) | ||||
| 				if !seen[num] { | ||||
| 					seen[num] = true | ||||
| 
 | ||||
| 					block := pm.chainman.GetBlockByNumber(uint64(num)) | ||||
| 					hashes = append(hashes, block.Hash()) | ||||
| 					if len(blocks) < tt.expected { | ||||
| 						blocks = append(blocks, block) | ||||
| 					} | ||||
| 					break | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 		for j, hash := range tt.explicit { | ||||
| 			hashes = append(hashes, hash) | ||||
| 			if tt.available[j] && len(blocks) < tt.expected { | ||||
| 				blocks = append(blocks, pm.chainman.GetBlock(hash)) | ||||
| 			} | ||||
| 		} | ||||
| 		// Send the hash request and verify the response
 | ||||
| 		p2p.Send(peer.app, 0x05, hashes) | ||||
| 		if err := p2p.ExpectMsg(peer.app, 0x06, blocks); err != nil { | ||||
| 			t.Errorf("test %d: blocks mismatch: %v", i, err) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that block headers can be retrieved from a remote chain based on user queries.
 | ||||
| func TestGetBlockHeaders62(t *testing.T) { testGetBlockHeaders(t, 62) } | ||||
| func TestGetBlockHeaders63(t *testing.T) { testGetBlockHeaders(t, 63) } | ||||
| func TestGetBlockHeaders64(t *testing.T) { testGetBlockHeaders(t, 64) } | ||||
| 
 | ||||
| func testGetBlockHeaders(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(downloader.MaxHashFetch+15, nil, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Create a "random" unknown hash for testing
 | ||||
| 	var unknown common.Hash | ||||
| 	for i, _ := range unknown { | ||||
| 		unknown[i] = byte(i) | ||||
| 	} | ||||
| 	// Create a batch of tests for various scenarios
 | ||||
| 	limit := uint64(downloader.MaxHeaderFetch) | ||||
| 	tests := []struct { | ||||
| 		query  *getBlockHeadersData // The query to execute for header retrieval
 | ||||
| 		expect []common.Hash        // The hashes of the block whose headers are expected
 | ||||
| 	}{ | ||||
| 		// A single random block should be retrievable by hash and number too
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Hash: pm.chainman.GetBlockByNumber(limit / 2).Hash()}, Amount: 1}, | ||||
| 			[]common.Hash{pm.chainman.GetBlockByNumber(limit / 2).Hash()}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: limit / 2}, Amount: 1}, | ||||
| 			[]common.Hash{pm.chainman.GetBlockByNumber(limit / 2).Hash()}, | ||||
| 		}, | ||||
| 		// Multiple headers should be retrievable in both directions
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: limit / 2}, Amount: 3}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(limit / 2).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 + 1).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 + 2).Hash(), | ||||
| 			}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: limit / 2}, Amount: 3, Reverse: true}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(limit / 2).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 - 1).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 - 2).Hash(), | ||||
| 			}, | ||||
| 		}, | ||||
| 		// Multiple headers with skip lists should be retrievable
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: limit / 2}, Skip: 3, Amount: 3}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(limit / 2).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 + 4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 + 8).Hash(), | ||||
| 			}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: limit / 2}, Skip: 3, Amount: 3, Reverse: true}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(limit / 2).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 - 4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(limit/2 - 8).Hash(), | ||||
| 			}, | ||||
| 		}, | ||||
| 		// The chain endpoints should be retrievable
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: 0}, Amount: 1}, | ||||
| 			[]common.Hash{pm.chainman.GetBlockByNumber(0).Hash()}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: pm.chainman.CurrentBlock().NumberU64()}, Amount: 1}, | ||||
| 			[]common.Hash{pm.chainman.CurrentBlock().Hash()}, | ||||
| 		}, | ||||
| 		// Ensure protocol limits are honored
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: pm.chainman.CurrentBlock().NumberU64() - 1}, Amount: limit + 10, Reverse: true}, | ||||
| 			pm.chainman.GetBlockHashesFromHash(pm.chainman.CurrentBlock().Hash(), limit), | ||||
| 		}, | ||||
| 		// Check that requesting more than available is handled gracefully
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: pm.chainman.CurrentBlock().NumberU64() - 4}, Skip: 3, Amount: 3}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(pm.chainman.CurrentBlock().NumberU64() - 4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(pm.chainman.CurrentBlock().NumberU64()).Hash(), | ||||
| 			}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: 4}, Skip: 3, Amount: 3, Reverse: true}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(0).Hash(), | ||||
| 			}, | ||||
| 		}, | ||||
| 		// Check that requesting more than available is handled gracefully, even if mid skip
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: pm.chainman.CurrentBlock().NumberU64() - 4}, Skip: 2, Amount: 3}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(pm.chainman.CurrentBlock().NumberU64() - 4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(pm.chainman.CurrentBlock().NumberU64() - 1).Hash(), | ||||
| 			}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: 4}, Skip: 2, Amount: 3, Reverse: true}, | ||||
| 			[]common.Hash{ | ||||
| 				pm.chainman.GetBlockByNumber(4).Hash(), | ||||
| 				pm.chainman.GetBlockByNumber(1).Hash(), | ||||
| 			}, | ||||
| 		}, | ||||
| 		// Check that non existing headers aren't returned
 | ||||
| 		{ | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Hash: unknown}, Amount: 1}, | ||||
| 			[]common.Hash{}, | ||||
| 		}, { | ||||
| 			&getBlockHeadersData{Origin: hashOrNumber{Number: pm.chainman.CurrentBlock().NumberU64() + 1}, Amount: 1}, | ||||
| 			[]common.Hash{}, | ||||
| 		}, | ||||
| 	} | ||||
| 	// Run each of the tests and verify the results against the chain
 | ||||
| 	for i, tt := range tests { | ||||
| 		// Collect the headers to expect in the response
 | ||||
| 		headers := []*types.Header{} | ||||
| 		for _, hash := range tt.expect { | ||||
| 			headers = append(headers, pm.chainman.GetBlock(hash).Header()) | ||||
| 		} | ||||
| 		// Send the hash request and verify the response
 | ||||
| 		p2p.Send(peer.app, 0x03, tt.query) | ||||
| 		if err := p2p.ExpectMsg(peer.app, 0x04, headers); err != nil { | ||||
| 			t.Errorf("test %d: headers mismatch: %v", i, err) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that block contents can be retrieved from a remote chain based on their hashes.
 | ||||
| func TestGetBlockBodies62(t *testing.T) { testGetBlockBodies(t, 62) } | ||||
| func TestGetBlockBodies63(t *testing.T) { testGetBlockBodies(t, 63) } | ||||
| func TestGetBlockBodies64(t *testing.T) { testGetBlockBodies(t, 64) } | ||||
| 
 | ||||
| func testGetBlockBodies(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(downloader.MaxBlockFetch+15, nil, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Create a batch of tests for various scenarios
 | ||||
| 	limit := downloader.MaxBlockFetch | ||||
| 	tests := []struct { | ||||
| 		random    int           // Number of blocks to fetch randomly from the chain
 | ||||
| 		explicit  []common.Hash // Explicitly requested blocks
 | ||||
| 		available []bool        // Availability of explicitly requested blocks
 | ||||
| 		expected  int           // Total number of existing blocks to expect
 | ||||
| 	}{ | ||||
| 		{1, nil, nil, 1},                                                       // A single random block should be retrievable
 | ||||
| 		{10, nil, nil, 10},                                                     // Multiple random blocks should be retrievable
 | ||||
| 		{limit, nil, nil, limit},                                               // The maximum possible blocks should be retrievable
 | ||||
| 		{limit + 1, nil, nil, limit},                                           // No more that the possible block count should be returned
 | ||||
| 		{0, []common.Hash{pm.chainman.Genesis().Hash()}, []bool{true}, 1},      // The genesis block should be retrievable
 | ||||
| 		{0, []common.Hash{pm.chainman.CurrentBlock().Hash()}, []bool{true}, 1}, // The chains head block should be retrievable
 | ||||
| 		{0, []common.Hash{common.Hash{}}, []bool{false}, 0},                    // A non existent block should not be returned
 | ||||
| 
 | ||||
| 		// Existing and non-existing blocks interleaved should not cause problems
 | ||||
| 		{0, []common.Hash{ | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(1).Hash(), | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(10).Hash(), | ||||
| 			common.Hash{}, | ||||
| 			pm.chainman.GetBlockByNumber(100).Hash(), | ||||
| 			common.Hash{}, | ||||
| 		}, []bool{false, true, false, true, false, true, false}, 3}, | ||||
| 	} | ||||
| 	// Run each of the tests and verify the results against the chain
 | ||||
| 	for i, tt := range tests { | ||||
| 		// Collect the hashes to request, and the response to expect
 | ||||
| 		hashes, seen := []common.Hash{}, make(map[int64]bool) | ||||
| 		bodies := []*blockBody{} | ||||
| 
 | ||||
| 		for j := 0; j < tt.random; j++ { | ||||
| 			for { | ||||
| 				num := rand.Int63n(int64(pm.chainman.CurrentBlock().NumberU64())) | ||||
| 				if !seen[num] { | ||||
| 					seen[num] = true | ||||
| 
 | ||||
| 					block := pm.chainman.GetBlockByNumber(uint64(num)) | ||||
| 					hashes = append(hashes, block.Hash()) | ||||
| 					if len(bodies) < tt.expected { | ||||
| 						bodies = append(bodies, &blockBody{Transactions: block.Transactions(), Uncles: block.Uncles()}) | ||||
| 					} | ||||
| 					break | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 		for j, hash := range tt.explicit { | ||||
| 			hashes = append(hashes, hash) | ||||
| 			if tt.available[j] && len(bodies) < tt.expected { | ||||
| 				block := pm.chainman.GetBlock(hash) | ||||
| 				bodies = append(bodies, &blockBody{Transactions: block.Transactions(), Uncles: block.Uncles()}) | ||||
| 			} | ||||
| 		} | ||||
| 		// Send the hash request and verify the response
 | ||||
| 		p2p.Send(peer.app, 0x05, hashes) | ||||
| 		if err := p2p.ExpectMsg(peer.app, 0x06, bodies); err != nil { | ||||
| 			t.Errorf("test %d: bodies mismatch: %v", i, err) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that the node state database can be retrieved based on hashes.
 | ||||
| func TestGetNodeData63(t *testing.T) { testGetNodeData(t, 63) } | ||||
| func TestGetNodeData64(t *testing.T) { testGetNodeData(t, 64) } | ||||
| 
 | ||||
| func testGetNodeData(t *testing.T, protocol int) { | ||||
| 	// Define three accounts to simulate transactions with
 | ||||
| 	acc1Key, _ := crypto.HexToECDSA("8a1f9a8f95be41cd7ccb6168179afb4504aefe388d1e14474d32c45c72ce7b7a") | ||||
| 	acc2Key, _ := crypto.HexToECDSA("49a7b37aa6f6645917e7b807e9d1c00d4fa71f18343b0d4122a4d2df64dd6fee") | ||||
| 	acc1Addr := crypto.PubkeyToAddress(acc1Key.PublicKey) | ||||
| 	acc2Addr := crypto.PubkeyToAddress(acc2Key.PublicKey) | ||||
| 
 | ||||
| 	// Create a chain generator with some simple transactions (blatantly stolen from @fjl/chain_makerts_test)
 | ||||
| 	generator := func(i int, block *core.BlockGen) { | ||||
| 		switch i { | ||||
| 		case 0: | ||||
| 			// In block 1, the test bank sends account #1 some ether.
 | ||||
| 			tx, _ := types.NewTransaction(block.TxNonce(testBankAddress), acc1Addr, big.NewInt(10000), params.TxGas, nil, nil).SignECDSA(testBankKey) | ||||
| 			block.AddTx(tx) | ||||
| 		case 1: | ||||
| 			// In block 2, the test bank sends some more ether to account #1.
 | ||||
| 			// acc1Addr passes it on to account #2.
 | ||||
| 			tx1, _ := types.NewTransaction(block.TxNonce(testBankAddress), acc1Addr, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(testBankKey) | ||||
| 			tx2, _ := types.NewTransaction(block.TxNonce(acc1Addr), acc2Addr, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(acc1Key) | ||||
| 			block.AddTx(tx1) | ||||
| 			block.AddTx(tx2) | ||||
| 		case 2: | ||||
| 			// Block 3 is empty but was mined by account #2.
 | ||||
| 			block.SetCoinbase(acc2Addr) | ||||
| 			block.SetExtra([]byte("yeehaw")) | ||||
| 		case 3: | ||||
| 			// Block 4 includes blocks 2 and 3 as uncle headers (with modified extra data).
 | ||||
| 			b2 := block.PrevBlock(1).Header() | ||||
| 			b2.Extra = []byte("foo") | ||||
| 			block.AddUncle(b2) | ||||
| 			b3 := block.PrevBlock(2).Header() | ||||
| 			b3.Extra = []byte("foo") | ||||
| 			block.AddUncle(b3) | ||||
| 		} | ||||
| 	} | ||||
| 	// Assemble the test environment
 | ||||
| 	pm := newTestProtocolManager(4, generator, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Fetch for now the entire chain db
 | ||||
| 	hashes := []common.Hash{} | ||||
| 	for _, key := range pm.chaindb.(*ethdb.MemDatabase).Keys() { | ||||
| 		hashes = append(hashes, common.BytesToHash(key)) | ||||
| 	} | ||||
| 	p2p.Send(peer.app, 0x0d, hashes) | ||||
| 	msg, err := peer.app.ReadMsg() | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("failed to read node data response: %v", err) | ||||
| 	} | ||||
| 	if msg.Code != 0x0e { | ||||
| 		t.Fatalf("response packet code mismatch: have %x, want %x", msg.Code, 0x0c) | ||||
| 	} | ||||
| 	var data [][]byte | ||||
| 	if err := msg.Decode(&data); err != nil { | ||||
| 		t.Fatalf("failed to decode response node data: %v", err) | ||||
| 	} | ||||
| 	// Verify that all hashes correspond to the requested data, and reconstruct a state tree
 | ||||
| 	for i, want := range hashes { | ||||
| 		if hash := crypto.Sha3Hash(data[i]); hash != want { | ||||
| 			fmt.Errorf("data hash mismatch: have %x, want %x", hash, want) | ||||
| 		} | ||||
| 	} | ||||
| 	statedb, _ := ethdb.NewMemDatabase() | ||||
| 	for i := 0; i < len(data); i++ { | ||||
| 		statedb.Put(hashes[i].Bytes(), data[i]) | ||||
| 	} | ||||
| 	accounts := []common.Address{testBankAddress, acc1Addr, acc2Addr} | ||||
| 	for i := uint64(0); i <= pm.chainman.CurrentBlock().NumberU64(); i++ { | ||||
| 		trie := state.New(pm.chainman.GetBlockByNumber(i).Root(), statedb) | ||||
| 
 | ||||
| 		for j, acc := range accounts { | ||||
| 			bw := pm.chainman.State().GetBalance(acc) | ||||
| 			bh := trie.GetBalance(acc) | ||||
| 
 | ||||
| 			if (bw != nil && bh == nil) || (bw == nil && bh != nil) { | ||||
| 				t.Errorf("test %d, account %d: balance mismatch: have %v, want %v", i, j, bh, bw) | ||||
| 			} | ||||
| 			if bw != nil && bh != nil && bw.Cmp(bw) != 0 { | ||||
| 				t.Errorf("test %d, account %d: balance mismatch: have %v, want %v", i, j, bh, bw) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Tests that the transaction receipts can be retrieved based on hashes.
 | ||||
| func TestGetReceipt63(t *testing.T) { testGetReceipt(t, 63) } | ||||
| func TestGetReceipt64(t *testing.T) { testGetReceipt(t, 64) } | ||||
| 
 | ||||
| func testGetReceipt(t *testing.T, protocol int) { | ||||
| 	// Define three accounts to simulate transactions with
 | ||||
| 	acc1Key, _ := crypto.HexToECDSA("8a1f9a8f95be41cd7ccb6168179afb4504aefe388d1e14474d32c45c72ce7b7a") | ||||
| 	acc2Key, _ := crypto.HexToECDSA("49a7b37aa6f6645917e7b807e9d1c00d4fa71f18343b0d4122a4d2df64dd6fee") | ||||
| 	acc1Addr := crypto.PubkeyToAddress(acc1Key.PublicKey) | ||||
| 	acc2Addr := crypto.PubkeyToAddress(acc2Key.PublicKey) | ||||
| 
 | ||||
| 	// Create a chain generator with some simple transactions (blatantly stolen from @fjl/chain_makerts_test)
 | ||||
| 	generator := func(i int, block *core.BlockGen) { | ||||
| 		switch i { | ||||
| 		case 0: | ||||
| 			// In block 1, the test bank sends account #1 some ether.
 | ||||
| 			tx, _ := types.NewTransaction(block.TxNonce(testBankAddress), acc1Addr, big.NewInt(10000), params.TxGas, nil, nil).SignECDSA(testBankKey) | ||||
| 			block.AddTx(tx) | ||||
| 		case 1: | ||||
| 			// In block 2, the test bank sends some more ether to account #1.
 | ||||
| 			// acc1Addr passes it on to account #2.
 | ||||
| 			tx1, _ := types.NewTransaction(block.TxNonce(testBankAddress), acc1Addr, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(testBankKey) | ||||
| 			tx2, _ := types.NewTransaction(block.TxNonce(acc1Addr), acc2Addr, big.NewInt(1000), params.TxGas, nil, nil).SignECDSA(acc1Key) | ||||
| 			block.AddTx(tx1) | ||||
| 			block.AddTx(tx2) | ||||
| 		case 2: | ||||
| 			// Block 3 is empty but was mined by account #2.
 | ||||
| 			block.SetCoinbase(acc2Addr) | ||||
| 			block.SetExtra([]byte("yeehaw")) | ||||
| 		case 3: | ||||
| 			// Block 4 includes blocks 2 and 3 as uncle headers (with modified extra data).
 | ||||
| 			b2 := block.PrevBlock(1).Header() | ||||
| 			b2.Extra = []byte("foo") | ||||
| 			block.AddUncle(b2) | ||||
| 			b3 := block.PrevBlock(2).Header() | ||||
| 			b3.Extra = []byte("foo") | ||||
| 			block.AddUncle(b3) | ||||
| 		} | ||||
| 	} | ||||
| 	// Assemble the test environment
 | ||||
| 	pm := newTestProtocolManager(4, generator, nil) | ||||
| 	peer, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer peer.close() | ||||
| 
 | ||||
| 	// Collect the hashes to request, and the response to expect
 | ||||
| 	hashes := []common.Hash{} | ||||
| 	for i := uint64(0); i <= pm.chainman.CurrentBlock().NumberU64(); i++ { | ||||
| 		for _, tx := range pm.chainman.GetBlockByNumber(i).Transactions() { | ||||
| 			hashes = append(hashes, tx.Hash()) | ||||
| 		} | ||||
| 	} | ||||
| 	receipts := make([]*types.Receipt, len(hashes)) | ||||
| 	for i, hash := range hashes { | ||||
| 		receipts[i] = core.GetReceipt(pm.chaindb, hash) | ||||
| 	} | ||||
| 	// Send the hash request and verify the response
 | ||||
| 	p2p.Send(peer.app, 0x0f, hashes) | ||||
| 	if err := p2p.ExpectMsg(peer.app, 0x10, receipts); err != nil { | ||||
| 		t.Errorf("receipts mismatch: %v", err) | ||||
| 	} | ||||
| } | ||||
							
								
								
									
										147
									
								
								eth/helper_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										147
									
								
								eth/helper_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,147 @@ | ||||
| // This file contains some shares testing functionality, common to  multiple
 | ||||
| // different files and modules being tested.
 | ||||
| 
 | ||||
| package eth | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| 	"math/big" | ||||
| 	"sync" | ||||
| 	"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/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/ethdb" | ||||
| 	"github.com/ethereum/go-ethereum/event" | ||||
| 	"github.com/ethereum/go-ethereum/p2p" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/discover" | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	testBankKey, _  = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") | ||||
| 	testBankAddress = crypto.PubkeyToAddress(testBankKey.PublicKey) | ||||
| 	testBankFunds   = big.NewInt(1000000) | ||||
| ) | ||||
| 
 | ||||
| // newTestProtocolManager creates a new protocol manager for testing purposes,
 | ||||
| // with the given number of blocks already known, and potential notification
 | ||||
| // channels for different events.
 | ||||
| func newTestProtocolManager(blocks int, generator func(int, *core.BlockGen), newtx chan<- []*types.Transaction) *ProtocolManager { | ||||
| 	var ( | ||||
| 		evmux       = new(event.TypeMux) | ||||
| 		pow         = new(core.FakePow) | ||||
| 		db, _       = ethdb.NewMemDatabase() | ||||
| 		genesis     = core.WriteGenesisBlockForTesting(db, testBankAddress, testBankFunds) | ||||
| 		chainman, _ = core.NewChainManager(db, pow, evmux) | ||||
| 		blockproc   = core.NewBlockProcessor(db, pow, chainman, evmux) | ||||
| 	) | ||||
| 	chainman.SetProcessor(blockproc) | ||||
| 	if _, err := chainman.InsertChain(core.GenerateChain(genesis, db, blocks, generator)); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 	pm := NewProtocolManager(NetworkId, evmux, &testTxPool{added: newtx}, pow, chainman, db) | ||||
| 	pm.Start() | ||||
| 	return pm | ||||
| } | ||||
| 
 | ||||
| // testTxPool is a fake, helper transaction pool for testing purposes
 | ||||
| type testTxPool struct { | ||||
| 	pool  []*types.Transaction        // Collection of all transactions
 | ||||
| 	added chan<- []*types.Transaction // Notification channel for new transactions
 | ||||
| 
 | ||||
| 	lock sync.RWMutex // Protects the transaction pool
 | ||||
| } | ||||
| 
 | ||||
| // AddTransactions appends a batch of transactions to the pool, and notifies any
 | ||||
| // listeners if the addition channel is non nil
 | ||||
| func (p *testTxPool) AddTransactions(txs []*types.Transaction) { | ||||
| 	p.lock.Lock() | ||||
| 	defer p.lock.Unlock() | ||||
| 
 | ||||
| 	p.pool = append(p.pool, txs...) | ||||
| 	if p.added != nil { | ||||
| 		p.added <- txs | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // GetTransactions returns all the transactions known to the pool
 | ||||
| func (p *testTxPool) GetTransactions() types.Transactions { | ||||
| 	p.lock.RLock() | ||||
| 	defer p.lock.RUnlock() | ||||
| 
 | ||||
| 	txs := make([]*types.Transaction, len(p.pool)) | ||||
| 	copy(txs, p.pool) | ||||
| 
 | ||||
| 	return txs | ||||
| } | ||||
| 
 | ||||
| // newTestTransaction create a new dummy transaction.
 | ||||
| func newTestTransaction(from *crypto.Key, nonce uint64, datasize int) *types.Transaction { | ||||
| 	tx := types.NewTransaction(nonce, common.Address{}, big.NewInt(0), big.NewInt(100000), big.NewInt(0), make([]byte, datasize)) | ||||
| 	tx, _ = tx.SignECDSA(from.PrivateKey) | ||||
| 
 | ||||
| 	return tx | ||||
| } | ||||
| 
 | ||||
| // testPeer is a simulated peer to allow testing direct network calls.
 | ||||
| type testPeer struct { | ||||
| 	net p2p.MsgReadWriter // Network layer reader/writer to simulate remote messaging
 | ||||
| 	app *p2p.MsgPipeRW    // Application layer reader/writer to simulate the local side
 | ||||
| 	*peer | ||||
| } | ||||
| 
 | ||||
| // newTestPeer creates a new peer registered at the given protocol manager.
 | ||||
| func newTestPeer(name string, version int, pm *ProtocolManager, shake bool) (*testPeer, <-chan error) { | ||||
| 	// Create a message pipe to communicate through
 | ||||
| 	app, net := p2p.MsgPipe() | ||||
| 
 | ||||
| 	// Generate a random id and create the peer
 | ||||
| 	var id discover.NodeID | ||||
| 	rand.Read(id[:]) | ||||
| 
 | ||||
| 	peer := pm.newPeer(version, NetworkId, p2p.NewPeer(id, name, nil), net) | ||||
| 
 | ||||
| 	// Start the peer on a new thread
 | ||||
| 	errc := make(chan error, 1) | ||||
| 	go func() { | ||||
| 		pm.newPeerCh <- peer | ||||
| 		errc <- pm.handle(peer) | ||||
| 	}() | ||||
| 	tp := &testPeer{ | ||||
| 		app:  app, | ||||
| 		net:  net, | ||||
| 		peer: peer, | ||||
| 	} | ||||
| 	// Execute any implicitly requested handshakes and return
 | ||||
| 	if shake { | ||||
| 		td, head, genesis := pm.chainman.Status() | ||||
| 		tp.handshake(nil, td, head, genesis) | ||||
| 	} | ||||
| 	return tp, errc | ||||
| } | ||||
| 
 | ||||
| // handshake simulates a trivial handshake that expects the same state from the
 | ||||
| // remote side as we are simulating locally.
 | ||||
| func (p *testPeer) handshake(t *testing.T, td *big.Int, head common.Hash, genesis common.Hash) { | ||||
| 	msg := &statusData{ | ||||
| 		ProtocolVersion: uint32(p.version), | ||||
| 		NetworkId:       uint32(NetworkId), | ||||
| 		TD:              td, | ||||
| 		CurrentBlock:    head, | ||||
| 		GenesisBlock:    genesis, | ||||
| 	} | ||||
| 	if err := p2p.ExpectMsg(p.app, StatusMsg, msg); err != nil { | ||||
| 		t.Fatalf("status recv: %v", err) | ||||
| 	} | ||||
| 	if err := p2p.Send(p.app, StatusMsg, msg); err != nil { | ||||
| 		t.Fatalf("status send: %v", err) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // close terminates the local side of the peer, notifying the remote protocol
 | ||||
| // manager of termination.
 | ||||
| func (p *testPeer) close() { | ||||
| 	p.app.Close() | ||||
| } | ||||
							
								
								
									
										153
									
								
								eth/metrics.go
									
									
									
									
									
								
							
							
						
						
									
										153
									
								
								eth/metrics.go
									
									
									
									
									
								
							| @ -18,27 +18,140 @@ package eth | ||||
| 
 | ||||
| import ( | ||||
| 	"github.com/ethereum/go-ethereum/metrics" | ||||
| 	"github.com/ethereum/go-ethereum/p2p" | ||||
| ) | ||||
| 
 | ||||
| var ( | ||||
| 	propTxnInPacketsMeter    = metrics.NewMeter("eth/prop/txns/in/packets") | ||||
| 	propTxnInTrafficMeter    = metrics.NewMeter("eth/prop/txns/in/traffic") | ||||
| 	propTxnOutPacketsMeter   = metrics.NewMeter("eth/prop/txns/out/packets") | ||||
| 	propTxnOutTrafficMeter   = metrics.NewMeter("eth/prop/txns/out/traffic") | ||||
| 	propHashInPacketsMeter   = metrics.NewMeter("eth/prop/hashes/in/packets") | ||||
| 	propHashInTrafficMeter   = metrics.NewMeter("eth/prop/hashes/in/traffic") | ||||
| 	propHashOutPacketsMeter  = metrics.NewMeter("eth/prop/hashes/out/packets") | ||||
| 	propHashOutTrafficMeter  = metrics.NewMeter("eth/prop/hashes/out/traffic") | ||||
| 	propBlockInPacketsMeter  = metrics.NewMeter("eth/prop/blocks/in/packets") | ||||
| 	propBlockInTrafficMeter  = metrics.NewMeter("eth/prop/blocks/in/traffic") | ||||
| 	propBlockOutPacketsMeter = metrics.NewMeter("eth/prop/blocks/out/packets") | ||||
| 	propBlockOutTrafficMeter = metrics.NewMeter("eth/prop/blocks/out/traffic") | ||||
| 	reqHashInPacketsMeter    = metrics.NewMeter("eth/req/hashes/in/packets") | ||||
| 	reqHashInTrafficMeter    = metrics.NewMeter("eth/req/hashes/in/traffic") | ||||
| 	reqHashOutPacketsMeter   = metrics.NewMeter("eth/req/hashes/out/packets") | ||||
| 	reqHashOutTrafficMeter   = metrics.NewMeter("eth/req/hashes/out/traffic") | ||||
| 	reqBlockInPacketsMeter   = metrics.NewMeter("eth/req/blocks/in/packets") | ||||
| 	reqBlockInTrafficMeter   = metrics.NewMeter("eth/req/blocks/in/traffic") | ||||
| 	reqBlockOutPacketsMeter  = metrics.NewMeter("eth/req/blocks/out/packets") | ||||
| 	reqBlockOutTrafficMeter  = metrics.NewMeter("eth/req/blocks/out/traffic") | ||||
| 	propTxnInPacketsMeter     = metrics.NewMeter("eth/prop/txns/in/packets") | ||||
| 	propTxnInTrafficMeter     = metrics.NewMeter("eth/prop/txns/in/traffic") | ||||
| 	propTxnOutPacketsMeter    = metrics.NewMeter("eth/prop/txns/out/packets") | ||||
| 	propTxnOutTrafficMeter    = metrics.NewMeter("eth/prop/txns/out/traffic") | ||||
| 	propHashInPacketsMeter    = metrics.NewMeter("eth/prop/hashes/in/packets") | ||||
| 	propHashInTrafficMeter    = metrics.NewMeter("eth/prop/hashes/in/traffic") | ||||
| 	propHashOutPacketsMeter   = metrics.NewMeter("eth/prop/hashes/out/packets") | ||||
| 	propHashOutTrafficMeter   = metrics.NewMeter("eth/prop/hashes/out/traffic") | ||||
| 	propBlockInPacketsMeter   = metrics.NewMeter("eth/prop/blocks/in/packets") | ||||
| 	propBlockInTrafficMeter   = metrics.NewMeter("eth/prop/blocks/in/traffic") | ||||
| 	propBlockOutPacketsMeter  = metrics.NewMeter("eth/prop/blocks/out/packets") | ||||
| 	propBlockOutTrafficMeter  = metrics.NewMeter("eth/prop/blocks/out/traffic") | ||||
| 	reqHashInPacketsMeter     = metrics.NewMeter("eth/req/hashes/in/packets") | ||||
| 	reqHashInTrafficMeter     = metrics.NewMeter("eth/req/hashes/in/traffic") | ||||
| 	reqHashOutPacketsMeter    = metrics.NewMeter("eth/req/hashes/out/packets") | ||||
| 	reqHashOutTrafficMeter    = metrics.NewMeter("eth/req/hashes/out/traffic") | ||||
| 	reqBlockInPacketsMeter    = metrics.NewMeter("eth/req/blocks/in/packets") | ||||
| 	reqBlockInTrafficMeter    = metrics.NewMeter("eth/req/blocks/in/traffic") | ||||
| 	reqBlockOutPacketsMeter   = metrics.NewMeter("eth/req/blocks/out/packets") | ||||
| 	reqBlockOutTrafficMeter   = metrics.NewMeter("eth/req/blocks/out/traffic") | ||||
| 	reqHeaderInPacketsMeter   = metrics.NewMeter("eth/req/headers/in/packets") | ||||
| 	reqHeaderInTrafficMeter   = metrics.NewMeter("eth/req/headers/in/traffic") | ||||
| 	reqHeaderOutPacketsMeter  = metrics.NewMeter("eth/req/headers/out/packets") | ||||
| 	reqHeaderOutTrafficMeter  = metrics.NewMeter("eth/req/headers/out/traffic") | ||||
| 	reqBodyInPacketsMeter     = metrics.NewMeter("eth/req/bodies/in/packets") | ||||
| 	reqBodyInTrafficMeter     = metrics.NewMeter("eth/req/bodies/in/traffic") | ||||
| 	reqBodyOutPacketsMeter    = metrics.NewMeter("eth/req/bodies/out/packets") | ||||
| 	reqBodyOutTrafficMeter    = metrics.NewMeter("eth/req/bodies/out/traffic") | ||||
| 	reqStateInPacketsMeter    = metrics.NewMeter("eth/req/states/in/packets") | ||||
| 	reqStateInTrafficMeter    = metrics.NewMeter("eth/req/states/in/traffic") | ||||
| 	reqStateOutPacketsMeter   = metrics.NewMeter("eth/req/states/out/packets") | ||||
| 	reqStateOutTrafficMeter   = metrics.NewMeter("eth/req/states/out/traffic") | ||||
| 	reqReceiptInPacketsMeter  = metrics.NewMeter("eth/req/receipts/in/packets") | ||||
| 	reqReceiptInTrafficMeter  = metrics.NewMeter("eth/req/receipts/in/traffic") | ||||
| 	reqReceiptOutPacketsMeter = metrics.NewMeter("eth/req/receipts/out/packets") | ||||
| 	reqReceiptOutTrafficMeter = metrics.NewMeter("eth/req/receipts/out/traffic") | ||||
| 	miscInPacketsMeter        = metrics.NewMeter("eth/misc/in/packets") | ||||
| 	miscInTrafficMeter        = metrics.NewMeter("eth/misc/in/traffic") | ||||
| 	miscOutPacketsMeter       = metrics.NewMeter("eth/misc/out/packets") | ||||
| 	miscOutTrafficMeter       = metrics.NewMeter("eth/misc/out/traffic") | ||||
| ) | ||||
| 
 | ||||
| // meteredMsgReadWriter is a wrapper around a p2p.MsgReadWriter, capable of
 | ||||
| // accumulating the above defined metrics based on the data stream contents.
 | ||||
| type meteredMsgReadWriter struct { | ||||
| 	p2p.MsgReadWriter     // Wrapped message stream to meter
 | ||||
| 	version           int // Protocol version to select correct meters
 | ||||
| } | ||||
| 
 | ||||
| // newMeteredMsgWriter wraps a p2p MsgReadWriter with metering support. If the
 | ||||
| // metrics system is disabled, this fucntion returns the original object.
 | ||||
| func newMeteredMsgWriter(rw p2p.MsgReadWriter) p2p.MsgReadWriter { | ||||
| 	if !metrics.Enabled { | ||||
| 		return rw | ||||
| 	} | ||||
| 	return &meteredMsgReadWriter{MsgReadWriter: rw} | ||||
| } | ||||
| 
 | ||||
| // Init sets the protocol version used by the stream to know which meters to
 | ||||
| // increment in case of overlapping message ids between protocol versions.
 | ||||
| func (rw *meteredMsgReadWriter) Init(version int) { | ||||
| 	rw.version = version | ||||
| } | ||||
| 
 | ||||
| func (rw *meteredMsgReadWriter) ReadMsg() (p2p.Msg, error) { | ||||
| 	// Read the message and short circuit in case of an error
 | ||||
| 	msg, err := rw.MsgReadWriter.ReadMsg() | ||||
| 	if err != nil { | ||||
| 		return msg, err | ||||
| 	} | ||||
| 	// Account for the data traffic
 | ||||
| 	packets, traffic := miscInPacketsMeter, miscInTrafficMeter | ||||
| 	switch { | ||||
| 	case rw.version < eth62 && msg.Code == BlockHashesMsg: | ||||
| 		packets, traffic = reqHashInPacketsMeter, reqHashInTrafficMeter | ||||
| 	case rw.version < eth62 && msg.Code == BlocksMsg: | ||||
| 		packets, traffic = reqBlockInPacketsMeter, reqBlockInTrafficMeter | ||||
| 
 | ||||
| 	case rw.version >= eth62 && msg.Code == BlockHeadersMsg: | ||||
| 		packets, traffic = reqBlockInPacketsMeter, reqBlockInTrafficMeter | ||||
| 	case rw.version >= eth62 && msg.Code == BlockBodiesMsg: | ||||
| 		packets, traffic = reqBodyInPacketsMeter, reqBodyInTrafficMeter | ||||
| 
 | ||||
| 	case rw.version >= eth63 && msg.Code == NodeDataMsg: | ||||
| 		packets, traffic = reqStateInPacketsMeter, reqStateInTrafficMeter | ||||
| 	case rw.version >= eth63 && msg.Code == ReceiptsMsg: | ||||
| 		packets, traffic = reqReceiptInPacketsMeter, reqReceiptInTrafficMeter | ||||
| 
 | ||||
| 	case msg.Code == NewBlockHashesMsg: | ||||
| 		packets, traffic = propHashInPacketsMeter, propHashInTrafficMeter | ||||
| 	case msg.Code == NewBlockMsg: | ||||
| 		packets, traffic = propBlockInPacketsMeter, propBlockInTrafficMeter | ||||
| 	case msg.Code == TxMsg: | ||||
| 		packets, traffic = propTxnInPacketsMeter, propTxnInTrafficMeter | ||||
| 	} | ||||
| 	packets.Mark(1) | ||||
| 	traffic.Mark(int64(msg.Size)) | ||||
| 
 | ||||
| 	return msg, err | ||||
| } | ||||
| 
 | ||||
| func (rw *meteredMsgReadWriter) WriteMsg(msg p2p.Msg) error { | ||||
| 	// Account for the data traffic
 | ||||
| 	packets, traffic := miscOutPacketsMeter, miscOutTrafficMeter | ||||
| 	switch { | ||||
| 	case rw.version < eth62 && msg.Code == BlockHashesMsg: | ||||
| 		packets, traffic = reqHashOutPacketsMeter, reqHashOutTrafficMeter | ||||
| 	case rw.version < eth62 && msg.Code == BlocksMsg: | ||||
| 		packets, traffic = reqBlockOutPacketsMeter, reqBlockOutTrafficMeter | ||||
| 
 | ||||
| 	case rw.version >= eth62 && msg.Code == BlockHeadersMsg: | ||||
| 		packets, traffic = reqHeaderOutPacketsMeter, reqHeaderOutTrafficMeter | ||||
| 	case rw.version >= eth62 && msg.Code == BlockBodiesMsg: | ||||
| 		packets, traffic = reqBodyOutPacketsMeter, reqBodyOutTrafficMeter | ||||
| 
 | ||||
| 	case rw.version >= eth63 && msg.Code == NodeDataMsg: | ||||
| 		packets, traffic = reqStateOutPacketsMeter, reqStateOutTrafficMeter | ||||
| 	case rw.version >= eth63 && msg.Code == ReceiptsMsg: | ||||
| 		packets, traffic = reqReceiptOutPacketsMeter, reqReceiptOutTrafficMeter | ||||
| 
 | ||||
| 	case msg.Code == NewBlockHashesMsg: | ||||
| 		packets, traffic = propHashOutPacketsMeter, propHashOutTrafficMeter | ||||
| 	case msg.Code == NewBlockMsg: | ||||
| 		packets, traffic = propBlockOutPacketsMeter, propBlockOutTrafficMeter | ||||
| 	case msg.Code == TxMsg: | ||||
| 		packets, traffic = propTxnOutPacketsMeter, propTxnOutTrafficMeter | ||||
| 	} | ||||
| 	packets.Mark(1) | ||||
| 	traffic.Mark(int64(msg.Size)) | ||||
| 
 | ||||
| 	// Send the packet to the p2p layer
 | ||||
| 	return rw.MsgReadWriter.WriteMsg(msg) | ||||
| } | ||||
|  | ||||
							
								
								
									
										106
									
								
								eth/peer.go
									
									
									
									
									
								
							
							
						
						
									
										106
									
								
								eth/peer.go
									
									
									
									
									
								
							| @ -129,9 +129,7 @@ func (p *peer) MarkTransaction(hash common.Hash) { | ||||
| // SendTransactions sends transactions to the peer and includes the hashes
 | ||||
| // in its transaction hash set for future reference.
 | ||||
| func (p *peer) SendTransactions(txs types.Transactions) error { | ||||
| 	propTxnOutPacketsMeter.Mark(1) | ||||
| 	for _, tx := range txs { | ||||
| 		propTxnOutTrafficMeter.Mark(tx.Size().Int64()) | ||||
| 		p.knownTxs.Add(tx.Hash()) | ||||
| 	} | ||||
| 	return p2p.Send(p.rw, TxMsg, txs) | ||||
| @ -139,62 +137,126 @@ func (p *peer) SendTransactions(txs types.Transactions) error { | ||||
| 
 | ||||
| // SendBlockHashes sends a batch of known hashes to the remote peer.
 | ||||
| func (p *peer) SendBlockHashes(hashes []common.Hash) error { | ||||
| 	reqHashOutPacketsMeter.Mark(1) | ||||
| 	reqHashOutTrafficMeter.Mark(int64(32 * len(hashes))) | ||||
| 
 | ||||
| 	return p2p.Send(p.rw, BlockHashesMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // SendBlocks sends a batch of blocks to the remote peer.
 | ||||
| func (p *peer) SendBlocks(blocks []*types.Block) error { | ||||
| 	reqBlockOutPacketsMeter.Mark(1) | ||||
| 	for _, block := range blocks { | ||||
| 		reqBlockOutTrafficMeter.Mark(block.Size().Int64()) | ||||
| 	} | ||||
| 	return p2p.Send(p.rw, BlocksMsg, blocks) | ||||
| } | ||||
| 
 | ||||
| // SendNewBlockHashes announces the availability of a number of blocks through
 | ||||
| // SendNewBlockHashes61 announces the availability of a number of blocks through
 | ||||
| // a hash notification.
 | ||||
| func (p *peer) SendNewBlockHashes(hashes []common.Hash) error { | ||||
| 	propHashOutPacketsMeter.Mark(1) | ||||
| 	propHashOutTrafficMeter.Mark(int64(32 * len(hashes))) | ||||
| 
 | ||||
| func (p *peer) SendNewBlockHashes61(hashes []common.Hash) error { | ||||
| 	for _, hash := range hashes { | ||||
| 		p.knownBlocks.Add(hash) | ||||
| 	} | ||||
| 	return p2p.Send(p.rw, NewBlockHashesMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // SendNewBlockHashes announces the availability of a number of blocks through
 | ||||
| // a hash notification.
 | ||||
| func (p *peer) SendNewBlockHashes(hashes []common.Hash, numbers []uint64) error { | ||||
| 	for _, hash := range hashes { | ||||
| 		p.knownBlocks.Add(hash) | ||||
| 	} | ||||
| 	request := make(newBlockHashesData, len(hashes)) | ||||
| 	for i := 0; i < len(hashes); i++ { | ||||
| 		request[i].Hash = hashes[i] | ||||
| 		request[i].Number = numbers[i] | ||||
| 	} | ||||
| 	return p2p.Send(p.rw, NewBlockHashesMsg, request) | ||||
| } | ||||
| 
 | ||||
| // SendNewBlock propagates an entire block to a remote peer.
 | ||||
| func (p *peer) SendNewBlock(block *types.Block, td *big.Int) error { | ||||
| 	propBlockOutPacketsMeter.Mark(1) | ||||
| 	propBlockOutTrafficMeter.Mark(block.Size().Int64()) | ||||
| 
 | ||||
| 	p.knownBlocks.Add(block.Hash()) | ||||
| 	return p2p.Send(p.rw, NewBlockMsg, []interface{}{block, td}) | ||||
| } | ||||
| 
 | ||||
| // SendBlockHeaders sends a batch of block headers to the remote peer.
 | ||||
| func (p *peer) SendBlockHeaders(headers []*types.Header) error { | ||||
| 	return p2p.Send(p.rw, BlockHeadersMsg, headers) | ||||
| } | ||||
| 
 | ||||
| // SendBlockBodies sends a batch of block contents to the remote peer.
 | ||||
| func (p *peer) SendBlockBodies(bodies []*blockBody) error { | ||||
| 	return p2p.Send(p.rw, BlockBodiesMsg, blockBodiesData(bodies)) | ||||
| } | ||||
| 
 | ||||
| // SendNodeData sends a batch of arbitrary internal data, corresponding to the
 | ||||
| // hashes requested.
 | ||||
| func (p *peer) SendNodeData(data [][]byte) error { | ||||
| 	return p2p.Send(p.rw, NodeDataMsg, data) | ||||
| } | ||||
| 
 | ||||
| // SendReceipts sends a batch of transaction receipts, corresponding to the ones
 | ||||
| // requested.
 | ||||
| func (p *peer) SendReceipts(receipts []*types.Receipt) error { | ||||
| 	return p2p.Send(p.rw, ReceiptsMsg, receipts) | ||||
| } | ||||
| 
 | ||||
| // RequestHashes fetches a batch of hashes from a peer, starting at from, going
 | ||||
| // towards the genesis block.
 | ||||
| func (p *peer) RequestHashes(from common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("Peer [%s] fetching hashes (%d) from %x...\n", p.id, downloader.MaxHashFetch, from[:4]) | ||||
| 	glog.V(logger.Debug).Infof("%v fetching hashes (%d) from %x...", p, downloader.MaxHashFetch, from[:4]) | ||||
| 	return p2p.Send(p.rw, GetBlockHashesMsg, getBlockHashesData{from, uint64(downloader.MaxHashFetch)}) | ||||
| } | ||||
| 
 | ||||
| // RequestHashesFromNumber fetches a batch of hashes from a peer, starting at the
 | ||||
| // requested block number, going upwards towards the genesis block.
 | ||||
| // RequestHashesFromNumber fetches a batch of hashes from a peer, starting at
 | ||||
| // the requested block number, going upwards towards the genesis block.
 | ||||
| func (p *peer) RequestHashesFromNumber(from uint64, count int) error { | ||||
| 	glog.V(logger.Debug).Infof("Peer [%s] fetching hashes (%d) from #%d...\n", p.id, count, from) | ||||
| 	glog.V(logger.Debug).Infof("%v fetching hashes (%d) from #%d...", p, count, from) | ||||
| 	return p2p.Send(p.rw, GetBlockHashesFromNumberMsg, getBlockHashesFromNumberData{from, uint64(count)}) | ||||
| } | ||||
| 
 | ||||
| // RequestBlocks fetches a batch of blocks corresponding to the specified hashes.
 | ||||
| func (p *peer) RequestBlocks(hashes []common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("[%s] fetching %v blocks\n", p.id, len(hashes)) | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %v blocks", p, len(hashes)) | ||||
| 	return p2p.Send(p.rw, GetBlocksMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // RequestHeaders is a wrapper around the header query functions to fetch a
 | ||||
| // single header. It is used solely by the fetcher.
 | ||||
| func (p *peer) RequestOneHeader(hash common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching a single header: %x", p, hash) | ||||
| 	return p2p.Send(p.rw, GetBlockHeadersMsg, &getBlockHeadersData{Origin: hashOrNumber{Hash: hash}, Amount: uint64(1), Skip: uint64(0), Reverse: false}) | ||||
| } | ||||
| 
 | ||||
| // RequestHeadersByHash fetches a batch of blocks' headers corresponding to the
 | ||||
| // specified header query, based on the hash of an origin block.
 | ||||
| func (p *peer) RequestHeadersByHash(origin common.Hash, amount int, skip int, reverse bool) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %d headers from %x, skipping %d (reverse = %v)", p, amount, origin[:4], skip, reverse) | ||||
| 	return p2p.Send(p.rw, GetBlockHeadersMsg, &getBlockHeadersData{Origin: hashOrNumber{Hash: origin}, Amount: uint64(amount), Skip: uint64(skip), Reverse: reverse}) | ||||
| } | ||||
| 
 | ||||
| // RequestHeadersByNumber fetches a batch of blocks' headers corresponding to the
 | ||||
| // specified header query, based on the number of an origin block.
 | ||||
| func (p *peer) RequestHeadersByNumber(origin uint64, amount int, skip int, reverse bool) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %d headers from #%d, skipping %d (reverse = %v)", p, amount, origin, skip, reverse) | ||||
| 	return p2p.Send(p.rw, GetBlockHeadersMsg, &getBlockHeadersData{Origin: hashOrNumber{Number: origin}, Amount: uint64(amount), Skip: uint64(skip), Reverse: reverse}) | ||||
| } | ||||
| 
 | ||||
| // RequestBodies fetches a batch of blocks' bodies corresponding to the hashes
 | ||||
| // specified.
 | ||||
| func (p *peer) RequestBodies(hashes []common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %d block bodies", p, len(hashes)) | ||||
| 	return p2p.Send(p.rw, GetBlockBodiesMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // RequestNodeData fetches a batch of arbitrary data from a node's known state
 | ||||
| // data, corresponding to the specified hashes.
 | ||||
| func (p *peer) RequestNodeData(hashes []common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %v state data", p, len(hashes)) | ||||
| 	return p2p.Send(p.rw, GetNodeDataMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // RequestReceipts fetches a batch of transaction receipts from a remote node.
 | ||||
| func (p *peer) RequestReceipts(hashes []common.Hash) error { | ||||
| 	glog.V(logger.Debug).Infof("%v fetching %v receipts", p, len(hashes)) | ||||
| 	return p2p.Send(p.rw, GetReceiptsMsg, hashes) | ||||
| } | ||||
| 
 | ||||
| // Handshake executes the eth protocol handshake, negotiating version number,
 | ||||
| // network IDs, difficulties, head and genesis blocks.
 | ||||
| func (p *peer) Handshake(td *big.Int, head common.Hash, genesis common.Hash) error { | ||||
|  | ||||
							
								
								
									
										126
									
								
								eth/protocol.go
									
									
									
									
									
								
							
							
						
						
									
										126
									
								
								eth/protocol.go
									
									
									
									
									
								
							| @ -17,17 +17,28 @@ | ||||
| package eth | ||||
| 
 | ||||
| import ( | ||||
| 	"fmt" | ||||
| 	"io" | ||||
| 	"math/big" | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/core/types" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| // Constants to match up protocol versions and messages
 | ||||
| const ( | ||||
| 	eth61 = 61 | ||||
| 	eth62 = 62 | ||||
| 	eth63 = 63 | ||||
| 	eth64 = 64 | ||||
| ) | ||||
| 
 | ||||
| // Supported versions of the eth protocol (first is primary).
 | ||||
| var ProtocolVersions = []uint{61, 60} | ||||
| var ProtocolVersions = []uint{eth64, eth63, eth62, eth61} | ||||
| 
 | ||||
| // Number of implemented message corresponding to different protocol versions.
 | ||||
| var ProtocolLengths = []uint64{9, 8} | ||||
| var ProtocolLengths = []uint64{15, 12, 8, 9} | ||||
| 
 | ||||
| const ( | ||||
| 	NetworkId          = 1 | ||||
| @ -36,15 +47,37 @@ const ( | ||||
| 
 | ||||
| // eth protocol message codes
 | ||||
| const ( | ||||
| 	StatusMsg = iota | ||||
| 	NewBlockHashesMsg | ||||
| 	TxMsg | ||||
| 	GetBlockHashesMsg | ||||
| 	BlockHashesMsg | ||||
| 	GetBlocksMsg | ||||
| 	BlocksMsg | ||||
| 	NewBlockMsg | ||||
| 	GetBlockHashesFromNumberMsg | ||||
| 	// Protocol messages belonging to eth/61
 | ||||
| 	StatusMsg                   = 0x00 | ||||
| 	NewBlockHashesMsg           = 0x01 | ||||
| 	TxMsg                       = 0x02 | ||||
| 	GetBlockHashesMsg           = 0x03 | ||||
| 	BlockHashesMsg              = 0x04 | ||||
| 	GetBlocksMsg                = 0x05 | ||||
| 	BlocksMsg                   = 0x06 | ||||
| 	NewBlockMsg                 = 0x07 | ||||
| 	GetBlockHashesFromNumberMsg = 0x08 | ||||
| 
 | ||||
| 	// Protocol messages belonging to eth/62 (new protocol from scratch)
 | ||||
| 	// StatusMsg          = 0x00 (uncomment after eth/61 deprecation)
 | ||||
| 	// NewBlockHashesMsg  = 0x01 (uncomment after eth/61 deprecation)
 | ||||
| 	// TxMsg              = 0x02 (uncomment after eth/61 deprecation)
 | ||||
| 	GetBlockHeadersMsg = 0x03 | ||||
| 	BlockHeadersMsg    = 0x04 | ||||
| 	GetBlockBodiesMsg  = 0x05 | ||||
| 	BlockBodiesMsg     = 0x06 | ||||
| 	// 	NewBlockMsg       = 0x07 (uncomment after eth/61 deprecation)
 | ||||
| 
 | ||||
| 	// Protocol messages belonging to eth/63
 | ||||
| 	GetNodeDataMsg = 0x0d | ||||
| 	NodeDataMsg    = 0x0e | ||||
| 	GetReceiptsMsg = 0x0f | ||||
| 	ReceiptsMsg    = 0x10 | ||||
| 
 | ||||
| 	// Protocol messages belonging to eth/64
 | ||||
| 	GetAcctProofMsg     = 0x11 | ||||
| 	GetStorageDataProof = 0x12 | ||||
| 	Proof               = 0x13 | ||||
| ) | ||||
| 
 | ||||
| type errCode int | ||||
| @ -102,22 +135,85 @@ type statusData struct { | ||||
| 	GenesisBlock    common.Hash | ||||
| } | ||||
| 
 | ||||
| // getBlockHashesData is the network packet for the hash based block retrieval
 | ||||
| // message.
 | ||||
| // newBlockHashesData is the network packet for the block announcements.
 | ||||
| type newBlockHashesData []struct { | ||||
| 	Hash   common.Hash // Hash of one particular block being announced
 | ||||
| 	Number uint64      // Number of one particular block being announced
 | ||||
| } | ||||
| 
 | ||||
| // getBlockHashesData is the network packet for the hash based hash retrieval.
 | ||||
| type getBlockHashesData struct { | ||||
| 	Hash   common.Hash | ||||
| 	Amount uint64 | ||||
| } | ||||
| 
 | ||||
| // getBlockHashesFromNumberData is the network packet for the number based block
 | ||||
| // retrieval message.
 | ||||
| // getBlockHashesFromNumberData is the network packet for the number based hash
 | ||||
| // retrieval.
 | ||||
| type getBlockHashesFromNumberData struct { | ||||
| 	Number uint64 | ||||
| 	Amount uint64 | ||||
| } | ||||
| 
 | ||||
| // getBlockHeadersData represents a block header query.
 | ||||
| type getBlockHeadersData struct { | ||||
| 	Origin  hashOrNumber // Block from which to retrieve headers
 | ||||
| 	Amount  uint64       // Maximum number of headers to retrieve
 | ||||
| 	Skip    uint64       // Blocks to skip between consecutive headers
 | ||||
| 	Reverse bool         // Query direction (false = rising towards latest, true = falling towards genesis)
 | ||||
| } | ||||
| 
 | ||||
| // hashOrNumber is a combined field for specifying an origin block.
 | ||||
| type hashOrNumber struct { | ||||
| 	Hash   common.Hash // Block hash from which to retrieve headers (excludes Number)
 | ||||
| 	Number uint64      // Block hash from which to retrieve headers (excludes Hash)
 | ||||
| } | ||||
| 
 | ||||
| // EncodeRLP is a specialized encoder for hashOrNumber to encode only one of the
 | ||||
| // two contained union fields.
 | ||||
| func (hn *hashOrNumber) EncodeRLP(w io.Writer) error { | ||||
| 	if hn.Hash == (common.Hash{}) { | ||||
| 		return rlp.Encode(w, hn.Number) | ||||
| 	} | ||||
| 	if hn.Number != 0 { | ||||
| 		return fmt.Errorf("both origin hash (%x) and number (%d) provided", hn.Hash, hn.Number) | ||||
| 	} | ||||
| 	return rlp.Encode(w, hn.Hash) | ||||
| } | ||||
| 
 | ||||
| // DecodeRLP is a specialized decoder for hashOrNumber to decode the contents
 | ||||
| // into either a block hash or a block number.
 | ||||
| func (hn *hashOrNumber) DecodeRLP(s *rlp.Stream) error { | ||||
| 	_, size, _ := s.Kind() | ||||
| 	origin, err := s.Raw() | ||||
| 	if err == nil { | ||||
| 		switch { | ||||
| 		case size == 32: | ||||
| 			err = rlp.DecodeBytes(origin, &hn.Hash) | ||||
| 		case size <= 8: | ||||
| 			err = rlp.DecodeBytes(origin, &hn.Number) | ||||
| 		default: | ||||
| 			err = fmt.Errorf("invalid input size %d for origin", size) | ||||
| 		} | ||||
| 	} | ||||
| 	return err | ||||
| } | ||||
| 
 | ||||
| // newBlockData is the network packet for the block propagation message.
 | ||||
| type newBlockData struct { | ||||
| 	Block *types.Block | ||||
| 	TD    *big.Int | ||||
| } | ||||
| 
 | ||||
| // blockBody represents the data content of a single block.
 | ||||
| type blockBody struct { | ||||
| 	Transactions []*types.Transaction // Transactions contained within a block
 | ||||
| 	Uncles       []*types.Header      // Uncles contained within a block
 | ||||
| } | ||||
| 
 | ||||
| // blockBodiesData is the network packet for block content distribution.
 | ||||
| type blockBodiesData []*blockBody | ||||
| 
 | ||||
| // nodeDataData is the network response packet for a node data retrieval.
 | ||||
| type nodeDataData []struct { | ||||
| 	Value []byte | ||||
| } | ||||
|  | ||||
| @ -18,19 +18,16 @@ package eth | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| 	"math/big" | ||||
| 	"fmt" | ||||
| 	"sync" | ||||
| 	"testing" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/ethereum/go-ethereum/common" | ||||
| 	"github.com/ethereum/go-ethereum/core" | ||||
| 	"github.com/ethereum/go-ethereum/core/types" | ||||
| 	"github.com/ethereum/go-ethereum/crypto" | ||||
| 	"github.com/ethereum/go-ethereum/ethdb" | ||||
| 	"github.com/ethereum/go-ethereum/event" | ||||
| 	"github.com/ethereum/go-ethereum/p2p" | ||||
| 	"github.com/ethereum/go-ethereum/p2p/discover" | ||||
| 	"github.com/ethereum/go-ethereum/rlp" | ||||
| ) | ||||
| 
 | ||||
| func init() { | ||||
| @ -40,8 +37,14 @@ func init() { | ||||
| 
 | ||||
| var testAccount = crypto.NewKey(rand.Reader) | ||||
| 
 | ||||
| func TestStatusMsgErrors(t *testing.T) { | ||||
| 	pm := newProtocolManagerForTesting(nil) | ||||
| // Tests that handshake failures are detected and reported correctly.
 | ||||
| func TestStatusMsgErrors61(t *testing.T) { testStatusMsgErrors(t, 61) } | ||||
| func TestStatusMsgErrors62(t *testing.T) { testStatusMsgErrors(t, 62) } | ||||
| func TestStatusMsgErrors63(t *testing.T) { testStatusMsgErrors(t, 63) } | ||||
| func TestStatusMsgErrors64(t *testing.T) { testStatusMsgErrors(t, 64) } | ||||
| 
 | ||||
| func testStatusMsgErrors(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(0, nil, nil) | ||||
| 	td, currentBlock, genesis := pm.chainman.Status() | ||||
| 	defer pm.Stop() | ||||
| 
 | ||||
| @ -56,23 +59,23 @@ func TestStatusMsgErrors(t *testing.T) { | ||||
| 		}, | ||||
| 		{ | ||||
| 			code: StatusMsg, data: statusData{10, NetworkId, td, currentBlock, genesis}, | ||||
| 			wantError: errResp(ErrProtocolVersionMismatch, "10 (!= 0)"), | ||||
| 			wantError: errResp(ErrProtocolVersionMismatch, "10 (!= %d)", protocol), | ||||
| 		}, | ||||
| 		{ | ||||
| 			code: StatusMsg, data: statusData{uint32(ProtocolVersions[0]), 999, td, currentBlock, genesis}, | ||||
| 			code: StatusMsg, data: statusData{uint32(protocol), 999, td, currentBlock, genesis}, | ||||
| 			wantError: errResp(ErrNetworkIdMismatch, "999 (!= 1)"), | ||||
| 		}, | ||||
| 		{ | ||||
| 			code: StatusMsg, data: statusData{uint32(ProtocolVersions[0]), NetworkId, td, currentBlock, common.Hash{3}}, | ||||
| 			code: StatusMsg, data: statusData{uint32(protocol), NetworkId, td, currentBlock, common.Hash{3}}, | ||||
| 			wantError: errResp(ErrGenesisBlockMismatch, "0300000000000000000000000000000000000000000000000000000000000000 (!= %x)", genesis), | ||||
| 		}, | ||||
| 	} | ||||
| 
 | ||||
| 	for i, test := range tests { | ||||
| 		p, errc := newTestPeer(pm) | ||||
| 		p, errc := newTestPeer("peer", protocol, pm, false) | ||||
| 		// The send call might hang until reset because
 | ||||
| 		// the protocol might not read the payload.
 | ||||
| 		go p2p.Send(p, test.code, test.data) | ||||
| 		go p2p.Send(p.app, test.code, test.data) | ||||
| 
 | ||||
| 		select { | ||||
| 		case err := <-errc: | ||||
| @ -89,16 +92,20 @@ func TestStatusMsgErrors(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| // This test checks that received transactions are added to the local pool.
 | ||||
| func TestRecvTransactions(t *testing.T) { | ||||
| func TestRecvTransactions61(t *testing.T) { testRecvTransactions(t, 61) } | ||||
| func TestRecvTransactions62(t *testing.T) { testRecvTransactions(t, 62) } | ||||
| func TestRecvTransactions63(t *testing.T) { testRecvTransactions(t, 63) } | ||||
| func TestRecvTransactions64(t *testing.T) { testRecvTransactions(t, 64) } | ||||
| 
 | ||||
| func testRecvTransactions(t *testing.T, protocol int) { | ||||
| 	txAdded := make(chan []*types.Transaction) | ||||
| 	pm := newProtocolManagerForTesting(txAdded) | ||||
| 	p, _ := newTestPeer(pm) | ||||
| 	pm := newTestProtocolManager(0, nil, txAdded) | ||||
| 	p, _ := newTestPeer("peer", protocol, pm, true) | ||||
| 	defer pm.Stop() | ||||
| 	defer p.close() | ||||
| 	p.handshake(t) | ||||
| 
 | ||||
| 	tx := newtx(testAccount, 0, 0) | ||||
| 	if err := p2p.Send(p, TxMsg, []interface{}{tx}); err != nil { | ||||
| 	tx := newTestTransaction(testAccount, 0, 0) | ||||
| 	if err := p2p.Send(p.app, TxMsg, []interface{}{tx}); err != nil { | ||||
| 		t.Fatalf("send error: %v", err) | ||||
| 	} | ||||
| 	select { | ||||
| @ -114,15 +121,20 @@ func TestRecvTransactions(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| // This test checks that pending transactions are sent.
 | ||||
| func TestSendTransactions(t *testing.T) { | ||||
| 	pm := newProtocolManagerForTesting(nil) | ||||
| func TestSendTransactions61(t *testing.T) { testSendTransactions(t, 61) } | ||||
| func TestSendTransactions62(t *testing.T) { testSendTransactions(t, 62) } | ||||
| func TestSendTransactions63(t *testing.T) { testSendTransactions(t, 63) } | ||||
| func TestSendTransactions64(t *testing.T) { testSendTransactions(t, 64) } | ||||
| 
 | ||||
| func testSendTransactions(t *testing.T, protocol int) { | ||||
| 	pm := newTestProtocolManager(0, nil, nil) | ||||
| 	defer pm.Stop() | ||||
| 
 | ||||
| 	// Fill the pool with big transactions.
 | ||||
| 	const txsize = txsyncPackSize / 10 | ||||
| 	alltxs := make([]*types.Transaction, 100) | ||||
| 	for nonce := range alltxs { | ||||
| 		alltxs[nonce] = newtx(testAccount, uint64(nonce), txsize) | ||||
| 		alltxs[nonce] = newTestTransaction(testAccount, uint64(nonce), txsize) | ||||
| 	} | ||||
| 	pm.txpool.AddTransactions(alltxs) | ||||
| 
 | ||||
| @ -137,7 +149,7 @@ func TestSendTransactions(t *testing.T) { | ||||
| 		} | ||||
| 		for n := 0; n < len(alltxs) && !t.Failed(); { | ||||
| 			var txs []*types.Transaction | ||||
| 			msg, err := p.ReadMsg() | ||||
| 			msg, err := p.app.ReadMsg() | ||||
| 			if err != nil { | ||||
| 				t.Errorf("%v: read error: %v", p.Peer, err) | ||||
| 			} else if msg.Code != TxMsg { | ||||
| @ -161,97 +173,53 @@ func TestSendTransactions(t *testing.T) { | ||||
| 		} | ||||
| 	} | ||||
| 	for i := 0; i < 3; i++ { | ||||
| 		p, _ := newTestPeer(pm) | ||||
| 		p.handshake(t) | ||||
| 		p, _ := newTestPeer(fmt.Sprintf("peer #%d", i), protocol, pm, true) | ||||
| 		wg.Add(1) | ||||
| 		go checktxs(p) | ||||
| 	} | ||||
| 	wg.Wait() | ||||
| } | ||||
| 
 | ||||
| // testPeer wraps all peer-related data for tests.
 | ||||
| type testPeer struct { | ||||
| 	p2p.MsgReadWriter                // writing to the test peer feeds the protocol
 | ||||
| 	pipe              *p2p.MsgPipeRW // the protocol read/writes on this end
 | ||||
| 	pm                *ProtocolManager | ||||
| 	*peer | ||||
| } | ||||
| 
 | ||||
| func newProtocolManagerForTesting(txAdded chan<- []*types.Transaction) *ProtocolManager { | ||||
| 	db, _ := ethdb.NewMemDatabase() | ||||
| 	core.WriteTestNetGenesisBlock(db, 0) | ||||
| 	var ( | ||||
| 		em       = new(event.TypeMux) | ||||
| 		chain, _ = core.NewChainManager(db, core.FakePow{}, em) | ||||
| 		txpool   = &fakeTxPool{added: txAdded} | ||||
| 		pm       = NewProtocolManager(NetworkId, em, txpool, core.FakePow{}, chain) | ||||
| 	) | ||||
| 	pm.Start() | ||||
| 	return pm | ||||
| } | ||||
| 
 | ||||
| func newTestPeer(pm *ProtocolManager) (*testPeer, <-chan error) { | ||||
| 	var id discover.NodeID | ||||
| 	rand.Read(id[:]) | ||||
| 	rw1, rw2 := p2p.MsgPipe() | ||||
| 	peer := pm.newPeer(pm.protVer, pm.netId, p2p.NewPeer(id, "test peer", nil), rw2) | ||||
| 	errc := make(chan error, 1) | ||||
| 	go func() { | ||||
| 		pm.newPeerCh <- peer | ||||
| 		errc <- pm.handle(peer) | ||||
| 	}() | ||||
| 	return &testPeer{rw1, rw2, pm, peer}, errc | ||||
| } | ||||
| 
 | ||||
| func (p *testPeer) handshake(t *testing.T) { | ||||
| 	td, currentBlock, genesis := p.pm.chainman.Status() | ||||
| 	msg := &statusData{ | ||||
| 		ProtocolVersion: uint32(p.pm.protVer), | ||||
| 		NetworkId:       uint32(p.pm.netId), | ||||
| 		TD:              td, | ||||
| 		CurrentBlock:    currentBlock, | ||||
| 		GenesisBlock:    genesis, | ||||
| // Tests that the custom union field encoder and decoder works correctly.
 | ||||
| func TestGetBlockHeadersDataEncodeDecode(t *testing.T) { | ||||
| 	// Create a "random" hash for testing
 | ||||
| 	var hash common.Hash | ||||
| 	for i, _ := range hash { | ||||
| 		hash[i] = byte(i) | ||||
| 	} | ||||
| 	if err := p2p.ExpectMsg(p, StatusMsg, msg); err != nil { | ||||
| 		t.Fatalf("status recv: %v", err) | ||||
| 	// Assemble some table driven tests
 | ||||
| 	tests := []struct { | ||||
| 		packet *getBlockHeadersData | ||||
| 		fail   bool | ||||
| 	}{ | ||||
| 		// Providing the origin as either a hash or a number should both work
 | ||||
| 		{fail: false, packet: &getBlockHeadersData{Origin: hashOrNumber{Number: 314}}}, | ||||
| 		{fail: false, packet: &getBlockHeadersData{Origin: hashOrNumber{Hash: hash}}}, | ||||
| 
 | ||||
| 		// Providing arbitrary query field should also work
 | ||||
| 		{fail: false, packet: &getBlockHeadersData{Origin: hashOrNumber{Number: 314}, Amount: 314, Skip: 1, Reverse: true}}, | ||||
| 		{fail: false, packet: &getBlockHeadersData{Origin: hashOrNumber{Hash: hash}, Amount: 314, Skip: 1, Reverse: true}}, | ||||
| 
 | ||||
| 		// Providing both the origin hash and origin number must fail
 | ||||
| 		{fail: true, packet: &getBlockHeadersData{Origin: hashOrNumber{Hash: hash, Number: 314}}}, | ||||
| 	} | ||||
| 	if err := p2p.Send(p, StatusMsg, msg); err != nil { | ||||
| 		t.Fatalf("status send: %v", err) | ||||
| 	// Iterate over each of the tests and try to encode and then decode
 | ||||
| 	for i, tt := range tests { | ||||
| 		bytes, err := rlp.EncodeToBytes(tt.packet) | ||||
| 		if err != nil && !tt.fail { | ||||
| 			t.Fatalf("test %d: failed to encode packet: %v", i, err) | ||||
| 		} else if err == nil && tt.fail { | ||||
| 			t.Fatalf("test %d: encode should have failed", i) | ||||
| 		} | ||||
| 		if !tt.fail { | ||||
| 			packet := new(getBlockHeadersData) | ||||
| 			if err := rlp.DecodeBytes(bytes, packet); err != nil { | ||||
| 				t.Fatalf("test %d: failed to decode packet: %v", i, err) | ||||
| 			} | ||||
| 			if packet.Origin.Hash != tt.packet.Origin.Hash || packet.Origin.Number != tt.packet.Origin.Number || packet.Amount != tt.packet.Amount || | ||||
| 				packet.Skip != tt.packet.Skip || packet.Reverse != tt.packet.Reverse { | ||||
| 				t.Fatalf("test %d: encode decode mismatch: have %+v, want %+v", i, packet, tt.packet) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (p *testPeer) close() { | ||||
| 	p.pipe.Close() | ||||
| } | ||||
| 
 | ||||
| type fakeTxPool struct { | ||||
| 	// all transactions are collected.
 | ||||
| 	mu  sync.Mutex | ||||
| 	all []*types.Transaction | ||||
| 	// if added is non-nil, it receives added transactions.
 | ||||
| 	added chan<- []*types.Transaction | ||||
| } | ||||
| 
 | ||||
| func (pool *fakeTxPool) AddTransactions(txs []*types.Transaction) { | ||||
| 	pool.mu.Lock() | ||||
| 	defer pool.mu.Unlock() | ||||
| 	pool.all = append(pool.all, txs...) | ||||
| 	if pool.added != nil { | ||||
| 		pool.added <- txs | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (pool *fakeTxPool) GetTransactions() types.Transactions { | ||||
| 	pool.mu.Lock() | ||||
| 	defer pool.mu.Unlock() | ||||
| 	txs := make([]*types.Transaction, len(pool.all)) | ||||
| 	copy(txs, pool.all) | ||||
| 	return types.Transactions(txs) | ||||
| } | ||||
| 
 | ||||
| func newtx(from *crypto.Key, nonce uint64, datasize int) *types.Transaction { | ||||
| 	data := make([]byte, datasize) | ||||
| 	tx := types.NewTransaction(nonce, common.Address{}, big.NewInt(0), big.NewInt(100000), big.NewInt(0), data) | ||||
| 	tx, _ = tx.SignECDSA(from.PrivateKey) | ||||
| 	return tx | ||||
| } | ||||
|  | ||||
| @ -49,6 +49,14 @@ func (db *MemDatabase) Get(key []byte) ([]byte, error) { | ||||
| 	return db.db[string(key)], nil | ||||
| } | ||||
| 
 | ||||
| func (db *MemDatabase) Keys() [][]byte { | ||||
| 	keys := [][]byte{} | ||||
| 	for key, _ := range db.db { | ||||
| 		keys = append(keys, []byte(key)) | ||||
| 	} | ||||
| 	return keys | ||||
| } | ||||
| 
 | ||||
| /* | ||||
| func (db *MemDatabase) GetKeys() []*common.Key { | ||||
| 	data, _ := db.Get([]byte("KeyRing")) | ||||
|  | ||||
| @ -31,8 +31,8 @@ import ( | ||||
| // MetricsEnabledFlag is the CLI flag name to use to enable metrics collections.
 | ||||
| var MetricsEnabledFlag = "metrics" | ||||
| 
 | ||||
| // enabled is the flag specifying if metrics are enable or not.
 | ||||
| var enabled = false | ||||
| // Enabled is the flag specifying if metrics are enable or not.
 | ||||
| var Enabled = false | ||||
| 
 | ||||
| // Init enables or disables the metrics system. Since we need this to run before
 | ||||
| // any other code gets to create meters and timers, we'll actually do an ugly hack
 | ||||
| @ -41,7 +41,7 @@ func init() { | ||||
| 	for _, arg := range os.Args { | ||||
| 		if strings.TrimLeft(arg, "-") == MetricsEnabledFlag { | ||||
| 			glog.V(logger.Info).Infof("Enabling metrics collection") | ||||
| 			enabled = true | ||||
| 			Enabled = true | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| @ -49,7 +49,7 @@ func init() { | ||||
| // NewMeter create a new metrics Meter, either a real one of a NOP stub depending
 | ||||
| // on the metrics flag.
 | ||||
| func NewMeter(name string) metrics.Meter { | ||||
| 	if !enabled { | ||||
| 	if !Enabled { | ||||
| 		return new(metrics.NilMeter) | ||||
| 	} | ||||
| 	return metrics.GetOrRegisterMeter(name, metrics.DefaultRegistry) | ||||
| @ -58,7 +58,7 @@ func NewMeter(name string) metrics.Meter { | ||||
| // NewTimer create a new metrics Timer, either a real one of a NOP stub depending
 | ||||
| // on the metrics flag.
 | ||||
| func NewTimer(name string) metrics.Timer { | ||||
| 	if !enabled { | ||||
| 	if !Enabled { | ||||
| 		return new(metrics.NilTimer) | ||||
| 	} | ||||
| 	return metrics.GetOrRegisterTimer(name, metrics.DefaultRegistry) | ||||
| @ -68,7 +68,7 @@ func NewTimer(name string) metrics.Timer { | ||||
| // process.
 | ||||
| func CollectProcessMetrics(refresh time.Duration) { | ||||
| 	// Short circuit if the metrics system is disabled
 | ||||
| 	if !enabled { | ||||
| 	if !Enabled { | ||||
| 		return | ||||
| 	} | ||||
| 	// Create the various data collectors
 | ||||
|  | ||||
| @ -38,8 +38,14 @@ type meteredConn struct { | ||||
| } | ||||
| 
 | ||||
| // newMeteredConn creates a new metered connection, also bumping the ingress or
 | ||||
| // egress connection meter.
 | ||||
| // egress connection meter. If the metrics system is disabled, this function
 | ||||
| // returns the original object.
 | ||||
| func newMeteredConn(conn net.Conn, ingress bool) net.Conn { | ||||
| 	// Short circuit if metrics are disabled
 | ||||
| 	if !metrics.Enabled { | ||||
| 		return conn | ||||
| 	} | ||||
| 	// Otherwise bump the connection counters and wrap the connection
 | ||||
| 	if ingress { | ||||
| 		ingressConnectMeter.Mark(1) | ||||
| 	} else { | ||||
|  | ||||
		Loading…
	
		Reference in New Issue
	
	Block a user