6a21c9ba6f
* notes from call * should_forward function * should_forward_block cleanup * some cleanup and notes * complete draft * corrections * some cleanup * ran cargo fmt * Revert "ran cargo fmt" This reverts commit 464a5c4b62401c68fff7d36d3cd8069560a57713. * ran cargo fmt after updating * proposer index mods * fmt * new way of reading state * fmt * fmt * compiles again * fmt * Correct stream timeout delay queue handling * Correct small typo * Support BlocksByRange step parameter * Add initial docs to syncing * Initial start of naive-attestation-aggregation * Remove ping protocol * Initial renaming of network services * Correct rebasing relative to latest master * Adds HashMapDelay struct to utils * Initial network restructure * Network restructure. Adds new types for v0.2.0 * Removes build artefacts * Shift validation to beacon chain * Temporarily remove gossip validation This is to be updated to match current optimisation efforts. * Adds AggregateAndProof * Begin rebuilding pubsub encoding/decoding * Temp commit * Shift gossipsup decoding in eth2_libp2p * Shifts block encoding/decoding into RPC * Progress on attestation service * Initial work on removing libp2p lock * Add LRU caches to store (rollup) * Update attestation validation for DB changes (WIP) * Initial version of should_forward_block * Scaffold * Progress on attestation validation Also, consolidate prod+testing slot clocks so that they share much of the same implementation and can both handle sub-slot time changes. * Removes lock from libp2p service * Completed network lock removal * Finish(?) attestation processing * Correct network termination future * Add slot check to block check * Correct fmt issues * Remove Drop implementation for network service * Address reviewers suggestions * Modification of validator for subscriptions * Add slot signing to validator client * Further progress on validation subscription * Register SSZ snappy protocol messages * Add initial idea of snappy compressed Codec by replacing the UVI codec with the snap library * Fix matching against protocol string * Adds TODOs for implementation work * Implements check against max packet size before attempting to decode snappy compressed packages * Add ssz_snappy codec * Adds necessary validator subscription functionality * Progress on snappy codec * Clean up validator <-> beacon node http types * Add aggregator status to ValidatorDuty * clear buffer after decoding * Impl Clone for manual slot clock * Fix minor errors * Further progress validator client subscription * Initial subscription and aggregation handling * Progress to modifying val client for attestation aggregation * First draft of validator client upgrade for aggregate attestations * Trying something * Length prefix compressed data * Fix gossipsub tests * Working snappy frames with compressed length prefix * Removes lock on a network channel * Partially implement beacon node subscription http api * Uncompressed length prefix working * Cleanup * Remove Testing request and response * Return codec from match statement; reduce code duplication * Fix unsafe unwrap in Outbound decode * Add length checks * All encode/decode functions use snappy frame format * Add a `full` BeaconBlock method * Add executable to test rpc against other impls * Remove unused code; minor fixes * Add PH & MS slot clock changes * Account for genesis time * Use checked mul * Account for genesis slot * Change API * Refactor "duration to..." functions * Re-merge updated block processing to v0.2.0 (#962) * Start updating types * WIP * Signature hacking * Existing EF tests passing with fake_crypto * Updates * Delete outdated API spec * The refactor continues * It compiles * WIP test fixes * All release tests passing bar genesis state parsing * Update and test YamlConfig * Update to spec v0.10 compatible BLS * Updates to BLS EF tests * Add EF test for AggregateVerify And delete unused hash2curve tests for uncompressed points * Update EF tests to v0.10.1 * Use optional block root correctly in block proc * Use genesis fork in deposit domain. All tests pass * Cargo fmt * Fast aggregate verify test * Update REST API docs * Cargo fmt * Fix unused import * Bump spec tags to v0.10.1 * Add `seconds_per_eth1_block` to chainspec * Update to timestamp based eth1 voting scheme * Return None from `get_votes_to_consider` if block cache is empty * Handle overflows in `is_candidate_block` * Revert to failing tests * Fix eth1 data sets test * Choose default vote according to spec * Fix collect_valid_votes tests * Fix `get_votes_to_consider` to choose all eligible blocks * Uncomment winning_vote tests * Add comments; remove unused code * Reduce seconds_per_eth1_block for simulation * Addressed review comments * Add test for default vote case * Fix logs * Remove unused functions * Meter default eth1 votes * Fix comments * Address review comments; remove unused dependency * Add first attempt at attestation proc. re-write * Add version 2 of attestation processing * Minor fixes * Add validator pubkey cache * Make get_indexed_attestation take a committee * Link signature processing into new attn verification * First working version * Ensure pubkey cache is updated * Add more metrics, slight optimizations * Clone committee cache during attestation processing * Update shuffling cache during block processing * Remove old commented-out code * Fix shuffling cache insert bug * Used indexed attestation in fork choice * Restructure attn processing, add metrics * Add more detailed metrics * Tidy, fix failing tests * Fix failing tests, tidy * Disable/delete two outdated tests * Add new Pubkeys struct to signature_sets * Refactor with functional approach * Update beacon chain * Remove decompressed member from pubkey bytes * Add hashmap for indices lookup * Add state cache, remove store cache * Only build the head committee cache * Change `get_attesting_indices` to use Vec * Fix failing test * Tidy * Add pubkey cache persistence file * Add more comments * Integrate persistence file into builder * Add pubkey cache tests * Add data_dir to beacon chain builder * Remove Option in pubkey cache persistence file * Ensure consistency between datadir/data_dir * Fix failing network test * Tidy * Fix todos * Improve tests * Fix compile error * Fix compile error from merge * Split up block processing metrics * Tidy * Refactor get_pubkey_from_state * Remove commented-out code * Rename state_cache -> checkpoint_cache * Rename Checkpoint -> Snapshot * Tidy, add comments * Tidy up find_head function * Change some checkpoint -> snapshot * Add tests * Expose max_len * Remove dead code * Tidy * Fix bug * Add sync-speed metric * Add first attempt at VerifiableBlock * Start integrating into beacon chain * Integrate VerifiableBlock * Rename VerifableBlock -> PartialBlockVerification * Add start of typed methods * Add progress * Add further progress * Rename structs * Add full block verification to block_processing.rs * Further beacon chain integration * Update checks for gossip * Add todo * Start adding segement verification * Add passing chain segement test * Initial integration with batch sync * Minor changes * Tidy, add more error checking * Start adding chain_segment tests * Finish invalid signature tests * Include single and gossip verified blocks in tests * Add gossip verification tests * Start adding docs * Finish adding comments to block_processing.rs * Rename block_processing.rs -> block_verification * Start removing old block processing code * Fixes beacon_chain compilation * Fix project-wide compile errors * Remove old code * Fix bug with beacon proposer index * Fix shim for BlockProcessingError * Only process one epoch at a time * Fix loop in chain segment processing * Add caching for state.eth1_data_votes * Add BeaconChain::validator_pubkey * Revert "Add caching for state.eth1_data_votes" This reverts commit cd73dcd6434fb8d8e6bf30c5356355598ea7b78e. * Add sync-speed metric (#898) * Add PH & MS slot clock changes * Account for genesis time * Use checked mul * Account for genesis slot * Change API * Allow for clock disparity * Refactor "duration to..." functions * Ensure errors are returned during batch processing Co-authored-by: Michael Sproul <michael@sigmaprime.io> Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: pawan <pawandhananjay@gmail.com> Co-authored-by: Paul Hauner <paul@paulhauner.com> * Enr fork (#967) * Start fixing enr-fork-id * Fix time-until-next-fork logic * Remove fork crate * Update any references to beacon_chain.spec.milliseconds_per_slot to beacon_chain.slot_clock.slot_diration().as_nillis() in the attestation service. (#968) * Process network attestations (#966) * Start updating types * WIP * Signature hacking * Existing EF tests passing with fake_crypto * Updates * Delete outdated API spec * The refactor continues * It compiles * WIP test fixes * All release tests passing bar genesis state parsing * Update and test YamlConfig * Update to spec v0.10 compatible BLS * Updates to BLS EF tests * Add EF test for AggregateVerify And delete unused hash2curve tests for uncompressed points * Update EF tests to v0.10.1 * Use optional block root correctly in block proc * Use genesis fork in deposit domain. All tests pass * Cargo fmt * Fast aggregate verify test * Update REST API docs * Cargo fmt * Fix unused import * Bump spec tags to v0.10.1 * Add `seconds_per_eth1_block` to chainspec * Update to timestamp based eth1 voting scheme * Return None from `get_votes_to_consider` if block cache is empty * Handle overflows in `is_candidate_block` * Revert to failing tests * Fix eth1 data sets test * Choose default vote according to spec * Fix collect_valid_votes tests * Fix `get_votes_to_consider` to choose all eligible blocks * Uncomment winning_vote tests * Add comments; remove unused code * Reduce seconds_per_eth1_block for simulation * Addressed review comments * Add test for default vote case * Fix logs * Remove unused functions * Meter default eth1 votes * Fix comments * Address review comments; remove unused dependency * Add first attempt at attestation proc. re-write * Add version 2 of attestation processing * Minor fixes * Add validator pubkey cache * Make get_indexed_attestation take a committee * Link signature processing into new attn verification * First working version * Ensure pubkey cache is updated * Add more metrics, slight optimizations * Clone committee cache during attestation processing * Update shuffling cache during block processing * Remove old commented-out code * Fix shuffling cache insert bug * Used indexed attestation in fork choice * Restructure attn processing, add metrics * Add more detailed metrics * Tidy, fix failing tests * Fix failing tests, tidy * Disable/delete two outdated tests * Add new Pubkeys struct to signature_sets * Refactor with functional approach * Update beacon chain * Remove decompressed member from pubkey bytes * Add hashmap for indices lookup * Add state cache, remove store cache * Only build the head committee cache * Change `get_attesting_indices` to use Vec * Fix failing test * Tidy * Add pubkey cache persistence file * Add more comments * Integrate persistence file into builder * Add pubkey cache tests * Add data_dir to beacon chain builder * Remove Option in pubkey cache persistence file * Ensure consistency between datadir/data_dir * Fix failing network test * Tidy * Fix todos * Improve tests * Fix compile error * Fix compile error from merge * Split up block processing metrics * Tidy * Refactor get_pubkey_from_state * Remove commented-out code * Rename state_cache -> checkpoint_cache * Rename Checkpoint -> Snapshot * Tidy, add comments * Tidy up find_head function * Change some checkpoint -> snapshot * Add tests * Expose max_len * Remove dead code * Tidy * Fix bug * Add sync-speed metric * Add first attempt at VerifiableBlock * Start integrating into beacon chain * Integrate VerifiableBlock * Rename VerifableBlock -> PartialBlockVerification * Add start of typed methods * Add progress * Add further progress * Rename structs * Add full block verification to block_processing.rs * Further beacon chain integration * Update checks for gossip * Add todo * Start adding segement verification * Add passing chain segement test * Initial integration with batch sync * Minor changes * Tidy, add more error checking * Start adding chain_segment tests * Finish invalid signature tests * Include single and gossip verified blocks in tests * Add gossip verification tests * Start adding docs * Finish adding comments to block_processing.rs * Rename block_processing.rs -> block_verification * Start removing old block processing code * Fixes beacon_chain compilation * Fix project-wide compile errors * Remove old code * Fix bug with beacon proposer index * Fix shim for BlockProcessingError * Only process one epoch at a time * Fix loop in chain segment processing * Add caching for state.eth1_data_votes * Add BeaconChain::validator_pubkey * Revert "Add caching for state.eth1_data_votes" This reverts commit cd73dcd6434fb8d8e6bf30c5356355598ea7b78e. * Allow for clock disparity * Ensure errors are returned during batch processing * Add block gossip verification * Connect attestation processing to beacon chain * Optimistically subscribe to subnets on the same slot Co-authored-by: Michael Sproul <michael@sigmaprime.io> Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: pawan <pawandhananjay@gmail.com> Co-authored-by: Paul Hauner <paul@paulhauner.com> * Update /validator/subscribe (#969) * Add progress on duties refactor * Add simple is_aggregator bool to val subscription * Remove unused function * Upgrade sim (#972) * Add progress on duties refactor * Add simple is_aggregator bool to val subscription * Add the no-eth1-sim, refactor sim * Sends discovery for persistent subnets (#973) * main takes cmdline arguments * Add test script * Fix errors * snappy uses Uvi to encode/decode length prefix * Add more comments * Run fmt Co-authored-by: Grant Wuerker <gwuerker@gmail.com> Co-authored-by: Age Manning <Age@AgeManning.com> Co-authored-by: Michael Sproul <micsproul@gmail.com> Co-authored-by: Michael Sproul <michael@sigmaprime.io> Co-authored-by: b-m-f <max@ehlers.berlin> Co-authored-by: Paul Hauner <paul@paulhauner.com> Co-authored-by: realbigsean <seananderson33@GMAIL.com>
604 lines
24 KiB
Rust
604 lines
24 KiB
Rust
#![cfg(test)]
|
|
use eth2_libp2p::rpc::methods::*;
|
|
use eth2_libp2p::rpc::*;
|
|
use eth2_libp2p::{BehaviourEvent, RPCEvent};
|
|
use slog::{warn, Level};
|
|
use std::sync::atomic::{AtomicBool, Ordering::Relaxed};
|
|
use std::sync::{Arc, Mutex};
|
|
use std::time::Duration;
|
|
use tokio::prelude::*;
|
|
use types::{
|
|
BeaconBlock, Epoch, EthSpec, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock, Slot,
|
|
};
|
|
|
|
mod common;
|
|
|
|
type E = MinimalEthSpec;
|
|
|
|
#[test]
|
|
// Tests the STATUS RPC message
|
|
fn test_status_rpc() {
|
|
// set up the logging. The level and enabled logging or not
|
|
let log_level = Level::Trace;
|
|
let enable_logging = false;
|
|
|
|
let log = common::build_log(log_level, enable_logging);
|
|
|
|
// get sender/receiver
|
|
let (mut sender, mut receiver) = common::build_node_pair(&log, 10500);
|
|
|
|
// Dummy STATUS RPC message
|
|
let rpc_request = RPCRequest::Status(StatusMessage {
|
|
fork_digest: [0; 4],
|
|
finalized_root: Hash256::from_low_u64_be(0),
|
|
finalized_epoch: Epoch::new(1),
|
|
head_root: Hash256::from_low_u64_be(0),
|
|
head_slot: Slot::new(1),
|
|
});
|
|
|
|
// Dummy STATUS RPC message
|
|
let rpc_response = RPCResponse::Status(StatusMessage {
|
|
fork_digest: [0; 4],
|
|
finalized_root: Hash256::from_low_u64_be(0),
|
|
finalized_epoch: Epoch::new(1),
|
|
head_root: Hash256::from_low_u64_be(0),
|
|
head_slot: Slot::new(1),
|
|
});
|
|
|
|
let sender_request = rpc_request.clone();
|
|
let sender_log = log.clone();
|
|
let sender_response = rpc_response.clone();
|
|
|
|
// build the sender future
|
|
let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match sender.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
|
|
// Send a STATUS message
|
|
warn!(sender_log, "Sending RPC");
|
|
sender
|
|
.swarm
|
|
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
|
|
}
|
|
Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
|
|
// Should receive the RPC response
|
|
RPCEvent::Response(id, response @ RPCErrorResponse::Success(_)) => {
|
|
if id == 1 {
|
|
warn!(sender_log, "Sender Received");
|
|
let response = {
|
|
match response {
|
|
RPCErrorResponse::Success(r) => r,
|
|
_ => unreachable!(),
|
|
}
|
|
};
|
|
assert_eq!(response, sender_response.clone());
|
|
|
|
warn!(sender_log, "Sender Completed");
|
|
return Ok(Async::Ready(true));
|
|
}
|
|
}
|
|
e => panic!("Received invalid RPC message {}", e),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
};
|
|
}
|
|
});
|
|
|
|
// build the receiver future
|
|
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match receiver.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
|
|
// Should receive sent RPC request
|
|
RPCEvent::Request(id, request) => {
|
|
if request == rpc_request {
|
|
// send the response
|
|
warn!(log, "Receiver Received");
|
|
receiver.swarm.send_rpc(
|
|
peer_id,
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::Success(rpc_response.clone()),
|
|
),
|
|
);
|
|
}
|
|
}
|
|
e => panic!("Received invalid RPC message {}", e),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
}
|
|
}
|
|
});
|
|
|
|
// execute the futures and check the result
|
|
let test_result = Arc::new(AtomicBool::new(false));
|
|
let error_result = test_result.clone();
|
|
let thread_result = test_result.clone();
|
|
tokio::run(
|
|
sender_future
|
|
.select(receiver_future)
|
|
.timeout(Duration::from_millis(1000))
|
|
.map_err(move |_| error_result.store(false, Relaxed))
|
|
.map(move |result| {
|
|
thread_result.store(result.0, Relaxed);
|
|
}),
|
|
);
|
|
assert!(test_result.load(Relaxed));
|
|
}
|
|
|
|
#[test]
|
|
// Tests a streamed BlocksByRange RPC Message
|
|
fn test_blocks_by_range_chunked_rpc() {
|
|
// set up the logging. The level and enabled logging or not
|
|
let log_level = Level::Trace;
|
|
let enable_logging = false;
|
|
|
|
let messages_to_send = 10;
|
|
|
|
let log = common::build_log(log_level, enable_logging);
|
|
|
|
// get sender/receiver
|
|
let (mut sender, mut receiver) = common::build_node_pair(&log, 10505);
|
|
|
|
// BlocksByRange Request
|
|
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest {
|
|
start_slot: 0,
|
|
count: messages_to_send,
|
|
step: 0,
|
|
});
|
|
|
|
// BlocksByRange Response
|
|
let spec = E::default_spec();
|
|
let empty_block = BeaconBlock::empty(&spec);
|
|
let empty_signed = SignedBeaconBlock {
|
|
message: empty_block,
|
|
signature: Signature::empty_signature(),
|
|
};
|
|
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed));
|
|
|
|
let sender_request = rpc_request.clone();
|
|
let sender_log = log.clone();
|
|
let sender_response = rpc_response.clone();
|
|
|
|
// keep count of the number of messages received
|
|
let messages_received = Arc::new(Mutex::new(0));
|
|
// build the sender future
|
|
let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match sender.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
|
|
// Send a BlocksByRange request
|
|
warn!(sender_log, "Sender sending RPC request");
|
|
sender
|
|
.swarm
|
|
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
|
|
}
|
|
Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
|
|
// Should receive the RPC response
|
|
RPCEvent::Response(id, response) => {
|
|
if id == 1 {
|
|
warn!(sender_log, "Sender received a response");
|
|
match response {
|
|
RPCErrorResponse::Success(res) => {
|
|
assert_eq!(res, sender_response.clone());
|
|
*messages_received.lock().unwrap() += 1;
|
|
warn!(sender_log, "Chunk received");
|
|
}
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRange,
|
|
) => {
|
|
// should be exactly 10 messages before terminating
|
|
assert_eq!(
|
|
*messages_received.lock().unwrap(),
|
|
messages_to_send
|
|
);
|
|
// end the test
|
|
return Ok(Async::Ready(true));
|
|
}
|
|
_ => panic!("Invalid RPC received"),
|
|
}
|
|
}
|
|
}
|
|
_ => panic!("Received invalid RPC message"),
|
|
},
|
|
Async::Ready(Some(_)) => {}
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
};
|
|
}
|
|
});
|
|
|
|
// build the receiver future
|
|
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match receiver.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
|
|
// Should receive the sent RPC request
|
|
RPCEvent::Request(id, request) => {
|
|
if request == rpc_request {
|
|
// send the response
|
|
warn!(log, "Receiver got request");
|
|
|
|
for _ in 1..=messages_to_send {
|
|
receiver.swarm.send_rpc(
|
|
peer_id.clone(),
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::Success(rpc_response.clone()),
|
|
),
|
|
);
|
|
}
|
|
// send the stream termination
|
|
receiver.swarm.send_rpc(
|
|
peer_id,
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRange,
|
|
),
|
|
),
|
|
);
|
|
}
|
|
}
|
|
_ => panic!("Received invalid RPC message"),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
}
|
|
}
|
|
});
|
|
|
|
// execute the futures and check the result
|
|
let test_result = Arc::new(AtomicBool::new(false));
|
|
let error_result = test_result.clone();
|
|
let thread_result = test_result.clone();
|
|
tokio::run(
|
|
sender_future
|
|
.select(receiver_future)
|
|
.timeout(Duration::from_millis(1000))
|
|
.map_err(move |_| error_result.store(false, Relaxed))
|
|
.map(move |result| {
|
|
thread_result.store(result.0, Relaxed);
|
|
}),
|
|
);
|
|
assert!(test_result.load(Relaxed));
|
|
}
|
|
|
|
#[test]
|
|
// Tests an empty response to a BlocksByRange RPC Message
|
|
fn test_blocks_by_range_single_empty_rpc() {
|
|
// set up the logging. The level and enabled logging or not
|
|
let log_level = Level::Trace;
|
|
let enable_logging = false;
|
|
|
|
let log = common::build_log(log_level, enable_logging);
|
|
|
|
// get sender/receiver
|
|
let (mut sender, mut receiver) = common::build_node_pair(&log, 10510);
|
|
|
|
// BlocksByRange Request
|
|
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest {
|
|
start_slot: 0,
|
|
count: 10,
|
|
step: 0,
|
|
});
|
|
|
|
// BlocksByRange Response
|
|
let spec = E::default_spec();
|
|
let empty_block = BeaconBlock::empty(&spec);
|
|
let empty_signed = SignedBeaconBlock {
|
|
message: empty_block,
|
|
signature: Signature::empty_signature(),
|
|
};
|
|
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed));
|
|
|
|
let sender_request = rpc_request.clone();
|
|
let sender_log = log.clone();
|
|
let sender_response = rpc_response.clone();
|
|
|
|
// keep count of the number of messages received
|
|
let messages_received = Arc::new(Mutex::new(0));
|
|
// build the sender future
|
|
let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match sender.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
|
|
// Send a BlocksByRange request
|
|
warn!(sender_log, "Sender sending RPC request");
|
|
sender
|
|
.swarm
|
|
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
|
|
}
|
|
Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
|
|
// Should receive the RPC response
|
|
RPCEvent::Response(id, response) => {
|
|
if id == 1 {
|
|
warn!(sender_log, "Sender received a response");
|
|
match response {
|
|
RPCErrorResponse::Success(res) => {
|
|
assert_eq!(res, sender_response.clone());
|
|
*messages_received.lock().unwrap() += 1;
|
|
warn!(sender_log, "Chunk received");
|
|
}
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRange,
|
|
) => {
|
|
// should be exactly 1 messages before terminating
|
|
assert_eq!(*messages_received.lock().unwrap(), 1);
|
|
// end the test
|
|
return Ok(Async::Ready(true));
|
|
}
|
|
_ => panic!("Invalid RPC received"),
|
|
}
|
|
}
|
|
}
|
|
m => panic!("Received invalid RPC message: {}", m),
|
|
},
|
|
Async::Ready(Some(_)) => {}
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
};
|
|
}
|
|
});
|
|
|
|
// build the receiver future
|
|
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match receiver.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
|
|
// Should receive the sent RPC request
|
|
RPCEvent::Request(id, request) => {
|
|
if request == rpc_request {
|
|
// send the response
|
|
warn!(log, "Receiver got request");
|
|
|
|
receiver.swarm.send_rpc(
|
|
peer_id.clone(),
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::Success(rpc_response.clone()),
|
|
),
|
|
);
|
|
// send the stream termination
|
|
receiver.swarm.send_rpc(
|
|
peer_id,
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRange,
|
|
),
|
|
),
|
|
);
|
|
}
|
|
}
|
|
_ => panic!("Received invalid RPC message"),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
}
|
|
}
|
|
});
|
|
|
|
// execute the futures and check the result
|
|
let test_result = Arc::new(AtomicBool::new(false));
|
|
let error_result = test_result.clone();
|
|
let thread_result = test_result.clone();
|
|
tokio::run(
|
|
sender_future
|
|
.select(receiver_future)
|
|
.timeout(Duration::from_millis(1000))
|
|
.map_err(move |_| error_result.store(false, Relaxed))
|
|
.map(move |result| {
|
|
thread_result.store(result.0, Relaxed);
|
|
}),
|
|
);
|
|
assert!(test_result.load(Relaxed));
|
|
}
|
|
|
|
#[test]
|
|
// Tests a streamed, chunked BlocksByRoot RPC Message
|
|
// The size of the reponse is a full `BeaconBlock`
|
|
// which is greater than the Snappy frame size. Hence, this test
|
|
// serves to test the snappy framing format as well.
|
|
fn test_blocks_by_root_chunked_rpc() {
|
|
// set up the logging. The level and enabled logging or not
|
|
let log_level = Level::Trace;
|
|
let enable_logging = false;
|
|
|
|
let messages_to_send = 3;
|
|
|
|
let log = common::build_log(log_level, enable_logging);
|
|
let spec = E::default_spec();
|
|
|
|
// get sender/receiver
|
|
let (mut sender, mut receiver) = common::build_node_pair(&log, 10515);
|
|
|
|
// BlocksByRoot Request
|
|
let rpc_request = RPCRequest::BlocksByRoot(BlocksByRootRequest {
|
|
block_roots: vec![Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0)],
|
|
});
|
|
|
|
// BlocksByRoot Response
|
|
let full_block = BeaconBlock::full(&spec);
|
|
let signed_full_block = SignedBeaconBlock {
|
|
message: full_block,
|
|
signature: Signature::empty_signature(),
|
|
};
|
|
let rpc_response = RPCResponse::BlocksByRoot(Box::new(signed_full_block));
|
|
|
|
let sender_request = rpc_request.clone();
|
|
let sender_log = log.clone();
|
|
let sender_response = rpc_response.clone();
|
|
|
|
// keep count of the number of messages received
|
|
let messages_received = Arc::new(Mutex::new(0));
|
|
// build the sender future
|
|
let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match sender.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
|
|
// Send a BlocksByRoot request
|
|
warn!(sender_log, "Sender sending RPC request");
|
|
sender
|
|
.swarm
|
|
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
|
|
}
|
|
Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
|
|
// Should receive the RPC response
|
|
RPCEvent::Response(id, response) => {
|
|
warn!(sender_log, "Sender received a response");
|
|
assert_eq!(id, 1);
|
|
match response {
|
|
RPCErrorResponse::Success(res) => {
|
|
assert_eq!(res, sender_response.clone());
|
|
*messages_received.lock().unwrap() += 1;
|
|
warn!(sender_log, "Chunk received");
|
|
}
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRoot,
|
|
) => {
|
|
// should be exactly 10 messages before terminating
|
|
assert_eq!(*messages_received.lock().unwrap(), messages_to_send);
|
|
// end the test
|
|
return Ok(Async::Ready(true));
|
|
}
|
|
m => panic!("Invalid RPC received: {}", m),
|
|
}
|
|
}
|
|
m => panic!("Received invalid RPC message: {}", m),
|
|
},
|
|
Async::Ready(Some(_)) => {}
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
};
|
|
}
|
|
});
|
|
|
|
// build the receiver future
|
|
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match receiver.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
|
|
// Should receive the sent RPC request
|
|
RPCEvent::Request(id, request) => {
|
|
if request == rpc_request {
|
|
// send the response
|
|
warn!(log, "Receiver got request");
|
|
|
|
for _ in 1..=messages_to_send {
|
|
receiver.swarm.send_rpc(
|
|
peer_id.clone(),
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::Success(rpc_response.clone()),
|
|
),
|
|
);
|
|
}
|
|
// send the stream termination
|
|
receiver.swarm.send_rpc(
|
|
peer_id,
|
|
RPCEvent::Response(
|
|
id,
|
|
RPCErrorResponse::StreamTermination(
|
|
ResponseTermination::BlocksByRange,
|
|
),
|
|
),
|
|
);
|
|
}
|
|
}
|
|
_ => panic!("Received invalid RPC message"),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
}
|
|
}
|
|
});
|
|
|
|
// execute the futures and check the result
|
|
let test_result = Arc::new(AtomicBool::new(false));
|
|
let error_result = test_result.clone();
|
|
let thread_result = test_result.clone();
|
|
tokio::run(
|
|
sender_future
|
|
.select(receiver_future)
|
|
.timeout(Duration::from_millis(1000))
|
|
.map_err(move |_| error_result.store(false, Relaxed))
|
|
.map(move |result| {
|
|
thread_result.store(result.0, Relaxed);
|
|
}),
|
|
);
|
|
assert!(test_result.load(Relaxed));
|
|
}
|
|
|
|
#[test]
|
|
// Tests a Goodbye RPC message
|
|
fn test_goodbye_rpc() {
|
|
// set up the logging. The level and enabled logging or not
|
|
let log_level = Level::Trace;
|
|
let enable_logging = false;
|
|
|
|
let log = common::build_log(log_level, enable_logging);
|
|
|
|
// get sender/receiver
|
|
let (mut sender, mut receiver) = common::build_node_pair(&log, 10520);
|
|
|
|
// Goodbye Request
|
|
let rpc_request = RPCRequest::Goodbye(GoodbyeReason::ClientShutdown);
|
|
|
|
let sender_request = rpc_request.clone();
|
|
let sender_log = log.clone();
|
|
|
|
// build the sender future
|
|
let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match sender.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
|
|
// Send a Goodbye request
|
|
warn!(sender_log, "Sender sending RPC request");
|
|
sender
|
|
.swarm
|
|
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
|
|
}
|
|
Async::Ready(Some(_)) => {}
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
};
|
|
}
|
|
});
|
|
|
|
// build the receiver future
|
|
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
|
|
loop {
|
|
match receiver.poll().unwrap() {
|
|
Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
|
|
// Should receive the sent RPC request
|
|
RPCEvent::Request(id, request) => {
|
|
if request == rpc_request {
|
|
assert_eq!(id, 0);
|
|
assert_eq!(rpc_request.clone(), request);
|
|
// receives the goodbye. Nothing left to do
|
|
return Ok(Async::Ready(true));
|
|
}
|
|
}
|
|
_ => panic!("Received invalid RPC message"),
|
|
},
|
|
Async::Ready(Some(_)) => (),
|
|
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
|
|
}
|
|
}
|
|
});
|
|
|
|
// execute the futures and check the result
|
|
let test_result = Arc::new(AtomicBool::new(false));
|
|
let error_result = test_result.clone();
|
|
let thread_result = test_result.clone();
|
|
tokio::run(
|
|
sender_future
|
|
.select(receiver_future)
|
|
.timeout(Duration::from_millis(1000))
|
|
.map_err(move |_| error_result.store(false, Relaxed))
|
|
.map(move |result| {
|
|
thread_result.store(result.0, Relaxed);
|
|
}),
|
|
);
|
|
assert!(test_result.load(Relaxed));
|
|
}
|