Increase number of requested block bodies in chain sync (#10247)
* Increase the number of block bodies requested during Sync. * Increase the number of block bodies requested during Sync. * Check if our peer is an older parity client with the bug of not handling large requests properly * Add a ClientVersion struct and a ClientCapabilites trait * Make ClientVersion its own module * Refactor and extend use of ClientVersion * Replace strings with ClientVersion in PeerInfo * Group further functionality in ClientCapabilities * Move parity client version data from tuple to its own struct. * Implement accessor methods for ParityClientData and remove them from ClientVersion. * Minor fixes * Make functions specific to parity return types specific to parity. * Test for shorter ID strings * Fix formatting and remove unneeded dependencies. * Roll back Cargo.lock * Commit last Cargo.lock * Convert from string to ClientVersion * * When checking if peer accepts service transactions just check if it's parity, remove version check. * Remove dependency on semver in ethcore-sync * Remove unnecessary String instantiation * Rename peer_info to peer_version * Update RPC test helpers * Simplify From<String> * Parse static version string only once * Update RPC tests to new ClientVersion struct * Document public members * More robust parsing of ID string * Minor changes. * Update version in which large block bodies requests appear. * Update ethcore/sync/src/block_sync.rs Co-Authored-By: elferdo <elferdo@gmail.com> * Update util/network/src/client_version.rs Co-Authored-By: elferdo <elferdo@gmail.com> * Update util/network/src/client_version.rs Co-Authored-By: elferdo <elferdo@gmail.com> * Update tests. * Minor fixes.
This commit is contained in:
		
							parent
							
								
									d5c19f8719
								
							
						
					
					
						commit
						b7e8621846
					
				
							
								
								
									
										5
									
								
								Cargo.lock
									
									
									
										generated
									
									
									
								
							
							
						
						
									
										5
									
								
								Cargo.lock
									
									
									
										generated
									
									
									
								
							@ -939,10 +939,14 @@ dependencies = [
 | 
			
		||||
 "ethereum-types 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "ethkey 0.3.0",
 | 
			
		||||
 "ipnetwork 0.12.8 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "lazy_static 1.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "libc 0.2.48 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "parity-crypto 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "parity-snappy 0.1.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "rlp 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "semver 0.9.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "serde 1.0.80 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "serde_derive 1.0.80 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
]
 | 
			
		||||
 | 
			
		||||
[[package]]
 | 
			
		||||
@ -3478,6 +3482,7 @@ version = "0.9.0"
 | 
			
		||||
source = "registry+https://github.com/rust-lang/crates.io-index"
 | 
			
		||||
dependencies = [
 | 
			
		||||
 "semver-parser 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
 "serde 1.0.80 (registry+https://github.com/rust-lang/crates.io-index)",
 | 
			
		||||
]
 | 
			
		||||
 | 
			
		||||
[[package]]
 | 
			
		||||
 | 
			
		||||
@ -24,6 +24,7 @@ use devp2p::NetworkService;
 | 
			
		||||
use network::{NetworkProtocolHandler, NetworkContext, PeerId, ProtocolId,
 | 
			
		||||
	NetworkConfiguration as BasicNetworkConfiguration, NonReservedPeerMode, Error, ErrorKind,
 | 
			
		||||
	ConnectionFilter};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
 | 
			
		||||
use types::pruning_info::PruningInfo;
 | 
			
		||||
use ethereum_types::{H256, H512, U256};
 | 
			
		||||
@ -158,7 +159,7 @@ pub struct PeerInfo {
 | 
			
		||||
	/// Public node id
 | 
			
		||||
	pub id: Option<String>,
 | 
			
		||||
	/// Node client ID
 | 
			
		||||
	pub client_version: String,
 | 
			
		||||
	pub client_version: ClientVersion,
 | 
			
		||||
	/// Capabilities
 | 
			
		||||
	pub capabilities: Vec<String>,
 | 
			
		||||
	/// Remote endpoint address
 | 
			
		||||
 | 
			
		||||
@ -29,10 +29,13 @@ use ethcore::error::{ImportErrorKind, QueueErrorKind, BlockError, Error as Ethco
 | 
			
		||||
use sync_io::SyncIo;
 | 
			
		||||
use blocks::{BlockCollection, SyncBody, SyncHeader};
 | 
			
		||||
use chain::BlockSet;
 | 
			
		||||
use network::PeerId;
 | 
			
		||||
use network::client_version::ClientCapabilities;
 | 
			
		||||
 | 
			
		||||
const MAX_HEADERS_TO_REQUEST: usize = 128;
 | 
			
		||||
const MAX_BODIES_TO_REQUEST: usize = 32;
 | 
			
		||||
const MAX_RECEPITS_TO_REQUEST: usize = 128;
 | 
			
		||||
const MAX_BODIES_TO_REQUEST_LARGE: usize = 128;
 | 
			
		||||
const MAX_BODIES_TO_REQUEST_SMALL: usize = 32; // Size request for parity clients prior to 2.4.0
 | 
			
		||||
const MAX_RECEPITS_TO_REQUEST: usize = 256;
 | 
			
		||||
const SUBCHAIN_SIZE: u64 = 256;
 | 
			
		||||
const MAX_ROUND_PARENTS: usize = 16;
 | 
			
		||||
const MAX_PARALLEL_SUBCHAIN_DOWNLOAD: usize = 5;
 | 
			
		||||
@ -464,12 +467,12 @@ impl BlockDownloader {
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	/// Find some headers or blocks to download for a peer.
 | 
			
		||||
	pub fn request_blocks(&mut self, io: &mut SyncIo, num_active_peers: usize) -> Option<BlockRequest> {
 | 
			
		||||
	pub fn request_blocks(&mut self, peer_id: PeerId, io: &mut SyncIo, num_active_peers: usize) -> Option<BlockRequest> {
 | 
			
		||||
		match self.state {
 | 
			
		||||
			State::Idle => {
 | 
			
		||||
				self.start_sync_round(io);
 | 
			
		||||
				if self.state == State::ChainHead {
 | 
			
		||||
					return self.request_blocks(io, num_active_peers);
 | 
			
		||||
					return self.request_blocks(peer_id, io, num_active_peers);
 | 
			
		||||
				}
 | 
			
		||||
			},
 | 
			
		||||
			State::ChainHead => {
 | 
			
		||||
@ -487,7 +490,15 @@ impl BlockDownloader {
 | 
			
		||||
			},
 | 
			
		||||
			State::Blocks => {
 | 
			
		||||
				// check to see if we need to download any block bodies first
 | 
			
		||||
				let needed_bodies = self.blocks.needed_bodies(MAX_BODIES_TO_REQUEST, false);
 | 
			
		||||
				let client_version = io.peer_version(peer_id);
 | 
			
		||||
 | 
			
		||||
				let number_of_bodies_to_request = if client_version.can_handle_large_requests() {
 | 
			
		||||
					MAX_BODIES_TO_REQUEST_LARGE
 | 
			
		||||
				} else {
 | 
			
		||||
					MAX_BODIES_TO_REQUEST_SMALL
 | 
			
		||||
				};
 | 
			
		||||
 | 
			
		||||
				let needed_bodies = self.blocks.needed_bodies(number_of_bodies_to_request, false);
 | 
			
		||||
				if !needed_bodies.is_empty() {
 | 
			
		||||
					return Some(BlockRequest::Bodies {
 | 
			
		||||
						hashes: needed_bodies,
 | 
			
		||||
 | 
			
		||||
@ -23,6 +23,7 @@ use ethcore::verification::queue::kind::blocks::Unverified;
 | 
			
		||||
use ethereum_types::{H256, U256};
 | 
			
		||||
use hash::keccak;
 | 
			
		||||
use network::PeerId;
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use rlp::Rlp;
 | 
			
		||||
use snapshot::ChunkType;
 | 
			
		||||
use std::time::Instant;
 | 
			
		||||
@ -107,7 +108,7 @@ impl SyncHandler {
 | 
			
		||||
 | 
			
		||||
	/// Called by peer when it is disconnecting
 | 
			
		||||
	pub fn on_peer_aborting(sync: &mut ChainSync, io: &mut SyncIo, peer_id: PeerId) {
 | 
			
		||||
		trace!(target: "sync", "== Disconnecting {}: {}", peer_id, io.peer_info(peer_id));
 | 
			
		||||
		trace!(target: "sync", "== Disconnecting {}: {}", peer_id, io.peer_version(peer_id));
 | 
			
		||||
		sync.handshaking_peers.remove(&peer_id);
 | 
			
		||||
		if sync.peers.contains_key(&peer_id) {
 | 
			
		||||
			debug!(target: "sync", "Disconnected {}", peer_id);
 | 
			
		||||
@ -133,7 +134,7 @@ impl SyncHandler {
 | 
			
		||||
 | 
			
		||||
	/// Called when a new peer is connected
 | 
			
		||||
	pub fn on_peer_connected(sync: &mut ChainSync, io: &mut SyncIo, peer: PeerId) {
 | 
			
		||||
		trace!(target: "sync", "== Connected {}: {}", peer, io.peer_info(peer));
 | 
			
		||||
		trace!(target: "sync", "== Connected {}: {}", peer, io.peer_version(peer));
 | 
			
		||||
		if let Err(e) = sync.send_status(io, peer) {
 | 
			
		||||
			debug!(target:"sync", "Error sending status request: {:?}", e);
 | 
			
		||||
			io.disconnect_peer(peer);
 | 
			
		||||
@ -579,6 +580,7 @@ impl SyncHandler {
 | 
			
		||||
			snapshot_number: if warp_protocol { Some(r.val_at(6)?) } else { None },
 | 
			
		||||
			block_set: None,
 | 
			
		||||
			private_tx_enabled: if private_tx_protocol { r.val_at(7).unwrap_or(false) } else { false },
 | 
			
		||||
			client_version: ClientVersion::from(io.peer_version(peer_id)),
 | 
			
		||||
		};
 | 
			
		||||
 | 
			
		||||
		trace!(target: "sync", "New peer {} (\
 | 
			
		||||
@ -599,12 +601,12 @@ impl SyncHandler {
 | 
			
		||||
			peer.private_tx_enabled
 | 
			
		||||
		);
 | 
			
		||||
		if io.is_expired() {
 | 
			
		||||
			trace!(target: "sync", "Status packet from expired session {}:{}", peer_id, io.peer_info(peer_id));
 | 
			
		||||
			trace!(target: "sync", "Status packet from expired session {}:{}", peer_id, io.peer_version(peer_id));
 | 
			
		||||
			return Ok(());
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if sync.peers.contains_key(&peer_id) {
 | 
			
		||||
			debug!(target: "sync", "Unexpected status packet from {}:{}", peer_id, io.peer_info(peer_id));
 | 
			
		||||
			debug!(target: "sync", "Unexpected status packet from {}:{}", peer_id, io.peer_version(peer_id));
 | 
			
		||||
			return Ok(());
 | 
			
		||||
		}
 | 
			
		||||
		let chain_info = io.chain().chain_info();
 | 
			
		||||
@ -633,7 +635,7 @@ impl SyncHandler {
 | 
			
		||||
		// Don't activate peer immediatelly when searching for common block.
 | 
			
		||||
		// Let the current sync round complete first.
 | 
			
		||||
		sync.active_peers.insert(peer_id.clone());
 | 
			
		||||
		debug!(target: "sync", "Connected {}:{}", peer_id, io.peer_info(peer_id));
 | 
			
		||||
		debug!(target: "sync", "Connected {}:{}", peer_id, io.peer_version(peer_id));
 | 
			
		||||
 | 
			
		||||
		if let Some((fork_block, _)) = sync.fork_block {
 | 
			
		||||
			SyncRequester::request_fork_header(sync, io, peer_id, fork_block);
 | 
			
		||||
 | 
			
		||||
@ -104,6 +104,7 @@ use parking_lot::{Mutex, RwLock, RwLockWriteGuard};
 | 
			
		||||
use bytes::Bytes;
 | 
			
		||||
use rlp::{RlpStream, DecoderError};
 | 
			
		||||
use network::{self, PeerId, PacketId};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use ethcore::client::{BlockChainClient, BlockStatus, BlockId, BlockChainInfo, BlockQueueInfo};
 | 
			
		||||
use ethcore::snapshot::{RestorationStatus};
 | 
			
		||||
use sync_io::SyncIo;
 | 
			
		||||
@ -342,6 +343,8 @@ pub struct PeerInfo {
 | 
			
		||||
	snapshot_number: Option<BlockNumber>,
 | 
			
		||||
	/// Block set requested
 | 
			
		||||
	block_set: Option<BlockSet>,
 | 
			
		||||
	/// Version of the software the peer is running
 | 
			
		||||
	client_version: ClientVersion,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl PeerInfo {
 | 
			
		||||
@ -964,7 +967,7 @@ impl ChainSync {
 | 
			
		||||
					if !have_latest && (higher_difficulty || force || self.state == SyncState::NewBlocks) {
 | 
			
		||||
						// check if got new blocks to download
 | 
			
		||||
						trace!(target: "sync", "Syncing with peer {}, force={}, td={:?}, our td={}, state={:?}", peer_id, force, peer_difficulty, syncing_difficulty, self.state);
 | 
			
		||||
						if let Some(request) = self.new_blocks.request_blocks(io, num_active_peers) {
 | 
			
		||||
						if let Some(request) = self.new_blocks.request_blocks(peer_id, io, num_active_peers) {
 | 
			
		||||
							SyncRequester::request_blocks(self, io, peer_id, request, BlockSet::NewBlocks);
 | 
			
		||||
							if self.state == SyncState::Idle {
 | 
			
		||||
								self.state = SyncState::Blocks;
 | 
			
		||||
@ -977,7 +980,7 @@ impl ChainSync {
 | 
			
		||||
					let equal_or_higher_difficulty = peer_difficulty.map_or(false, |pd| pd >= syncing_difficulty);
 | 
			
		||||
 | 
			
		||||
					if force || equal_or_higher_difficulty {
 | 
			
		||||
						if let Some(request) = self.old_blocks.as_mut().and_then(|d| d.request_blocks(io, num_active_peers)) {
 | 
			
		||||
						if let Some(request) = self.old_blocks.as_mut().and_then(|d| d.request_blocks(peer_id, io, num_active_peers)) {
 | 
			
		||||
							SyncRequester::request_blocks(self, io, peer_id, request, BlockSet::OldBlocks);
 | 
			
		||||
							return;
 | 
			
		||||
						}
 | 
			
		||||
@ -1459,6 +1462,7 @@ pub mod tests {
 | 
			
		||||
				snapshot_hash: None,
 | 
			
		||||
				asking_snapshot_data: None,
 | 
			
		||||
				block_set: None,
 | 
			
		||||
				client_version: ClientVersion::from(""),
 | 
			
		||||
			});
 | 
			
		||||
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
@ -21,6 +21,7 @@ use bytes::Bytes;
 | 
			
		||||
use ethereum_types::H256;
 | 
			
		||||
use fastmap::H256FastSet;
 | 
			
		||||
use network::{PeerId, PacketId};
 | 
			
		||||
use network::client_version::ClientCapabilities;
 | 
			
		||||
use rand::Rng;
 | 
			
		||||
use rlp::{Encodable, RlpStream};
 | 
			
		||||
use sync_io::SyncIo;
 | 
			
		||||
@ -41,28 +42,6 @@ use super::{
 | 
			
		||||
	TRANSACTIONS_PACKET,
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
/// Checks if peer is able to process service transactions
 | 
			
		||||
fn accepts_service_transaction(client_id: &str) -> bool {
 | 
			
		||||
	// Parity versions starting from this will accept service-transactions
 | 
			
		||||
	const SERVICE_TRANSACTIONS_VERSION: (u32, u32) = (1u32, 6u32);
 | 
			
		||||
	// Parity client string prefix
 | 
			
		||||
	const LEGACY_CLIENT_ID_PREFIX: &'static str = "Parity/";
 | 
			
		||||
	const PARITY_CLIENT_ID_PREFIX: &'static str = "Parity-Ethereum/";
 | 
			
		||||
	const VERSION_PREFIX: &'static str = "/v";
 | 
			
		||||
	
 | 
			
		||||
	let idx = client_id.rfind(VERSION_PREFIX).map(|idx| idx + VERSION_PREFIX.len()).unwrap_or(client_id.len());
 | 
			
		||||
	let splitted = if client_id.starts_with(LEGACY_CLIENT_ID_PREFIX) || client_id.starts_with(PARITY_CLIENT_ID_PREFIX) {
 | 
			
		||||
		client_id[idx..].split('.')
 | 
			
		||||
	} else {
 | 
			
		||||
		return false;
 | 
			
		||||
	};
 | 
			
		||||
 | 
			
		||||
	let ver: Vec<u32> = splitted
 | 
			
		||||
			.take(2)
 | 
			
		||||
			.filter_map(|s| s.parse().ok())
 | 
			
		||||
			.collect();
 | 
			
		||||
	ver.len() == 2 && (ver[0] > SERVICE_TRANSACTIONS_VERSION.0 || (ver[0] == SERVICE_TRANSACTIONS_VERSION.0 && ver[1] >= SERVICE_TRANSACTIONS_VERSION.1))
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// The Chain Sync Propagator: propagates data to peers
 | 
			
		||||
pub struct SyncPropagator;
 | 
			
		||||
@ -146,7 +125,7 @@ impl SyncPropagator {
 | 
			
		||||
		// most of times service_transactions will be empty
 | 
			
		||||
		// => there's no need to merge packets
 | 
			
		||||
		if !service_transactions.is_empty() {
 | 
			
		||||
			let service_transactions_peers = SyncPropagator::select_peers_for_transactions(sync, |peer_id| accepts_service_transaction(&io.peer_info(*peer_id)));
 | 
			
		||||
			let service_transactions_peers = SyncPropagator::select_peers_for_transactions(sync, |peer_id| io.peer_version(*peer_id).accepts_service_transaction());
 | 
			
		||||
			let service_transactions_affected_peers = SyncPropagator::propagate_transactions_to_peers(
 | 
			
		||||
				sync, io, service_transactions_peers, service_transactions, &mut should_continue
 | 
			
		||||
			);
 | 
			
		||||
@ -451,6 +430,7 @@ mod tests {
 | 
			
		||||
				snapshot_hash: None,
 | 
			
		||||
				asking_snapshot_data: None,
 | 
			
		||||
				block_set: None,
 | 
			
		||||
				client_version: ClientVersion::from(""),
 | 
			
		||||
			});
 | 
			
		||||
		let ss = TestSnapshotService::new();
 | 
			
		||||
		let mut io = TestIo::new(&mut client, &ss, &queue, None);
 | 
			
		||||
@ -598,20 +578,17 @@ mod tests {
 | 
			
		||||
		io.peers_info.insert(1, "Geth".to_owned());
 | 
			
		||||
		// and peer#2 is Parity, accepting service transactions
 | 
			
		||||
		insert_dummy_peer(&mut sync, 2, block_hash);
 | 
			
		||||
		io.peers_info.insert(2, "Parity-Ethereum/v2.6".to_owned());
 | 
			
		||||
		// and peer#3 is Parity, discarding service transactions
 | 
			
		||||
		io.peers_info.insert(2, "Parity-Ethereum/v2.6.0/linux/rustc".to_owned());
 | 
			
		||||
		// and peer#3 is Parity, accepting service transactions
 | 
			
		||||
		insert_dummy_peer(&mut sync, 3, block_hash);
 | 
			
		||||
		io.peers_info.insert(3, "Parity/v1.5".to_owned());
 | 
			
		||||
		// and peer#4 is Parity, accepting service transactions
 | 
			
		||||
		insert_dummy_peer(&mut sync, 4, block_hash);
 | 
			
		||||
		io.peers_info.insert(4, "Parity-Ethereum/ABCDEFGH/v2.7.3".to_owned());
 | 
			
		||||
		io.peers_info.insert(3, "Parity-Ethereum/ABCDEFGH/v2.7.3/linux/rustc".to_owned());
 | 
			
		||||
 | 
			
		||||
		// and new service transaction is propagated to peers
 | 
			
		||||
		SyncPropagator::propagate_new_transactions(&mut sync, &mut io, || true);
 | 
			
		||||
 | 
			
		||||
		// peer#2 && peer#4 are receiving service transaction
 | 
			
		||||
		// peer#2 && peer#3 are receiving service transaction
 | 
			
		||||
		assert!(io.packets.iter().any(|p| p.packet_id == 0x02 && p.recipient == 2)); // TRANSACTIONS_PACKET
 | 
			
		||||
		assert!(io.packets.iter().any(|p| p.packet_id == 0x02 && p.recipient == 4)); // TRANSACTIONS_PACKET
 | 
			
		||||
		assert!(io.packets.iter().any(|p| p.packet_id == 0x02 && p.recipient == 3)); // TRANSACTIONS_PACKET
 | 
			
		||||
		assert_eq!(io.packets.len(), 2);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
@ -628,7 +605,7 @@ mod tests {
 | 
			
		||||
 | 
			
		||||
		// when peer#1 is Parity, accepting service transactions
 | 
			
		||||
		insert_dummy_peer(&mut sync, 1, block_hash);
 | 
			
		||||
		io.peers_info.insert(1, "Parity-Ethereum/v2.6".to_owned());
 | 
			
		||||
		io.peers_info.insert(1, "Parity-Ethereum/v2.6.0/linux/rustc".to_owned());
 | 
			
		||||
 | 
			
		||||
		// and service + non-service transactions are propagated to peers
 | 
			
		||||
		SyncPropagator::propagate_new_transactions(&mut sync, &mut io, || true);
 | 
			
		||||
 | 
			
		||||
@ -62,6 +62,7 @@ impl SyncRequester {
 | 
			
		||||
		for h in &hashes {
 | 
			
		||||
			rlp.append(&h.clone());
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		SyncRequester::send_request(sync, io, peer_id, PeerAsking::BlockBodies, GET_BLOCK_BODIES_PACKET, rlp.out());
 | 
			
		||||
		let peer = sync.peers.get_mut(&peer_id).expect("peer_id may originate either from on_packet, where it is already validated or from enumerating self.peers. qed");
 | 
			
		||||
		peer.asking_blocks = hashes;
 | 
			
		||||
 | 
			
		||||
@ -91,7 +91,7 @@ impl SyncSupplier {
 | 
			
		||||
			// Packets that require the peer to be confirmed
 | 
			
		||||
			_ => {
 | 
			
		||||
				if !sync.read().peers.contains_key(&peer) {
 | 
			
		||||
					debug!(target:"sync", "Unexpected packet {} from unregistered peer: {}:{}", packet_id, peer, io.peer_info(peer));
 | 
			
		||||
					debug!(target:"sync", "Unexpected packet {} from unregistered peer: {}:{}", packet_id, peer, io.peer_version(peer));
 | 
			
		||||
					return;
 | 
			
		||||
				}
 | 
			
		||||
				debug!(target: "sync", "{} -> Dispatching packet: {}", peer, packet_id);
 | 
			
		||||
 | 
			
		||||
@ -16,6 +16,7 @@
 | 
			
		||||
 | 
			
		||||
use std::collections::HashMap;
 | 
			
		||||
use network::{NetworkContext, PeerId, PacketId, Error, SessionInfo, ProtocolId};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use bytes::Bytes;
 | 
			
		||||
use ethcore::client::BlockChainClient;
 | 
			
		||||
use types::BlockNumber;
 | 
			
		||||
@ -40,9 +41,9 @@ pub trait SyncIo {
 | 
			
		||||
	fn chain(&self) -> &BlockChainClient;
 | 
			
		||||
	/// Get the snapshot service.
 | 
			
		||||
	fn snapshot_service(&self) -> &SnapshotService;
 | 
			
		||||
	/// Returns peer identifier string
 | 
			
		||||
	fn peer_info(&self, peer_id: PeerId) -> String {
 | 
			
		||||
		peer_id.to_string()
 | 
			
		||||
	/// Returns peer version identifier
 | 
			
		||||
	fn peer_version(&self, peer_id: PeerId) -> ClientVersion {
 | 
			
		||||
		ClientVersion::from(peer_id.to_string())
 | 
			
		||||
	}
 | 
			
		||||
	/// Returns information on p2p session
 | 
			
		||||
	fn peer_session_info(&self, peer_id: PeerId) -> Option<SessionInfo>;
 | 
			
		||||
@ -134,7 +135,7 @@ impl<'s> SyncIo for NetSyncIo<'s> {
 | 
			
		||||
		self.network.protocol_version(*protocol, peer_id).unwrap_or(0)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn peer_info(&self, peer_id: PeerId) -> String {
 | 
			
		||||
	fn peer_version(&self, peer_id: PeerId) -> ClientVersion {
 | 
			
		||||
		self.network.peer_client_version(peer_id)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -20,6 +20,7 @@ use ethereum_types::H256;
 | 
			
		||||
use parking_lot::{RwLock, Mutex};
 | 
			
		||||
use bytes::Bytes;
 | 
			
		||||
use network::{self, PeerId, ProtocolId, PacketId, SessionInfo};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use tests::snapshot::*;
 | 
			
		||||
use ethcore::client::{TestBlockChainClient, BlockChainClient, Client as EthcoreClient,
 | 
			
		||||
	ClientConfig, ChainNotify, NewBlocks, ChainMessageType, ClientIoMessage};
 | 
			
		||||
@ -118,10 +119,12 @@ impl<'p, C> SyncIo for TestIo<'p, C> where C: FlushingBlockChainClient, C: 'p {
 | 
			
		||||
		&*self.chain
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn peer_info(&self, peer_id: PeerId) -> String {
 | 
			
		||||
		self.peers_info.get(&peer_id)
 | 
			
		||||
	fn peer_version(&self, peer_id: PeerId) -> ClientVersion {
 | 
			
		||||
		let client_id = self.peers_info.get(&peer_id)
 | 
			
		||||
			.cloned()
 | 
			
		||||
			.unwrap_or_else(|| peer_id.to_string())
 | 
			
		||||
			.unwrap_or_else(|| peer_id.to_string());
 | 
			
		||||
 | 
			
		||||
		ClientVersion::from(client_id)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn snapshot_service(&self) -> &SnapshotService {
 | 
			
		||||
 | 
			
		||||
@ -42,6 +42,7 @@ ethcore-io = { path = "../util/io" }
 | 
			
		||||
ethcore-light = { path = "../ethcore/light" }
 | 
			
		||||
ethcore-logger = { path = "../parity/logger" }
 | 
			
		||||
ethcore-miner = { path = "../miner" }
 | 
			
		||||
ethcore-network = { path = "../util/network" }
 | 
			
		||||
ethcore-private-tx = { path = "../ethcore/private-tx" }
 | 
			
		||||
ethcore-sync = { path = "../ethcore/sync" }
 | 
			
		||||
ethereum-types = "0.4"
 | 
			
		||||
 | 
			
		||||
@ -52,6 +52,7 @@ extern crate ethcore_io as io;
 | 
			
		||||
extern crate ethcore_light as light;
 | 
			
		||||
extern crate ethcore_logger;
 | 
			
		||||
extern crate ethcore_miner as miner;
 | 
			
		||||
extern crate ethcore_network as network;
 | 
			
		||||
extern crate ethcore_private_tx;
 | 
			
		||||
extern crate ethcore_sync as sync;
 | 
			
		||||
extern crate ethereum_types;
 | 
			
		||||
 | 
			
		||||
@ -20,6 +20,7 @@ use std::collections::BTreeMap;
 | 
			
		||||
use ethereum_types::H256;
 | 
			
		||||
use parking_lot::RwLock;
 | 
			
		||||
use sync::{SyncProvider, EthProtocolInfo, SyncStatus, SyncState, PeerInfo, TransactionStats};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
 | 
			
		||||
/// TestSyncProvider config.
 | 
			
		||||
pub struct Config {
 | 
			
		||||
@ -75,7 +76,7 @@ impl SyncProvider for TestSyncProvider {
 | 
			
		||||
		vec![
 | 
			
		||||
			PeerInfo {
 | 
			
		||||
				id: Some("node1".to_owned()),
 | 
			
		||||
				client_version: "Parity-Ethereum/1".to_owned(),
 | 
			
		||||
				client_version: ClientVersion::from("Parity-Ethereum/1/v2.4.0/linux/rustc"),
 | 
			
		||||
				capabilities: vec!["eth/62".to_owned(), "eth/63".to_owned()],
 | 
			
		||||
				remote_address: "127.0.0.1:7777".to_owned(),
 | 
			
		||||
				local_address: "127.0.0.1:8888".to_owned(),
 | 
			
		||||
@ -88,7 +89,7 @@ impl SyncProvider for TestSyncProvider {
 | 
			
		||||
			},
 | 
			
		||||
			PeerInfo {
 | 
			
		||||
				id: None,
 | 
			
		||||
				client_version: "Parity-Ethereum/2".to_owned(),
 | 
			
		||||
				client_version: ClientVersion::from("Parity-Ethereum/2/v2.4.0/linux/rustc"),
 | 
			
		||||
				capabilities: vec!["eth/63".to_owned(), "eth/64".to_owned()],
 | 
			
		||||
				remote_address: "Handshake".to_owned(),
 | 
			
		||||
				local_address: "127.0.0.1:3333".to_owned(),
 | 
			
		||||
 | 
			
		||||
@ -251,7 +251,7 @@ fn rpc_parity_net_peers() {
 | 
			
		||||
	let io = deps.default_client();
 | 
			
		||||
 | 
			
		||||
	let request = r#"{"jsonrpc": "2.0", "method": "parity_netPeers", "params":[], "id": 1}"#;
 | 
			
		||||
	let response = r#"{"jsonrpc":"2.0","result":{"active":0,"connected":120,"max":50,"peers":[{"caps":["eth/62","eth/63"],"id":"node1","name":"Parity-Ethereum/1","network":{"localAddress":"127.0.0.1:8888","remoteAddress":"127.0.0.1:7777"},"protocols":{"eth":{"difficulty":"0x28","head":"0000000000000000000000000000000000000000000000000000000000000032","version":62},"pip":null}},{"caps":["eth/63","eth/64"],"id":null,"name":"Parity-Ethereum/2","network":{"localAddress":"127.0.0.1:3333","remoteAddress":"Handshake"},"protocols":{"eth":{"difficulty":null,"head":"000000000000000000000000000000000000000000000000000000000000003c","version":64},"pip":null}}]},"id":1}"#;
 | 
			
		||||
	let response = r#"{"jsonrpc":"2.0","result":{"active":0,"connected":120,"max":50,"peers":[{"caps":["eth/62","eth/63"],"id":"node1","name":{"ParityClient":{"can_handle_large_requests":true,"compiler":"rustc","identity":"1","name":"Parity-Ethereum","os":"linux","semver":"2.4.0"}},"network":{"localAddress":"127.0.0.1:8888","remoteAddress":"127.0.0.1:7777"},"protocols":{"eth":{"difficulty":"0x28","head":"0000000000000000000000000000000000000000000000000000000000000032","version":62},"pip":null}},{"caps":["eth/63","eth/64"],"id":null,"name":{"ParityClient":{"can_handle_large_requests":true,"compiler":"rustc","identity":"2","name":"Parity-Ethereum","os":"linux","semver":"2.4.0"}},"network":{"localAddress":"127.0.0.1:3333","remoteAddress":"Handshake"},"protocols":{"eth":{"difficulty":null,"head":"000000000000000000000000000000000000000000000000000000000000003c","version":64},"pip":null}}]},"id":1}"#;
 | 
			
		||||
 | 
			
		||||
	assert_eq!(io.handle_request_sync(request), Some(response.to_owned()));
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -14,6 +14,7 @@
 | 
			
		||||
// You should have received a copy of the GNU General Public License
 | 
			
		||||
// along with Parity Ethereum.  If not, see <http://www.gnu.org/licenses/>.
 | 
			
		||||
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use std::collections::BTreeMap;
 | 
			
		||||
use sync::{self, PeerInfo as SyncPeerInfo, TransactionStats as SyncTransactionStats};
 | 
			
		||||
use serde::{Serialize, Serializer};
 | 
			
		||||
@ -54,7 +55,7 @@ pub struct PeerInfo {
 | 
			
		||||
	/// Public node id
 | 
			
		||||
	pub id: Option<String>,
 | 
			
		||||
	/// Node client ID
 | 
			
		||||
	pub name: String,
 | 
			
		||||
	pub name: ClientVersion,
 | 
			
		||||
	/// Capabilities
 | 
			
		||||
	pub caps: Vec<String>,
 | 
			
		||||
	/// Network information
 | 
			
		||||
 | 
			
		||||
@ -42,6 +42,7 @@ use network::{NetworkConfiguration, NetworkIoMessage, ProtocolId, PeerId, Packet
 | 
			
		||||
use network::{NonReservedPeerMode, NetworkContext as NetworkContextTrait};
 | 
			
		||||
use network::{SessionInfo, Error, ErrorKind, DisconnectReason, NetworkProtocolHandler};
 | 
			
		||||
use discovery::{Discovery, TableUpdates, NodeEntry, MAX_DATAGRAM_SIZE};
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use ip_utils::{map_external_address, select_public_address};
 | 
			
		||||
use parity_path::restrict_permissions_owner;
 | 
			
		||||
use parking_lot::{Mutex, RwLock};
 | 
			
		||||
@ -180,8 +181,8 @@ impl<'s> NetworkContextTrait for NetworkContext<'s> {
 | 
			
		||||
		Ok(())
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> String {
 | 
			
		||||
		self.resolve_session(peer).map_or("unknown".to_owned(), |s| s.lock().info.client_version.clone())
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> ClientVersion {
 | 
			
		||||
		self.resolve_session(peer).map_or(ClientVersion::from("unknown").to_owned(), |s| s.lock().info.client_version.clone())
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn session_info(&self, peer: PeerId) -> Option<SessionInfo> {
 | 
			
		||||
 | 
			
		||||
@ -29,6 +29,7 @@ use handshake::Handshake;
 | 
			
		||||
use io::{IoContext, StreamToken};
 | 
			
		||||
use network::{Error, ErrorKind, DisconnectReason, SessionInfo, ProtocolId, PeerCapabilityInfo};
 | 
			
		||||
use network::SessionCapabilityInfo;
 | 
			
		||||
use network::client_version::ClientVersion;
 | 
			
		||||
use host::*;
 | 
			
		||||
use node_table::NodeId;
 | 
			
		||||
use snappy;
 | 
			
		||||
@ -112,7 +113,7 @@ impl Session {
 | 
			
		||||
			had_hello: false,
 | 
			
		||||
			info: SessionInfo {
 | 
			
		||||
				id: id.cloned(),
 | 
			
		||||
				client_version: String::new(),
 | 
			
		||||
				client_version: ClientVersion::from(""),
 | 
			
		||||
				protocol_version: 0,
 | 
			
		||||
				capabilities: Vec::new(),
 | 
			
		||||
				peer_capabilities: Vec::new(),
 | 
			
		||||
@ -419,7 +420,8 @@ impl Session {
 | 
			
		||||
	fn read_hello<Message>(&mut self, io: &IoContext<Message>, rlp: &Rlp, host: &HostInfo) -> Result<(), Error>
 | 
			
		||||
	where Message: Send + Sync + Clone {
 | 
			
		||||
		let protocol = rlp.val_at::<u32>(0)?;
 | 
			
		||||
		let client_version = rlp.val_at::<String>(1)?;
 | 
			
		||||
		let client_version_string = rlp.val_at::<String>(1)?;
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string);
 | 
			
		||||
		let peer_caps: Vec<PeerCapabilityInfo> = rlp.list_at(2)?;
 | 
			
		||||
		let id = rlp.val_at::<NodeId>(4)?;
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -80,7 +80,7 @@ impl NetworkProtocolHandler for TestProtocol {
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn connected(&self, io: &NetworkContext, peer: &PeerId) {
 | 
			
		||||
		assert!(io.peer_client_version(*peer).contains("Parity"));
 | 
			
		||||
		assert!(io.peer_client_version(*peer).to_string().contains("Parity"));
 | 
			
		||||
		if self.drop_session {
 | 
			
		||||
			io.disconnect_peer(*peer)
 | 
			
		||||
		} else {
 | 
			
		||||
 | 
			
		||||
@ -13,9 +13,13 @@ ethcore-io = { path = "../io" }
 | 
			
		||||
ethereum-types = "0.4"
 | 
			
		||||
ethkey = { path = "../../accounts/ethkey" }
 | 
			
		||||
ipnetwork = "0.12.6"
 | 
			
		||||
lazy_static = "1.0"
 | 
			
		||||
rlp = { version = "0.3.0", features = ["ethereum"] }
 | 
			
		||||
libc = "0.2"
 | 
			
		||||
parity-snappy = "0.1"
 | 
			
		||||
semver = {version="0.9.0", features=["serde"]}
 | 
			
		||||
serde = "1.0"
 | 
			
		||||
serde_derive = "1.0"
 | 
			
		||||
 | 
			
		||||
[dev-dependencies]
 | 
			
		||||
assert_matches = "1.2"
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										515
									
								
								util/network/src/client_version.rs
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										515
									
								
								util/network/src/client_version.rs
									
									
									
									
									
										Normal file
									
								
							@ -0,0 +1,515 @@
 | 
			
		||||
// Copyright 2015-2019 Parity Technologies (UK) Ltd.
 | 
			
		||||
// This file is part of Parity Ethereum.
 | 
			
		||||
 | 
			
		||||
// Parity Ethereum is free software: you can redistribute it and/or modify
 | 
			
		||||
// it under the terms of the GNU General Public License as published by
 | 
			
		||||
// the Free Software Foundation, either version 3 of the License, or
 | 
			
		||||
// (at your option) any later version.
 | 
			
		||||
 | 
			
		||||
// Parity Ethereum 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 General Public License for more details.
 | 
			
		||||
 | 
			
		||||
// You should have received a copy of the GNU General Public License
 | 
			
		||||
// along with Parity Ethereum.  If not, see <http://www.gnu.org/licenses/>.
 | 
			
		||||
 | 
			
		||||
#![warn(missing_docs)]
 | 
			
		||||
 | 
			
		||||
//! Parse ethereum client ID strings and provide querying functionality
 | 
			
		||||
 | 
			
		||||
use semver::Version;
 | 
			
		||||
use std::fmt;
 | 
			
		||||
 | 
			
		||||
/// Parity client string prefix
 | 
			
		||||
const LEGACY_CLIENT_ID_PREFIX: &str = "Parity";
 | 
			
		||||
const PARITY_CLIENT_ID_PREFIX: &str = "Parity-Ethereum";
 | 
			
		||||
 | 
			
		||||
lazy_static! {
 | 
			
		||||
/// Parity versions starting from this will accept block bodies requests
 | 
			
		||||
/// of 256 bodies
 | 
			
		||||
	static ref PARITY_CLIENT_LARGE_REQUESTS_VERSION: Version = Version::parse("2.4.0").unwrap();
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Description of the software version running in a peer
 | 
			
		||||
/// according to https://github.com/ethereum/wiki/wiki/Client-Version-Strings
 | 
			
		||||
/// This structure as it is represents the format used by Parity clients. Other
 | 
			
		||||
/// vendors may provide additional fields.
 | 
			
		||||
#[derive(Clone,Debug,PartialEq,Eq,Serialize)]
 | 
			
		||||
pub struct ParityClientData {
 | 
			
		||||
	name: String,
 | 
			
		||||
	identity: Option<String>,
 | 
			
		||||
	semver: Version,
 | 
			
		||||
	os: String,
 | 
			
		||||
	compiler: String,
 | 
			
		||||
 | 
			
		||||
	// Capability flags, should be calculated in constructor
 | 
			
		||||
	can_handle_large_requests: bool,
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Accessor methods for ParityClientData. This will probably
 | 
			
		||||
/// need to be abstracted away into a trait.
 | 
			
		||||
impl ParityClientData {
 | 
			
		||||
	fn new(
 | 
			
		||||
		name: String,
 | 
			
		||||
		identity: Option<String>,
 | 
			
		||||
		semver: Version,
 | 
			
		||||
		os: String,
 | 
			
		||||
		compiler: String,
 | 
			
		||||
	) -> Self {
 | 
			
		||||
		// Flags logic
 | 
			
		||||
		let can_handle_large_requests = &semver >= &PARITY_CLIENT_LARGE_REQUESTS_VERSION;
 | 
			
		||||
 | 
			
		||||
		// Instantiate and return
 | 
			
		||||
		ParityClientData {
 | 
			
		||||
			name: name,
 | 
			
		||||
			identity: identity,
 | 
			
		||||
			semver: semver,
 | 
			
		||||
			os: os,
 | 
			
		||||
			compiler: compiler,
 | 
			
		||||
 | 
			
		||||
			can_handle_large_requests: can_handle_large_requests,
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn name(&self) -> &str {
 | 
			
		||||
		self.name.as_str()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn identity(&self) -> Option<&str> {
 | 
			
		||||
		self.identity.as_ref().map(String::as_str)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn semver(&self) -> &Version {
 | 
			
		||||
		&self.semver
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn os(&self) -> &str {
 | 
			
		||||
		self.os.as_str()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn compiler(&self) -> &str {
 | 
			
		||||
		self.compiler.as_str()
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn can_handle_large_requests(&self) -> bool {
 | 
			
		||||
		self.can_handle_large_requests
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Enum describing the version of the software running on a peer.
 | 
			
		||||
#[derive(Clone,Debug,Eq,PartialEq,Serialize)]
 | 
			
		||||
pub enum ClientVersion {
 | 
			
		||||
	/// The peer runs software from parity and the string format is known
 | 
			
		||||
	ParityClient(
 | 
			
		||||
		/// The actual information fields: name, version, os, ...
 | 
			
		||||
		ParityClientData
 | 
			
		||||
	),
 | 
			
		||||
	/// The string ID is recognized as Parity but the overall format
 | 
			
		||||
	/// could not be parsed
 | 
			
		||||
	ParityUnknownFormat(String),
 | 
			
		||||
	/// Other software vendors than Parity
 | 
			
		||||
	Other(String),
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl Default for ClientVersion {
 | 
			
		||||
	fn default() -> Self {
 | 
			
		||||
		ClientVersion::Other("".to_owned())
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Provide information about what a particular version of a
 | 
			
		||||
/// peer software can do
 | 
			
		||||
pub trait ClientCapabilities {
 | 
			
		||||
	/// Parity versions before PARITY_CLIENT_LARGE_REQUESTS_VERSION would not
 | 
			
		||||
	/// check the accumulated size of a packet when building a response to a
 | 
			
		||||
	/// GET_BLOCK_BODIES request. If the packet was larger than a given limit,
 | 
			
		||||
	/// instead of sending fewer blocks no packet would get sent at all. Query
 | 
			
		||||
	/// if this version can handle requests for a large number of block bodies.
 | 
			
		||||
	fn can_handle_large_requests(&self) -> bool;
 | 
			
		||||
 | 
			
		||||
	/// Service transactions are specific to parity. Query if this version
 | 
			
		||||
	/// accepts them.
 | 
			
		||||
	fn accepts_service_transaction(&self) -> bool;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl ClientCapabilities for ClientVersion {
 | 
			
		||||
	fn can_handle_large_requests(&self) -> bool {
 | 
			
		||||
		match self {
 | 
			
		||||
			ClientVersion::ParityClient(data) => data.can_handle_large_requests(),
 | 
			
		||||
			ClientVersion::ParityUnknownFormat(_) => false, // Play it safe
 | 
			
		||||
			ClientVersion::Other(_) => true // As far as we know
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn accepts_service_transaction(&self) -> bool {
 | 
			
		||||
		match self {
 | 
			
		||||
			ClientVersion::ParityClient(_) => true,
 | 
			
		||||
			ClientVersion::ParityUnknownFormat(_) => true,
 | 
			
		||||
			ClientVersion::Other(_) => false
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
fn is_parity(client_id: &str) -> bool {
 | 
			
		||||
	client_id.starts_with(LEGACY_CLIENT_ID_PREFIX) || client_id.starts_with(PARITY_CLIENT_ID_PREFIX)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Parse known parity formats. Recognizes either a short format with four fields
 | 
			
		||||
/// or a long format which includes the same fields and an identity one.
 | 
			
		||||
fn parse_parity_format(client_version: &str) -> Result<ParityClientData, ()> {
 | 
			
		||||
	const PARITY_ID_STRING_MINIMUM_TOKENS: usize = 4;
 | 
			
		||||
 | 
			
		||||
	let tokens: Vec<&str> = client_version.split("/").collect();
 | 
			
		||||
 | 
			
		||||
	if tokens.len() < PARITY_ID_STRING_MINIMUM_TOKENS {
 | 
			
		||||
		return Err(())
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	let name = tokens[0];
 | 
			
		||||
 | 
			
		||||
	let identity = if tokens.len() - 3 > 1 {
 | 
			
		||||
		Some(tokens[1..(tokens.len() - 3)].join("/"))
 | 
			
		||||
	} else {
 | 
			
		||||
		None
 | 
			
		||||
	};
 | 
			
		||||
 | 
			
		||||
	let compiler = tokens[tokens.len() - 1];
 | 
			
		||||
	let os = tokens[tokens.len() - 2];
 | 
			
		||||
 | 
			
		||||
	// If version is in the right position and valid format return a valid
 | 
			
		||||
	// result. Otherwise return an error.
 | 
			
		||||
	get_number_from_version(tokens[tokens.len() - 3])
 | 
			
		||||
		.and_then(|v| Version::parse(v).ok())
 | 
			
		||||
		.map(|semver| ParityClientData::new(
 | 
			
		||||
			name.to_owned(),
 | 
			
		||||
			identity,
 | 
			
		||||
			semver,
 | 
			
		||||
			os.to_owned(),
 | 
			
		||||
			compiler.to_owned(),
 | 
			
		||||
		))
 | 
			
		||||
		.ok_or(())
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/// Parse a version string and return the corresponding
 | 
			
		||||
/// ClientVersion. Only Parity clients are destructured right now, other
 | 
			
		||||
/// strings will just get wrapped in a variant so that the information is
 | 
			
		||||
/// not lost.
 | 
			
		||||
/// The parsing for parity may still fail, in which case return a ParityUnknownFormat with
 | 
			
		||||
/// the original version string. TryFrom would be a better trait to implement.
 | 
			
		||||
impl<T> From<T> for ClientVersion
 | 
			
		||||
where T: AsRef<str> {
 | 
			
		||||
	fn from(client_version: T) -> Self {
 | 
			
		||||
		let client_version_str: &str = client_version.as_ref();
 | 
			
		||||
 | 
			
		||||
		if !is_parity(client_version_str) {
 | 
			
		||||
			return ClientVersion::Other(client_version_str.to_owned());
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if let Ok(data) = parse_parity_format(client_version_str) {
 | 
			
		||||
			ClientVersion::ParityClient(data)
 | 
			
		||||
		} else {
 | 
			
		||||
			ClientVersion::ParityUnknownFormat(client_version_str.to_owned())
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
fn format_parity_version_string(client_version: &ParityClientData, f: &mut fmt::Formatter) -> std::fmt::Result {
 | 
			
		||||
	let name = client_version.name();
 | 
			
		||||
	let semver = client_version.semver();
 | 
			
		||||
	let os = client_version.os();
 | 
			
		||||
	let compiler = client_version.compiler();
 | 
			
		||||
 | 
			
		||||
	match client_version.identity() {
 | 
			
		||||
		None => write!(f, "{}/v{}/{}/{}", name, semver, os, compiler),
 | 
			
		||||
		Some(identity) => write!(f, "{}/{}/v{}/{}/{}", name, identity, semver, os, compiler),
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
impl fmt::Display for ClientVersion {
 | 
			
		||||
	fn fmt(&self, f: &mut fmt::Formatter) -> std::fmt::Result {
 | 
			
		||||
		match self {
 | 
			
		||||
			ClientVersion::ParityClient(data) => format_parity_version_string(data, f),
 | 
			
		||||
			ClientVersion::ParityUnknownFormat(id) => write!(f, "{}", id),
 | 
			
		||||
			ClientVersion::Other(id) => write!(f, "{}", id)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
fn get_number_from_version(version: &str) -> Option<&str> {
 | 
			
		||||
	if version.starts_with("v") {
 | 
			
		||||
		return version.get(1..);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	None
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
#[cfg(test)]
 | 
			
		||||
pub mod tests {
 | 
			
		||||
	use super::*;
 | 
			
		||||
 | 
			
		||||
	const PARITY_CLIENT_SEMVER: &str = "2.4.0";
 | 
			
		||||
	const PARITY_CLIENT_OLD_SEMVER: &str = "2.2.0";
 | 
			
		||||
	const PARITY_CLIENT_OS: &str = "linux";
 | 
			
		||||
	const PARITY_CLIENT_COMPILER: &str = "rustc";
 | 
			
		||||
	const PARITY_CLIENT_IDENTITY: &str = "ExpanseSOLO";
 | 
			
		||||
	const PARITY_CLIENT_MULTITOKEN_IDENTITY: &str = "ExpanseSOLO/abc/v1.2.3";
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
	fn make_default_version_string() -> String {
 | 
			
		||||
		format!(
 | 
			
		||||
			"{}/v{}/{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			PARITY_CLIENT_COMPILER
 | 
			
		||||
		)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn make_default_long_version_string() -> String {
 | 
			
		||||
		format!(
 | 
			
		||||
			"{}/{}/v{}/{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_IDENTITY,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			PARITY_CLIENT_COMPILER
 | 
			
		||||
		)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn make_multitoken_identity_long_version_string() -> String {
 | 
			
		||||
		format!(
 | 
			
		||||
			"{}/{}/v{}/{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_MULTITOKEN_IDENTITY,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			PARITY_CLIENT_COMPILER
 | 
			
		||||
		)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn make_old_semver_version_string() -> String {
 | 
			
		||||
		format!(
 | 
			
		||||
			"{}/v{}/{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_OLD_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			PARITY_CLIENT_COMPILER
 | 
			
		||||
		)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_from_empty_string_then_default() {
 | 
			
		||||
		let default = ClientVersion::default();
 | 
			
		||||
 | 
			
		||||
		assert_eq!(ClientVersion::from(""), default);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn get_number_from_version_when_valid_then_number() {
 | 
			
		||||
		let version_string = format!("v{}", PARITY_CLIENT_SEMVER);
 | 
			
		||||
 | 
			
		||||
		assert_eq!(get_number_from_version(&version_string).unwrap(), PARITY_CLIENT_SEMVER);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_str_parity_format_and_valid_then_all_fields_match() {
 | 
			
		||||
		let client_version_string = make_default_version_string();
 | 
			
		||||
 | 
			
		||||
		if let ClientVersion::ParityClient(client_version) = ClientVersion::from(client_version_string.as_str()) {
 | 
			
		||||
			assert_eq!(client_version.name(), PARITY_CLIENT_ID_PREFIX);
 | 
			
		||||
			assert_eq!(*client_version.semver(), Version::parse(PARITY_CLIENT_SEMVER).unwrap());
 | 
			
		||||
			assert_eq!(client_version.os(), PARITY_CLIENT_OS);
 | 
			
		||||
			assert_eq!(client_version.compiler(), PARITY_CLIENT_COMPILER);
 | 
			
		||||
		} else {
 | 
			
		||||
			panic!("shouldn't be here");
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_str_parity_long_format_and_valid_then_all_fields_match() {
 | 
			
		||||
		let client_version_string = make_default_long_version_string();
 | 
			
		||||
 | 
			
		||||
		if let ClientVersion::ParityClient(client_version) = ClientVersion::from(client_version_string.as_str()) {
 | 
			
		||||
			assert_eq!(client_version.name(), PARITY_CLIENT_ID_PREFIX);
 | 
			
		||||
			assert_eq!(client_version.identity().unwrap(), PARITY_CLIENT_IDENTITY);
 | 
			
		||||
			assert_eq!(*client_version.semver(), Version::parse(PARITY_CLIENT_SEMVER).unwrap());
 | 
			
		||||
			assert_eq!(client_version.os(), PARITY_CLIENT_OS);
 | 
			
		||||
			assert_eq!(client_version.compiler(), PARITY_CLIENT_COMPILER);
 | 
			
		||||
		} else {
 | 
			
		||||
			panic!("shouldnt be here");
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_str_parity_long_format_and_valid_and_identity_multiple_tokens_then_all_fields_match() {
 | 
			
		||||
		let client_version_string = make_multitoken_identity_long_version_string();
 | 
			
		||||
 | 
			
		||||
		if let ClientVersion::ParityClient(client_version) = ClientVersion::from(client_version_string.as_str()) {
 | 
			
		||||
			assert_eq!(client_version.name(), PARITY_CLIENT_ID_PREFIX);
 | 
			
		||||
			assert_eq!(client_version.identity().unwrap(), PARITY_CLIENT_MULTITOKEN_IDENTITY);
 | 
			
		||||
			assert_eq!(*client_version.semver(), Version::parse(PARITY_CLIENT_SEMVER).unwrap());
 | 
			
		||||
			assert_eq!(client_version.os(), PARITY_CLIENT_OS);
 | 
			
		||||
			assert_eq!(client_version.compiler(), PARITY_CLIENT_COMPILER);
 | 
			
		||||
		} else {
 | 
			
		||||
			panic!("shouldnt be here");
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_string_parity_format_and_valid_then_all_fields_match() {
 | 
			
		||||
		let client_version_string: String = make_default_version_string();
 | 
			
		||||
 | 
			
		||||
		if let ClientVersion::ParityClient(client_version) = ClientVersion::from(client_version_string.as_str()) {
 | 
			
		||||
			assert_eq!(client_version.name(), PARITY_CLIENT_ID_PREFIX);
 | 
			
		||||
			assert_eq!(*client_version.semver(), Version::parse(PARITY_CLIENT_SEMVER).unwrap());
 | 
			
		||||
			assert_eq!(client_version.os(), PARITY_CLIENT_OS);
 | 
			
		||||
			assert_eq!(client_version.compiler(), PARITY_CLIENT_COMPILER);
 | 
			
		||||
		} else {
 | 
			
		||||
			panic!("shouldn't be here");
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_parity_format_and_invalid_then_equals_parity_unknown_client_version_string() {
 | 
			
		||||
		// This is invalid because version has no leading 'v'
 | 
			
		||||
		let client_version_string = format!(
 | 
			
		||||
			"{}/{}/{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			PARITY_CLIENT_COMPILER);
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		let parity_unknown = ClientVersion::ParityUnknownFormat(client_version_string.to_string());
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version, parity_unknown);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_parity_format_without_identity_and_missing_compiler_field_then_equals_parity_unknown_client_version_string() {
 | 
			
		||||
		let client_version_string = format!(
 | 
			
		||||
			"{}/v{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			);
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		let parity_unknown = ClientVersion::ParityUnknownFormat(client_version_string.to_string());
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version, parity_unknown);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_parity_format_with_identity_and_missing_compiler_field_then_equals_parity_unknown_client_version_string() {
 | 
			
		||||
		let client_version_string = format!(
 | 
			
		||||
			"{}/{}/v{}/{}",
 | 
			
		||||
			PARITY_CLIENT_ID_PREFIX,
 | 
			
		||||
			PARITY_CLIENT_IDENTITY,
 | 
			
		||||
			PARITY_CLIENT_SEMVER,
 | 
			
		||||
			PARITY_CLIENT_OS,
 | 
			
		||||
			);
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		let parity_unknown = ClientVersion::ParityUnknownFormat(client_version_string.to_string());
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version, parity_unknown);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_not_parity_format_and_valid_then_other_with_client_version_string() {
 | 
			
		||||
		let client_version_string = "Geth/main.jnode.network/v1.8.21-stable-9dc5d1a9/linux";
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string);
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version, ClientVersion::Other(client_version_string.to_string()));
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_parity_format_and_valid_then_to_string_equal() {
 | 
			
		||||
		let client_version_string: String = make_default_version_string();
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version.to_string(), client_version_string);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_other_then_to_string_equal_input_string() {
 | 
			
		||||
		let client_version_string: String = "Other".to_string();
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from("Other");
 | 
			
		||||
 | 
			
		||||
		assert_eq!(client_version.to_string(), client_version_string);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_capabilities_when_parity_old_version_then_handles_large_requests_false() {
 | 
			
		||||
		let client_version_string: String = make_old_semver_version_string();
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		assert!(!client_version.can_handle_large_requests());
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_capabilities_when_parity_beta_version_then_not_handles_large_requests_true() {
 | 
			
		||||
		let client_version_string: String = format!(
 | 
			
		||||
			"{}/v{}/{}/{}",
 | 
			
		||||
			"Parity-Ethereum",
 | 
			
		||||
			"2.4.0-beta",
 | 
			
		||||
			"x86_64-linux-gnu",
 | 
			
		||||
			"rustc1.31.1")
 | 
			
		||||
			.to_string();
 | 
			
		||||
 | 
			
		||||
		let client_version = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		assert!(!client_version.can_handle_large_requests());
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	pub fn client_version_when_to_owned_then_both_objects_equal() {
 | 
			
		||||
		let client_version_string: String = make_old_semver_version_string();
 | 
			
		||||
 | 
			
		||||
		let origin = ClientVersion::from(client_version_string.as_str());
 | 
			
		||||
 | 
			
		||||
		let borrowed = &origin;
 | 
			
		||||
 | 
			
		||||
		let owned = origin.to_owned();
 | 
			
		||||
 | 
			
		||||
		assert_eq!(*borrowed, owned);
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	fn client_version_accepts_service_transaction_for_different_versions() {
 | 
			
		||||
		assert!(!ClientVersion::from("Geth").accepts_service_transaction());
 | 
			
		||||
		assert!(ClientVersion::from("Parity-Ethereum/v2.6.0/linux/rustc").accepts_service_transaction());
 | 
			
		||||
		assert!(ClientVersion::from("Parity-Ethereum/ABCDEFGH/v2.7.3/linux/rustc").accepts_service_transaction());
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	fn is_parity_when_parity_then_true() {
 | 
			
		||||
		let client_id = format!("{}/", PARITY_CLIENT_ID_PREFIX);
 | 
			
		||||
 | 
			
		||||
		assert!(is_parity(&client_id));
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	fn is_parity_when_empty_then_false() {
 | 
			
		||||
		let client_id = "";
 | 
			
		||||
 | 
			
		||||
		assert!(!is_parity(&client_id));
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	#[test]
 | 
			
		||||
	fn is_parity_when_other_then_false() {
 | 
			
		||||
		let client_id = "other";
 | 
			
		||||
 | 
			
		||||
		assert!(!is_parity(&client_id));
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
@ -24,6 +24,11 @@ extern crate rlp;
 | 
			
		||||
extern crate ipnetwork;
 | 
			
		||||
extern crate parity_snappy as snappy;
 | 
			
		||||
extern crate libc;
 | 
			
		||||
extern crate semver;
 | 
			
		||||
extern crate serde;
 | 
			
		||||
 | 
			
		||||
#[macro_use]
 | 
			
		||||
extern crate serde_derive;
 | 
			
		||||
 | 
			
		||||
#[cfg(test)] #[macro_use]
 | 
			
		||||
extern crate assert_matches;
 | 
			
		||||
@ -31,6 +36,11 @@ extern crate assert_matches;
 | 
			
		||||
#[macro_use]
 | 
			
		||||
extern crate error_chain;
 | 
			
		||||
 | 
			
		||||
#[macro_use]
 | 
			
		||||
extern crate lazy_static;
 | 
			
		||||
 | 
			
		||||
pub mod client_version;
 | 
			
		||||
 | 
			
		||||
mod connection_filter;
 | 
			
		||||
mod error;
 | 
			
		||||
 | 
			
		||||
@ -38,6 +48,7 @@ pub use connection_filter::{ConnectionFilter, ConnectionDirection};
 | 
			
		||||
pub use io::TimerToken;
 | 
			
		||||
pub use error::{Error, ErrorKind, DisconnectReason};
 | 
			
		||||
 | 
			
		||||
use client_version::ClientVersion;
 | 
			
		||||
use std::cmp::Ordering;
 | 
			
		||||
use std::collections::HashMap;
 | 
			
		||||
use std::net::{SocketAddr, SocketAddrV4, Ipv4Addr};
 | 
			
		||||
@ -97,7 +108,7 @@ pub struct SessionInfo {
 | 
			
		||||
	/// Peer public key
 | 
			
		||||
	pub id: Option<NodeId>,
 | 
			
		||||
	/// Peer client ID
 | 
			
		||||
	pub client_version: String,
 | 
			
		||||
	pub client_version: ClientVersion,
 | 
			
		||||
	/// Peer RLPx protocol version
 | 
			
		||||
	pub protocol_version: u32,
 | 
			
		||||
	/// Session protocol capabilities
 | 
			
		||||
@ -275,7 +286,7 @@ pub trait NetworkContext {
 | 
			
		||||
	fn register_timer(&self, token: TimerToken, delay: Duration) -> Result<(), Error>;
 | 
			
		||||
 | 
			
		||||
	/// Returns peer identification string
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> String;
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> ClientVersion;
 | 
			
		||||
 | 
			
		||||
	/// Returns information on p2p session
 | 
			
		||||
	fn session_info(&self, peer: PeerId) -> Option<SessionInfo>;
 | 
			
		||||
@ -322,7 +333,7 @@ impl<'a, T> NetworkContext for &'a T where T: ?Sized + NetworkContext {
 | 
			
		||||
		(**self).register_timer(token, delay)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> String {
 | 
			
		||||
	fn peer_client_version(&self, peer: PeerId) -> ClientVersion {
 | 
			
		||||
		(**self).peer_client_version(peer)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
		Loading…
	
		Reference in New Issue
	
	Block a user