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:
elferdo 2019-02-07 15:27:09 +01:00 committed by Afri Schoedon
parent d5c19f8719
commit b7e8621846
21 changed files with 607 additions and 65 deletions

5
Cargo.lock generated
View File

@ -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]]

View File

@ -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

View File

@ -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,

View File

@ -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);

View File

@ -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(""),
});
}

View File

@ -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);

View File

@ -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;

View File

@ -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);

View File

@ -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)
}

View File

@ -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 {

View File

@ -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"

View File

@ -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;

View File

@ -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(),

View File

@ -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()));
}

View File

@ -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

View File

@ -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> {

View File

@ -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)?;

View File

@ -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 {

View File

@ -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"

View 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));
}
}

View File

@ -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)
}