Replace legacy Rlp with UntrustedRlp and use in ethcore rlp views (#8316)

* WIP

* Replace Rlp with UntrustedRlp in views, explicity unwrap with expect

First pass to get it to compile. Need to figure out whether to do this or to propogate Errors upstream, which would require many more changes to dependent code. If we do this way we are assuming that the views are always used in a context where the rlp is trusted to be valid e.g. when reading from our own DB. So need to fid out whether views are used with data received from an untrusted (e.g. extrernal peer).

* Remove original Rlp impl, rename UntrustedRlp -> Rlp

* Create rlp views with view! macro to record debug info

Views are assumed to be over valid rlp, so if there is a decoding error we record where the view was created in the first place and report it in the expect

* Use $crate in view! macro to avoid import, fix tests

* Expect valid rlp in decode functions for now

* Replace spaces with tabs in new file

* Add doc tests for creating views with macro

* Update rlp docs to reflect removing of UntrustedRlp

* Replace UntrustedRlp usages in private-tx merge
This commit is contained in:
Andrew Jones 2018-04-16 14:52:12 +01:00 committed by Afri Schoedon
parent db7a8c4ac7
commit a04c5b180a
86 changed files with 944 additions and 1055 deletions

View File

@ -26,7 +26,7 @@ use hashdb::HashDB;
use memorydb::MemoryDB;
use bytes::Bytes;
use trie::{self, TrieMut, TrieDBMut, Trie, TrieDB, Recorder};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
// encode a key.
macro_rules! key {
@ -150,7 +150,7 @@ pub fn check_proof(proof: &[Bytes], num: u64, root: H256) -> Option<(H256, U256)
let res = match TrieDB::new(&db, &root) {
Err(_) => return None,
Ok(trie) => trie.get_with(&key!(num), |val: &[u8]| {
let rlp = UntrustedRlp::new(val);
let rlp = Rlp::new(val);
rlp.val_at::<H256>(0)
.and_then(|h| rlp.val_at::<U256>(1).map(|td| (h, td)))
.ok()

View File

@ -41,7 +41,7 @@ use ethcore::engines::epoch::{
PendingTransition as PendingEpochTransition
};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use heapsize::HeapSizeOf;
use ethereum_types::{H256, H264, U256};
use plain_hasher::H256FastMap;
@ -125,7 +125,7 @@ impl Encodable for Entry {
}
impl Decodable for Entry {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let mut candidates = SmallVec::<[Candidate; 3]>::new();
for item in rlp.iter() {
@ -186,7 +186,7 @@ fn encode_canonical_transition(header: &Header, proof: &[u8]) -> Vec<u8> {
// decode last canonical transition entry.
fn decode_canonical_transition(t: &[u8]) -> Result<(Header, &[u8]), DecoderError> {
let rlp = UntrustedRlp::new(t);
let rlp = Rlp::new(t);
Ok((rlp.val_at(0)?, rlp.at(1)?.data()?))
}

View File

@ -22,7 +22,7 @@ use transaction::UnverifiedTransaction;
use io::TimerToken;
use network::{HostInfo, NetworkProtocolHandler, NetworkContext, PeerId};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use ethereum_types::{H256, U256};
use kvdb::DBValue;
use parking_lot::{Mutex, RwLock};
@ -528,7 +528,7 @@ impl LightProtocol {
// - check whether peer exists
// - check whether request was made
// - check whether request kinds match
fn pre_verify_response(&self, peer: &PeerId, raw: &UntrustedRlp) -> Result<IdGuard, Error> {
fn pre_verify_response(&self, peer: &PeerId, raw: &Rlp) -> Result<IdGuard, Error> {
let req_id = ReqId(raw.val_at(0)?);
let cur_credits: U256 = raw.val_at(1)?;
@ -572,7 +572,7 @@ impl LightProtocol {
/// Packet data is _untrusted_, which means that invalid data won't lead to
/// issues.
pub fn handle_packet(&self, io: &IoContext, peer: &PeerId, packet_id: u8, data: &[u8]) {
let rlp = UntrustedRlp::new(data);
let rlp = Rlp::new(data);
trace!(target: "pip", "Incoming packet {} from peer {}", packet_id, peer);
@ -794,7 +794,7 @@ impl LightProtocol {
impl LightProtocol {
// Handle status message from peer.
fn status(&self, peer: &PeerId, io: &IoContext, data: UntrustedRlp) -> Result<(), Error> {
fn status(&self, peer: &PeerId, io: &IoContext, data: Rlp) -> Result<(), Error> {
let pending = match self.pending_peers.write().remove(peer) {
Some(pending) => pending,
None => {
@ -855,7 +855,7 @@ impl LightProtocol {
}
// Handle an announcement.
fn announcement(&self, peer: &PeerId, io: &IoContext, data: UntrustedRlp) -> Result<(), Error> {
fn announcement(&self, peer: &PeerId, io: &IoContext, data: Rlp) -> Result<(), Error> {
if !self.peers.read().contains_key(peer) {
debug!(target: "pip", "Ignoring announcement from unknown peer");
return Ok(())
@ -900,7 +900,7 @@ impl LightProtocol {
}
// Receive requests from a peer.
fn request(&self, peer_id: &PeerId, io: &IoContext, raw: UntrustedRlp) -> Result<(), Error> {
fn request(&self, peer_id: &PeerId, io: &IoContext, raw: Rlp) -> Result<(), Error> {
// the maximum amount of requests we'll fill in a single packet.
const MAX_REQUESTS: usize = 256;
@ -968,7 +968,7 @@ impl LightProtocol {
}
// handle a packet with responses.
fn response(&self, peer: &PeerId, io: &IoContext, raw: UntrustedRlp) -> Result<(), Error> {
fn response(&self, peer: &PeerId, io: &IoContext, raw: Rlp) -> Result<(), Error> {
let (req_id, responses) = {
let id_guard = self.pre_verify_response(peer, &raw)?;
let responses: Vec<Response> = raw.list_at(2)?;
@ -987,7 +987,7 @@ impl LightProtocol {
}
// handle an update of request credits parameters.
fn update_credits(&self, peer_id: &PeerId, io: &IoContext, raw: UntrustedRlp) -> Result<(), Error> {
fn update_credits(&self, peer_id: &PeerId, io: &IoContext, raw: Rlp) -> Result<(), Error> {
let peers = self.peers.read();
let peer = peers.get(peer_id).ok_or(Error::UnknownPeer)?;
@ -1022,7 +1022,7 @@ impl LightProtocol {
}
// handle an acknowledgement of request credits update.
fn acknowledge_update(&self, peer_id: &PeerId, _io: &IoContext, _raw: UntrustedRlp) -> Result<(), Error> {
fn acknowledge_update(&self, peer_id: &PeerId, _io: &IoContext, _raw: Rlp) -> Result<(), Error> {
let peers = self.peers.read();
let peer = peers.get(peer_id).ok_or(Error::UnknownPeer)?;
let mut peer = peer.lock();
@ -1041,7 +1041,7 @@ impl LightProtocol {
}
// Receive a set of transactions to relay.
fn relay_transactions(&self, peer: &PeerId, io: &IoContext, data: UntrustedRlp) -> Result<(), Error> {
fn relay_transactions(&self, peer: &PeerId, io: &IoContext, data: Rlp) -> Result<(), Error> {
const MAX_TRANSACTIONS: usize = 256;
let txs: Vec<_> = data.iter()

View File

@ -29,7 +29,7 @@
use request::{self, Request};
use super::error::Error;
use rlp::{UntrustedRlp, RlpStream, Decodable, Encodable, DecoderError};
use rlp::{Rlp, RlpStream, Decodable, Encodable, DecoderError};
use ethereum_types::U256;
use std::time::{Duration, Instant};
@ -162,7 +162,7 @@ impl Encodable for CostTable {
}
impl Decodable for CostTable {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let base = rlp.val_at(0)?;
let mut headers = None;

View File

@ -16,7 +16,7 @@
//! Peer status and capabilities.
use rlp::{DecoderError, Encodable, Decodable, RlpStream, UntrustedRlp};
use rlp::{DecoderError, Encodable, Decodable, RlpStream, Rlp};
use ethereum_types::{H256, U256};
use super::request_credits::FlowParams;
@ -85,7 +85,7 @@ impl Key {
// helper for decoding key-value pairs in the handshake or an announcement.
struct Parser<'a> {
pos: usize,
rlp: UntrustedRlp<'a>,
rlp: Rlp<'a>,
}
impl<'a> Parser<'a> {
@ -97,7 +97,7 @@ impl<'a> Parser<'a> {
// expect a specific next key, and get the value's RLP.
// if the key isn't found, the position isn't advanced.
fn expect_raw(&mut self, key: Key) -> Result<UntrustedRlp<'a>, DecoderError> {
fn expect_raw(&mut self, key: Key) -> Result<Rlp<'a>, DecoderError> {
trace!(target: "les", "Expecting key {}", key.as_str());
let pre_pos = self.pos;
if let Some((k, val)) = self.get_next()? {
@ -109,7 +109,7 @@ impl<'a> Parser<'a> {
}
// get the next key and value RLP.
fn get_next(&mut self) -> Result<Option<(Key, UntrustedRlp<'a>)>, DecoderError> {
fn get_next(&mut self) -> Result<Option<(Key, Rlp<'a>)>, DecoderError> {
while self.pos < self.rlp.item_count()? {
let pair = self.rlp.at(self.pos)?;
let k: String = pair.val_at(0)?;
@ -208,7 +208,7 @@ impl Capabilities {
/// - chain status
/// - serving capabilities
/// - request credit parameters
pub fn parse_handshake(rlp: UntrustedRlp) -> Result<(Status, Capabilities, Option<FlowParams>), DecoderError> {
pub fn parse_handshake(rlp: Rlp) -> Result<(Status, Capabilities, Option<FlowParams>), DecoderError> {
let mut parser = Parser {
pos: 0,
rlp: rlp,
@ -304,7 +304,7 @@ pub struct Announcement {
}
/// Parse an announcement.
pub fn parse_announcement(rlp: UntrustedRlp) -> Result<Announcement, DecoderError> {
pub fn parse_announcement(rlp: Rlp) -> Result<Announcement, DecoderError> {
let mut last_key = None;
let mut announcement = Announcement {
@ -374,7 +374,7 @@ mod tests {
use super::*;
use super::super::request_credits::FlowParams;
use ethereum_types::{U256, H256};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
#[test]
fn full_handshake() {
@ -404,7 +404,7 @@ mod tests {
let handshake = write_handshake(&status, &capabilities, Some(&flow_params));
let (read_status, read_capabilities, read_flow)
= parse_handshake(UntrustedRlp::new(&handshake)).unwrap();
= parse_handshake(Rlp::new(&handshake)).unwrap();
assert_eq!(read_status, status);
assert_eq!(read_capabilities, capabilities);
@ -439,7 +439,7 @@ mod tests {
let handshake = write_handshake(&status, &capabilities, Some(&flow_params));
let (read_status, read_capabilities, read_flow)
= parse_handshake(UntrustedRlp::new(&handshake)).unwrap();
= parse_handshake(Rlp::new(&handshake)).unwrap();
assert_eq!(read_status, status);
assert_eq!(read_capabilities, capabilities);
@ -473,7 +473,7 @@ mod tests {
let handshake = write_handshake(&status, &capabilities, Some(&flow_params));
let interleaved = {
let handshake = UntrustedRlp::new(&handshake);
let handshake = Rlp::new(&handshake);
let mut stream = RlpStream::new_list(handshake.item_count().unwrap_or(0) * 3);
for item in handshake.iter() {
@ -489,7 +489,7 @@ mod tests {
};
let (read_status, read_capabilities, read_flow)
= parse_handshake(UntrustedRlp::new(&interleaved)).unwrap();
= parse_handshake(Rlp::new(&interleaved)).unwrap();
assert_eq!(read_status, status);
assert_eq!(read_capabilities, capabilities);
@ -510,7 +510,7 @@ mod tests {
};
let serialized = write_announcement(&announcement);
let read = parse_announcement(UntrustedRlp::new(&serialized)).unwrap();
let read = parse_announcement(Rlp::new(&serialized)).unwrap();
assert_eq!(read, announcement);
}
@ -529,7 +529,7 @@ mod tests {
.append_raw(&encode_flag(Key::ServeHeaders), 1);
let out = stream.drain();
assert!(parse_announcement(UntrustedRlp::new(&out)).is_err());
assert!(parse_announcement(Rlp::new(&out)).is_err());
let mut stream = RlpStream::new_list(6);
stream
@ -541,7 +541,7 @@ mod tests {
.append_raw(&encode_pair(Key::ServeStateSince, &44u64), 1);
let out = stream.drain();
assert!(parse_announcement(UntrustedRlp::new(&out)).is_ok());
assert!(parse_announcement(Rlp::new(&out)).is_ok());
}
#[test]
@ -566,7 +566,7 @@ mod tests {
let handshake = write_handshake(&status, &capabilities, None);
let (read_status, read_capabilities, read_flow)
= parse_handshake(UntrustedRlp::new(&handshake)).unwrap();
= parse_handshake(Rlp::new(&handshake)).unwrap();
assert_eq!(read_status, status);
assert_eq!(read_capabilities, capabilities);

View File

@ -31,7 +31,7 @@ use provider::Provider;
use request;
use request::*;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use ethereum_types::{H256, U256, Address};
use std::sync::Arc;
@ -688,7 +688,7 @@ fn id_guard() {
stream.begin_list(2).append(&125usize).append(&3usize);
let packet = stream.out();
assert!(proto.response(&peer_id, &Expect::Nothing, UntrustedRlp::new(&packet)).is_err());
assert!(proto.response(&peer_id, &Expect::Nothing, Rlp::new(&packet)).is_err());
}
// next, do an unexpected response.
@ -699,7 +699,7 @@ fn id_guard() {
stream.begin_list(0);
let packet = stream.out();
assert!(proto.response(&peer_id, &Expect::Nothing, UntrustedRlp::new(&packet)).is_err());
assert!(proto.response(&peer_id, &Expect::Nothing, Rlp::new(&packet)).is_err());
}
// lastly, do a valid (but empty) response.
@ -710,7 +710,7 @@ fn id_guard() {
stream.begin_list(0);
let packet = stream.out();
assert!(proto.response(&peer_id, &Expect::Nothing, UntrustedRlp::new(&packet)).is_ok());
assert!(proto.response(&peer_id, &Expect::Nothing, Rlp::new(&packet)).is_ok());
}
let peers = proto.peers.read();

View File

@ -30,7 +30,7 @@ use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY, KECCAK_EMPTY_LIST_RLP, keccak};
use request::{self as net_request, IncompleteRequest, CompleteRequest, Output, OutputKind, Field};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use ethereum_types::{H256, U256, Address};
use parking_lot::Mutex;
use hashdb::HashDB;
@ -831,7 +831,7 @@ impl Account {
match TrieDB::new(&db, &state_root).and_then(|t| t.get(&keccak(&self.address)))? {
Some(val) => {
let rlp = UntrustedRlp::new(&val);
let rlp = Rlp::new(&val);
Ok(Some(BasicAccount {
nonce: rlp.val_at(0)?,
balance: rlp.val_at(1)?,

View File

@ -16,7 +16,7 @@
//! Light protocol request types.
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use ethereum_types::H256;
mod batch;
@ -148,7 +148,7 @@ impl<T> From<T> for Field<T> {
}
impl<T: Decodable> Decodable for Field<T> {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
match rlp.val_at::<u8>(0)? {
0 => Ok(Field::Scalar(rlp.val_at::<T>(1)?)),
1 => Ok({
@ -224,7 +224,7 @@ impl From<u64> for HashOrNumber {
}
impl Decodable for HashOrNumber {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.as_val::<H256>().map(HashOrNumber::Hash)
.or_else(|_| rlp.as_val().map(HashOrNumber::Number))
}
@ -331,7 +331,7 @@ impl Request {
}
impl Decodable for Request {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
match rlp.val_at::<Kind>(0)? {
Kind::Headers => Ok(Request::Headers(rlp.val_at(1)?)),
Kind::HeaderProof => Ok(Request::HeaderProof(rlp.val_at(1)?)),
@ -493,7 +493,7 @@ pub enum Kind {
}
impl Decodable for Kind {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
match rlp.as_val::<u8>()? {
0 => Ok(Kind::Headers),
1 => Ok(Kind::HeaderProof),
@ -578,7 +578,7 @@ impl Response {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
match rlp.val_at::<Kind>(0)? {
Kind::Headers => Ok(Response::Headers(rlp.val_at(1)?)),
Kind::HeaderProof => Ok(Response::HeaderProof(rlp.val_at(1)?)),
@ -673,7 +673,7 @@ pub trait ResponseLike {
pub mod header {
use super::{Field, HashOrNumber, NoSuchOutput, OutputKind, Output};
use ethcore::encoded;
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
/// Potentially incomplete headers request.
#[derive(Debug, Clone, PartialEq, Eq, RlpEncodable, RlpDecodable)]
@ -754,7 +754,7 @@ pub mod header {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
use ethcore::header::Header as FullHeader;
let mut headers = Vec::new();
@ -785,7 +785,7 @@ pub mod header {
/// Request and response for header proofs.
pub mod header_proof {
use super::{Field, NoSuchOutput, OutputKind, Output};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use ethereum_types::{H256, U256};
use bytes::Bytes;
@ -859,7 +859,7 @@ pub mod header_proof {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
Ok(Response {
proof: rlp.list_at(0)?,
hash: rlp.val_at(1)?,
@ -1027,7 +1027,7 @@ pub mod block_receipts {
pub mod block_body {
use super::{Field, NoSuchOutput, OutputKind, Output};
use ethcore::encoded;
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use ethereum_types::H256;
/// Potentially incomplete block body request.
@ -1092,7 +1092,7 @@ pub mod block_body {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
use ethcore::header::Header as FullHeader;
use transaction::UnverifiedTransaction;
@ -1411,7 +1411,7 @@ pub mod contract_code {
pub mod execution {
use super::{Field, NoSuchOutput, OutputKind, Output};
use transaction::Action;
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use ethereum_types::{H256, U256, Address};
use kvdb::DBValue;
use bytes::Bytes;
@ -1508,7 +1508,7 @@ pub mod execution {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let mut items = Vec::new();
for raw_item in rlp.iter() {
let mut item = DBValue::new();
@ -1536,7 +1536,7 @@ pub mod execution {
/// A request for epoch signal data.
pub mod epoch_signal {
use super::{Field, NoSuchOutput, OutputKind, Output};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
use ethereum_types::H256;
use bytes::Bytes;
@ -1548,7 +1548,7 @@ pub mod epoch_signal {
}
impl Decodable for Incomplete {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
Ok(Incomplete {
block_hash: rlp.val_at(0)?,
})
@ -1617,7 +1617,7 @@ pub mod epoch_signal {
}
impl Decodable for Response {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
Ok(Response {
signal: rlp.as_val()?,
@ -1891,7 +1891,7 @@ mod tests {
stream.append(&100usize).append_list(&reqs);
let out = stream.out();
let rlp = UntrustedRlp::new(&out);
let rlp = Rlp::new(&out);
assert_eq!(rlp.val_at::<usize>(0).unwrap(), 100usize);
assert_eq!(rlp.list_at::<Request>(1).unwrap(), reqs);
}

View File

@ -238,14 +238,14 @@ impl Provider where {
fn extract_original_transaction(&self, private: PrivateTransaction, contract: &Address) -> Result<UnverifiedTransaction, Error> {
let encrypted_transaction = private.encrypted;
let transaction_bytes = self.decrypt(contract, &encrypted_transaction)?;
let original_transaction: UnverifiedTransaction = UntrustedRlp::new(&transaction_bytes).as_val()?;
let original_transaction: UnverifiedTransaction = Rlp::new(&transaction_bytes).as_val()?;
Ok(original_transaction)
}
/// Process received private transaction
pub fn import_private_transaction(&self, rlp: &[u8]) -> Result<(), Error> {
trace!("Private transaction received");
let private_tx: PrivateTransaction = UntrustedRlp::new(rlp).as_val()?;
let private_tx: PrivateTransaction = Rlp::new(rlp).as_val()?;
let contract = private_tx.contract;
let contract_validators = self.get_validators(BlockId::Latest, &contract)?;
@ -356,7 +356,7 @@ impl Provider where {
/// Add signed private transaction into the store
/// Creates corresponding public transaction if last required singature collected and sends it to the chain
pub fn import_signed_private_transaction(&self, rlp: &[u8]) -> Result<(), Error> {
let tx: SignedPrivateTransaction = UntrustedRlp::new(rlp).as_val()?;
let tx: SignedPrivateTransaction = Rlp::new(rlp).as_val()?;
trace!("Signature for private transaction received: {:?}", tx);
let private_hash = tx.private_transaction_hash();
let desc = match self.transactions_for_signing.lock().get(&private_hash) {

View File

@ -22,7 +22,7 @@ use std::collections::HashSet;
use hash::{keccak, KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP};
use triehash::ordered_trie_root;
use rlp::{UntrustedRlp, RlpStream, Encodable, Decodable, DecoderError, encode_list};
use rlp::{Rlp, RlpStream, Encodable, Decodable, DecoderError, encode_list};
use ethereum_types::{H256, U256, Address, Bloom};
use bytes::Bytes;
use unexpected::{Mismatch, OutOfBounds};
@ -54,7 +54,7 @@ pub struct Block {
impl Block {
/// Returns true if the given bytes form a valid encoding of a block in RLP.
pub fn is_good(b: &[u8]) -> bool {
UntrustedRlp::new(b).as_val::<Block>().is_ok()
Rlp::new(b).as_val::<Block>().is_ok()
}
/// Get the RLP-encoding of the block with the seal.
@ -68,7 +68,7 @@ impl Block {
}
impl Decodable for Block {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
if rlp.as_raw().len() != rlp.payload_info()?.total() {
return Err(DecoderError::RlpIsTooBig);
}
@ -622,7 +622,7 @@ pub fn enact_verified(
// Remove state root from transaction receipts to make them EIP-98 compatible.
strip_receipts: bool,
) -> Result<LockedBlock, Error> {
let view = BlockView::new(&block.bytes);
let view = view!(BlockView, &block.bytes);
enact(
block.header,
@ -664,7 +664,7 @@ mod tests {
last_hashes: Arc<LastHashes>,
factories: Factories,
) -> Result<LockedBlock, Error> {
let block = BlockView::new(block_bytes);
let block = view!(BlockView, block_bytes);
let header = block.header();
let transactions: Result<Vec<_>, Error> = block
.transactions()
@ -715,7 +715,7 @@ mod tests {
last_hashes: Arc<LastHashes>,
factories: Factories,
) -> Result<SealedBlock, Error> {
let header = BlockView::new(block_bytes).header_view();
let header = view!(BlockView, block_bytes).header_view();
Ok(enact_bytes(block_bytes, engine, tracing, db, parent, last_hashes, factories)?.seal(engine, header.seal())?)
}
@ -781,7 +781,7 @@ mod tests {
let bytes = e.rlp_bytes();
assert_eq!(bytes, orig_bytes);
let uncles = BlockView::new(&bytes).uncles();
let uncles = view!(BlockView, &bytes).uncles();
assert_eq!(uncles[1].extra_data(), b"uncle2");
let db = e.drain();

View File

@ -25,11 +25,11 @@ use heapsize::HeapSizeOf;
use ethereum_types::{H256, Bloom, U256};
use parking_lot::{Mutex, RwLock};
use bytes::Bytes;
use rlp::{Rlp, RlpStream};
use rlp::RlpStream;
use rlp_compress::{compress, decompress, blocks_swapper};
use header::*;
use transaction::*;
use views::*;
use views::{BlockView, HeaderView};
use log_entry::{LogEntry, LocalizedLogEntry};
use receipt::Receipt;
use blooms::{BloomGroup, GroupPosition};
@ -231,13 +231,7 @@ impl BlockProvider for BlockChain {
fn block(&self, hash: &H256) -> Option<encoded::Block> {
let header = self.block_header_data(hash)?;
let body = self.block_body(hash)?;
let mut block = RlpStream::new_list(3);
let body_rlp = body.rlp();
block.append_raw(header.rlp().as_raw(), 1);
block.append_raw(body_rlp.at(0).as_raw(), 1);
block.append_raw(body_rlp.at(1).as_raw(), 1);
Some(encoded::Block::new(block.out()))
Some(encoded::Block::new_from_header_and_body(&header.view(), &body.view()))
}
/// Get block header data
@ -499,7 +493,7 @@ impl BlockChain {
None => {
// best block does not exist
// we need to insert genesis into the cache
let block = BlockView::new(genesis);
let block = view!(BlockView, genesis);
let header = block.header_view();
let hash = block.hash();
@ -701,7 +695,7 @@ impl BlockChain {
/// Supply a dummy parent total difficulty when the parent block may not be in the chain.
/// Returns true if the block is disconnected.
pub fn insert_unordered_block(&self, batch: &mut DBTransaction, bytes: &[u8], receipts: Vec<Receipt>, parent_td: Option<U256>, is_best: bool, is_ancient: bool) -> bool {
let block = BlockView::new(bytes);
let block = view!(BlockView, bytes);
let header = block.header_view();
let hash = header.hash();
@ -898,7 +892,7 @@ impl BlockChain {
/// If the block is already known, does nothing.
pub fn insert_block(&self, batch: &mut DBTransaction, bytes: &[u8], receipts: Vec<Receipt>) -> ImportRoute {
// create views onto rlp
let block = BlockView::new(bytes);
let block = view!(BlockView, bytes);
let header = block.header_view();
let hash = header.hash();
@ -1138,7 +1132,7 @@ impl BlockChain {
/// This function returns modified block hashes.
fn prepare_block_hashes_update(&self, block_bytes: &[u8], info: &BlockInfo) -> HashMap<BlockNumber, H256> {
let mut block_hashes = HashMap::new();
let block = BlockView::new(block_bytes);
let block = view!(BlockView, block_bytes);
let header = block.header_view();
let number = header.number();
@ -1165,7 +1159,7 @@ impl BlockChain {
/// This function returns modified block details.
/// Uses the given parent details or attempts to load them from the database.
fn prepare_block_details_update(&self, block_bytes: &[u8], info: &BlockInfo) -> HashMap<H256, BlockDetails> {
let block = BlockView::new(block_bytes);
let block = view!(BlockView, block_bytes);
let header = block.header_view();
let parent_hash = header.parent_hash();
@ -1197,7 +1191,7 @@ impl BlockChain {
/// This function returns modified transaction addresses.
fn prepare_transaction_addresses_update(&self, block_bytes: &[u8], info: &BlockInfo) -> HashMap<H256, Option<TransactionAddress>> {
let block = BlockView::new(block_bytes);
let block = view!(BlockView, block_bytes);
let transaction_hashes = block.transaction_hashes();
match info.location {
@ -1265,7 +1259,7 @@ impl BlockChain {
/// to bloom location in database (BlocksBloomLocation).
///
fn prepare_block_blooms_update(&self, block_bytes: &[u8], info: &BlockInfo) -> HashMap<GroupPosition, BloomGroup> {
let block = BlockView::new(block_bytes);
let block = view!(BlockView, block_bytes);
let header = block.header_view();
let log_blooms = match info.location {
@ -1384,9 +1378,9 @@ impl BlockChain {
/// Create a block body from a block.
pub fn block_to_body(block: &[u8]) -> Bytes {
let mut body = RlpStream::new_list(2);
let block_rlp = Rlp::new(block);
body.append_raw(block_rlp.at(1).as_raw(), 1);
body.append_raw(block_rlp.at(2).as_raw(), 1);
let block_view = view!(BlockView, block);
body.append_raw(block_view.transactions_rlp().as_raw(), 1);
body.append_raw(block_view.uncles_rlp().as_raw(), 1);
body.out()
}

View File

@ -41,7 +41,7 @@ impl Block {
#[inline]
pub fn hash(&self) -> H256 {
BlockView::new(&self.encoded()).header_view().hash()
view!(BlockView, &self.encoded()).header_view().hash()
}
#[inline]

View File

@ -62,7 +62,7 @@ use ethcore_miner::pool::VerifiedTransaction;
use parking_lot::{Mutex, RwLock};
use rand::OsRng;
use receipt::{Receipt, LocalizedReceipt};
use rlp::UntrustedRlp;
use rlp::Rlp;
use snapshot::{self, io as snapshot_io};
use spec::Spec;
use state_db::StateDB;
@ -446,7 +446,7 @@ impl Importer {
/// The block is guaranteed to be the next best blocks in the
/// first block sequence. Does no sealing or transaction validation.
fn import_old_block(&self, block_bytes: Bytes, receipts_bytes: Bytes, db: &KeyValueDB, chain: &BlockChain) -> Result<H256, ::error::Error> {
let block = BlockView::new(&block_bytes);
let block = view!(BlockView, &block_bytes);
let header = block.header();
let receipts = ::rlp::decode_list(&receipts_bytes);
let hash = header.hash();
@ -514,7 +514,7 @@ impl Importer {
let receipts = block.receipts().to_owned();
let traces = block.traces().clone().drain();
assert_eq!(header.hash(), BlockView::new(block_data).header_view().hash());
assert_eq!(header.hash(), view!(BlockView, block_data).header_view().hash());
//let traces = From::from(block.traces().clone().unwrap_or_else(Vec::new));
@ -988,7 +988,7 @@ impl Client {
let txs: Vec<UnverifiedTransaction> = transactions
.iter()
.filter_map(|bytes| UntrustedRlp::new(bytes).as_val().ok())
.filter_map(|bytes| Rlp::new(bytes).as_val().ok())
.collect();
self.notify(|notify| {
@ -1423,7 +1423,7 @@ impl ImportBlock for Client {
fn import_block_with_receipts(&self, block_bytes: Bytes, receipts_bytes: Bytes) -> Result<H256, BlockImportError> {
{
// check block order
let header = BlockView::new(&block_bytes).header_view();
let header = view!(BlockView, &block_bytes).header_view();
if self.chain.read().is_known(&header.hash()) {
return Err(BlockImportError::Import(ImportError::AlreadyInChain));
}

View File

@ -29,7 +29,7 @@ use journaldb;
use kvdb::DBValue;
use kvdb_memorydb;
use bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use ethkey::{Generator, Random};
use ethcore_miner::pool::VerifiedTransaction;
use transaction::{self, Transaction, LocalizedTransaction, SignedTransaction, Action};
@ -471,7 +471,7 @@ impl ChainInfo for TestBlockChainClient {
impl BlockInfo for TestBlockChainClient {
fn block_header(&self, id: BlockId) -> Option<encoded::Header> {
self.block_hash(id)
.and_then(|hash| self.blocks.read().get(&hash).map(|r| BlockView::new(r).header_rlp().as_raw().to_vec()))
.and_then(|hash| self.blocks.read().get(&hash).map(|r| view!(BlockView, r).header_rlp().as_raw().to_vec()))
.map(encoded::Header::new)
}
@ -513,7 +513,7 @@ impl RegistryInfo for TestBlockChainClient {
impl ImportBlock for TestBlockChainClient {
fn import_block(&self, b: Bytes) -> Result<H256, BlockImportError> {
let header = BlockView::new(&b).header();
let header = view!(BlockView, &b).header();
let h = header.hash();
let number: usize = header.number() as usize;
if number > self.blocks.read().len() {
@ -522,7 +522,7 @@ impl ImportBlock for TestBlockChainClient {
if number > 0 {
match self.blocks.read().get(header.parent_hash()) {
Some(parent) => {
let parent = BlockView::new(parent).header();
let parent = view!(BlockView, parent).header();
if parent.number() != (header.number() - 1) {
panic!("Unexpected block parent");
}
@ -547,7 +547,7 @@ impl ImportBlock for TestBlockChainClient {
while n > 0 && self.numbers.read()[&n] != parent_hash {
*self.numbers.write().get_mut(&n).unwrap() = parent_hash.clone();
n -= 1;
parent_hash = BlockView::new(&self.blocks.read()[&parent_hash]).header().parent_hash().clone();
parent_hash = view!(BlockView, &self.blocks.read()[&parent_hash]).header().parent_hash().clone();
}
}
}
@ -692,7 +692,7 @@ impl BlockChainClient for TestBlockChainClient {
fn block_body(&self, id: BlockId) -> Option<encoded::Body> {
self.block_hash(id).and_then(|hash| self.blocks.read().get(&hash).map(|r| {
let block = BlockView::new(r);
let block = view!(BlockView, r);
let mut stream = RlpStream::new_list(2);
stream.append_raw(block.transactions_rlp().as_raw(), 1);
stream.append_raw(block.uncles_rlp().as_raw(), 1);
@ -811,7 +811,7 @@ impl BlockChainClient for TestBlockChainClient {
fn queue_transactions(&self, transactions: Vec<Bytes>, _peer_id: usize) {
// import right here
let txs = transactions.into_iter().filter_map(|bytes| UntrustedRlp::new(&bytes).as_val().ok()).collect();
let txs = transactions.into_iter().filter_map(|bytes| Rlp::new(&bytes).as_val().ok()).collect();
self.miner.import_external_transactions(self, txs);
}

View File

@ -26,12 +26,12 @@
use block::Block as FullBlock;
use header::{BlockNumber, Header as FullHeader};
use transaction::UnverifiedTransaction;
use views;
use hash::keccak;
use heapsize::HeapSizeOf;
use ethereum_types::{H256, Bloom, U256, Address};
use rlp::{Rlp, RlpStream};
use views::{self, BlockView, HeaderView, BodyView};
/// Owning header view.
#[derive(Debug, Clone, PartialEq, Eq)]
@ -52,7 +52,7 @@ impl Header {
/// Get a borrowed header view onto the data.
#[inline]
pub fn view(&self) -> views::HeaderView { views::HeaderView::new(&self.0) }
pub fn view(&self) -> HeaderView { view!(HeaderView, &self.0) }
/// Get the rlp of the header.
#[inline]
@ -125,7 +125,7 @@ impl Body {
/// Get a borrowed view of the data within.
#[inline]
pub fn view(&self) -> views::BodyView { views::BodyView::new(&self.0) }
pub fn view(&self) -> BodyView { view!(BodyView, &self.0) }
/// Fully decode this block body.
pub fn decode(&self) -> (Vec<UnverifiedTransaction>, Vec<FullHeader>) {
@ -145,7 +145,7 @@ impl Body {
// forwarders to borrowed view.
impl Body {
/// Get raw rlp of transactions
pub fn transactions_rlp(&self) -> Rlp { self.view().transactions_rlp() }
pub fn transactions_rlp(&self) -> Rlp { self.view().transactions_rlp().rlp }
/// Get a vector of all transactions.
pub fn transactions(&self) -> Vec<UnverifiedTransaction> { self.view().transactions() }
@ -160,7 +160,7 @@ impl Body {
pub fn transaction_hashes(&self) -> Vec<H256> { self.view().transaction_hashes() }
/// Get raw rlp of uncle headers
pub fn uncles_rlp(&self) -> Rlp { self.view().uncles_rlp() }
pub fn uncles_rlp(&self) -> Rlp { self.view().uncles_rlp().rlp }
/// Decode uncle headers.
pub fn uncles(&self) -> Vec<FullHeader> { self.view().uncles() }
@ -198,20 +198,20 @@ impl Block {
/// Get a borrowed view of the whole block.
#[inline]
pub fn view(&self) -> views::BlockView { views::BlockView::new(&self.0) }
pub fn view(&self) -> BlockView { view!(BlockView, &self.0) }
/// Get a borrowed view of the block header.
#[inline]
pub fn header_view(&self) -> views::HeaderView { self.view().header_view() }
pub fn header_view(&self) -> HeaderView { self.view().header_view() }
/// Decode to a full block.
pub fn decode(&self) -> FullBlock { ::rlp::decode(&self.0) }
/// Decode the header.
pub fn decode_header(&self) -> FullHeader { self.rlp().val_at(0) }
pub fn decode_header(&self) -> FullHeader { self.view().rlp().val_at(0) }
/// Clone the encoded header.
pub fn header(&self) -> Header { Header(self.rlp().at(0).as_raw().to_vec()) }
pub fn header(&self) -> Header { Header(self.view().rlp().at(0).as_raw().to_vec()) }
/// Get the rlp of this block.
#[inline]

View File

@ -41,7 +41,7 @@ use self::finality::RollingFinality;
use ethkey::{self, Signature};
use io::{IoContext, IoHandler, TimerToken, IoService};
use itertools::{self, Itertools};
use rlp::{encode, Decodable, DecoderError, Encodable, RlpStream, UntrustedRlp};
use rlp::{encode, Decodable, DecoderError, Encodable, RlpStream, Rlp};
use ethereum_types::{H256, H520, Address, U128, U256};
use parking_lot::{Mutex, RwLock};
use unexpected::{Mismatch, OutOfBounds};
@ -325,7 +325,7 @@ impl Encodable for EmptyStep {
}
impl Decodable for EmptyStep {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let signature = rlp.val_at(0)?;
let empty_step_rlp = rlp.at(1)?;
@ -366,7 +366,7 @@ impl Encodable for SealedEmptyStep {
}
impl Decodable for SealedEmptyStep {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let signature = rlp.val_at(0)?;
let step = rlp.val_at(1)?;
@ -415,7 +415,7 @@ impl super::EpochVerifier<EthereumMachine> for EpochVerifier {
let mut finality_checker = RollingFinality::blank(self.subchain_validators.clone().into_inner());
let mut finalized = Vec::new();
let headers: Vec<Header> = UntrustedRlp::new(proof).as_list().ok()?;
let headers: Vec<Header> = Rlp::new(proof).as_list().ok()?;
{
let mut push_header = |parent_header: &Header, header: Option<&Header>| {
@ -479,13 +479,13 @@ fn header_expected_seal_fields(header: &Header, empty_steps_transition: u64) ->
fn header_step(header: &Header, empty_steps_transition: u64) -> Result<usize, ::rlp::DecoderError> {
let expected_seal_fields = header_expected_seal_fields(header, empty_steps_transition);
UntrustedRlp::new(&header.seal().get(0).expect(
Rlp::new(&header.seal().get(0).expect(
&format!("was either checked with verify_block_basic or is genesis; has {} fields; qed (Make sure the spec file has a correct genesis seal)", expected_seal_fields))).as_val()
}
fn header_signature(header: &Header, empty_steps_transition: u64) -> Result<Signature, ::rlp::DecoderError> {
let expected_seal_fields = header_expected_seal_fields(header, empty_steps_transition);
UntrustedRlp::new(&header.seal().get(1).expect(
Rlp::new(&header.seal().get(1).expect(
&format!("was checked with verify_block_basic; has {} fields; qed", expected_seal_fields))).as_val::<H520>().map(Into::into)
}
@ -498,7 +498,7 @@ fn header_empty_steps_raw(header: &Header) -> &[u8] {
// extracts the empty steps from the header seal. should only be called when there are 3 fields in the seal
// (i.e. header.number() >= self.empty_steps_transition).
fn header_empty_steps(header: &Header) -> Result<Vec<EmptyStep>, ::rlp::DecoderError> {
let empty_steps = UntrustedRlp::new(header_empty_steps_raw(header)).as_list::<SealedEmptyStep>()?;
let empty_steps = Rlp::new(header_empty_steps_raw(header)).as_list::<SealedEmptyStep>()?;
Ok(empty_steps.into_iter().map(|s| EmptyStep::from_sealed(s, header.parent_hash())).collect())
}
@ -575,7 +575,7 @@ fn combine_proofs(signal_number: BlockNumber, set_proof: &[u8], finality_proof:
}
fn destructure_proofs(combined: &[u8]) -> Result<(BlockNumber, &[u8], &[u8]), Error> {
let rlp = UntrustedRlp::new(combined);
let rlp = Rlp::new(combined);
Ok((
rlp.at(0)?.as_val()?,
rlp.at(1)?.data()?,
@ -801,7 +801,7 @@ impl Engine<EthereumMachine> for AuthorityRound {
EngineError::MalformedMessage(format!("{:?}", x))
}
let rlp = UntrustedRlp::new(rlp);
let rlp = Rlp::new(rlp);
let empty_step: EmptyStep = rlp.as_val().map_err(fmt_err)?;;
if empty_step.verify(&*self.validators).unwrap_or(false) {

View File

@ -57,10 +57,10 @@ impl super::EpochVerifier<EthereumMachine> for EpochVerifier {
}
fn verify_external(header: &Header, validators: &ValidatorSet) -> Result<(), Error> {
use rlp::UntrustedRlp;
use rlp::Rlp;
// Check if the signature belongs to a validator, can depend on parent state.
let sig = UntrustedRlp::new(&header.seal()[0]).as_val::<H520>()?;
let sig = Rlp::new(&header.seal()[0]).as_val::<H520>()?;
let signer = ethkey::public_to_address(&ethkey::recover(&sig.into(), &header.bare_hash())?);
if *header.author() != signer {

View File

@ -18,7 +18,7 @@
use ethereum_types::H256;
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
/// A full epoch transition.
#[derive(Debug, Clone)]
@ -41,7 +41,7 @@ impl Encodable for Transition {
}
impl Decodable for Transition {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
Ok(Transition {
block_hash: rlp.val_at(0)?,
block_number: rlp.val_at(1)?,
@ -64,7 +64,7 @@ impl Encodable for PendingTransition {
}
impl Decodable for PendingTransition {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
Ok(PendingTransition {
proof: rlp.as_val()?,
})

View File

@ -23,7 +23,7 @@ use bytes::Bytes;
use super::{Height, View, BlockHash, Step};
use error::Error;
use header::Header;
use rlp::{UntrustedRlp, RlpStream, Encodable, Decodable, DecoderError};
use rlp::{Rlp, RlpStream, Encodable, Decodable, DecoderError};
use ethkey::{recover, public_to_address};
use super::super::vote_collector::Message;
@ -61,12 +61,12 @@ impl VoteStep {
/// Header consensus view.
pub fn consensus_view(header: &Header) -> Result<View, ::rlp::DecoderError> {
let view_rlp = header.seal().get(0).expect("seal passed basic verification; seal has 3 fields; qed");
UntrustedRlp::new(view_rlp.as_slice()).as_val()
Rlp::new(view_rlp.as_slice()).as_val()
}
/// Proposal signature.
pub fn proposal_signature(header: &Header) -> Result<H520, ::rlp::DecoderError> {
UntrustedRlp::new(header.seal().get(1).expect("seal passed basic verification; seal has 3 fields; qed").as_slice()).as_val()
Rlp::new(header.seal().get(1).expect("seal passed basic verification; seal has 3 fields; qed").as_slice()).as_val()
}
impl Message for ConsensusMessage {
@ -100,7 +100,7 @@ impl ConsensusMessage {
pub fn verify(&self) -> Result<Address, Error> {
let full_rlp = ::rlp::encode(self);
let block_info = UntrustedRlp::new(&full_rlp).at(1)?;
let block_info = Rlp::new(&full_rlp).at(1)?;
let public_key = recover(&self.signature.into(), &keccak(block_info.as_raw()))?;
Ok(public_to_address(&public_key))
}
@ -142,7 +142,7 @@ impl Step {
}
impl Decodable for Step {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
match rlp.as_val()? {
0u8 => Ok(Step::Propose),
1 => Ok(Step::Prevote),
@ -160,7 +160,7 @@ impl Encodable for Step {
/// (signature, (height, view, step, block_hash))
impl Decodable for ConsensusMessage {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let m = rlp.at(1)?;
let block_message: H256 = m.val_at(3)?;
Ok(ConsensusMessage {
@ -234,7 +234,7 @@ mod tests {
};
let raw_rlp = ::rlp::encode(&message).into_vec();
let rlp = Rlp::new(&raw_rlp);
assert_eq!(message, rlp.as_val());
assert_eq!(Ok(message), rlp.as_val());
let message = ConsensusMessage {
signature: H520::default(),
@ -247,7 +247,7 @@ mod tests {
};
let raw_rlp = ::rlp::encode(&message);
let rlp = Rlp::new(&raw_rlp);
assert_eq!(message, rlp.as_val());
assert_eq!(Ok(message), rlp.as_val());
}
#[test]
@ -260,7 +260,7 @@ mod tests {
let raw_rlp = message_full_rlp(&tap.sign(addr, None, keccak(&mi)).unwrap().into(), &mi);
let rlp = UntrustedRlp::new(&raw_rlp);
let rlp = Rlp::new(&raw_rlp);
let message: ConsensusMessage = rlp.as_val().unwrap();
match message.verify() { Ok(a) if a == addr => {}, _ => panic!(), };
}

View File

@ -36,7 +36,7 @@ use client::EngineClient;
use bytes::Bytes;
use error::{Error, BlockError};
use header::{Header, BlockNumber};
use rlp::UntrustedRlp;
use rlp::Rlp;
use ethkey::{self, Message, Signature};
use account_provider::AccountProvider;
use block::*;
@ -118,7 +118,7 @@ impl <F> super::EpochVerifier<EthereumMachine> for EpochVerifier<F>
let mut addresses = HashSet::new();
let ref header_signatures_field = header.seal().get(2).ok_or(BlockError::InvalidSeal)?;
for rlp in UntrustedRlp::new(header_signatures_field).iter() {
for rlp in Rlp::new(header_signatures_field).iter() {
let signature: H520 = rlp.as_val()?;
let address = (self.recover)(&signature.into(), &message)?;
@ -154,7 +154,7 @@ fn combine_proofs(signal_number: BlockNumber, set_proof: &[u8], finality_proof:
}
fn destructure_proofs(combined: &[u8]) -> Result<(BlockNumber, &[u8], &[u8]), Error> {
let rlp = UntrustedRlp::new(combined);
let rlp = Rlp::new(combined);
Ok((
rlp.at(0)?.as_val()?,
rlp.at(1)?.data()?,
@ -503,7 +503,8 @@ impl Engine<EthereumMachine> for Tendermint {
fn fmt_err<T: ::std::fmt::Debug>(x: T) -> EngineError {
EngineError::MalformedMessage(format!("{:?}", x))
}
let rlp = UntrustedRlp::new(rlp);
let rlp = Rlp::new(rlp);
let message: ConsensusMessage = rlp.as_val().map_err(fmt_err)?;
if !self.votes.is_old_or_known(&message) {
let msg_hash = keccak(rlp.at(1).map_err(fmt_err)?.as_raw());
@ -595,7 +596,7 @@ impl Engine<EthereumMachine> for Tendermint {
let precommit_hash = message_hash(vote_step.clone(), header.bare_hash());
let ref signatures_field = header.seal().get(2).expect("block went through verify_block_basic; block has .seal_fields() fields; qed");
let mut origins = HashSet::new();
for rlp in UntrustedRlp::new(signatures_field).iter() {
for rlp in Rlp::new(signatures_field).iter() {
let precommit = ConsensusMessage {
signature: rlp.as_val()?,
block_hash: Some(header.bare_hash()),

View File

@ -25,7 +25,7 @@ use parking_lot::RwLock;
use bytes::Bytes;
use memory_cache::MemoryLruCache;
use unexpected::Mismatch;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use kvdb::DBValue;
use client::EngineClient;
@ -63,7 +63,7 @@ impl ::engines::StateDependentProof<EthereumMachine> for StateProof {
}
fn check_proof(&self, machine: &EthereumMachine, proof: &[u8]) -> Result<(), String> {
let (header, state_items) = decode_first_proof(&UntrustedRlp::new(proof))
let (header, state_items) = decode_first_proof(&Rlp::new(proof))
.map_err(|e| format!("proof incorrectly encoded: {}", e))?;
if &header != &self.header {
return Err("wrong header in proof".into());
@ -145,7 +145,7 @@ fn check_first_proof(machine: &EthereumMachine, provider: &validator_set::Valida
}).map_err(|err| err.to_string())
}
fn decode_first_proof(rlp: &UntrustedRlp) -> Result<(Header, Vec<DBValue>), ::error::Error> {
fn decode_first_proof(rlp: &Rlp) -> Result<(Header, Vec<DBValue>), ::error::Error> {
let header = rlp.val_at(0)?;
let state_items = rlp.at(1)?.iter().map(|x| {
let mut val = DBValue::new();
@ -165,7 +165,7 @@ fn encode_proof(header: &Header, receipts: &[Receipt]) -> Bytes {
stream.drain().into_vec()
}
fn decode_proof(rlp: &UntrustedRlp) -> Result<(Header, Vec<Receipt>), ::error::Error> {
fn decode_proof(rlp: &Rlp) -> Result<(Header, Vec<Receipt>), ::error::Error> {
Ok((rlp.val_at(0)?, rlp.list_at(1)?))
}
@ -357,7 +357,7 @@ impl ValidatorSet for ValidatorSafeContract {
fn epoch_set(&self, first: bool, machine: &EthereumMachine, _number: ::header::BlockNumber, proof: &[u8])
-> Result<(SimpleList, Option<H256>), ::error::Error>
{
let rlp = UntrustedRlp::new(proof);
let rlp = Rlp::new(proof);
if first {
trace!(target: "engine", "Recovering initial epoch set");

View File

@ -27,7 +27,7 @@ use error::{BlockError, Error};
use header::{Header, BlockNumber};
use engines::{self, Engine};
use ethjson;
use rlp::UntrustedRlp;
use rlp::Rlp;
use machine::EthereumMachine;
/// Number of blocks in an ethash snapshot.
@ -59,8 +59,8 @@ impl Seal {
).into());
}
let mix_hash = UntrustedRlp::new(seal[0].as_ref()).as_val::<H256>()?;
let nonce = UntrustedRlp::new(seal[1].as_ref()).as_val::<H64>()?;
let mix_hash = Rlp::new(seal[0].as_ref()).as_val::<H256>()?;
let nonce = Rlp::new(seal[1].as_ref()).as_val::<H64>()?;
let seal = Seal {
mix_hash,
nonce,

View File

@ -174,7 +174,7 @@ mod tests {
assert_eq!(morden.state_root(), "f3f4696bbf3b3b07775128eb7a3763279a394e382130f27c21e70233e04946a9".into());
let genesis = morden.genesis_block();
assert_eq!(BlockView::new(&genesis).header_view().hash(), "0cd786a2425d16f152c658316c423e6ce1181e15c3295826d7c9904cba9ce303".into());
assert_eq!(view!(BlockView, &genesis).header_view().hash(), "0cd786a2425d16f152c658316c423e6ce1181e15c3295826d7c9904cba9ce303".into());
let _ = morden.engine;
}
@ -185,7 +185,7 @@ mod tests {
assert_eq!(frontier.state_root(), "d7f8974fb5ac78d9ac099b9ad5018bedc2ce0a72dad1827a1709da30580f0544".into());
let genesis = frontier.genesis_block();
assert_eq!(BlockView::new(&genesis).header_view().hash(), "d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3".into());
assert_eq!(view!(BlockView, &genesis).header_view().hash(), "d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3".into());
let _ = frontier.engine;
}

View File

@ -21,7 +21,7 @@ use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP, keccak};
use heapsize::HeapSizeOf;
use ethereum_types::{H256, U256, Address, Bloom};
use bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream, Encodable, DecoderError, Decodable};
use rlp::{Rlp, RlpStream, Encodable, DecoderError, Decodable};
pub use types::BlockNumber;
@ -177,7 +177,7 @@ impl Header {
/// Get the seal field with RLP-decoded values as bytes.
pub fn decode_seal<'a, T: ::std::iter::FromIterator<&'a [u8]>>(&'a self) -> Result<T, DecoderError> {
self.seal.iter().map(|rlp| {
UntrustedRlp::new(rlp).data()
Rlp::new(rlp).data()
}).collect()
}
@ -327,7 +327,7 @@ fn change_field<T>(hash: &mut Option<H256>, field: &mut T, value: T) where T: Pa
impl Decodable for Header {
fn decode(r: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(r: &Rlp) -> Result<Self, DecoderError> {
let mut blockheader = Header {
parent_hash: r.val_at(0)?,
uncles_hash: r.val_at(1)?,

View File

@ -17,7 +17,7 @@
use super::test_common::*;
use evm;
use ethjson;
use rlp::UntrustedRlp;
use rlp::Rlp;
use transaction::{Action, UnverifiedTransaction, SignedTransaction};
fn do_json_test(json_data: &[u8]) -> Vec<String> {
@ -40,7 +40,7 @@ fn do_json_test(json_data: &[u8]) -> Vec<String> {
let allow_unsigned = number.map_or(false, |n| n >= 3_000_000);
let rlp: Vec<u8> = test.rlp.into();
let res = UntrustedRlp::new(&rlp)
let res = Rlp::new(&rlp)
.as_val()
.map_err(::error::Error::from)
.and_then(|t: UnverifiedTransaction| {

View File

@ -127,6 +127,9 @@ extern crate evm;
pub extern crate ethstore;
#[macro_use]
pub mod views;
#[cfg(test)]
extern crate kvdb_rocksdb;
@ -152,7 +155,6 @@ pub mod state_db;
pub mod test_helpers;
pub mod trace;
pub mod verification;
pub mod views;
mod cache_manager;
mod blooms;

View File

@ -25,7 +25,7 @@ use ethereum_types::{H256, U256};
use hashdb::HashDB;
use bytes::Bytes;
use trie::{TrieDB, Trie};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use std::collections::HashSet;
@ -148,7 +148,7 @@ pub fn to_fat_rlps(account_hash: &H256, acc: &BasicAccount, acct_db: &AccountDB,
// if it exists.
pub fn from_fat_rlp(
acct_db: &mut AccountDBMut,
rlp: UntrustedRlp,
rlp: Rlp,
mut storage_root: H256,
) -> Result<(BasicAccount, Option<Bytes>), Error> {
use trie::{TrieDBMut, TrieMut};
@ -217,7 +217,7 @@ mod tests {
use ethereum_types::{H256, Address};
use hashdb::HashDB;
use kvdb::DBValue;
use rlp::UntrustedRlp;
use rlp::Rlp;
use std::collections::HashSet;
@ -239,7 +239,7 @@ mod tests {
assert_eq!(::rlp::decode::<BasicAccount>(&thin_rlp), account);
let fat_rlps = to_fat_rlps(&keccak(&addr), &account, &AccountDB::new(db.as_hashdb(), &addr), &mut Default::default(), usize::max_value(), usize::max_value()).unwrap();
let fat_rlp = UntrustedRlp::new(&fat_rlps[0]).at(1).unwrap();
let fat_rlp = Rlp::new(&fat_rlps[0]).at(1).unwrap();
assert_eq!(from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &addr), fat_rlp, H256::zero()).unwrap().0, account);
}
@ -264,7 +264,7 @@ mod tests {
assert_eq!(::rlp::decode::<BasicAccount>(&thin_rlp), account);
let fat_rlp = to_fat_rlps(&keccak(&addr), &account, &AccountDB::new(db.as_hashdb(), &addr), &mut Default::default(), usize::max_value(), usize::max_value()).unwrap();
let fat_rlp = UntrustedRlp::new(&fat_rlp[0]).at(1).unwrap();
let fat_rlp = Rlp::new(&fat_rlp[0]).at(1).unwrap();
assert_eq!(from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &addr), fat_rlp, H256::zero()).unwrap().0, account);
}
@ -292,7 +292,7 @@ mod tests {
let mut root = KECCAK_NULL_RLP;
let mut restored_account = None;
for rlp in fat_rlps {
let fat_rlp = UntrustedRlp::new(&rlp).at(1).unwrap();
let fat_rlp = Rlp::new(&rlp).at(1).unwrap();
restored_account = Some(from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &addr), fat_rlp, root).unwrap().0);
root = restored_account.as_ref().unwrap().storage_root.clone();
}
@ -336,8 +336,8 @@ mod tests {
let fat_rlp2 = to_fat_rlps(&keccak(&addr2), &account2, &AccountDB::new(db.as_hashdb(), &addr2), &mut used_code, usize::max_value(), usize::max_value()).unwrap();
assert_eq!(used_code.len(), 1);
let fat_rlp1 = UntrustedRlp::new(&fat_rlp1[0]).at(1).unwrap();
let fat_rlp2 = UntrustedRlp::new(&fat_rlp2[0]).at(1).unwrap();
let fat_rlp1 = Rlp::new(&fat_rlp1[0]).at(1).unwrap();
let fat_rlp2 = Rlp::new(&fat_rlp2[0]).at(1).unwrap();
let (acc, maybe_code) = from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &addr2), fat_rlp2, H256::zero()).unwrap();
assert!(maybe_code.is_none());
@ -351,6 +351,6 @@ mod tests {
#[test]
fn encoding_empty_acc() {
let mut db = get_temp_state_db();
assert_eq!(from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &Address::default()), UntrustedRlp::new(&::rlp::NULL_RLP), H256::zero()).unwrap(), (ACC_EMPTY, None));
assert_eq!(from_fat_rlp(&mut AccountDBMut::new(db.as_hashdb_mut(), &Address::default()), Rlp::new(&::rlp::NULL_RLP), H256::zero()).unwrap(), (ACC_EMPTY, None));
}
}

View File

@ -21,7 +21,7 @@ use header::Header;
use hash::keccak;
use views::BlockView;
use rlp::{DecoderError, RlpStream, UntrustedRlp};
use rlp::{DecoderError, RlpStream, Rlp};
use ethereum_types::H256;
use bytes::Bytes;
use triehash::ordered_trie_root;
@ -89,7 +89,7 @@ impl AbridgedBlock {
///
/// Will fail if contains invalid rlp.
pub fn to_block(&self, parent_hash: H256, number: u64, receipts_root: H256) -> Result<Block, DecoderError> {
let rlp = UntrustedRlp::new(&self.rlp);
let rlp = Rlp::new(&self.rlp);
let mut header: Header = Default::default();
header.set_parent_hash(parent_hash);
@ -151,7 +151,7 @@ mod tests {
let receipts_root = b.header.receipts_root().clone();
let encoded = encode_block(&b);
let abridged = AbridgedBlock::from_block_view(&BlockView::new(&encoded));
let abridged = AbridgedBlock::from_block_view(&view!(BlockView, &encoded));
assert_eq!(abridged.to_block(H256::new(), 0, receipts_root).unwrap(), b);
}
@ -162,7 +162,7 @@ mod tests {
let receipts_root = b.header.receipts_root().clone();
let encoded = encode_block(&b);
let abridged = AbridgedBlock::from_block_view(&BlockView::new(&encoded));
let abridged = AbridgedBlock::from_block_view(&view!(BlockView, &encoded));
assert_eq!(abridged.to_block(H256::new(), 2, receipts_root).unwrap(), b);
}
@ -198,7 +198,7 @@ mod tests {
let encoded = encode_block(&b);
let abridged = AbridgedBlock::from_block_view(&BlockView::new(&encoded[..]));
let abridged = AbridgedBlock::from_block_view(&view!(BlockView, &encoded[..]));
assert_eq!(abridged.to_block(H256::new(), 0, receipts_root).unwrap(), b);
}
}

View File

@ -33,7 +33,7 @@ use receipt::Receipt;
use snapshot::{Error, ManifestData};
use itertools::{Position, Itertools};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use ethereum_types::{H256, U256};
use kvdb::KeyValueDB;
use bytes::Bytes;
@ -182,7 +182,7 @@ impl ChunkRebuilder {
fn verify_transition(
&mut self,
last_verifier: &mut Option<Box<EpochVerifier<EthereumMachine>>>,
transition_rlp: UntrustedRlp,
transition_rlp: Rlp,
engine: &EthEngine,
) -> Result<Verified, ::error::Error> {
use engines::ConstructedVerifier;
@ -242,7 +242,7 @@ impl Rebuilder for ChunkRebuilder {
engine: &EthEngine,
abort_flag: &AtomicBool,
) -> Result<(), ::error::Error> {
let rlp = UntrustedRlp::new(chunk);
let rlp = Rlp::new(chunk);
let is_last_chunk: bool = rlp.val_at(0)?;
let num_items = rlp.item_count()?;

View File

@ -33,7 +33,7 @@ use snapshot::block::AbridgedBlock;
use ethereum_types::H256;
use kvdb::KeyValueDB;
use bytes::Bytes;
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use rand::OsRng;
/// Snapshot creation and restoration for PoW chains.
@ -225,7 +225,7 @@ impl Rebuilder for PowRebuilder {
use ethereum_types::U256;
use triehash::ordered_trie_root;
let rlp = UntrustedRlp::new(chunk);
let rlp = Rlp::new(chunk);
let item_count = rlp.item_count()?;
let num_blocks = (item_count - 3) as u64;
@ -284,7 +284,7 @@ impl Rebuilder for PowRebuilder {
self.db.write_buffered(batch);
self.chain.commit();
parent_hash = BlockView::new(&block_bytes).hash();
parent_hash = view!(BlockView, &block_bytes).hash();
cur_number += 1;
}

View File

@ -27,7 +27,7 @@ use std::path::{Path, PathBuf};
use bytes::Bytes;
use ethereum_types::H256;
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use super::ManifestData;
@ -238,7 +238,7 @@ impl PackedReader {
file.seek(SeekFrom::Start(manifest_off))?;
file.read_exact(&mut manifest_buf)?;
let rlp = UntrustedRlp::new(&manifest_buf);
let rlp = Rlp::new(&manifest_buf);
let (start, version) = if rlp.item_count()? == 5 {
(0, 1)

View File

@ -39,7 +39,7 @@ use parking_lot::Mutex;
use journaldb::{self, Algorithm, JournalDB};
use kvdb::KeyValueDB;
use trie::{TrieDB, TrieDBMut, Trie, TrieMut};
use rlp::{RlpStream, UntrustedRlp};
use rlp::{RlpStream, Rlp};
use bloom_journal::Bloom;
use self::io::SnapshotWriter;
@ -327,7 +327,7 @@ impl StateRebuilder {
/// Feed an uncompressed state chunk into the rebuilder.
pub fn feed(&mut self, chunk: &[u8], flag: &AtomicBool) -> Result<(), ::error::Error> {
let rlp = UntrustedRlp::new(chunk);
let rlp = Rlp::new(chunk);
let empty_rlp = StateAccount::new_basic(U256::zero(), U256::zero()).rlp();
let mut pairs = Vec::with_capacity(rlp.item_count()?);
@ -415,7 +415,7 @@ struct RebuiltStatus {
// returns a status detailing newly-loaded code and accounts missing code.
fn rebuild_accounts(
db: &mut HashDB,
account_fat_rlps: UntrustedRlp,
account_fat_rlps: Rlp,
out_chunk: &mut [(H256, Bytes)],
known_code: &HashMap<H256, H256>,
known_storage_roots: &mut HashMap<H256, H256>,

View File

@ -913,7 +913,7 @@ mod tests {
);
let genesis = test_spec.genesis_block();
assert_eq!(
BlockView::new(&genesis).header_view().hash(),
view!(BlockView, &genesis).header_view().hash(),
"0cd786a2425d16f152c658316c423e6ce1181e15c3295826d7c9904cba9ce303".into()
);
}

View File

@ -171,7 +171,7 @@ pub fn generate_dummy_client_with_spec_accounts_and_data<F>(test_spec: F, accoun
panic!("error importing block which is valid by definition: {:?}", e);
}
last_header = BlockView::new(&b.rlp_bytes()).header();
last_header = view!(BlockView, &b.rlp_bytes()).header();
db = b.drain();
}
client.flush_queue();

View File

@ -141,7 +141,7 @@ fn query_bad_block() {
fn returns_chain_info() {
let dummy_block = get_good_dummy_block();
let client = get_test_client_with_blocks(vec![dummy_block.clone()]);
let block = BlockView::new(&dummy_block);
let block = view!(BlockView, &dummy_block);
let info = client.chain_info();
assert_eq!(info.best_block_hash, block.header().hash());
}
@ -178,12 +178,12 @@ fn returns_logs_with_limit() {
fn returns_block_body() {
let dummy_block = get_good_dummy_block();
let client = get_test_client_with_blocks(vec![dummy_block.clone()]);
let block = BlockView::new(&dummy_block);
let block = view!(BlockView, &dummy_block);
let body = client.block_body(BlockId::Hash(block.header().hash())).unwrap();
let body = body.rlp();
assert_eq!(body.item_count(), 2);
assert_eq!(body.at(0).as_raw()[..], block.rlp().at(1).as_raw()[..]);
assert_eq!(body.at(1).as_raw()[..], block.rlp().at(2).as_raw()[..]);
assert_eq!(body.item_count().unwrap(), 2);
assert_eq!(body.at(0).unwrap().as_raw()[..], block.rlp().at(1).as_raw()[..]);
assert_eq!(body.at(1).unwrap().as_raw()[..], block.rlp().at(2).as_raw()[..]);
}
#[test]
@ -259,7 +259,7 @@ fn can_mine() {
let b = client.prepare_open_block(Address::default(), (3141562.into(), 31415620.into()), vec![]).close();
assert_eq!(*b.block().header().parent_hash(), BlockView::new(&dummy_blocks[0]).header_view().hash());
assert_eq!(*b.block().header().parent_hash(), view!(BlockView, &dummy_blocks[0]).header_view().hash());
}
#[test]

View File

@ -97,7 +97,7 @@ fn can_trace_block_and_uncle_reward() {
panic!("error importing block which is valid by definition: {:?}", e);
}
last_header = BlockView::new(&root_block.rlp_bytes()).header();
last_header = view!(BlockView, &root_block.rlp_bytes()).header();
let root_header = last_header.clone();
db = root_block.drain();
@ -125,7 +125,7 @@ fn can_trace_block_and_uncle_reward() {
panic!("error importing block which is valid by definition: {:?}", e);
}
last_header = BlockView::new(&parent_block.rlp_bytes()).header();
last_header = view!(BlockView,&parent_block.rlp_bytes()).header();
db = parent_block.drain();
last_hashes.push(last_header.hash());

View File

@ -17,7 +17,7 @@
//! Trace errors.
use std::fmt;
use rlp::{Encodable, RlpStream, Decodable, DecoderError, UntrustedRlp};
use rlp::{Encodable, RlpStream, Decodable, DecoderError, Rlp};
use vm::Error as VmError;
/// Trace evm errors.
@ -115,7 +115,7 @@ impl Encodable for Error {
}
impl Decodable for Error {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
use self::Error::*;
let value: u8 = rlp.as_val()?;
match value {

View File

@ -17,7 +17,7 @@
//! Flat trace module
use std::collections::VecDeque;
use rlp::{UntrustedRlp, RlpStream, Decodable, Encodable, DecoderError};
use rlp::{Rlp, RlpStream, Decodable, Encodable, DecoderError};
use heapsize::HeapSizeOf;
use ethereum_types::Bloom;
use super::trace::{Action, Res};
@ -63,7 +63,7 @@ impl Encodable for FlatTrace {
}
impl Decodable for FlatTrace {
fn decode(d: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(d: &Rlp) -> Result<Self, DecoderError> {
let v: Vec<usize> = d.list_at(3)?;
let res = FlatTrace {
action: d.val_at(0)?,

View File

@ -18,7 +18,7 @@
use ethereum_types::{U256, Address, Bloom, BloomInput};
use bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream, Encodable, DecoderError, Decodable};
use rlp::{Rlp, RlpStream, Encodable, DecoderError, Decodable};
use vm::ActionParams;
use evm::CallType;
@ -154,7 +154,7 @@ impl Encodable for RewardType {
}
impl Decodable for RewardType {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.as_val().and_then(|v| Ok(match v {
0u32 => RewardType::Block,
1 => RewardType::Uncle,
@ -191,7 +191,7 @@ impl Encodable for Reward {
}
impl Decodable for Reward {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let res = Reward {
author: rlp.val_at(0)?,
value: rlp.val_at(1)?,
@ -263,7 +263,7 @@ impl Encodable for Action {
}
impl Decodable for Action {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let action_type: u8 = rlp.val_at(0)?;
match action_type {
0 => rlp.val_at(1).map(Action::Call),
@ -334,7 +334,7 @@ impl Encodable for Res {
}
impl Decodable for Res {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let action_type: u8 = rlp.val_at(0)?;
match action_type {
0 => rlp.val_at(1).map(Res::Call),

View File

@ -122,7 +122,7 @@ pub mod blocks {
pub fn new(bytes: Bytes) -> Self {
use views::BlockView;
let header = BlockView::new(&bytes).header();
let header = view!(BlockView, &bytes).header();
Unverified {
header: header,
bytes: bytes,

View File

@ -733,7 +733,7 @@ mod tests {
use super::kind::blocks::Unverified;
use test_helpers::{get_good_dummy_block_seq, get_good_dummy_block};
use error::*;
use views::*;
use views::BlockView;
// create a test block queue.
// auto_scaling enables verifier adjustment.
@ -785,7 +785,7 @@ mod tests {
fn returns_total_difficulty() {
let queue = get_test_queue(false);
let block = get_good_dummy_block();
let hash = BlockView::new(&block).header().hash().clone();
let hash = view!(BlockView, &block).header().hash().clone();
if let Err(e) = queue.import(Unverified::new(block)) {
panic!("error importing block that is valid by definition({:?})", e);
}
@ -801,7 +801,7 @@ mod tests {
fn returns_ok_for_drained_duplicates() {
let queue = get_test_queue(false);
let block = get_good_dummy_block();
let hash = BlockView::new(&block).header().hash().clone();
let hash = view!(BlockView, &block).header().hash().clone();
if let Err(e) = queue.import(Unverified::new(block)) {
panic!("error importing block that is valid by definition({:?})", e);
}

View File

@ -28,7 +28,7 @@ use bytes::Bytes;
use ethereum_types::H256;
use hash::keccak;
use heapsize::HeapSizeOf;
use rlp::UntrustedRlp;
use rlp::Rlp;
use triehash::ordered_trie_root;
use unexpected::{Mismatch, OutOfBounds};
@ -63,13 +63,13 @@ pub fn verify_block_basic(header: &Header, bytes: &[u8], engine: &EthEngine) ->
verify_header_params(&header, engine, true)?;
verify_block_integrity(bytes, &header.transactions_root(), &header.uncles_hash())?;
engine.verify_block_basic(&header)?;
for u in UntrustedRlp::new(bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
for u in Rlp::new(bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
let u = u?;
verify_header_params(&u, engine, false)?;
engine.verify_block_basic(&u)?;
}
for t in UntrustedRlp::new(bytes).at(1)?.iter().map(|rlp| rlp.as_val::<UnverifiedTransaction>()) {
for t in Rlp::new(bytes).at(1)?.iter().map(|rlp| rlp.as_val::<UnverifiedTransaction>()) {
engine.verify_transaction_basic(&t?, &header)?;
}
Ok(())
@ -81,7 +81,7 @@ pub fn verify_block_basic(header: &Header, bytes: &[u8], engine: &EthEngine) ->
pub fn verify_block_unordered(header: Header, bytes: Bytes, engine: &EthEngine, check_seal: bool) -> Result<PreverifiedBlock, Error> {
if check_seal {
engine.verify_block_unordered(&header)?;
for u in UntrustedRlp::new(&bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
for u in Rlp::new(&bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
engine.verify_block_unordered(&u?)?;
}
}
@ -91,7 +91,7 @@ pub fn verify_block_unordered(header: Header, bytes: Bytes, engine: &EthEngine,
Some((engine.params().nonce_cap_increment * header.number()).into())
} else { None };
{
let v = BlockView::new(&bytes);
let v = view!(BlockView, &bytes);
for t in v.transactions() {
let t = engine.verify_transaction_unordered(t, &header)?;
if let Some(max_nonce) = nonce_cap {
@ -145,7 +145,7 @@ pub fn verify_block_family<C: BlockInfo + CallContract>(header: &Header, parent:
}
fn verify_uncles(header: &Header, bytes: &[u8], bc: &BlockProvider, engine: &EthEngine) -> Result<(), Error> {
let num_uncles = UntrustedRlp::new(bytes).at(2)?.item_count()?;
let num_uncles = Rlp::new(bytes).at(2)?.item_count()?;
let max_uncles = engine.maximum_uncle_count(header.number());
if num_uncles != 0 {
if num_uncles > max_uncles {
@ -174,7 +174,7 @@ fn verify_uncles(header: &Header, bytes: &[u8], bc: &BlockProvider, engine: &Eth
}
let mut verified = HashSet::new();
for uncle in UntrustedRlp::new(bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
for uncle in Rlp::new(bytes).at(2)?.iter().map(|rlp| rlp.as_val::<Header>()) {
let uncle = uncle?;
if excluded.contains(&uncle.hash()) {
return Err(From::from(BlockError::UncleInChain(uncle.hash())))
@ -333,7 +333,7 @@ fn verify_parent(header: &Header, parent: &Header, engine: &EthEngine) -> Result
/// Verify block data against header: transactions root and uncles hash.
fn verify_block_integrity(block: &[u8], transactions_root: &H256, uncles_hash: &H256) -> Result<(), Error> {
let block = UntrustedRlp::new(block);
let block = Rlp::new(block);
let tx = block.at(1)?;
let expected_root = &ordered_trie_root(tx.iter().map(|r| r.as_raw()));
if expected_root != transactions_root {
@ -408,8 +408,8 @@ mod tests {
}
pub fn insert(&mut self, bytes: Bytes) {
let number = BlockView::new(&bytes).header_view().number();
let hash = BlockView::new(&bytes).header_view().hash();
let number = view!(BlockView, &bytes).header_view().number();
let hash = view!(BlockView, &bytes).header_view().hash();
self.blocks.insert(hash.clone(), bytes);
self.numbers.insert(number, hash.clone());
}
@ -448,7 +448,7 @@ mod tests {
/// Get the familial details concerning a block.
fn block_details(&self, hash: &H256) -> Option<BlockDetails> {
self.blocks.get(hash).map(|bytes| {
let header = BlockView::new(bytes).header();
let header = view!(BlockView, bytes).header();
BlockDetails {
number: header.number(),
total_difficulty: header.difficulty().clone(),
@ -482,12 +482,12 @@ mod tests {
}
fn basic_test(bytes: &[u8], engine: &EthEngine) -> Result<(), Error> {
let header = BlockView::new(bytes).header();
let header = view!(BlockView, bytes).header();
verify_block_basic(&header, bytes, engine)
}
fn family_test<BC>(bytes: &[u8], engine: &EthEngine, bc: &BC) -> Result<(), Error> where BC: BlockProvider {
let view = BlockView::new(bytes);
let view = view!(BlockView, bytes);
let header = view.header();
let transactions: Vec<_> = view.transactions()
.into_iter()
@ -514,7 +514,7 @@ mod tests {
}
fn unordered_test(bytes: &[u8], engine: &EthEngine) -> Result<(), Error> {
let header = BlockView::new(bytes).header();
let header = view!(BlockView, bytes).header();
verify_block_unordered(header, bytes.to_vec(), engine, false)?;
Ok(())
}

View File

@ -20,25 +20,34 @@ use bytes::Bytes;
use ethereum_types::H256;
use hash::keccak;
use header::Header;
use rlp::Rlp;
use transaction::{UnverifiedTransaction, LocalizedTransaction};
use views::{TransactionView, HeaderView};
use super::ViewRlp;
/// View onto block rlp.
pub struct BlockView<'a> {
rlp: Rlp<'a>
rlp: ViewRlp<'a>
}
impl<'a> BlockView<'a> {
/// Creates new view onto block from raw bytes.
pub fn new(bytes: &'a [u8]) -> BlockView<'a> {
BlockView {
rlp: Rlp::new(bytes)
}
}
impl<'a> BlockView<'a> {
/// Creates new view onto block from rlp.
pub fn new_from_rlp(rlp: Rlp<'a>) -> BlockView<'a> {
/// Use the `view!` macro to create this view in order to capture debugging info.
///
/// # Example
///
/// ```
/// #[macro_use]
/// extern crate ethcore;
///
/// use ethcore::views::{BlockView};
///
/// fn main() {
/// let bytes : &[u8] = &[];
/// let block_view = view!(BlockView, bytes);
/// }
/// ```
pub fn new(rlp: ViewRlp<'a>) -> BlockView<'a> {
BlockView {
rlp: rlp
}
@ -50,7 +59,7 @@ impl<'a> BlockView<'a> {
}
/// Return reference to underlaying rlp.
pub fn rlp(&self) -> &Rlp<'a> {
pub fn rlp(&self) -> &ViewRlp<'a> {
&self.rlp
}
@ -60,13 +69,13 @@ impl<'a> BlockView<'a> {
}
/// Return header rlp.
pub fn header_rlp(&self) -> Rlp {
pub fn header_rlp(&self) -> ViewRlp<'a> {
self.rlp.at(0)
}
/// Create new header view obto block head rlp.
pub fn header_view(&self) -> HeaderView<'a> {
HeaderView::new_from_rlp(self.rlp.at(0))
HeaderView::new(self.header_rlp())
}
/// Return List of transactions in given block.
@ -92,28 +101,28 @@ impl<'a> BlockView<'a> {
}
/// Return the raw rlp for the transactions in the given block.
pub fn transactions_rlp(&self) -> Rlp<'a> {
pub fn transactions_rlp(&self) -> ViewRlp<'a> {
self.rlp.at(1)
}
/// Return number of transactions in given block, without deserializing them.
pub fn transactions_count(&self) -> usize {
self.rlp.at(1).iter().count()
self.transactions_rlp().iter().count()
}
/// Return List of transactions in given block.
pub fn transaction_views(&self) -> Vec<TransactionView<'a>> {
self.rlp.at(1).iter().map(TransactionView::new_from_rlp).collect()
self.transactions_rlp().iter().map(TransactionView::new).collect()
}
/// Return transaction hashes.
pub fn transaction_hashes(&self) -> Vec<H256> {
self.rlp.at(1).iter().map(|rlp| keccak(rlp.as_raw())).collect()
self.transactions_rlp().iter().map(|rlp| keccak(rlp.as_raw())).collect()
}
/// Returns transaction at given index without deserializing unnecessary data.
pub fn transaction_at(&self, index: usize) -> Option<UnverifiedTransaction> {
self.rlp.at(1).iter().nth(index).map(|rlp| rlp.as_val())
self.transactions_rlp().iter().nth(index).map(|rlp| rlp.as_val())
}
/// Returns localized transaction at given index.
@ -131,7 +140,7 @@ impl<'a> BlockView<'a> {
}
/// Returns raw rlp for the uncles in the given block
pub fn uncles_rlp(&self) -> Rlp<'a> {
pub fn uncles_rlp(&self) -> ViewRlp<'a> {
self.rlp.at(2)
}
@ -142,27 +151,27 @@ impl<'a> BlockView<'a> {
/// Return number of uncles in given block, without deserializing them.
pub fn uncles_count(&self) -> usize {
self.rlp.at(2).iter().count()
self.uncles_rlp().iter().count()
}
/// Return List of transactions in given block.
pub fn uncle_views(&self) -> Vec<HeaderView<'a>> {
self.rlp.at(2).iter().map(HeaderView::new_from_rlp).collect()
self.uncles_rlp().iter().map(HeaderView::new).collect()
}
/// Return list of uncle hashes of given block.
pub fn uncle_hashes(&self) -> Vec<H256> {
self.rlp.at(2).iter().map(|rlp| keccak(rlp.as_raw())).collect()
self.uncles_rlp().iter().map(|rlp| keccak(rlp.as_raw())).collect()
}
/// Return nth uncle.
pub fn uncle_at(&self, index: usize) -> Option<Header> {
self.rlp.at(2).iter().nth(index).map(|rlp| rlp.as_val())
self.uncles_rlp().iter().nth(index).map(|rlp| rlp.as_val())
}
/// Return nth uncle rlp.
pub fn uncle_rlp_at(&self, index: usize) -> Option<Bytes> {
self.rlp.at(2).iter().nth(index).map(|rlp| rlp.as_raw().to_vec())
self.uncles_rlp().iter().nth(index).map(|rlp| rlp.as_raw().to_vec())
}
}
@ -176,7 +185,7 @@ mod tests {
// that's rlp of block created with ethash engine.
let rlp = "f90261f901f9a0d405da4e66f1445d455195229624e133f5baafe72b5cf7b3c36c12c8146e98b7a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347948888f1f195afa192cfee860698584c030f4c9db1a05fb2b4bfdef7b314451cb138a534d225c922fc0e5fbe25e451142732c3e25c25a088d2ec6b9860aae1a2c3b299f72b6a5d70d7f7ba4722c78f2c49ba96273c2158a007c6fdfa8eea7e86b81f5b0fc0f78f90cc19f4aa60d323151e0cac660199e9a1b90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302008003832fefba82524d84568e932a80a0a0349d8c3df71f1a48a9df7d03fd5f14aeee7d91332c009ecaff0a71ead405bd88ab4e252a7e8c2a23f862f86002018304cb2f94ec0e71ad0a90ffe1909d27dac207f7680abba42d01801ba03a347e72953c860f32b1eb2c78a680d8734b2ea08085d949d729479796f218d5a047ea6239d9e31ccac8af3366f5ca37184d26e7646e3191a3aeb81c4cf74de500c0".from_hex().unwrap();
let view = BlockView::new(&rlp);
let view = view!(BlockView, &rlp);
assert_eq!(view.hash(), "2c9747e804293bd3f1a986484343f23bc88fd5be75dfe9d5c2860aff61e6f259".into());
assert_eq!(view.transactions_count(), 1);
assert_eq!(view.uncles_count(), 0);

View File

@ -20,32 +20,40 @@ use bytes::Bytes;
use ethereum_types::H256;
use hash::keccak;
use header::{Header, BlockNumber};
use rlp::Rlp;
use transaction::{LocalizedTransaction, UnverifiedTransaction};
use views::{TransactionView, HeaderView};
use super::ViewRlp;
/// View onto block rlp.
pub struct BodyView<'a> {
rlp: Rlp<'a>
rlp: ViewRlp<'a>
}
impl<'a> BodyView<'a> {
/// Creates new view onto block from raw bytes.
pub fn new(bytes: &'a [u8]) -> BodyView<'a> {
BodyView {
rlp: Rlp::new(bytes)
}
}
/// Creates new view onto block from rlp.
pub fn new_from_rlp(rlp: Rlp<'a>) -> BodyView<'a> {
/// Creates new view onto block body from rlp.
/// Use the `view!` macro to create this view in order to capture debugging info.
///
/// # Example
///
/// ```
/// #[macro_use]
/// extern crate ethcore;
///
/// use ethcore::views::{BodyView};
///
/// fn main() {
/// let bytes : &[u8] = &[];
/// let body_view = view!(BodyView, bytes);
/// }
/// ```
pub fn new(rlp: ViewRlp<'a>) -> BodyView<'a> {
BodyView {
rlp: rlp
}
}
/// Return reference to underlaying rlp.
pub fn rlp(&self) -> &Rlp<'a> {
pub fn rlp(&self) -> &ViewRlp<'a> {
&self.rlp
}
@ -69,27 +77,27 @@ impl<'a> BodyView<'a> {
}
/// Return the raw rlp for the transactions in the given block.
pub fn transactions_rlp(&self) -> Rlp<'a> {
pub fn transactions_rlp(&self) -> ViewRlp<'a> {
self.rlp.at(0)
}
/// Return number of transactions in given block, without deserializing them.
pub fn transactions_count(&self) -> usize {
self.rlp.at(0).item_count()
self.transactions_rlp().item_count()
}
/// Return List of transactions in given block.
pub fn transaction_views(&self) -> Vec<TransactionView<'a>> {
self.rlp.at(0).iter().map(TransactionView::new_from_rlp).collect()
self.transactions_rlp().iter().map(TransactionView::new).collect()
}
/// Return transaction hashes.
pub fn transaction_hashes(&self) -> Vec<H256> {
self.rlp.at(0).iter().map(|rlp| keccak(rlp.as_raw())).collect()
self.transactions_rlp().iter().map(|rlp| keccak(rlp.as_raw())).collect()
}
/// Returns transaction at given index without deserializing unnecessary data.
pub fn transaction_at(&self, index: usize) -> Option<UnverifiedTransaction> {
self.rlp.at(0).iter().nth(index).map(|rlp| rlp.as_val())
self.transactions_rlp().iter().nth(index).map(|rlp| rlp.as_val())
}
/// Returns localized transaction at given index.
@ -104,7 +112,7 @@ impl<'a> BodyView<'a> {
}
/// Returns raw rlp for the uncles in the given block
pub fn uncles_rlp(&self) -> Rlp<'a> {
pub fn uncles_rlp(&self) -> ViewRlp<'a> {
self.rlp.at(1)
}
@ -115,27 +123,27 @@ impl<'a> BodyView<'a> {
/// Return number of uncles in given block, without deserializing them.
pub fn uncles_count(&self) -> usize {
self.rlp.at(1).item_count()
self.uncles_rlp().item_count()
}
/// Return List of transactions in given block.
pub fn uncle_views(&self) -> Vec<HeaderView<'a>> {
self.rlp.at(1).iter().map(HeaderView::new_from_rlp).collect()
self.uncles_rlp().iter().map(HeaderView::new).collect()
}
/// Return list of uncle hashes of given block.
pub fn uncle_hashes(&self) -> Vec<H256> {
self.rlp.at(1).iter().map(|rlp| keccak(rlp.as_raw())).collect()
self.uncles_rlp().iter().map(|rlp| keccak(rlp.as_raw())).collect()
}
/// Return nth uncle.
pub fn uncle_at(&self, index: usize) -> Option<Header> {
self.rlp.at(1).iter().nth(index).map(|rlp| rlp.as_val())
self.uncles_rlp().iter().nth(index).map(|rlp| rlp.as_val())
}
/// Return nth uncle rlp.
pub fn uncle_rlp_at(&self, index: usize) -> Option<Bytes> {
self.rlp.at(1).iter().nth(index).map(|rlp| rlp.as_raw().to_vec())
self.uncles_rlp().iter().nth(index).map(|rlp| rlp.as_raw().to_vec())
}
}
@ -150,7 +158,7 @@ mod tests {
// that's rlp of block created with ethash engine.
let rlp = "f90261f901f9a0d405da4e66f1445d455195229624e133f5baafe72b5cf7b3c36c12c8146e98b7a01dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347948888f1f195afa192cfee860698584c030f4c9db1a05fb2b4bfdef7b314451cb138a534d225c922fc0e5fbe25e451142732c3e25c25a088d2ec6b9860aae1a2c3b299f72b6a5d70d7f7ba4722c78f2c49ba96273c2158a007c6fdfa8eea7e86b81f5b0fc0f78f90cc19f4aa60d323151e0cac660199e9a1b90100000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008302008003832fefba82524d84568e932a80a0a0349d8c3df71f1a48a9df7d03fd5f14aeee7d91332c009ecaff0a71ead405bd88ab4e252a7e8c2a23f862f86002018304cb2f94ec0e71ad0a90ffe1909d27dac207f7680abba42d01801ba03a347e72953c860f32b1eb2c78a680d8734b2ea08085d949d729479796f218d5a047ea6239d9e31ccac8af3366f5ca37184d26e7646e3191a3aeb81c4cf74de500c0".from_hex().unwrap();
let body = BlockChain::block_to_body(&rlp);
let view = BodyView::new(&body);
let view = view!(BodyView, &body);
assert_eq!(view.transactions_count(), 1);
assert_eq!(view.uncles_count(), 0);
}

View File

@ -20,35 +20,44 @@ use bytes::Bytes;
use ethereum_types::{H256, Bloom, U256, Address};
use hash::keccak;
use header::BlockNumber;
use rlp::{self, Rlp};
use rlp::{self};
use super::ViewRlp;
/// View onto block header rlp.
pub struct HeaderView<'a> {
rlp: Rlp<'a>
rlp: ViewRlp<'a>
}
impl<'a> HeaderView<'a> {
/// Creates new view onto header from raw bytes.
pub fn new(bytes: &'a [u8]) -> HeaderView<'a> {
/// Creates a new Header view from valid ViewRlp
/// Use the `view!` macro to create this view in order to capture debugging info.
///
/// # Example
///
/// ```
/// #[macro_use]
/// extern crate ethcore;
///
/// use ethcore::views::{HeaderView};
///
/// fn main() {
/// let bytes : &[u8] = &[];
/// let tx_view = view!(HeaderView, bytes);
/// }
/// ```
pub fn new(rlp: ViewRlp<'a>) -> HeaderView<'a> {
HeaderView {
rlp: Rlp::new(bytes)
}
}
/// Creates new view onto header from rlp.
pub fn new_from_rlp(rlp: Rlp<'a>) -> HeaderView<'a> {
HeaderView {
rlp: rlp
rlp
}
}
/// Returns header hash.
pub fn hash(&self) -> H256 {
keccak(self.rlp.as_raw())
keccak(self.rlp.rlp.as_raw())
}
/// Returns raw rlp.
pub fn rlp(&self) -> &Rlp<'a> { &self.rlp }
pub fn rlp(&self) -> &ViewRlp<'a> { &self.rlp }
/// Returns parent hash.
pub fn parent_hash(&self) -> H256 { self.rlp.val_at(0) }
@ -102,9 +111,10 @@ impl<'a> HeaderView<'a> {
pub fn decode_seal(&self) -> Result<Vec<Bytes>, rlp::DecoderError> {
let seal = self.seal();
seal.into_iter()
.map(|s| rlp::UntrustedRlp::new(&s).data().map(|x| x.to_vec()))
.map(|s| rlp::Rlp::new(&s).data().map(|x| x.to_vec()))
.collect()
}
}
#[cfg(test)]
@ -120,7 +130,7 @@ mod tests {
let mix_hash = "a0a0349d8c3df71f1a48a9df7d03fd5f14aeee7d91332c009ecaff0a71ead405bd".from_hex().unwrap();
let nonce = "88ab4e252a7e8c2a23".from_hex().unwrap();
let view = HeaderView::new(&rlp);
let view = view!(HeaderView, &rlp);
assert_eq!(view.hash(), "2c9747e804293bd3f1a986484343f23bc88fd5be75dfe9d5c2860aff61e6f259".into());
assert_eq!(view.parent_hash(), "d405da4e66f1445d455195229624e133f5baafe72b5cf7b3c36c12c8146e98b7".into());
assert_eq!(view.uncles_hash(), "1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347".into());

View File

@ -16,12 +16,26 @@
//! Block oriented views onto rlp.
#[macro_use]
mod view_rlp;
mod block;
mod body;
mod header;
mod transaction;
pub use self::view_rlp::ViewRlp;
pub use self::block::BlockView;
pub use self::body::BodyView;
pub use self::header::HeaderView;
pub use self::transaction::TransactionView;
#[cfg(test)]
mod tests {
use super::HeaderView;
#[test]
#[should_panic(expected="View rlp is trusted and should be valid. Constructed in ethcore/src/views/mod.rs on line 39: RlpExpectedToBeList")]
fn should_include_file_line_number_in_panic_for_invalid_rlp() {
let _ = view!(HeaderView, &[]).parent_hash();
}
}

View File

@ -18,30 +18,39 @@
use bytes::Bytes;
use ethereum_types::{H256, U256};
use hash::keccak;
use rlp::Rlp;
// use rlp::{Rlp, Decodable};
use super::ViewRlp;
/// View onto transaction rlp.
pub struct TransactionView<'a> {
rlp: Rlp<'a>
rlp: ViewRlp<'a>
}
impl<'a> TransactionView<'a> {
/// Creates new view onto block from raw bytes.
pub fn new(bytes: &'a [u8]) -> TransactionView<'a> {
TransactionView {
rlp: Rlp::new(bytes)
}
}
/// Creates new view onto block from rlp.
pub fn new_from_rlp(rlp: Rlp<'a>) -> TransactionView<'a> {
/// Creates new view onto valid transaction rlp.
/// Use the `view!` macro to create this view in order to capture debugging info.
///
/// # Example
///
/// ```
/// #[macro_use]
/// extern crate ethcore;
///
/// use ethcore::views::{TransactionView};
///
/// fn main() {
/// let bytes : &[u8] = &[];
/// let tx_view = view!(TransactionView, bytes);
/// }
/// ```
pub fn new(rlp: ViewRlp<'a>) -> TransactionView<'a> {
TransactionView {
rlp: rlp
}
}
/// Return reference to underlaying rlp.
pub fn rlp(&self) -> &Rlp<'a> {
pub fn rlp(&self) -> &ViewRlp<'a> {
&self.rlp
}
@ -84,7 +93,7 @@ mod tests {
fn test_transaction_view() {
let rlp = "f87c80018261a894095e7baea6a6c7c4c2dfeb977efac326af552d870a9d00000000000000000000000000000000000000000000000000000000001ba048b55bfa915ac795c431978d8a6a992b628d557da5ff759b307d495a36649353a0efffd310ac743f371de3b9f7f9cb56c0b28ad43601b4ab949f53faa07bd2c804".from_hex().unwrap();
let view = TransactionView::new(&rlp);
let view = view!(TransactionView, &rlp);
assert_eq!(view.nonce(), 0.into());
assert_eq!(view.gas_price(), 1.into());
assert_eq!(view.gas(), 0x61a8.into());

View File

@ -0,0 +1,130 @@
// Copyright 2015-2017 Parity Technologies (UK) Ltd.
// This file is part of Parity.
// Parity 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 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. If not, see <http://www.gnu.org/licenses/>.
//! Wrapper for view rlp expected to be valid with debug info
use rlp::{Rlp, Decodable, DecoderError};
/// Wrapper for trusted rlp, which is expected to be valid, for use in views
/// When created with view!, records the file and line where it was created for debugging
pub struct ViewRlp<'a> {
/// Wrapped Rlp, expected to be valid
pub rlp: Rlp<'a>,
file: &'a str,
line: u32,
}
impl<'a, 'view> ViewRlp<'a> where 'a : 'view {
#[doc(hidden)]
pub fn new(bytes: &'a [u8], file: &'a str, line: u32) -> Self {
ViewRlp {
rlp: Rlp::new(bytes),
file,
line
}
}
/// Returns a new instance replacing existing rlp with new rlp, maintaining debug info
fn new_from_rlp(&self, rlp: Rlp<'a>) -> Self {
ViewRlp {
rlp,
file: self.file,
line: self.line
}
}
fn maybe_at(&self, index: usize) -> Option<ViewRlp<'a>> {
self.rlp.at(index)
.map(|rlp| self.new_from_rlp(rlp))
.ok()
}
fn expect_valid_rlp<T>(&self, r: Result<T, DecoderError>) -> T {
r.expect(&format!("View rlp is trusted and should be valid. Constructed in {} on line {}", self.file, self.line))
}
/// Returns rlp at the given index, panics if no rlp at that index
pub fn at(&self, index: usize) -> ViewRlp<'a> {
let rlp = self.expect_valid_rlp(self.rlp.at(index));
self.new_from_rlp(rlp)
}
/// Returns an iterator over all rlp values
pub fn iter(&'view self) -> ViewRlpIterator<'a, 'view> {
self.into_iter()
}
/// Returns decoded value of this rlp, panics if rlp not valid
pub fn as_val<T>(&self) -> T where T: Decodable {
self.expect_valid_rlp(self.rlp.as_val())
}
/// Returns decoded value at the given index, panics not present or valid at that index
pub fn val_at<T>(&self, index: usize) -> T where T : Decodable {
self.expect_valid_rlp(self.rlp.val_at(index))
}
/// Returns decoded list of values, panics if rlp is invalid
pub fn list_at<T>(&self, index: usize) -> Vec<T> where T: Decodable {
self.expect_valid_rlp(self.rlp.list_at(index))
}
/// Returns the number of items in the rlp, panics if it is not a list of rlp values
pub fn item_count(&self) -> usize {
self.expect_valid_rlp(self.rlp.item_count())
}
/// Returns raw rlp bytes
pub fn as_raw(&'view self) -> &'a [u8] {
self.rlp.as_raw()
}
}
/// Iterator over rlp-slice list elements.
pub struct ViewRlpIterator<'a, 'view> where 'a: 'view {
rlp: &'view ViewRlp<'a>,
index: usize,
}
impl<'a, 'view> IntoIterator for &'view ViewRlp<'a> where 'a: 'view {
type Item = ViewRlp<'a>;
type IntoIter = ViewRlpIterator<'a, 'view>;
fn into_iter(self) -> Self::IntoIter {
ViewRlpIterator {
rlp: self,
index: 0,
}
}
}
impl<'a, 'view> Iterator for ViewRlpIterator<'a, 'view> {
type Item = ViewRlp<'a>;
fn next(&mut self) -> Option<ViewRlp<'a>> {
let index = self.index;
let result = self.rlp.maybe_at(index);
self.index += 1;
result
}
}
#[macro_export]
macro_rules! view {
($view: ident, $bytes: expr) => {
$view::new($crate::views::ViewRlp::new($bytes, file!(), line!()))
};
}

View File

@ -22,8 +22,8 @@ use std::collections::{HashSet, VecDeque};
use std::cmp;
use heapsize::HeapSizeOf;
use ethereum_types::H256;
use rlp::UntrustedRlp;
use ethcore::views::{BlockView};
use rlp::Rlp;
use ethcore::views::BlockView;
use ethcore::header::{BlockNumber, Header as BlockHeader};
use ethcore::client::{BlockStatus, BlockId, BlockImportError};
use ethcore::block::Block;
@ -216,7 +216,7 @@ impl BlockDownloader {
}
/// Add new block headers.
pub fn import_headers(&mut self, io: &mut SyncIo, r: &UntrustedRlp, expected_hash: Option<H256>) -> Result<DownloadAction, BlockDownloaderImportError> {
pub fn import_headers(&mut self, io: &mut SyncIo, r: &Rlp, expected_hash: Option<H256>) -> Result<DownloadAction, BlockDownloaderImportError> {
let item_count = r.item_count().unwrap_or(0);
if self.state == State::Idle {
trace!(target: "sync", "Ignored unexpected block headers");
@ -316,7 +316,7 @@ impl BlockDownloader {
}
/// Called by peer once it has new block bodies
pub fn import_bodies(&mut self, _io: &mut SyncIo, r: &UntrustedRlp) -> Result<(), BlockDownloaderImportError> {
pub fn import_bodies(&mut self, _io: &mut SyncIo, r: &Rlp) -> Result<(), BlockDownloaderImportError> {
let item_count = r.item_count().unwrap_or(0);
if item_count == 0 {
return Err(BlockDownloaderImportError::Useless);
@ -342,7 +342,7 @@ impl BlockDownloader {
}
/// Called by peer once it has new block bodies
pub fn import_receipts(&mut self, _io: &mut SyncIo, r: &UntrustedRlp) -> Result<(), BlockDownloaderImportError> {
pub fn import_receipts(&mut self, _io: &mut SyncIo, r: &Rlp) -> Result<(), BlockDownloaderImportError> {
let item_count = r.item_count().unwrap_or(0);
if item_count == 0 {
return Err(BlockDownloaderImportError::Useless);
@ -478,7 +478,7 @@ impl BlockDownloader {
let block = block_and_receipts.block;
let receipts = block_and_receipts.receipts;
let (h, number, parent) = {
let header = BlockView::new(&block).header_view();
let header = view!(BlockView, &block).header_view();
(header.hash(), header.number(), header.parent_hash())
};

View File

@ -22,7 +22,7 @@ use heapsize::HeapSizeOf;
use ethereum_types::H256;
use triehash::ordered_trie_root;
use bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream, DecoderError};
use rlp::{Rlp, RlpStream, DecoderError};
use network;
use ethcore::encoded::Block;
use ethcore::views::{HeaderView, BodyView};
@ -292,8 +292,9 @@ impl BlockCollection {
}
for block in blocks {
let body = BodyView::new(block.body.as_ref().expect("blocks contains only full blocks; qed"));
let block_view = Block::new_from_header_and_body(&HeaderView::new(&block.header), &body);
let body = view!(BodyView, block.body.as_ref().expect("blocks contains only full blocks; qed"));
let header = view!(HeaderView, &block.header);
let block_view = Block::new_from_header_and_body(&header, &body);
drained.push(BlockAndReceipts {
block: block_view.rlp().as_raw().to_vec(),
receipts: block.receipts.clone(),
@ -340,7 +341,7 @@ impl BlockCollection {
fn insert_body(&mut self, b: Bytes) -> Result<(), network::Error> {
let header_id = {
let body = UntrustedRlp::new(&b);
let body = Rlp::new(&b);
let tx = body.at(0)?;
let tx_root = ordered_trie_root(tx.iter().map(|r| r.as_raw()));
let uncles = keccak(body.at(1)?.as_raw());
@ -375,7 +376,7 @@ impl BlockCollection {
fn insert_receipt(&mut self, r: Bytes) -> Result<(), network::Error> {
let receipt_root = {
let receipts = UntrustedRlp::new(&r);
let receipts = Rlp::new(&r);
ordered_trie_root(receipts.iter().map(|r| r.as_raw()))
};
self.downloading_receipts.remove(&receipt_root);
@ -403,7 +404,7 @@ impl BlockCollection {
}
fn insert_header(&mut self, header: Bytes) -> Result<H256, DecoderError> {
let info: BlockHeader = UntrustedRlp::new(&header).as_val()?;
let info: BlockHeader = Rlp::new(&header).as_val()?;
let hash = info.hash();
if self.blocks.contains_key(&hash) {
return Ok(hash);
@ -525,8 +526,8 @@ mod test {
let blocks: Vec<_> = (0..nblocks)
.map(|i| (&client as &BlockChainClient).block(BlockId::Number(i as BlockNumber)).unwrap().into_inner())
.collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| HeaderView::new(h).hash()).collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).unwrap().as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| view!(HeaderView, h).hash()).collect();
let heads: Vec<_> = hashes.iter().enumerate().filter_map(|(i, h)| if i % 20 == 0 { Some(h.clone()) } else { None }).collect();
bc.reset_to(heads);
assert!(!bc.is_empty());
@ -580,8 +581,8 @@ mod test {
let blocks: Vec<_> = (0..nblocks)
.map(|i| (&client as &BlockChainClient).block(BlockId::Number(i as BlockNumber)).unwrap().into_inner())
.collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| HeaderView::new(h).hash()).collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).unwrap().as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| view!(HeaderView, h).hash()).collect();
let heads: Vec<_> = hashes.iter().enumerate().filter_map(|(i, h)| if i % 20 == 0 { Some(h.clone()) } else { None }).collect();
bc.reset_to(heads);
@ -604,8 +605,8 @@ mod test {
let blocks: Vec<_> = (0..nblocks)
.map(|i| (&client as &BlockChainClient).block(BlockId::Number(i as BlockNumber)).unwrap().into_inner())
.collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| HeaderView::new(h).hash()).collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).unwrap().as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| view!(HeaderView, h).hash()).collect();
let heads: Vec<_> = hashes.iter().enumerate().filter_map(|(i, h)| if i % 20 == 0 { Some(h.clone()) } else { None }).collect();
bc.reset_to(heads);

View File

@ -98,7 +98,7 @@ use ethereum_types::{H256, U256};
use plain_hasher::H256FastMap;
use parking_lot::RwLock;
use bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream, DecoderError, Encodable};
use rlp::{Rlp, RlpStream, DecoderError, Encodable};
use network::{self, PeerId, PacketId};
use ethcore::header::{BlockNumber, Header as BlockHeader};
use ethcore::client::{BlockChainClient, BlockStatus, BlockId, BlockChainInfo, BlockImportError, BlockQueueInfo};
@ -632,7 +632,7 @@ impl ChainSync {
}
/// Called by peer to report status
fn on_peer_status(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_status(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
self.handshaking_peers.remove(&peer_id);
let protocol_version: u8 = r.val_at(0)?;
let warp_protocol = io.protocol_version(&WARP_SYNC_PROTOCOL_ID, peer_id) != 0;
@ -702,7 +702,7 @@ impl ChainSync {
}
/// Called by peer once it has new block headers during sync
fn on_peer_block_headers(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_block_headers(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
let confirmed = match self.peers.get_mut(&peer_id) {
Some(ref mut peer) if peer.asking == PeerAsking::ForkHeader => {
peer.asking = PeerAsking::Nothing;
@ -803,7 +803,7 @@ impl ChainSync {
}
/// Called by peer once it has new block bodies
fn on_peer_block_bodies(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_block_bodies(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
self.clear_peer_download(peer_id);
let block_set = self.peers.get(&peer_id).and_then(|p| p.block_set).unwrap_or(BlockSet::NewBlocks);
if !self.reset_peer_asking(peer_id, PeerAsking::BlockBodies) {
@ -857,7 +857,7 @@ impl ChainSync {
}
/// Called by peer once it has new block receipts
fn on_peer_block_receipts(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_block_receipts(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
self.clear_peer_download(peer_id);
let block_set = self.peers.get(&peer_id).and_then(|p| p.block_set).unwrap_or(BlockSet::NewBlocks);
if !self.reset_peer_asking(peer_id, PeerAsking::BlockReceipts) {
@ -911,7 +911,7 @@ impl ChainSync {
}
/// Called by peer once it has new block bodies
fn on_peer_new_block(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_new_block(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "Ignoring new block from unconfirmed peer {}", peer_id);
return Ok(());
@ -978,7 +978,7 @@ impl ChainSync {
}
/// Handles `NewHashes` packet. Initiates headers download for any unknown hashes.
fn on_peer_new_hashes(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_new_hashes(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "Ignoring new hashes from unconfirmed peer {}", peer_id);
return Ok(());
@ -1053,7 +1053,7 @@ impl ChainSync {
}
/// Called when snapshot manifest is downloaded from a peer.
fn on_snapshot_manifest(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_snapshot_manifest(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "Ignoring snapshot manifest from unconfirmed peer {}", peer_id);
return Ok(());
@ -1097,7 +1097,7 @@ impl ChainSync {
}
/// Called when snapshot data is downloaded from a peer.
fn on_snapshot_data(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_snapshot_data(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "Ignoring snapshot data from unconfirmed peer {}", peer_id);
return Ok(());
@ -1501,7 +1501,7 @@ impl ChainSync {
}
/// Called when peer sends us new transactions
fn on_peer_transactions(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_peer_transactions(&mut self, io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
// Accept transactions only when fully synced
if !io.is_chain_queue_empty() || (self.state != SyncState::Idle && self.state != SyncState::NewBlocks) {
trace!(target: "sync", "{} Ignoring transactions while syncing", peer_id);
@ -1555,7 +1555,7 @@ impl ChainSync {
}
/// Respond to GetBlockHeaders request
fn return_block_headers(io: &SyncIo, r: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_block_headers(io: &SyncIo, r: &Rlp, peer_id: PeerId) -> RlpResponseResult {
// Packet layout:
// [ block: { P , B_32 }, maxHeaders: P, skip: P, reverse: P in { 0 , 1 } ]
let max_headers: usize = r.val_at(1)?;
@ -1628,7 +1628,7 @@ impl ChainSync {
}
/// Respond to GetBlockBodies request
fn return_block_bodies(io: &SyncIo, r: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_block_bodies(io: &SyncIo, r: &Rlp, peer_id: PeerId) -> RlpResponseResult {
let mut count = r.item_count().unwrap_or(0);
if count == 0 {
debug!(target: "sync", "Empty GetBlockBodies request, ignoring.");
@ -1650,7 +1650,7 @@ impl ChainSync {
}
/// Respond to GetNodeData request
fn return_node_data(io: &SyncIo, r: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_node_data(io: &SyncIo, r: &Rlp, peer_id: PeerId) -> RlpResponseResult {
let mut count = r.item_count().unwrap_or(0);
trace!(target: "sync", "{} -> GetNodeData: {} entries", peer_id, count);
if count == 0 {
@ -1674,7 +1674,7 @@ impl ChainSync {
Ok(Some((NODE_DATA_PACKET, rlp)))
}
fn return_receipts(io: &SyncIo, rlp: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_receipts(io: &SyncIo, rlp: &Rlp, peer_id: PeerId) -> RlpResponseResult {
let mut count = rlp.item_count().unwrap_or(0);
trace!(target: "sync", "{} -> GetReceipts: {} entries", peer_id, count);
if count == 0 {
@ -1699,7 +1699,7 @@ impl ChainSync {
}
/// Respond to GetSnapshotManifest request
fn return_snapshot_manifest(io: &SyncIo, r: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_snapshot_manifest(io: &SyncIo, r: &Rlp, peer_id: PeerId) -> RlpResponseResult {
let count = r.item_count().unwrap_or(0);
trace!(target: "sync", "{} -> GetSnapshotManifest", peer_id);
if count != 0 {
@ -1722,7 +1722,7 @@ impl ChainSync {
}
/// Respond to GetSnapshotData request
fn return_snapshot_data(io: &SyncIo, r: &UntrustedRlp, peer_id: PeerId) -> RlpResponseResult {
fn return_snapshot_data(io: &SyncIo, r: &Rlp, peer_id: PeerId) -> RlpResponseResult {
let hash: H256 = r.val_at(0)?;
trace!(target: "sync", "{} -> GetSnapshotData {:?}", peer_id, hash);
let rlp = match io.snapshot_service().chunk(hash) {
@ -1739,8 +1739,8 @@ impl ChainSync {
Ok(Some((SNAPSHOT_DATA_PACKET, rlp)))
}
fn return_rlp<FRlp, FError>(io: &mut SyncIo, rlp: &UntrustedRlp, peer: PeerId, rlp_func: FRlp, error_func: FError) -> Result<(), PacketDecodeError>
where FRlp : Fn(&SyncIo, &UntrustedRlp, PeerId) -> RlpResponseResult,
fn return_rlp<FRlp, FError>(io: &mut SyncIo, rlp: &Rlp, peer: PeerId, rlp_func: FRlp, error_func: FError) -> Result<(), PacketDecodeError>
where FRlp : Fn(&SyncIo, &Rlp, PeerId) -> RlpResponseResult,
FError : FnOnce(network::Error) -> String
{
let response = rlp_func(io, rlp, peer);
@ -1757,7 +1757,7 @@ impl ChainSync {
/// Dispatch incoming requests and responses
pub fn dispatch_packet(sync: &RwLock<ChainSync>, io: &mut SyncIo, peer: PeerId, packet_id: u8, data: &[u8]) {
let rlp = UntrustedRlp::new(data);
let rlp = Rlp::new(data);
let result = match packet_id {
GET_BLOCK_BODIES_PACKET => ChainSync::return_rlp(io, &rlp, peer,
ChainSync::return_block_bodies,
@ -1798,7 +1798,7 @@ impl ChainSync {
debug!(target:"sync", "Unexpected packet {} from unregistered peer: {}:{}", packet_id, peer, io.peer_info(peer));
return;
}
let rlp = UntrustedRlp::new(data);
let rlp = Rlp::new(data);
let result = match packet_id {
STATUS_PACKET => self.on_peer_status(io, peer, &rlp),
TRANSACTIONS_PACKET => self.on_peer_transactions(io, peer, &rlp),
@ -2233,7 +2233,7 @@ impl ChainSync {
}
/// Called when peer sends us new consensus packet
fn on_consensus_packet(io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_consensus_packet(io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
trace!(target: "sync", "Received consensus packet from {:?}", peer_id);
io.chain().queue_consensus_message(r.as_raw().to_vec());
Ok(())
@ -2249,7 +2249,7 @@ impl ChainSync {
}
/// Called when peer sends us new private transaction packet
fn on_private_transaction(&self, _io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_private_transaction(&self, _io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "{} Ignoring packet from unconfirmed/unknown peer", peer_id);
return Ok(());
@ -2273,7 +2273,7 @@ impl ChainSync {
}
/// Called when peer sends us signed private transaction packet
fn on_signed_private_transaction(&self, _io: &mut SyncIo, peer_id: PeerId, r: &UntrustedRlp) -> Result<(), PacketDecodeError> {
fn on_signed_private_transaction(&self, _io: &mut SyncIo, peer_id: PeerId, r: &Rlp) -> Result<(), PacketDecodeError> {
if !self.peers.get(&peer_id).map_or(false, |p| p.can_sync()) {
trace!(target: "sync", "{} Ignoring packet from unconfirmed/unknown peer", peer_id);
return Ok(());
@ -2324,7 +2324,7 @@ mod tests {
use ethereum_types::{H256, U256, Address};
use parking_lot::RwLock;
use bytes::Bytes;
use rlp::{Rlp, RlpStream, UntrustedRlp};
use rlp::{Rlp, RlpStream};
use super::*;
use ::SyncConfig;
use super::{PeerInfo, PeerAsking};
@ -2421,7 +2421,7 @@ mod tests {
let ss = TestSnapshotService::new();
let io = TestIo::new(&mut client, &ss, &queue, None);
let result = ChainSync::return_receipts(&io, &UntrustedRlp::new(&[0xc0]), 0);
let result = ChainSync::return_receipts(&io, &Rlp::new(&[0xc0]), 0);
assert!(result.is_ok());
}
@ -2442,7 +2442,7 @@ mod tests {
let receipts_request = receipt_list.out();
// it returns rlp ONLY for hashes started with "f"
let result = ChainSync::return_receipts(&io, &UntrustedRlp::new(&receipts_request.clone()), 0);
let result = ChainSync::return_receipts(&io, &Rlp::new(&receipts_request.clone()), 0);
assert!(result.is_ok());
let rlp_result = result.unwrap();
@ -2484,41 +2484,41 @@ mod tests {
client.add_blocks(100, EachBlockWith::Nothing);
let blocks: Vec<_> = (0 .. 100)
.map(|i| (&client as &BlockChainClient).block(BlockId::Number(i as BlockNumber)).map(|b| b.into_inner()).unwrap()).collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| HeaderView::new(h).hash()).collect();
let headers: Vec<_> = blocks.iter().map(|b| Rlp::new(b).at(0).unwrap().as_raw().to_vec()).collect();
let hashes: Vec<_> = headers.iter().map(|h| view!(HeaderView, h).hash()).collect();
let queue = RwLock::new(VecDeque::new());
let ss = TestSnapshotService::new();
let io = TestIo::new(&mut client, &ss, &queue, None);
let unknown: H256 = H256::new();
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&unknown, 1, 0, false)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&unknown, 1, 0, false)), 0);
assert!(to_header_vec(result).is_empty());
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&unknown, 1, 0, true)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&unknown, 1, 0, true)), 0);
assert!(to_header_vec(result).is_empty());
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&hashes[2], 1, 0, true)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&hashes[2], 1, 0, true)), 0);
assert_eq!(to_header_vec(result), vec![headers[2].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&hashes[2], 1, 0, false)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&hashes[2], 1, 0, false)), 0);
assert_eq!(to_header_vec(result), vec![headers[2].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&hashes[50], 3, 5, false)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&hashes[50], 3, 5, false)), 0);
assert_eq!(to_header_vec(result), vec![headers[50].clone(), headers[56].clone(), headers[62].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_hash_req(&hashes[50], 3, 5, true)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_hash_req(&hashes[50], 3, 5, true)), 0);
assert_eq!(to_header_vec(result), vec![headers[50].clone(), headers[44].clone(), headers[38].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_num_req(2, 1, 0, true)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_num_req(2, 1, 0, true)), 0);
assert_eq!(to_header_vec(result), vec![headers[2].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_num_req(2, 1, 0, false)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_num_req(2, 1, 0, false)), 0);
assert_eq!(to_header_vec(result), vec![headers[2].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_num_req(50, 3, 5, false)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_num_req(50, 3, 5, false)), 0);
assert_eq!(to_header_vec(result), vec![headers[50].clone(), headers[56].clone(), headers[62].clone()]);
let result = ChainSync::return_block_headers(&io, &UntrustedRlp::new(&make_num_req(50, 3, 5, true)), 0);
let result = ChainSync::return_block_headers(&io, &Rlp::new(&make_num_req(50, 3, 5, true)), 0);
assert_eq!(to_header_vec(result), vec![headers[50].clone(), headers[44].clone(), headers[38].clone()]);
}
@ -2537,7 +2537,7 @@ mod tests {
let node_request = node_list.out();
// it returns rlp ONLY for hashes started with "f"
let result = ChainSync::return_node_data(&io, &UntrustedRlp::new(&node_request.clone()), 0);
let result = ChainSync::return_node_data(&io, &Rlp::new(&node_request.clone()), 0);
assert!(result.is_ok());
let rlp_result = result.unwrap();
@ -2546,7 +2546,7 @@ mod tests {
// the length of one rlp-encoded hashe
let rlp = rlp_result.unwrap().1.out();
let rlp = Rlp::new(&rlp);
assert_eq!(1, rlp.item_count());
assert_eq!(Ok(1), rlp.item_count());
io.sender = Some(2usize);
@ -2891,7 +2891,7 @@ mod tests {
return None;
}
let rlp = UntrustedRlp::new(&*p.data);
let rlp = Rlp::new(&*p.data);
let item_count = rlp.item_count().unwrap_or(0);
if item_count != 1 {
return None;
@ -2918,7 +2918,7 @@ mod tests {
let ss = TestSnapshotService::new();
let mut io = TestIo::new(&mut client, &ss, &queue, None);
let block = UntrustedRlp::new(&block_data);
let block = Rlp::new(&block_data);
let result = sync.on_peer_new_block(&mut io, 0, &block);
@ -2937,7 +2937,7 @@ mod tests {
let ss = TestSnapshotService::new();
let mut io = TestIo::new(&mut client, &ss, &queue, None);
let block = UntrustedRlp::new(&block_data);
let block = Rlp::new(&block_data);
let result = sync.on_peer_new_block(&mut io, 0, &block);
@ -2954,7 +2954,7 @@ mod tests {
let mut io = TestIo::new(&mut client, &ss, &queue, None);
let empty_data = vec![];
let block = UntrustedRlp::new(&empty_data);
let block = Rlp::new(&empty_data);
let result = sync.on_peer_new_block(&mut io, 0, &block);
@ -2971,7 +2971,7 @@ mod tests {
let mut io = TestIo::new(&mut client, &ss, &queue, None);
let hashes_data = get_dummy_hashes();
let hashes_rlp = UntrustedRlp::new(&hashes_data);
let hashes_rlp = Rlp::new(&hashes_data);
let result = sync.on_peer_new_hashes(&mut io, 0, &hashes_rlp);
@ -2988,7 +2988,7 @@ mod tests {
let mut io = TestIo::new(&mut client, &ss, &queue, None);
let empty_hashes_data = vec![];
let hashes_rlp = UntrustedRlp::new(&empty_hashes_data);
let hashes_rlp = Rlp::new(&empty_hashes_data);
let result = sync.on_peer_new_hashes(&mut io, 0, &hashes_rlp);
@ -3011,7 +3011,7 @@ mod tests {
sync.propagate_new_hashes(&chain_info, &mut io, &peers);
let data = &io.packets[0].data.clone();
let result = sync.on_peer_new_hashes(&mut io, 0, &UntrustedRlp::new(data));
let result = sync.on_peer_new_hashes(&mut io, 0, &Rlp::new(data));
assert!(result.is_ok());
}
@ -3031,7 +3031,7 @@ mod tests {
sync.propagate_blocks(&chain_info, &mut io, &[], &peers);
let data = &io.packets[0].data.clone();
let result = sync.on_peer_new_block(&mut io, 0, &UntrustedRlp::new(data));
let result = sync.on_peer_new_block(&mut io, 0, &Rlp::new(data));
assert!(result.is_ok());
}

View File

@ -26,6 +26,7 @@ extern crate ethcore_network_devp2p as devp2p;
extern crate ethcore_bytes as bytes;
extern crate ethcore_io as io;
extern crate ethcore_transaction as transaction;
#[macro_use]
extern crate ethcore;
extern crate ethereum_types;
extern crate env_logger;

View File

@ -24,7 +24,7 @@ use ethkey::{self, Signature, Secret, Public, recover, public_to_address};
use evm::Schedule;
use hash::keccak;
use heapsize::HeapSizeOf;
use rlp::{self, RlpStream, UntrustedRlp, DecoderError, Encodable};
use rlp::{self, RlpStream, Rlp, DecoderError, Encodable};
type Bytes = Vec<u8>;
type BlockNumber = u64;
@ -50,7 +50,7 @@ impl Default for Action {
}
impl rlp::Decodable for Action {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
if rlp.is_empty() {
Ok(Action::Create)
} else {
@ -291,7 +291,7 @@ impl Deref for UnverifiedTransaction {
}
impl rlp::Decodable for UnverifiedTransaction {
fn decode(d: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(d: &Rlp) -> Result<Self, DecoderError> {
if d.item_count()? != 9 {
return Err(DecoderError::RlpIncorrectListLen);
}

View File

@ -18,7 +18,7 @@
use ethereum_types::{H256, U256, Address, Bloom};
use heapsize::HeapSizeOf;
use rlp::{UntrustedRlp, RlpStream, Encodable, Decodable, DecoderError};
use rlp::{Rlp, RlpStream, Encodable, Decodable, DecoderError};
use {BlockNumber};
use log_entry::{LogEntry, LocalizedLogEntry};
@ -81,7 +81,7 @@ impl Encodable for Receipt {
}
impl Decodable for Receipt {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
if rlp.item_count()? == 3 {
Ok(Receipt {
outcome: TransactionOutcome::Unknown,

View File

@ -17,7 +17,7 @@
//! Snapshot manifest type definition
use ethereum_types::H256;
use rlp::{UntrustedRlp, RlpStream, DecoderError};
use rlp::{Rlp, RlpStream, DecoderError};
use bytes::Bytes;
/// Manifest data.
@ -53,7 +53,7 @@ impl ManifestData {
/// Try to restore manifest data from raw bytes, interpreted as RLP.
pub fn from_rlp(raw: &[u8]) -> Result<Self, DecoderError> {
let decoder = UntrustedRlp::new(raw);
let decoder = Rlp::new(raw);
let (start, version) = if decoder.item_count()? == 5 {
(0, 1)
} else {

View File

@ -1,6 +1,6 @@
//! EVM call types.
use rlp::{Encodable, Decodable, DecoderError, RlpStream, UntrustedRlp};
use rlp::{Encodable, Decodable, DecoderError, RlpStream, Rlp};
/// The type of the call-like instruction.
#[derive(Debug, PartialEq, Clone)]
@ -31,7 +31,7 @@ impl Encodable for CallType {
}
impl Decodable for CallType {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.as_val().and_then(|v| Ok(match v {
0u32 => CallType::None,
1 => CallType::Call,

View File

@ -26,7 +26,7 @@ use transaction::{
};
use ethcore::client::ClientIoMessage;
use io::IoHandler;
use rlp::UntrustedRlp;
use rlp::Rlp;
use kvdb::KeyValueDB;
extern crate ethcore;
@ -103,7 +103,7 @@ struct TransactionEntry {
impl TransactionEntry {
fn into_pending(self) -> Option<PendingTransaction> {
let tx: UnverifiedTransaction = match UntrustedRlp::new(&self.rlp_bytes).as_val() {
let tx: UnverifiedTransaction = match Rlp::new(&self.rlp_bytes).as_val() {
Err(e) => {
warn!(target: "local_store", "Invalid persistent transaction stored: {}", e);
return None

View File

@ -247,7 +247,7 @@ fn execute_import_light(cmd: ImportBlockchain) -> Result<(), String> {
let do_import = |bytes: Vec<u8>| {
while client.queue_info().is_full() { sleep(Duration::from_secs(1)); }
let header: ::ethcore::header::Header = ::rlp::UntrustedRlp::new(&bytes).val_at(0)
let header: ::ethcore::header::Header = ::rlp::Rlp::new(&bytes).val_at(0)
.map_err(|e| format!("Bad block: {}", e))?;
if client.best_block_header().number() >= header.number() { return Ok(()) }

View File

@ -44,6 +44,7 @@ extern crate jsonrpc_ipc_server as ipc;
extern crate jsonrpc_pubsub;
extern crate ethash;
#[cfg_attr(test, macro_use)]
extern crate ethcore;
extern crate ethcore_bytes as bytes;
extern crate ethcore_crypto as crypto;

View File

@ -20,7 +20,7 @@ use std::thread;
use std::time::{Instant, Duration, SystemTime, UNIX_EPOCH};
use std::sync::Arc;
use rlp::{self, UntrustedRlp};
use rlp::{self, Rlp};
use ethereum_types::{U256, H64, H160, H256, Address};
use parking_lot::Mutex;
@ -813,7 +813,7 @@ impl<C, SN: ?Sized, S: ?Sized, M, EM, T: StateInfo + 'static> Eth for EthClient<
}
fn send_raw_transaction(&self, raw: Bytes) -> Result<RpcH256> {
UntrustedRlp::new(&raw.into_vec()).as_val()
Rlp::new(&raw.into_vec()).as_val()
.map_err(errors::rlp)
.and_then(|tx| SignedTransaction::new(tx).map_err(errors::transaction))
.and_then(|signed_transaction| {

View File

@ -36,7 +36,7 @@ use sync::LightSync;
use hash::{KECCAK_NULL_RLP, KECCAK_EMPTY_LIST_RLP};
use ethereum_types::U256;
use parking_lot::{RwLock, Mutex};
use rlp::UntrustedRlp;
use rlp::Rlp;
use transaction::SignedTransaction;
use v1::impls::eth_filter::Filterable;
@ -373,7 +373,7 @@ impl<T: LightChainClient + 'static> Eth for EthClient<T> {
fn send_raw_transaction(&self, raw: Bytes) -> Result<RpcH256> {
let best_header = self.client.best_block_header().decode();
UntrustedRlp::new(&raw.into_vec()).as_val()
Rlp::new(&raw.into_vec()).as_val()
.map_err(errors::rlp)
.and_then(|tx| {
self.client.engine().verify_transaction_basic(&tx, &best_header)

View File

@ -18,7 +18,7 @@
use std::sync::Arc;
use rlp::UntrustedRlp;
use rlp::Rlp;
use ethcore_private_tx::Provider as PrivateTransactionManager;
use ethereum_types::Address;
@ -56,7 +56,7 @@ impl Private for PrivateClient {
type Metadata = Metadata;
fn send_transaction(&self, request: Bytes) -> Result<PrivateTransactionReceipt, Error> {
let signed_transaction = UntrustedRlp::new(&request.into_vec()).as_val()
let signed_transaction = Rlp::new(&request.into_vec()).as_val()
.map_err(errors::rlp)
.and_then(|tx| SignedTransaction::new(tx).map_err(errors::transaction))?;
let client = self.unwrap_manager()?;
@ -65,7 +65,7 @@ impl Private for PrivateClient {
}
fn compose_deployment_transaction(&self, block_number: BlockNumber, request: Bytes, validators: Vec<H160>, gas_price: U256) -> Result<PrivateTransactionReceiptAndTransaction, Error> {
let signed_transaction = UntrustedRlp::new(&request.into_vec()).as_val()
let signed_transaction = Rlp::new(&request.into_vec()).as_val()
.map_err(errors::rlp)
.and_then(|tx| SignedTransaction::new(tx).map_err(errors::transaction))?;
let client = self.unwrap_manager()?;

View File

@ -22,7 +22,7 @@ use ethcore::account_provider::AccountProvider;
use ethkey;
use parity_reactor::Remote;
use parking_lot::Mutex;
use rlp::UntrustedRlp;
use rlp::Rlp;
use transaction::{SignedTransaction, PendingTransaction};
use jsonrpc_core::{Result, BoxFuture, Error};
@ -127,7 +127,7 @@ impl<D: Dispatcher + 'static> SignerClient<D> {
fn verify_transaction<F>(bytes: Bytes, request: FilledTransactionRequest, process: F) -> Result<ConfirmationResponse> where
F: FnOnce(PendingTransaction) -> Result<ConfirmationResponse>,
{
let signed_transaction = UntrustedRlp::new(&bytes.0).as_val().map_err(errors::rlp)?;
let signed_transaction = Rlp::new(&bytes.0).as_val().map_err(errors::rlp)?;
let signed_transaction = SignedTransaction::new(signed_transaction).map_err(|e| errors::invalid_params("Invalid signature.", e))?;
let sender = signed_transaction.sender();

View File

@ -19,7 +19,7 @@
use std::sync::Arc;
use ethcore::client::{BlockChainClient, CallAnalytics, TransactionId, TraceId, StateClient, StateInfo, Call, BlockId};
use rlp::UntrustedRlp;
use rlp::Rlp;
use transaction::SignedTransaction;
use jsonrpc_core::Result;
@ -139,7 +139,7 @@ impl<C, S> Traces for TracesClient<C> where
fn raw_transaction(&self, raw_transaction: Bytes, flags: TraceOptions, block: Trailing<BlockNumber>) -> Result<TraceResults> {
let block = block.unwrap_or_default();
let tx = UntrustedRlp::new(&raw_transaction.into_vec()).as_val().map_err(|e| errors::invalid_params("Transaction is not valid RLP", e))?;
let tx = Rlp::new(&raw_transaction.into_vec()).as_val().map_err(|e| errors::invalid_params("Transaction is not valid RLP", e))?;
let signed = SignedTransaction::new(tx).map_err(errors::transaction)?;
let id = match block {

View File

@ -411,7 +411,7 @@ fn verify_transaction_counts(name: String, chain: BlockChain) {
let tester = EthTester::from_chain(&chain);
let mut id = 1;
for b in chain.blocks_rlp().iter().filter(|b| Block::is_good(b)).map(|b| BlockView::new(b)) {
for b in chain.blocks_rlp().iter().filter(|b| Block::is_good(b)).map(|b| view!(BlockView, b)) {
let count = b.transactions_count();
let hash = b.hash();

View File

@ -21,7 +21,7 @@ use std::collections::HashMap;
use std::collections::hash_map::Entry;
use error::{Result, BaseDataError};
use ethereum_types::H256;
use rlp::{UntrustedRlp, RlpStream, Encodable, DecoderError, Decodable, encode, decode};
use rlp::{Rlp, RlpStream, Encodable, DecoderError, Decodable, encode, decode};
use hashdb::*;
use memorydb::*;
use kvdb::{KeyValueDB, DBTransaction};
@ -64,7 +64,7 @@ impl Encodable for Payload {
}
impl Decodable for Payload {
fn decode(rlp: &UntrustedRlp) -> ::std::result::Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> ::std::result::Result<Self, DecoderError> {
let payload = Payload {
count: rlp.val_at(0)?,
value: DBValue::from_slice(rlp.at(1)?.data()?),

View File

@ -21,7 +21,7 @@ use std::collections::hash_map::Entry;
use std::sync::Arc;
use parking_lot::RwLock;
use heapsize::HeapSizeOf;
use rlp::{UntrustedRlp, RlpStream, encode, decode, DecoderError, Decodable, Encodable};
use rlp::{Rlp, RlpStream, encode, decode, DecoderError, Decodable, Encodable};
use hashdb::*;
use memorydb::*;
use super::{DB_PREFIX_LEN, LATEST_ERA_KEY};
@ -78,7 +78,7 @@ struct DatabaseValue {
}
impl Decodable for DatabaseValue {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let id = rlp.val_at(0)?;
let inserts = rlp.at(1)?.iter().map(|r| {
let k = r.val_at(0)?;

View File

@ -15,7 +15,7 @@
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
use ethereum_types::H256;
use rlp::{RlpStream, Encodable, UntrustedRlp, DecoderError};
use rlp::{RlpStream, Encodable, Rlp, DecoderError};
const PADDING : [u8; 10] = [ 0u8; 10 ];
@ -34,13 +34,13 @@ impl Encodable for DatabaseKey {
}
pub struct DatabaseValueView<'a> {
rlp: UntrustedRlp<'a>,
rlp: Rlp<'a>,
}
impl<'a> DatabaseValueView<'a> {
pub fn from_rlp(data: &'a [u8]) -> Self {
DatabaseValueView {
rlp: UntrustedRlp::new(data),
rlp: Rlp::new(data),
}
}

View File

@ -24,7 +24,7 @@ use mio::deprecated::{Handler, EventLoop, TryRead, TryWrite};
use mio::tcp::*;
use ethereum_types::{H128, H256, H512};
use ethcore_bytes::*;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use std::io::{self, Cursor, Read, Write};
use io::{IoContext, StreamToken};
use handshake::Handshake;
@ -391,7 +391,7 @@ impl EncryptedConnection {
self.decoder.decrypt(&mut RefReadBuffer::new(&header[0..16]), &mut RefWriteBuffer::new(&mut hdec), false).expect("Invalid length or padding");
let length = ((((hdec[0] as u32) << 8) + (hdec[1] as u32)) << 8) + (hdec[2] as u32);
let header_rlp = UntrustedRlp::new(&hdec[3..6]);
let header_rlp = Rlp::new(&hdec[3..6]);
let protocol_id = header_rlp.val_at::<u16>(0)?;
self.payload_len = length as usize;

View File

@ -25,7 +25,7 @@ use mio::deprecated::{Handler, EventLoop};
use mio::udp::*;
use hash::keccak;
use ethereum_types::{H256, H520};
use rlp::{UntrustedRlp, RlpStream, encode_list};
use rlp::{Rlp, RlpStream, encode_list};
use node_table::*;
use network::{Error, ErrorKind};
use io::{StreamToken, IoContext};
@ -259,7 +259,7 @@ impl Discovery {
fn send_packet(&mut self, packet_id: u8, address: &SocketAddr, payload: &[u8]) -> Result<(), Error> {
let mut rlp = RlpStream::new();
rlp.append_raw(&[packet_id], 1);
let source = UntrustedRlp::new(payload);
let source = Rlp::new(payload);
rlp.begin_list(source.item_count()? + 1);
for i in 0 .. source.item_count()? {
rlp.append_raw(source.at(i)?.as_raw(), 1);
@ -382,7 +382,7 @@ impl Discovery {
let node_id = recover(&signature.into(), &keccak(signed))?;
let packet_id = signed[0];
let rlp = UntrustedRlp::new(&signed[1..]);
let rlp = Rlp::new(&signed[1..]);
match packet_id {
PACKET_PING => self.on_ping(&rlp, &node_id, &from, &hash_signed),
PACKET_PONG => self.on_pong(&rlp, &node_id, &from),
@ -409,7 +409,7 @@ impl Discovery {
entry.endpoint.is_allowed(&self.ip_filter) && entry.id != self.id
}
fn on_ping(&mut self, rlp: &UntrustedRlp, node: &NodeId, from: &SocketAddr, echo_hash: &[u8]) -> Result<Option<TableUpdates>, Error> {
fn on_ping(&mut self, rlp: &Rlp, node: &NodeId, from: &SocketAddr, echo_hash: &[u8]) -> Result<Option<TableUpdates>, Error> {
trace!(target: "discovery", "Got Ping from {:?}", &from);
let source = NodeEndpoint::from_rlp(&rlp.at(1)?)?;
let dest = NodeEndpoint::from_rlp(&rlp.at(2)?)?;
@ -433,7 +433,7 @@ impl Discovery {
Ok(Some(TableUpdates { added: added_map, removed: HashSet::new() }))
}
fn on_pong(&mut self, rlp: &UntrustedRlp, node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
fn on_pong(&mut self, rlp: &Rlp, node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
trace!(target: "discovery", "Got Pong from {:?}", &from);
// TODO: validate pong packet in rlp.val_at(1)
let dest = NodeEndpoint::from_rlp(&rlp.at(0)?)?;
@ -448,7 +448,7 @@ impl Discovery {
Ok(None)
}
fn on_find_node(&mut self, rlp: &UntrustedRlp, _node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
fn on_find_node(&mut self, rlp: &Rlp, _node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
trace!(target: "discovery", "Got FindNode from {:?}", &from);
let target: NodeId = rlp.val_at(0)?;
let timestamp: u64 = rlp.val_at(1)?;
@ -481,7 +481,7 @@ impl Discovery {
packets.collect()
}
fn on_neighbours(&mut self, rlp: &UntrustedRlp, _node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
fn on_neighbours(&mut self, rlp: &Rlp, _node: &NodeId, from: &SocketAddr) -> Result<Option<TableUpdates>, Error> {
// TODO: validate packet
let mut added = HashMap::new();
trace!(target: "discovery", "Got {} Neighbours from {:?}", rlp.at(0)?.item_count()?, &from);
@ -725,7 +725,7 @@ mod tests {
discovery2.on_packet(&ping_data.payload, ep1.address.clone()).ok();
let pong_data = discovery2.send_queue.pop_front().unwrap();
let data = &pong_data.payload[(32 + 65)..];
let rlp = UntrustedRlp::new(&data[1..]);
let rlp = Rlp::new(&data[1..]);
assert_eq!(ping_data.payload[0..32], rlp.val_at::<Vec<u8>>(1).unwrap()[..])
}
}

View File

@ -20,7 +20,7 @@ use hash::write_keccak;
use mio::tcp::*;
use ethereum_types::{H256, H520};
use ethcore_bytes::Bytes;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use connection::{Connection};
use node_table::NodeId;
use io::{IoContext, StreamToken};
@ -205,7 +205,7 @@ impl Handshake {
trace!(target: "network", "Received EIP8 handshake auth from {:?}", self.connection.remote_addr_str());
self.auth_cipher.extend_from_slice(data);
let auth = ecies::decrypt(secret, &self.auth_cipher[0..2], &self.auth_cipher[2..])?;
let rlp = UntrustedRlp::new(&auth);
let rlp = Rlp::new(&auth);
let signature: H520 = rlp.val_at(0)?;
let remote_public: Public = rlp.val_at(1)?;
let remote_nonce: H256 = rlp.val_at(2)?;
@ -248,7 +248,7 @@ impl Handshake {
trace!(target: "network", "Received EIP8 handshake auth from {:?}", self.connection.remote_addr_str());
self.ack_cipher.extend_from_slice(data);
let ack = ecies::decrypt(secret, &self.ack_cipher[0..2], &self.ack_cipher[2..])?;
let rlp = UntrustedRlp::new(&ack);
let rlp = Rlp::new(&ack);
self.remote_ephemeral = rlp.val_at(0)?;
self.remote_nonce = rlp.val_at(1)?;
self.remote_version = rlp.val_at(2)?;

View File

@ -22,7 +22,7 @@ use std::path::PathBuf;
use std::str::FromStr;
use std::{fs, mem, slice};
use ethereum_types::H512;
use rlp::{UntrustedRlp, RlpStream, DecoderError};
use rlp::{Rlp, RlpStream, DecoderError};
use network::{Error, ErrorKind, AllowIP, IpFilter};
use discovery::{TableUpdates, NodeEntry};
use ip_utils::*;
@ -66,7 +66,7 @@ impl NodeEndpoint {
}
}
pub fn from_rlp(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
pub fn from_rlp(rlp: &Rlp) -> Result<Self, DecoderError> {
let tcp_port = rlp.val_at::<u16>(2)?;
let udp_port = rlp.val_at::<u16>(1)?;
let addr_bytes = rlp.at(0)?.data()?;

View File

@ -23,7 +23,7 @@ use mio::*;
use mio::deprecated::{Handler, EventLoop};
use mio::tcp::*;
use ethereum_types::H256;
use rlp::{UntrustedRlp, RlpStream, EMPTY_LIST_RLP};
use rlp::{Rlp, RlpStream, EMPTY_LIST_RLP};
use connection::{EncryptedConnection, Packet, Connection, MAX_PAYLOAD_SIZE};
use handshake::Handshake;
use io::{IoContext, StreamToken};
@ -349,12 +349,12 @@ impl Session {
};
match packet_id {
PACKET_HELLO => {
let rlp = UntrustedRlp::new(&data); //TODO: validate rlp expected size
let rlp = Rlp::new(&data); //TODO: validate rlp expected size
self.read_hello(io, &rlp, host)?;
Ok(SessionData::Ready)
},
PACKET_DISCONNECT => {
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let reason: u8 = rlp.val_at(0)?;
if self.had_hello {
debug!(target:"network", "Disconnected: {}: {:?}", self.token(), DisconnectReason::from_u8(reason));
@ -419,7 +419,7 @@ impl Session {
self.send(io, &rlp.drain())
}
fn read_hello<Message>(&mut self, io: &IoContext<Message>, rlp: &UntrustedRlp, host: &HostInfo) -> Result<(), Error>
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)?;

View File

@ -42,7 +42,7 @@ use ipnetwork::{IpNetwork, IpNetworkError};
use io::IoChannel;
use ethkey::Secret;
use ethereum_types::{H256, H512};
use rlp::{Decodable, DecoderError, UntrustedRlp};
use rlp::{Decodable, DecoderError, Rlp};
/// Protocol handler level packet id
pub type PacketId = u8;
@ -118,7 +118,7 @@ pub struct PeerCapabilityInfo {
}
impl Decodable for PeerCapabilityInfo {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let p: Vec<u8> = rlp.val_at(0)?;
if p.len() != 3 {
return Err(DecoderError::Custom("Invalid subprotocol string length. Should be 3"));

View File

@ -19,7 +19,7 @@ use elastic_array::ElasticArray36;
use nibbleslice::NibbleSlice;
use nibblevec::NibbleVec;
use bytes::*;
use rlp::{UntrustedRlp, RlpStream, Prototype, DecoderError};
use rlp::{Rlp, RlpStream, Prototype, DecoderError};
use hashdb::DBValue;
/// Partial node key type.
@ -41,7 +41,7 @@ pub enum Node<'a> {
impl<'a> Node<'a> {
/// Decode the `node_rlp` and return the Node.
pub fn decoded(node_rlp: &'a [u8]) -> Result<Self, DecoderError> {
let r = UntrustedRlp::new(node_rlp);
let r = Rlp::new(node_rlp);
match r.prototype()? {
// either leaf or extension - decode first item with NibbleSlice::???
// and use is_leaf return to figure out which.
@ -105,7 +105,7 @@ impl<'a> Node<'a> {
}
pub fn try_decode_hash(node_data: &[u8]) -> Option<H256> {
let r = UntrustedRlp::new(node_data);
let r = Rlp::new(node_data);
if r.is_data() && r.size() == 32 {
Some(r.as_val().expect("Hash is the correct size of 32 bytes; qed"))
} else {

View File

@ -24,7 +24,7 @@ use super::node::NodeKey;
use hashdb::HashDB;
use bytes::ToPretty;
use nibbleslice::NibbleSlice;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use hashdb::DBValue;
use std::collections::{HashSet, VecDeque};
@ -107,7 +107,7 @@ impl Node {
RlpNode::Branch(ref children_rlp, val) => {
let mut child = |i| {
let raw = children_rlp[i];
let child_rlp = UntrustedRlp::new(raw);
let child_rlp = Rlp::new(raw);
if !child_rlp.is_empty() {
Some(Self::inline_or_hash(raw, db, storage))
} else {

View File

@ -11,7 +11,7 @@ use byteorder::{ByteOrder, BigEndian};
use bigint::{U128, U256, H64, H128, H160, H256, H512, H520, Bloom};
use traits::{Encodable, Decodable};
use stream::RlpStream;
use {UntrustedRlp, DecoderError};
use {Rlp, DecoderError};
pub fn decode_usize(bytes: &[u8]) -> Result<usize, DecoderError> {
match bytes.len() {
@ -41,7 +41,7 @@ impl Encodable for bool {
}
impl Decodable for bool {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
match bytes.len() {
0 => Ok(false),
@ -65,7 +65,7 @@ impl Encodable for Vec<u8> {
}
impl Decodable for Vec<u8> {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
Ok(bytes.to_vec())
})
@ -87,7 +87,7 @@ impl<T> Encodable for Option<T> where T: Encodable {
}
impl<T> Decodable for Option<T> where T: Decodable {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
let items = rlp.item_count()?;
match items {
1 => rlp.val_at(0).map(Some),
@ -108,7 +108,7 @@ impl Encodable for u8 {
}
impl Decodable for u8 {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
match bytes.len() {
1 if bytes[0] != 0 => Ok(bytes[0]),
@ -136,7 +136,7 @@ macro_rules! impl_encodable_for_u {
macro_rules! impl_decodable_for_u {
($name: ident) => {
impl Decodable for $name {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
match bytes.len() {
0 | 1 => u8::decode(rlp).map(|v| v as $name),
@ -174,7 +174,7 @@ impl Encodable for usize {
}
impl Decodable for usize {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
u64::decode(rlp).map(|value| value as usize)
}
}
@ -192,7 +192,7 @@ macro_rules! impl_encodable_for_hash {
macro_rules! impl_decodable_for_hash {
($name: ident, $size: expr) => {
impl Decodable for $name {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| match bytes.len().cmp(&$size) {
cmp::Ordering::Less => Err(DecoderError::RlpIsTooShort),
cmp::Ordering::Greater => Err(DecoderError::RlpIsTooBig),
@ -239,7 +239,7 @@ macro_rules! impl_encodable_for_uint {
macro_rules! impl_decodable_for_uint {
($name: ident, $size: expr) => {
impl Decodable for $name {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
if !bytes.is_empty() && bytes[0] == 0 {
Err(DecoderError::RlpInvalidIndirection)
@ -273,7 +273,7 @@ impl Encodable for String {
}
impl Decodable for String {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
rlp.decoder().decode_value(|bytes| {
match str::from_utf8(bytes) {
Ok(s) => Ok(s.to_owned()),

View File

@ -27,12 +27,6 @@
//! * You encode a big set of data.
//!
//!### Use `Rlp` when:
//! * You are working on trusted data (not corrupted).
//! * You want to get view onto rlp-slice.
//! * You don't want to decode whole rlp at once.
//!
//!### Use `UntrustedRlp` when:
//! * You are working on untrusted data (~corrupted).
//! * You need to handle data corruption errors.
//! * You are working on input data.
//! * You want to get view onto rlp-slice.
@ -46,7 +40,6 @@ extern crate rustc_hex;
mod traits;
mod error;
mod rlpin;
mod untrusted_rlp;
mod stream;
mod impls;
@ -55,8 +48,7 @@ use elastic_array::ElasticArray1024;
pub use error::DecoderError;
pub use traits::{Decodable, Encodable};
pub use untrusted_rlp::{UntrustedRlp, UntrustedRlpIterator, PayloadInfo, Prototype};
pub use rlpin::{Rlp, RlpIterator};
pub use rlpin::{Rlp, RlpIterator, PayloadInfo, Prototype};
pub use stream::RlpStream;
/// The RLP encoded empty data (used to mean "null value").
@ -77,12 +69,12 @@ pub const EMPTY_LIST_RLP: [u8; 1] = [0xC0; 1];
/// ```
pub fn decode<T>(bytes: &[u8]) -> T where T: Decodable {
let rlp = Rlp::new(bytes);
rlp.as_val()
rlp.as_val().expect("trusted rlp should be valid")
}
pub fn decode_list<T>(bytes: &[u8]) -> Vec<T> where T: Decodable {
let rlp = Rlp::new(bytes);
rlp.as_list()
rlp.as_list().expect("trusted rlp should be valid")
}
/// Shortcut function to encode structure into rlp.

View File

@ -6,246 +6,295 @@
// option. This file may not be copied, modified, or distributed
// except according to those terms.
use std::cell::Cell;
use std::fmt;
use {UntrustedRlp, PayloadInfo, Prototype, Decodable};
use rustc_hex::ToHex;
use impls::decode_usize;
use {Decodable, DecoderError};
impl<'a> From<UntrustedRlp<'a>> for Rlp<'a> {
fn from(rlp: UntrustedRlp<'a>) -> Rlp<'a> {
Rlp { rlp: rlp }
/// rlp offset
#[derive(Copy, Clone, Debug)]
struct OffsetCache {
index: usize,
offset: usize,
}
impl OffsetCache {
fn new(index: usize, offset: usize) -> OffsetCache {
OffsetCache {
index: index,
offset: offset,
}
}
}
/// Data-oriented view onto trusted rlp-slice.
#[derive(Debug)]
/// RLP prototype
pub enum Prototype {
/// Empty
Null,
/// Value
Data(usize),
/// List
List(usize),
}
/// Stores basic information about item
pub struct PayloadInfo {
/// Header length in bytes
pub header_len: usize,
/// Value length in bytes
pub value_len: usize,
}
fn calculate_payload_info(header_bytes: &[u8], len_of_len: usize) -> Result<PayloadInfo, DecoderError> {
let header_len = 1 + len_of_len;
match header_bytes.get(1) {
Some(&0) => return Err(DecoderError::RlpDataLenWithZeroPrefix),
None => return Err(DecoderError::RlpIsTooShort),
_ => (),
}
if header_bytes.len() < header_len { return Err(DecoderError::RlpIsTooShort); }
let value_len = decode_usize(&header_bytes[1..header_len])?;
Ok(PayloadInfo::new(header_len, value_len))
}
impl PayloadInfo {
fn new(header_len: usize, value_len: usize) -> PayloadInfo {
PayloadInfo {
header_len: header_len,
value_len: value_len,
}
}
/// Total size of the RLP.
pub fn total(&self) -> usize { self.header_len + self.value_len }
/// Create a new object from the given bytes RLP. The bytes
pub fn from(header_bytes: &[u8]) -> Result<PayloadInfo, DecoderError> {
match header_bytes.first().cloned() {
None => Err(DecoderError::RlpIsTooShort),
Some(0...0x7f) => Ok(PayloadInfo::new(0, 1)),
Some(l @ 0x80...0xb7) => Ok(PayloadInfo::new(1, l as usize - 0x80)),
Some(l @ 0xb8...0xbf) => {
let len_of_len = l as usize - 0xb7;
calculate_payload_info(header_bytes, len_of_len)
}
Some(l @ 0xc0...0xf7) => Ok(PayloadInfo::new(1, l as usize - 0xc0)),
Some(l @ 0xf8...0xff) => {
let len_of_len = l as usize - 0xf7;
calculate_payload_info(header_bytes, len_of_len)
},
// we cant reach this place, but rust requires _ to be implemented
_ => { unreachable!(); }
}
}
}
/// Data-oriented view onto rlp-slice.
///
/// Unlikely to `UntrustedRlp` doesn't bother you with error
/// handling. It assumes that you know what you are doing.
/// This is an immutable structure. No operations change it.
///
/// Should be used in places where, error handling is required,
/// eg. on input
#[derive(Debug)]
pub struct Rlp<'a> {
rlp: UntrustedRlp<'a>
bytes: &'a [u8],
offset_cache: Cell<OffsetCache>,
count_cache: Cell<Option<usize>>,
}
impl<'a> Clone for Rlp<'a> {
fn clone(&self) -> Rlp<'a> {
Rlp {
bytes: self.bytes,
offset_cache: self.offset_cache.clone(),
count_cache: self.count_cache.clone(),
}
}
}
impl<'a> fmt::Display for Rlp<'a> {
fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> {
write!(f, "{}", self.rlp)
match self.prototype() {
Ok(Prototype::Null) => write!(f, "null"),
Ok(Prototype::Data(_)) => write!(f, "\"0x{}\"", self.data().unwrap().to_hex()),
Ok(Prototype::List(len)) => {
write!(f, "[")?;
for i in 0..len-1 {
write!(f, "{}, ", self.at(i).unwrap())?;
}
write!(f, "{}", self.at(len - 1).unwrap())?;
write!(f, "]")
},
Err(err) => write!(f, "{:?}", err)
}
}
}
impl<'a, 'view> Rlp<'a> where 'a: 'view {
/// Create a new instance of `Rlp`
pub fn new(bytes: &'a [u8]) -> Rlp<'a> {
Rlp {
rlp: UntrustedRlp::new(bytes)
bytes: bytes,
offset_cache: Cell::new(OffsetCache::new(usize::max_value(), 0)),
count_cache: Cell::new(None)
}
}
/// The raw data of the RLP as slice.
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// let dog = rlp.at(1).as_raw();
/// assert_eq!(dog, &[0x83, b'd', b'o', b'g']);
/// }
/// ```
pub fn as_raw(&'view self) -> &'a [u8] {
self.rlp.as_raw()
self.bytes
}
/// Get the prototype of the RLP.
pub fn prototype(&self) -> Prototype {
self.rlp.prototype().unwrap()
pub fn prototype(&self) -> Result<Prototype, DecoderError> {
// optimize? && return appropriate errors
if self.is_data() {
Ok(Prototype::Data(self.size()))
} else if self.is_list() {
self.item_count().map(Prototype::List)
} else {
Ok(Prototype::Null)
}
}
/// Get payload info.
pub fn payload_info(&self) -> PayloadInfo {
self.rlp.payload_info().unwrap()
pub fn payload_info(&self) -> Result<PayloadInfo, DecoderError> {
BasicDecoder::payload_info(self.bytes)
}
/// Get underlieing data.
pub fn data(&'view self) -> &'a [u8] {
self.rlp.data().unwrap()
pub fn data(&'view self) -> Result<&'a [u8], DecoderError> {
let pi = BasicDecoder::payload_info(self.bytes)?;
Ok(&self.bytes[pi.header_len..(pi.header_len + pi.value_len)])
}
/// Returns number of RLP items.
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// assert_eq!(rlp.item_count(), 2);
/// let view = rlp.at(1);
/// assert_eq!(view.item_count(), 0);
/// }
/// ```
pub fn item_count(&self) -> usize {
self.rlp.item_count().unwrap_or(0)
pub fn item_count(&self) -> Result<usize, DecoderError> {
match self.is_list() {
true => match self.count_cache.get() {
Some(c) => Ok(c),
None => {
let c = self.iter().count();
self.count_cache.set(Some(c));
Ok(c)
}
},
false => Err(DecoderError::RlpExpectedToBeList),
}
}
/// Returns the number of bytes in the data, or zero if it isn't data.
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// assert_eq!(rlp.size(), 0);
/// let view = rlp.at(1);
/// assert_eq!(view.size(), 3);
/// }
/// ```
pub fn size(&self) -> usize {
self.rlp.size()
match self.is_data() {
// TODO: No panic on malformed data, but ideally would Err on no PayloadInfo.
true => BasicDecoder::payload_info(self.bytes).map(|b| b.value_len).unwrap_or(0),
false => 0
}
}
/// Get view onto RLP-slice at index.
///
/// Caches offset to given index, so access to successive
/// slices is faster.
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// let dog: String = rlp.at(1).as_val();
/// assert_eq!(dog, "dog".to_string());
/// }
/// ```
pub fn at(&'view self, index: usize) -> Rlp<'a> {
From::from(self.rlp.at(index).unwrap())
pub fn at(&'view self, index: usize) -> Result<Rlp<'a>, DecoderError> {
if !self.is_list() {
return Err(DecoderError::RlpExpectedToBeList);
}
// move to cached position if its index is less or equal to
// current search index, otherwise move to beginning of list
let c = self.offset_cache.get();
let (mut bytes, to_skip) = match c.index <= index {
true => (Rlp::consume(self.bytes, c.offset)?, index - c.index),
false => (self.consume_list_payload()?, index),
};
// skip up to x items
bytes = Rlp::consume_items(bytes, to_skip)?;
// update the cache
self.offset_cache.set(OffsetCache::new(index, self.bytes.len() - bytes.len()));
// construct new rlp
let found = BasicDecoder::payload_info(bytes)?;
Ok(Rlp::new(&bytes[0..found.header_len + found.value_len]))
}
/// No value
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![];
/// let rlp = Rlp::new(&data);
/// assert!(rlp.is_null());
/// }
/// ```
pub fn is_null(&self) -> bool {
self.rlp.is_null()
self.bytes.len() == 0
}
/// Contains a zero-length string or zero-length list.
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc0];
/// let rlp = Rlp::new(&data);
/// assert!(rlp.is_empty());
/// }
/// ```
pub fn is_empty(&self) -> bool {
self.rlp.is_empty()
!self.is_null() && (self.bytes[0] == 0xc0 || self.bytes[0] == 0x80)
}
/// List value
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// assert!(rlp.is_list());
/// }
/// ```
pub fn is_list(&self) -> bool {
self.rlp.is_list()
!self.is_null() && self.bytes[0] >= 0xc0
}
/// String value
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// assert!(rlp.at(1).is_data());
/// }
/// ```
pub fn is_data(&self) -> bool {
self.rlp.is_data()
!self.is_null() && self.bytes[0] < 0xc0
}
/// Int value
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc1, 0x10];
/// let rlp = Rlp::new(&data);
/// assert_eq!(rlp.is_int(), false);
/// assert_eq!(rlp.at(0).is_int(), true);
/// }
/// ```
pub fn is_int(&self) -> bool {
self.rlp.is_int()
if self.is_null() {
return false;
}
match self.bytes[0] {
0...0x80 => true,
0x81...0xb7 => self.bytes[1] != 0,
b @ 0xb8...0xbf => self.bytes[1 + b as usize - 0xb7] != 0,
_ => false
}
}
/// Get iterator over rlp-slices
///
/// ```rust
/// extern crate rlp;
/// use rlp::*;
///
/// fn main () {
/// let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
/// let rlp = Rlp::new(&data);
/// let strings: Vec<String> = rlp.iter().map(| i | i.as_val()).collect();
/// }
/// ```
pub fn iter(&'view self) -> RlpIterator<'a, 'view> {
self.into_iter()
}
/// Decode data into an object
pub fn as_val<T>(&self) -> T where T: Decodable {
self.rlp.as_val().expect("Unexpected rlp error")
pub fn as_val<T>(&self) -> Result<T, DecoderError> where T: Decodable {
T::decode(self)
}
pub fn as_list<T>(&self) -> Vec<T> where T: Decodable {
pub fn as_list<T>(&self) -> Result<Vec<T>, DecoderError> where T: Decodable {
self.iter().map(|rlp| rlp.as_val()).collect()
}
/// Decode data at given list index into an object
pub fn val_at<T>(&self, index: usize) -> T where T: Decodable {
self.at(index).as_val()
pub fn val_at<T>(&self, index: usize) -> Result<T, DecoderError> where T: Decodable {
self.at(index)?.as_val()
}
pub fn list_at<T>(&self, index: usize) -> Vec<T> where T: Decodable {
self.at(index).as_list()
pub fn list_at<T>(&self, index: usize) -> Result<Vec<T>, DecoderError> where T: Decodable {
self.at(index)?.as_list()
}
pub fn decoder(&self) -> BasicDecoder {
BasicDecoder::new(self.clone())
}
/// consumes first found prefix
fn consume_list_payload(&self) -> Result<&'a [u8], DecoderError> {
let item = BasicDecoder::payload_info(self.bytes)?;
let bytes = Rlp::consume(self.bytes, item.header_len)?;
Ok(bytes)
}
/// consumes fixed number of items
fn consume_items(bytes: &'a [u8], items: usize) -> Result<&'a [u8], DecoderError> {
let mut result = bytes;
for _ in 0..items {
let i = BasicDecoder::payload_info(result)?;
result = Rlp::consume(result, i.header_len + i.value_len)?;
}
Ok(result)
}
/// consumes slice prefix of length `len`
fn consume(bytes: &'a [u8], len: usize) -> Result<&'a [u8], DecoderError> {
match bytes.len() >= len {
true => Ok(&bytes[len..]),
false => Err(DecoderError::RlpIsTooShort),
}
}
}
/// Iterator over trusted rlp-slice list elements.
/// Iterator over rlp-slice list elements.
pub struct RlpIterator<'a, 'view> where 'a: 'view {
rlp: &'view Rlp<'a>,
index: usize
index: usize,
}
impl<'a, 'view> IntoIterator for &'view Rlp<'a> where 'a: 'view {
@ -265,19 +314,93 @@ impl<'a, 'view> Iterator for RlpIterator<'a, 'view> {
fn next(&mut self) -> Option<Rlp<'a>> {
let index = self.index;
let result = self.rlp.rlp.at(index).ok().map(From::from);
let result = self.rlp.at(index).ok();
self.index += 1;
result
}
}
#[test]
fn break_it() {
use rustc_hex::FromHex;
use bigint::U256;
let h: Vec<u8> = FromHex::from_hex("f84d0589010efbef67941f79b2a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470").unwrap();
let r: Rlp = Rlp::new(&h);
let u: U256 = r.val_at(1);
assert_eq!(format!("{}", u), "19526463837540678066");
pub struct BasicDecoder<'a> {
rlp: Rlp<'a>
}
impl<'a> BasicDecoder<'a> {
pub fn new(rlp: Rlp<'a>) -> BasicDecoder<'a> {
BasicDecoder {
rlp: rlp
}
}
/// Return first item info.
fn payload_info(bytes: &[u8]) -> Result<PayloadInfo, DecoderError> {
let item = PayloadInfo::from(bytes)?;
match item.header_len.checked_add(item.value_len) {
Some(x) if x <= bytes.len() => Ok(item),
_ => Err(DecoderError::RlpIsTooShort),
}
}
pub fn decode_value<T, F>(&self, f: F) -> Result<T, DecoderError>
where F: Fn(&[u8]) -> Result<T, DecoderError> {
let bytes = self.rlp.as_raw();
match bytes.first().cloned() {
// RLP is too short.
None => Err(DecoderError::RlpIsTooShort),
// Single byte value.
Some(l @ 0...0x7f) => Ok(f(&[l])?),
// 0-55 bytes
Some(l @ 0x80...0xb7) => {
let last_index_of = 1 + l as usize - 0x80;
if bytes.len() < last_index_of {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
let d = &bytes[1..last_index_of];
if l == 0x81 && d[0] < 0x80 {
return Err(DecoderError::RlpInvalidIndirection);
}
Ok(f(d)?)
},
// Longer than 55 bytes.
Some(l @ 0xb8...0xbf) => {
let len_of_len = l as usize - 0xb7;
let begin_of_value = 1 as usize + len_of_len;
if bytes.len() < begin_of_value {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
let len = decode_usize(&bytes[1..begin_of_value])?;
let last_index_of_value = begin_of_value.checked_add(len)
.ok_or(DecoderError::RlpInvalidLength)?;
if bytes.len() < last_index_of_value {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
Ok(f(&bytes[begin_of_value..last_index_of_value])?)
}
// We are reading value, not a list!
_ => Err(DecoderError::RlpExpectedToBeData)
}
}
}
#[cfg(test)]
mod tests {
use {Rlp, DecoderError};
#[test]
fn test_rlp_display() {
use rustc_hex::FromHex;
let data = "f84d0589010efbef67941f79b2a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470".from_hex().unwrap();
let rlp = Rlp::new(&data);
assert_eq!(format!("{}", rlp), "[\"0x05\", \"0x010efbef67941f79b2\", \"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421\", \"0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470\"]");
}
#[test]
fn length_overflow() {
let bs = [0xbf, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xe5];
let rlp = Rlp::new(&bs);
let res: Result<u8, DecoderError> = rlp.as_val();
assert_eq!(Err(DecoderError::RlpInvalidLength), res);
}
}

View File

@ -8,12 +8,12 @@
//! Common RLP traits
use elastic_array::ElasticArray1024;
use {DecoderError, UntrustedRlp, RlpStream};
use {DecoderError, Rlp, RlpStream};
/// RLP decodable trait
pub trait Decodable: Sized {
/// Decode a value from RLP bytes
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError>;
fn decode(rlp: &Rlp) -> Result<Self, DecoderError>;
}
/// Structure encodable to RLP

View File

@ -1,406 +0,0 @@
// Copyright 2015-2017 Parity Technologies
//
// Licensed under the Apache License, Version 2.0 <LICENSE-APACHE or
// http://www.apache.org/licenses/LICENSE-2.0> or the MIT license
// <LICENSE-MIT or http://opensource.org/licenses/MIT>, at your
// option. This file may not be copied, modified, or distributed
// except according to those terms.
use std::cell::Cell;
use std::fmt;
use rustc_hex::ToHex;
use impls::decode_usize;
use {Decodable, DecoderError};
/// rlp offset
#[derive(Copy, Clone, Debug)]
struct OffsetCache {
index: usize,
offset: usize,
}
impl OffsetCache {
fn new(index: usize, offset: usize) -> OffsetCache {
OffsetCache {
index: index,
offset: offset,
}
}
}
#[derive(Debug)]
/// RLP prototype
pub enum Prototype {
/// Empty
Null,
/// Value
Data(usize),
/// List
List(usize),
}
/// Stores basic information about item
pub struct PayloadInfo {
/// Header length in bytes
pub header_len: usize,
/// Value length in bytes
pub value_len: usize,
}
fn calculate_payload_info(header_bytes: &[u8], len_of_len: usize) -> Result<PayloadInfo, DecoderError> {
let header_len = 1 + len_of_len;
match header_bytes.get(1) {
Some(&0) => return Err(DecoderError::RlpDataLenWithZeroPrefix),
None => return Err(DecoderError::RlpIsTooShort),
_ => (),
}
if header_bytes.len() < header_len { return Err(DecoderError::RlpIsTooShort); }
let value_len = decode_usize(&header_bytes[1..header_len])?;
Ok(PayloadInfo::new(header_len, value_len))
}
impl PayloadInfo {
fn new(header_len: usize, value_len: usize) -> PayloadInfo {
PayloadInfo {
header_len: header_len,
value_len: value_len,
}
}
/// Total size of the RLP.
pub fn total(&self) -> usize { self.header_len + self.value_len }
/// Create a new object from the given bytes RLP. The bytes
pub fn from(header_bytes: &[u8]) -> Result<PayloadInfo, DecoderError> {
match header_bytes.first().cloned() {
None => Err(DecoderError::RlpIsTooShort),
Some(0...0x7f) => Ok(PayloadInfo::new(0, 1)),
Some(l @ 0x80...0xb7) => Ok(PayloadInfo::new(1, l as usize - 0x80)),
Some(l @ 0xb8...0xbf) => {
let len_of_len = l as usize - 0xb7;
calculate_payload_info(header_bytes, len_of_len)
}
Some(l @ 0xc0...0xf7) => Ok(PayloadInfo::new(1, l as usize - 0xc0)),
Some(l @ 0xf8...0xff) => {
let len_of_len = l as usize - 0xf7;
calculate_payload_info(header_bytes, len_of_len)
},
// we cant reach this place, but rust requires _ to be implemented
_ => { unreachable!(); }
}
}
}
/// Data-oriented view onto rlp-slice.
///
/// This is an immutable structure. No operations change it.
///
/// Should be used in places where, error handling is required,
/// eg. on input
#[derive(Debug)]
pub struct UntrustedRlp<'a> {
bytes: &'a [u8],
offset_cache: Cell<OffsetCache>,
count_cache: Cell<Option<usize>>,
}
impl<'a> Clone for UntrustedRlp<'a> {
fn clone(&self) -> UntrustedRlp<'a> {
UntrustedRlp {
bytes: self.bytes,
offset_cache: self.offset_cache.clone(),
count_cache: self.count_cache.clone(),
}
}
}
impl<'a> fmt::Display for UntrustedRlp<'a> {
fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> {
match self.prototype() {
Ok(Prototype::Null) => write!(f, "null"),
Ok(Prototype::Data(_)) => write!(f, "\"0x{}\"", self.data().unwrap().to_hex()),
Ok(Prototype::List(len)) => {
write!(f, "[")?;
for i in 0..len-1 {
write!(f, "{}, ", self.at(i).unwrap())?;
}
write!(f, "{}", self.at(len - 1).unwrap())?;
write!(f, "]")
},
Err(err) => write!(f, "{:?}", err)
}
}
}
impl<'a, 'view> UntrustedRlp<'a> where 'a: 'view {
pub fn new(bytes: &'a [u8]) -> UntrustedRlp<'a> {
UntrustedRlp {
bytes: bytes,
offset_cache: Cell::new(OffsetCache::new(usize::max_value(), 0)),
count_cache: Cell::new(None)
}
}
pub fn as_raw(&'view self) -> &'a [u8] {
self.bytes
}
pub fn prototype(&self) -> Result<Prototype, DecoderError> {
// optimize? && return appropriate errors
if self.is_data() {
Ok(Prototype::Data(self.size()))
} else if self.is_list() {
self.item_count().map(Prototype::List)
} else {
Ok(Prototype::Null)
}
}
pub fn payload_info(&self) -> Result<PayloadInfo, DecoderError> {
BasicDecoder::payload_info(self.bytes)
}
pub fn data(&'view self) -> Result<&'a [u8], DecoderError> {
let pi = BasicDecoder::payload_info(self.bytes)?;
Ok(&self.bytes[pi.header_len..(pi.header_len + pi.value_len)])
}
pub fn item_count(&self) -> Result<usize, DecoderError> {
match self.is_list() {
true => match self.count_cache.get() {
Some(c) => Ok(c),
None => {
let c = self.iter().count();
self.count_cache.set(Some(c));
Ok(c)
}
},
false => Err(DecoderError::RlpExpectedToBeList),
}
}
pub fn size(&self) -> usize {
match self.is_data() {
// TODO: No panic on malformed data, but ideally would Err on no PayloadInfo.
true => BasicDecoder::payload_info(self.bytes).map(|b| b.value_len).unwrap_or(0),
false => 0
}
}
pub fn at(&'view self, index: usize) -> Result<UntrustedRlp<'a>, DecoderError> {
if !self.is_list() {
return Err(DecoderError::RlpExpectedToBeList);
}
// move to cached position if its index is less or equal to
// current search index, otherwise move to beginning of list
let c = self.offset_cache.get();
let (mut bytes, to_skip) = match c.index <= index {
true => (UntrustedRlp::consume(self.bytes, c.offset)?, index - c.index),
false => (self.consume_list_payload()?, index),
};
// skip up to x items
bytes = UntrustedRlp::consume_items(bytes, to_skip)?;
// update the cache
self.offset_cache.set(OffsetCache::new(index, self.bytes.len() - bytes.len()));
// construct new rlp
let found = BasicDecoder::payload_info(bytes)?;
Ok(UntrustedRlp::new(&bytes[0..found.header_len + found.value_len]))
}
pub fn is_null(&self) -> bool {
self.bytes.len() == 0
}
pub fn is_empty(&self) -> bool {
!self.is_null() && (self.bytes[0] == 0xc0 || self.bytes[0] == 0x80)
}
pub fn is_list(&self) -> bool {
!self.is_null() && self.bytes[0] >= 0xc0
}
pub fn is_data(&self) -> bool {
!self.is_null() && self.bytes[0] < 0xc0
}
pub fn is_int(&self) -> bool {
if self.is_null() {
return false;
}
match self.bytes[0] {
0...0x80 => true,
0x81...0xb7 => self.bytes[1] != 0,
b @ 0xb8...0xbf => self.bytes[1 + b as usize - 0xb7] != 0,
_ => false
}
}
pub fn iter(&'view self) -> UntrustedRlpIterator<'a, 'view> {
self.into_iter()
}
pub fn as_val<T>(&self) -> Result<T, DecoderError> where T: Decodable {
T::decode(self)
}
pub fn as_list<T>(&self) -> Result<Vec<T>, DecoderError> where T: Decodable {
self.iter().map(|rlp| rlp.as_val()).collect()
}
pub fn val_at<T>(&self, index: usize) -> Result<T, DecoderError> where T: Decodable {
self.at(index)?.as_val()
}
pub fn list_at<T>(&self, index: usize) -> Result<Vec<T>, DecoderError> where T: Decodable {
self.at(index)?.as_list()
}
pub fn decoder(&self) -> BasicDecoder {
BasicDecoder::new(self.clone())
}
/// consumes first found prefix
fn consume_list_payload(&self) -> Result<&'a [u8], DecoderError> {
let item = BasicDecoder::payload_info(self.bytes)?;
let bytes = UntrustedRlp::consume(self.bytes, item.header_len)?;
Ok(bytes)
}
/// consumes fixed number of items
fn consume_items(bytes: &'a [u8], items: usize) -> Result<&'a [u8], DecoderError> {
let mut result = bytes;
for _ in 0..items {
let i = BasicDecoder::payload_info(result)?;
result = UntrustedRlp::consume(result, i.header_len + i.value_len)?;
}
Ok(result)
}
/// consumes slice prefix of length `len`
fn consume(bytes: &'a [u8], len: usize) -> Result<&'a [u8], DecoderError> {
match bytes.len() >= len {
true => Ok(&bytes[len..]),
false => Err(DecoderError::RlpIsTooShort),
}
}
}
/// Iterator over rlp-slice list elements.
pub struct UntrustedRlpIterator<'a, 'view> where 'a: 'view {
rlp: &'view UntrustedRlp<'a>,
index: usize,
}
impl<'a, 'view> IntoIterator for &'view UntrustedRlp<'a> where 'a: 'view {
type Item = UntrustedRlp<'a>;
type IntoIter = UntrustedRlpIterator<'a, 'view>;
fn into_iter(self) -> Self::IntoIter {
UntrustedRlpIterator {
rlp: self,
index: 0,
}
}
}
impl<'a, 'view> Iterator for UntrustedRlpIterator<'a, 'view> {
type Item = UntrustedRlp<'a>;
fn next(&mut self) -> Option<UntrustedRlp<'a>> {
let index = self.index;
let result = self.rlp.at(index).ok();
self.index += 1;
result
}
}
pub struct BasicDecoder<'a> {
rlp: UntrustedRlp<'a>
}
impl<'a> BasicDecoder<'a> {
pub fn new(rlp: UntrustedRlp<'a>) -> BasicDecoder<'a> {
BasicDecoder {
rlp: rlp
}
}
/// Return first item info.
fn payload_info(bytes: &[u8]) -> Result<PayloadInfo, DecoderError> {
let item = PayloadInfo::from(bytes)?;
match item.header_len.checked_add(item.value_len) {
Some(x) if x <= bytes.len() => Ok(item),
_ => Err(DecoderError::RlpIsTooShort),
}
}
pub fn decode_value<T, F>(&self, f: F) -> Result<T, DecoderError>
where F: Fn(&[u8]) -> Result<T, DecoderError> {
let bytes = self.rlp.as_raw();
match bytes.first().cloned() {
// RLP is too short.
None => Err(DecoderError::RlpIsTooShort),
// Single byte value.
Some(l @ 0...0x7f) => Ok(f(&[l])?),
// 0-55 bytes
Some(l @ 0x80...0xb7) => {
let last_index_of = 1 + l as usize - 0x80;
if bytes.len() < last_index_of {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
let d = &bytes[1..last_index_of];
if l == 0x81 && d[0] < 0x80 {
return Err(DecoderError::RlpInvalidIndirection);
}
Ok(f(d)?)
},
// Longer than 55 bytes.
Some(l @ 0xb8...0xbf) => {
let len_of_len = l as usize - 0xb7;
let begin_of_value = 1 as usize + len_of_len;
if bytes.len() < begin_of_value {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
let len = decode_usize(&bytes[1..begin_of_value])?;
let last_index_of_value = begin_of_value.checked_add(len)
.ok_or(DecoderError::RlpInvalidLength)?;
if bytes.len() < last_index_of_value {
return Err(DecoderError::RlpInconsistentLengthAndData);
}
Ok(f(&bytes[begin_of_value..last_index_of_value])?)
}
// We are reading value, not a list!
_ => Err(DecoderError::RlpExpectedToBeData)
}
}
}
#[cfg(test)]
mod tests {
use {UntrustedRlp, DecoderError};
#[test]
fn test_rlp_display() {
use rustc_hex::FromHex;
let data = "f84d0589010efbef67941f79b2a056e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421a0c5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470".from_hex().unwrap();
let rlp = UntrustedRlp::new(&data);
assert_eq!(format!("{}", rlp), "[\"0x05\", \"0x010efbef67941f79b2\", \"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421\", \"0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470\"]");
}
#[test]
fn length_overflow() {
let bs = [0xbf, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xe5];
let rlp = UntrustedRlp::new(&bs);
let res: Result<u8, DecoderError> = rlp.as_val();
assert_eq!(Err(DecoderError::RlpInvalidLength), res);
}
}

View File

@ -11,13 +11,13 @@ extern crate rlp;
use std::{fmt, cmp};
use bigint::{U256, H160};
use rlp::{Encodable, Decodable, UntrustedRlp, RlpStream, DecoderError};
use rlp::{Encodable, Decodable, Rlp, RlpStream, DecoderError};
#[test]
fn rlp_at() {
let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
{
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
assert!(rlp.is_list());
let animals: Vec<String> = rlp.as_list().unwrap();
assert_eq!(animals, vec!["cat".to_owned(), "dog".to_owned()]);
@ -43,7 +43,7 @@ fn rlp_at() {
fn rlp_at_err() {
let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o'];
{
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
assert!(rlp.is_list());
let cat_err = rlp.at(0).unwrap_err();
@ -58,7 +58,7 @@ fn rlp_at_err() {
fn rlp_iter() {
let data = vec![0xc8, 0x83, b'c', b'a', b't', 0x83, b'd', b'o', b'g'];
{
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let mut iter = rlp.iter();
let cat = iter.next().unwrap();
@ -337,7 +337,7 @@ fn decode_untrusted_vector_str() {
fn test_rlp_data_length_check()
{
let data = vec![0x84, b'c', b'a', b't'];
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let as_val: Result<String, DecoderError> = rlp.as_val();
assert_eq!(Err(DecoderError::RlpInconsistentLengthAndData), as_val);
@ -351,7 +351,7 @@ fn test_rlp_long_data_length_check()
data.push(b'c');
}
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let as_val: Result<String, DecoderError> = rlp.as_val();
assert_eq!(Err(DecoderError::RlpInconsistentLengthAndData), as_val);
@ -365,7 +365,7 @@ fn test_the_exact_long_string()
data.push(b'c');
}
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let as_val: Result<String, DecoderError> = rlp.as_val();
assert!(as_val.is_ok());
@ -379,7 +379,7 @@ fn test_rlp_2bytes_data_length_check()
data.push(b'c');
}
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let as_val: Result<String, DecoderError> = rlp.as_val();
assert_eq!(Err(DecoderError::RlpInconsistentLengthAndData), as_val);
@ -396,7 +396,7 @@ fn test_rlp_nested_empty_list_encode() {
#[test]
fn test_rlp_list_length_overflow() {
let data: Vec<u8> = vec![0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x00, 0x00, 0x00];
let rlp = UntrustedRlp::new(&data);
let rlp = Rlp::new(&data);
let as_val: Result<String, DecoderError> = rlp.val_at(0);
assert_eq!(Err(DecoderError::RlpIsTooShort), as_val);
}

View File

@ -16,7 +16,7 @@ mod common;
use std::cmp;
use std::collections::HashMap;
use elastic_array::ElasticArray1024;
use rlp::{UntrustedRlp, RlpStream};
use rlp::{Rlp, RlpStream};
use common::{SNAPSHOT_SWAPPER, BLOCKS_SWAPPER};
pub fn snapshot_swapper() -> &'static Swapper<'static> {
@ -41,7 +41,7 @@ pub trait Decompressor {
/// Call this function to compress rlp.
pub fn compress(c: &[u8], swapper: &Compressor) -> ElasticArray1024<u8> {
let rlp = UntrustedRlp::new(c);
let rlp = Rlp::new(c);
if rlp.is_data() {
ElasticArray1024::from_slice(swapper.compressed(rlp.as_raw()).unwrap_or_else(|| rlp.as_raw()))
} else {
@ -51,7 +51,7 @@ pub fn compress(c: &[u8], swapper: &Compressor) -> ElasticArray1024<u8> {
/// Call this function to decompress rlp.
pub fn decompress(c: &[u8], swapper: &Decompressor) -> ElasticArray1024<u8> {
let rlp = UntrustedRlp::new(c);
let rlp = Rlp::new(c);
if rlp.is_data() {
ElasticArray1024::from_slice(swapper.decompressed(rlp.as_raw()).unwrap_or_else(|| rlp.as_raw()))
} else {
@ -59,7 +59,7 @@ pub fn decompress(c: &[u8], swapper: &Decompressor) -> ElasticArray1024<u8> {
}
}
fn map_rlp<F: Fn(&UntrustedRlp) -> ElasticArray1024<u8>>(rlp: &UntrustedRlp, f: F) -> ElasticArray1024<u8> {
fn map_rlp<F: Fn(&Rlp) -> ElasticArray1024<u8>>(rlp: &Rlp, f: F) -> ElasticArray1024<u8> {
let mut stream = RlpStream::new_list(rlp.item_count().unwrap_or_default());
for subrlp in rlp.iter() {
stream.append_raw(&f(&subrlp), 1);

View File

@ -35,7 +35,7 @@ pub fn impl_decodable(ast: &syn::DeriveInput) -> quote::Tokens {
let dummy_const: syn::Ident = format!("_IMPL_RLP_DECODABLE_FOR_{}", name).into();
let impl_block = quote! {
impl rlp::Decodable for #name {
fn decode(rlp: &rlp::UntrustedRlp) -> Result<Self, rlp::DecoderError> {
fn decode(rlp: &rlp::Rlp) -> Result<Self, rlp::DecoderError> {
let result = #name {
#(#stmts)*
};
@ -75,7 +75,7 @@ pub fn impl_decodable_wrapper(ast: &syn::DeriveInput) -> quote::Tokens {
let dummy_const: syn::Ident = format!("_IMPL_RLP_DECODABLE_FOR_{}", name).into();
let impl_block = quote! {
impl rlp::Decodable for #name {
fn decode(rlp: &rlp::UntrustedRlp) -> Result<Self, rlp::DecoderError> {
fn decode(rlp: &rlp::Rlp) -> Result<Self, rlp::DecoderError> {
let result = #name {
#stmt
};

View File

@ -20,7 +20,7 @@ use std::fmt;
use std::time::{self, SystemTime, Duration, Instant};
use ethereum_types::{H256, H512};
use rlp::{self, DecoderError, RlpStream, UntrustedRlp};
use rlp::{self, DecoderError, RlpStream, Rlp};
use smallvec::SmallVec;
use tiny_keccak::{keccak256, Keccak};
@ -85,7 +85,7 @@ impl rlp::Encodable for Topic {
}
impl rlp::Decodable for Topic {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
use std::cmp;
rlp.decoder().decode_value(|bytes| match bytes.len().cmp(&4) {
@ -145,7 +145,7 @@ fn append_topics<'a>(s: &'a mut RlpStream, topics: &[Topic]) -> &'a mut RlpStrea
}
}
fn decode_topics(rlp: UntrustedRlp) -> Result<SmallVec<[Topic; 4]>, DecoderError> {
fn decode_topics(rlp: Rlp) -> Result<SmallVec<[Topic; 4]>, DecoderError> {
if rlp.is_list() {
rlp.iter().map(|r| r.as_val::<Topic>()).collect()
} else {
@ -212,7 +212,7 @@ impl rlp::Encodable for Envelope {
}
impl rlp::Decodable for Envelope {
fn decode(rlp: &UntrustedRlp) -> Result<Self, DecoderError> {
fn decode(rlp: &Rlp) -> Result<Self, DecoderError> {
if rlp.item_count()? != 5 { return Err(DecoderError::RlpIncorrectListLen) }
Ok(Envelope {
@ -332,7 +332,7 @@ impl Message {
}
/// Decode message from RLP and check for validity against system time.
pub fn decode(rlp: UntrustedRlp, now: SystemTime) -> Result<Self, Error> {
pub fn decode(rlp: Rlp, now: SystemTime) -> Result<Self, Error> {
let envelope: Envelope = rlp.as_val()?;
let encoded_size = rlp.as_raw().len();
let hash = H256(keccak256(rlp.as_raw()));
@ -418,7 +418,7 @@ impl Message {
mod tests {
use super::*;
use std::time::{self, Duration, SystemTime};
use rlp::UntrustedRlp;
use rlp::Rlp;
use smallvec::SmallVec;
fn unix_time(x: u64) -> SystemTime {
@ -481,7 +481,7 @@ mod tests {
for i in 0..30 {
let now = unix_time(100_000 - i);
Message::decode(UntrustedRlp::new(&*encoded), now).unwrap();
Message::decode(Rlp::new(&*encoded), now).unwrap();
}
}
@ -499,7 +499,7 @@ mod tests {
let encoded = ::rlp::encode(&envelope);
let now = unix_time(100_000 - 1_000);
Message::decode(UntrustedRlp::new(&*encoded), now).unwrap();
Message::decode(Rlp::new(&*encoded), now).unwrap();
}
#[test]
@ -516,6 +516,6 @@ mod tests {
let encoded = ::rlp::encode(&envelope);
let now = unix_time(95_000);
Message::decode(UntrustedRlp::new(&*encoded), now).unwrap();
Message::decode(Rlp::new(&*encoded), now).unwrap();
}
}

View File

@ -26,7 +26,7 @@ use ethereum_types::{H256, H512};
use network::{self, HostInfo, NetworkContext, NodeId, PeerId, ProtocolId, TimerToken};
use ordered_float::OrderedFloat;
use parking_lot::{Mutex, RwLock};
use rlp::{DecoderError, RlpStream, UntrustedRlp};
use rlp::{DecoderError, RlpStream, Rlp};
use message::{Message, Error as MessageError};
@ -506,7 +506,7 @@ impl<T: MessageHandler> Network<T> {
}
// handle status packet from peer.
fn on_status(&self, peer: &PeerId, _status: UntrustedRlp)
fn on_status(&self, peer: &PeerId, _status: Rlp)
-> Result<(), Error>
{
let peers = self.peers.read();
@ -523,7 +523,7 @@ impl<T: MessageHandler> Network<T> {
}
}
fn on_messages(&self, peer: &PeerId, message_packet: UntrustedRlp)
fn on_messages(&self, peer: &PeerId, message_packet: Rlp)
-> Result<(), Error>
{
let mut messages_vec = {
@ -568,7 +568,7 @@ impl<T: MessageHandler> Network<T> {
Ok(())
}
fn on_pow_requirement(&self, peer: &PeerId, requirement: UntrustedRlp)
fn on_pow_requirement(&self, peer: &PeerId, requirement: Rlp)
-> Result<(), Error>
{
use byteorder::{ByteOrder, BigEndian};
@ -604,7 +604,7 @@ impl<T: MessageHandler> Network<T> {
Ok(())
}
fn on_topic_filter(&self, peer: &PeerId, filter: UntrustedRlp)
fn on_topic_filter(&self, peer: &PeerId, filter: Rlp)
-> Result<(), Error>
{
let peers = self.peers.read();
@ -661,7 +661,7 @@ impl<T: MessageHandler> Network<T> {
}
fn on_packet<C: ?Sized + Context>(&self, io: &C, peer: &PeerId, packet_id: u8, data: &[u8]) {
let rlp = UntrustedRlp::new(data);
let rlp = Rlp::new(data);
let res = match packet_id {
packet::STATUS => self.on_status(peer, rlp),
packet::MESSAGES => self.on_messages(peer, rlp),