Rust Cuckatoo for verifier and test-miner (#1558)

* cuck placeholder

* rustfmt

* cuckatoo, early days

* rustfmt

* data structures are in place, siphash key creation is consistent with @tromp

* solver in place, (not yet working)

* cuckatoo test solver working with test nonce

* rustfmt

* update solver to remove adjacency list removals

* verifier functioning

* rustfmt

* Proper error handing in Cuckatoo module, couple of tests

* modify cuckoo/cuckatoo solvers and verifiers to function identically, in advance of trait refactoring

* rustfmt

* refactor PoW context into trait, default to using cuckoo context

* rustfmt

* create macros for integer casting/unwraps

* don't instantiate structs when just verifying, add test validation vector for cuckatoo 29

* rustfmt

* don't init cuckoo structs if just validating

* test fix

* ensure BH hashing for POW is only done within miner/validators
This commit is contained in:
Yeastplume 2018-09-28 11:53:14 +01:00 committed by GitHub
parent a13c20ceb2
commit e64f4fbcd1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
16 changed files with 1171 additions and 329 deletions

3
Cargo.lock generated
View file

@ -740,6 +740,7 @@ dependencies = [
"grin_wallet 0.3.0", "grin_wallet 0.3.0",
"lazy_static 1.1.0 (registry+https://github.com/rust-lang/crates.io-index)", "lazy_static 1.1.0 (registry+https://github.com/rust-lang/crates.io-index)",
"lru-cache 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", "lru-cache 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)",
"num 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
"num-bigint 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)", "num-bigint 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
"rand 0.5.5 (registry+https://github.com/rust-lang/crates.io-index)", "rand 0.5.5 (registry+https://github.com/rust-lang/crates.io-index)",
"serde 1.0.79 (registry+https://github.com/rust-lang/crates.io-index)", "serde 1.0.79 (registry+https://github.com/rust-lang/crates.io-index)",
@ -1411,6 +1412,7 @@ name = "num"
version = "0.2.0" version = "0.2.0"
source = "registry+https://github.com/rust-lang/crates.io-index" source = "registry+https://github.com/rust-lang/crates.io-index"
dependencies = [ dependencies = [
"num-bigint 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
"num-complex 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)", "num-complex 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
"num-integer 0.1.39 (registry+https://github.com/rust-lang/crates.io-index)", "num-integer 0.1.39 (registry+https://github.com/rust-lang/crates.io-index)",
"num-iter 0.1.37 (registry+https://github.com/rust-lang/crates.io-index)", "num-iter 0.1.37 (registry+https://github.com/rust-lang/crates.io-index)",
@ -1488,6 +1490,7 @@ name = "num-rational"
version = "0.2.1" version = "0.2.1"
source = "registry+https://github.com/rust-lang/crates.io-index" source = "registry+https://github.com/rust-lang/crates.io-index"
dependencies = [ dependencies = [
"num-bigint 0.2.0 (registry+https://github.com/rust-lang/crates.io-index)",
"num-integer 0.1.39 (registry+https://github.com/rust-lang/crates.io-index)", "num-integer 0.1.39 (registry+https://github.com/rust-lang/crates.io-index)",
"num-traits 0.2.6 (registry+https://github.com/rust-lang/crates.io-index)", "num-traits 0.2.6 (registry+https://github.com/rust-lang/crates.io-index)",
] ]

View file

@ -28,7 +28,7 @@ use core::core::merkle_proof::MerkleProof;
use core::core::verifier_cache::VerifierCache; use core::core::verifier_cache::VerifierCache;
use core::core::{Block, BlockHeader, BlockSums, Output, OutputIdentifier, Transaction, TxKernel}; use core::core::{Block, BlockHeader, BlockSums, Output, OutputIdentifier, Transaction, TxKernel};
use core::global; use core::global;
use core::pow::Difficulty; use core::pow::{self, Difficulty};
use error::{Error, ErrorKind}; use error::{Error, ErrorKind};
use grin_store::Error::NotFoundErr; use grin_store::Error::NotFoundErr;
use pipe; use pipe;
@ -153,7 +153,7 @@ pub struct Chain {
block_hashes_cache: Arc<RwLock<VecDeque<Hash>>>, block_hashes_cache: Arc<RwLock<VecDeque<Hash>>>,
verifier_cache: Arc<RwLock<VerifierCache>>, verifier_cache: Arc<RwLock<VerifierCache>>,
// POW verification function // POW verification function
pow_verifier: fn(&BlockHeader, u8) -> bool, pow_verifier: fn(&BlockHeader, u8) -> Result<(), pow::Error>,
archive_mode: bool, archive_mode: bool,
} }
@ -169,7 +169,7 @@ impl Chain {
db_env: Arc<lmdb::Environment>, db_env: Arc<lmdb::Environment>,
adapter: Arc<ChainAdapter>, adapter: Arc<ChainAdapter>,
genesis: Block, genesis: Block,
pow_verifier: fn(&BlockHeader, u8) -> bool, pow_verifier: fn(&BlockHeader, u8) -> Result<(), pow::Error>,
verifier_cache: Arc<RwLock<VerifierCache>>, verifier_cache: Arc<RwLock<VerifierCache>>,
archive_mode: bool, archive_mode: bool,
) -> Result<Chain, Error> { ) -> Result<Chain, Error> {

View file

@ -27,7 +27,7 @@ use core::core::verifier_cache::VerifierCache;
use core::core::Committed; use core::core::Committed;
use core::core::{Block, BlockHeader, BlockSums}; use core::core::{Block, BlockHeader, BlockSums};
use core::global; use core::global;
use core::pow::Difficulty; use core::pow::{self, Difficulty};
use error::{Error, ErrorKind}; use error::{Error, ErrorKind};
use grin_store; use grin_store;
use store; use store;
@ -49,7 +49,7 @@ pub struct BlockContext {
/// The sync head /// The sync head
pub sync_head: Tip, pub sync_head: Tip,
/// The POW verification function /// The POW verification function
pub pow_verifier: fn(&BlockHeader, u8) -> bool, pub pow_verifier: fn(&BlockHeader, u8) -> Result<(), pow::Error>,
/// MMR sum tree states /// MMR sum tree states
pub txhashset: Arc<RwLock<txhashset::TxHashSet>>, pub txhashset: Arc<RwLock<txhashset::TxHashSet>>,
/// Recently processed blocks to avoid double-processing /// Recently processed blocks to avoid double-processing
@ -439,7 +439,7 @@ fn validate_header(
if shift != consensus::SECOND_POW_SIZESHIFT && header.pow.scaling_difficulty != 1 { if shift != consensus::SECOND_POW_SIZESHIFT && header.pow.scaling_difficulty != 1 {
return Err(ErrorKind::InvalidScaling.into()); return Err(ErrorKind::InvalidScaling.into());
} }
if !(ctx.pow_verifier)(header, shift) { if !(ctx.pow_verifier)(header, shift).is_ok() {
error!( error!(
LOGGER, LOGGER,
"pipe: validate_header bad cuckoo shift size {}", shift "pipe: validate_header bad cuckoo shift size {}", shift
@ -527,7 +527,8 @@ fn validate_block(
&prev.total_kernel_offset, &prev.total_kernel_offset,
&prev.total_kernel_sum, &prev.total_kernel_sum,
verifier_cache, verifier_cache,
).map_err(|e| ErrorKind::InvalidBlockProof(e))?; )
.map_err(|e| ErrorKind::InvalidBlockProof(e))?;
Ok(()) Ok(())
} }
@ -567,8 +568,7 @@ fn verify_block_sums(b: &Block, ext: &mut txhashset::Extension) -> Result<(), Er
let offset = b.header.total_kernel_offset(); let offset = b.header.total_kernel_offset();
// Verify the kernel sums for the block_sums with the new block applied. // Verify the kernel sums for the block_sums with the new block applied.
let (utxo_sum, kernel_sum) = let (utxo_sum, kernel_sum) = (block_sums, b as &Committed).verify_kernel_sums(overage, offset)?;
(block_sums, b as &Committed).verify_kernel_sums(overage, offset)?;
// Save the new block_sums for the new block to the db via the batch. // Save the new block_sums for the new block to the db via the batch.
ext.batch.save_block_sums( ext.batch.save_block_sums(

View file

@ -14,6 +14,7 @@ failure = "0.1"
failure_derive = "0.1" failure_derive = "0.1"
lazy_static = "1" lazy_static = "1"
lru-cache = "0.1" lru-cache = "0.1"
num = "0.2"
num-bigint = "0.2" num-bigint = "0.2"
rand = "0.5" rand = "0.5"
serde = "1" serde = "1"

View file

@ -25,7 +25,7 @@ use std::sync::{Arc, RwLock};
use consensus::{self, reward, REWARD}; use consensus::{self, reward, REWARD};
use core::committed::{self, Committed}; use core::committed::{self, Committed};
use core::compact_block::{CompactBlock, CompactBlockBody}; use core::compact_block::{CompactBlock, CompactBlockBody};
use core::hash::{Hash, HashWriter, Hashed, ZERO_HASH}; use core::hash::{Hash, Hashed, ZERO_HASH};
use core::verifier_cache::VerifierCache; use core::verifier_cache::VerifierCache;
use core::{ use core::{
transaction, Commitment, Input, KernelFeatures, Output, OutputFeatures, Transaction, transaction, Commitment, Input, KernelFeatures, Output, OutputFeatures, Transaction,
@ -278,16 +278,19 @@ impl BlockHeader {
Ok(()) Ok(())
} }
/// Returns the pre-pow hash, as the post-pow hash /// Return the pre-pow, unhashed
/// should just be the hash of the POW /// Let the cuck(at)oo miner/verifier handle the hashing
pub fn pre_pow_hash(&self) -> Hash { /// for consistency with how this call is performed everywhere
let mut hasher = HashWriter::default(); /// else
self.write_pre_pow(&mut hasher).unwrap(); pub fn pre_pow(&self) -> Vec<u8> {
self.pow.write_pre_pow(self.version, &mut hasher).unwrap(); let mut header_buf = vec![];
hasher.write_u64(self.pow.nonce).unwrap(); {
let mut ret = [0; 32]; let mut writer = ser::BinWriter::new(&mut header_buf);
hasher.finalize(&mut ret); self.write_pre_pow(&mut writer).unwrap();
Hash(ret) self.pow.write_pre_pow(self.version, &mut writer).unwrap();
writer.write_u64(self.pow.nonce).unwrap();
}
header_buf
} }
/// Total difficulty accumulated by the proof of work on this header /// Total difficulty accumulated by the proof of work on this header

View file

@ -22,7 +22,7 @@ use consensus::{
DIFFICULTY_ADJUST_WINDOW, INITIAL_DIFFICULTY, MEDIAN_TIME_WINDOW, PROOFSIZE, DIFFICULTY_ADJUST_WINDOW, INITIAL_DIFFICULTY, MEDIAN_TIME_WINDOW, PROOFSIZE,
REFERENCE_SIZESHIFT, REFERENCE_SIZESHIFT,
}; };
use pow::Difficulty; use pow::{self, CuckooContext, Difficulty, EdgeType, PoWContext};
/// An enum collecting sets of parameters used throughout the /// An enum collecting sets of parameters used throughout the
/// code wherever mining is needed. This should allow for /// code wherever mining is needed. This should allow for
/// different sets of parameters for different purposes, /// different sets of parameters for different purposes,
@ -93,10 +93,23 @@ impl Default for ChainTypes {
} }
} }
/// PoW test mining and verifier context
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub enum PoWContextTypes {
/// Classic Cuckoo
Cuckoo,
/// Bleeding edge Cuckatoo
Cuckatoo,
}
lazy_static!{ lazy_static!{
/// The mining parameter mode /// The mining parameter mode
pub static ref CHAIN_TYPE: RwLock<ChainTypes> = pub static ref CHAIN_TYPE: RwLock<ChainTypes> =
RwLock::new(ChainTypes::Mainnet); RwLock::new(ChainTypes::Mainnet);
/// PoW context type to instantiate
pub static ref POW_CONTEXT_TYPE: RwLock<PoWContextTypes> =
RwLock::new(PoWContextTypes::Cuckoo);
} }
/// Set the mining mode /// Set the mining mode
@ -105,6 +118,25 @@ pub fn set_mining_mode(mode: ChainTypes) {
*param_ref = mode; *param_ref = mode;
} }
/// Return either a cuckoo context or a cuckatoo context
/// Single change point
pub fn create_pow_context<T>(
edge_bits: u8,
proof_size: usize,
easiness_pct: u32,
max_sols: u32,
) -> Result<Box<impl PoWContext<T>>, pow::Error>
where
T: EdgeType,
{
// Perform whatever tests, configuration etc are needed to determine desired context + edge size
// + params
// Hardcode to regular cuckoo for now
CuckooContext::<T>::new(edge_bits, proof_size, easiness_pct, max_sols)
// Or switch to cuckatoo as follows:
// CuckatooContext::<T>::new(edge_bits, proof_size, easiness_pct, max_sols)
}
/// The minimum acceptable sizeshift /// The minimum acceptable sizeshift
pub fn min_sizeshift() -> u8 { pub fn min_sizeshift() -> u8 {
let param_ref = CHAIN_TYPE.read().unwrap(); let param_ref = CHAIN_TYPE.read().unwrap();

153
core/src/pow/common.rs Normal file
View file

@ -0,0 +1,153 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Common types and traits for cuckoo/cuckatoo family of solvers
use blake2::blake2b::blake2b;
use pow::error::{Error, ErrorKind};
use pow::num::{PrimInt, ToPrimitive};
use pow::siphash::siphash24;
use std::hash::Hash;
use std::io::Cursor;
use std::ops::{BitOrAssign, Mul};
use std::{fmt, mem};
use byteorder::{LittleEndian, ReadBytesExt, WriteBytesExt};
/// Operations needed for edge type (going to be u32 or u64)
pub trait EdgeType: PrimInt + ToPrimitive + Mul + BitOrAssign + Hash {}
impl EdgeType for u32 {}
impl EdgeType for u64 {}
/// An edge in the Cuckoo graph, simply references two u64 nodes.
#[derive(Debug, Copy, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
pub struct Edge<T>
where
T: EdgeType,
{
pub u: T,
pub v: T,
}
impl<T> fmt::Display for Edge<T>
where
T: EdgeType,
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"(u: {}, v: {})",
self.u.to_u64().unwrap_or(0),
self.v.to_u64().unwrap_or(0)
)
}
}
/// An element of an adjencency list
#[derive(Debug, Clone, Eq, PartialEq)]
pub struct Link<T>
where
T: EdgeType,
{
pub next: T,
pub to: T,
}
impl<T> fmt::Display for Link<T>
where
T: EdgeType,
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"(next: {}, to: {})",
self.next.to_u64().unwrap_or(0),
self.to.to_u64().unwrap_or(0)
)
}
}
pub fn set_header_nonce(header: Vec<u8>, nonce: Option<u32>) -> Result<[u64; 4], Error> {
let len = header.len();
let mut header = header.clone();
if let Some(n) = nonce {
header.truncate(len - mem::size_of::<u32>());
header.write_u32::<LittleEndian>(n)?;
}
create_siphash_keys(header)
}
pub fn create_siphash_keys(header: Vec<u8>) -> Result<[u64; 4], Error> {
let h = blake2b(32, &[], &header);
let hb = h.as_bytes();
let mut rdr = Cursor::new(hb);
Ok([
rdr.read_u64::<LittleEndian>()?,
rdr.read_u64::<LittleEndian>()?,
rdr.read_u64::<LittleEndian>()?,
rdr.read_u64::<LittleEndian>()?,
])
}
/// Return siphash masked for type
pub fn sipnode<T>(
keys: &[u64; 4],
edge: T,
edge_mask: &T,
uorv: u64,
shift: bool,
) -> Result<T, Error>
where
T: EdgeType,
{
let hash_u64 = siphash24(
keys,
2 * edge.to_u64().ok_or(ErrorKind::IntegerCast)? + uorv,
);
let mut masked = hash_u64 & edge_mask.to_u64().ok_or(ErrorKind::IntegerCast)?;
if shift {
masked = masked << 1;
masked |= uorv;
}
Ok(T::from(masked).ok_or(ErrorKind::IntegerCast)?)
}
/// Macros to clean up integer unwrapping
#[macro_export]
macro_rules! to_u64 {
($n:expr) => {
$n.to_u64().ok_or(ErrorKind::IntegerCast)?
};
}
#[macro_export]
macro_rules! to_u32 {
($n:expr) => {
$n.to_u64().ok_or(ErrorKind::IntegerCast)? as u32
};
}
#[macro_export]
macro_rules! to_usize {
($n:expr) => {
$n.to_u64().ok_or(ErrorKind::IntegerCast)? as usize
};
}
#[macro_export]
macro_rules! to_edge {
($n:expr) => {
T::from($n).ok_or(ErrorKind::IntegerCast)?
};
}

461
core/src/pow/cuckatoo.rs Normal file
View file

@ -0,0 +1,461 @@
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Implementation of Cuckatoo Cycle designed by John Tromp.
use pow::num::ToPrimitive;
use std::mem;
use byteorder::{BigEndian, LittleEndian, WriteBytesExt};
use croaring::Bitmap;
use pow::common::{self, EdgeType, Link};
use pow::error::{Error, ErrorKind};
use pow::{PoWContext, Proof};
use util;
struct Graph<T>
where
T: EdgeType,
{
/// Maximum number of edges
max_edges: T,
/// Maximum nodes
max_nodes: u64,
/// Adjacency links
links: Vec<Link<T>>,
/// Index into links array
adj_list: Vec<T>,
///
visited: Bitmap,
/// Maximum solutions
max_sols: u32,
///
pub solutions: Vec<Proof>,
/// proof size
proof_size: usize,
/// define NIL type
nil: T,
}
impl<T> Graph<T>
where
T: EdgeType,
{
/// Create a new graph with given parameters
pub fn new(max_edges: T, max_sols: u32, proof_size: usize) -> Result<Graph<T>, Error> {
let max_nodes = 2 * to_u64!(max_edges);
Ok(Graph {
max_edges: max_edges,
max_nodes: max_nodes,
links: vec![],
adj_list: vec![],
visited: Bitmap::create(),
max_sols: max_sols,
solutions: vec![],
proof_size: proof_size,
nil: T::max_value(),
})
}
pub fn reset(&mut self) -> Result<(), Error> {
//TODO: Can be optimised
self.links = Vec::with_capacity(2 * self.max_nodes as usize);
self.adj_list = vec![T::max_value(); 2 * self.max_nodes as usize];
self.solutions = vec![Proof::zero(self.proof_size); 1];
self.visited = Bitmap::create();
Ok(())
}
pub fn byte_count(&self) -> Result<u64, Error> {
Ok(
2 * to_u64!(self.max_edges) * mem::size_of::<Link<T>>() as u64
+ mem::size_of::<T>() as u64 * 2 * self.max_nodes,
)
}
/// Add an edge to the graph
pub fn add_edge(&mut self, u: T, mut v: T) -> Result<(), Error> {
let max_nodes_t = to_edge!(self.max_nodes);
if u >= max_nodes_t || v >= max_nodes_t {
return Err(ErrorKind::EdgeAddition)?;
}
v = v + to_edge!(self.max_nodes);
let adj_u = self.adj_list[to_usize!(u ^ T::one())];
let adj_v = self.adj_list[to_usize!(v ^ T::one())];
if adj_u != self.nil && adj_v != self.nil {
let sol_index = self.solutions.len() - 1;
self.solutions[sol_index].nonces[0] = self.links.len() as u64 / 2;
self.cycles_with_link(1, u, v)?;
}
let ulink = self.links.len();
let vlink = self.links.len() + 1;
if to_edge!(vlink) == self.nil {
return Err(ErrorKind::EdgeAddition)?;
}
self.links.push(Link {
next: self.adj_list[to_usize!(u)],
to: u,
});
self.links.push(Link {
next: self.adj_list[to_usize!(v)],
to: v,
});
self.adj_list[to_usize!(u)] = T::from(ulink).ok_or(ErrorKind::IntegerCast)?;
self.adj_list[to_usize!(v)] = T::from(vlink).ok_or(ErrorKind::IntegerCast)?;
Ok(())
}
fn test_bit(&mut self, u: u64) -> bool {
self.visited.contains(u as u32)
}
fn cycles_with_link(&mut self, len: u32, u: T, dest: T) -> Result<(), Error> {
if self.test_bit(to_u64!(u >> 1)) {
return Ok(());
}
if (u ^ T::one()) == dest {
if len == self.proof_size as u32 {
if self.solutions.len() < self.max_sols as usize {
// create next solution
self.solutions.push(Proof::zero(self.proof_size));
}
return Ok(());
}
} else if len == self.proof_size as u32 {
return Ok(());
}
let mut au1 = self.adj_list[to_usize!(u ^ T::one())];
if au1 != self.nil {
self.visited.add(to_u32!(u >> 1));
while au1 != self.nil {
let i = self.solutions.len() - 1;
self.solutions[i].nonces[len as usize] = to_u64!(au1) / 2;
let link_index = to_usize!(au1 ^ T::one());
let link = self.links[link_index].to;
if link != self.nil {
self.cycles_with_link(len + 1, link, dest)?;
}
au1 = self.links[to_usize!(au1)].next;
}
self.visited.remove(to_u32!(u >> 1));
}
Ok(())
}
}
/// Cuckatoo solver context
pub struct CuckatooContext<T>
where
T: EdgeType,
{
siphash_keys: [u64; 4],
easiness: T,
graph: Graph<T>,
proof_size: usize,
edge_mask: T,
}
impl<T> PoWContext<T> for CuckatooContext<T>
where
T: EdgeType,
{
fn new(
edge_bits: u8,
proof_size: usize,
easiness_pct: u32,
max_sols: u32,
) -> Result<Box<Self>, Error> {
Ok(Box::new(CuckatooContext::<T>::new_impl(
edge_bits,
proof_size,
easiness_pct,
max_sols,
)?))
}
fn set_header_nonce(
&mut self,
header: Vec<u8>,
nonce: Option<u32>,
solve: bool,
) -> Result<(), Error> {
self.set_header_nonce_impl(header, nonce, solve)
}
fn find_cycles(&mut self) -> Result<Vec<Proof>, Error> {
self.find_cycles_impl()
}
fn verify(&self, proof: &Proof) -> Result<(), Error> {
self.verify_impl(proof)
}
}
impl<T> CuckatooContext<T>
where
T: EdgeType,
{
/// New Solver context
pub fn new_impl(
edge_bits: u8,
proof_size: usize,
easiness_pct: u32,
max_sols: u32,
) -> Result<CuckatooContext<T>, Error> {
let num_edges = 1 << edge_bits;
let num_nodes = 2 * num_edges as u64;
let easiness = to_u64!(easiness_pct) * num_nodes / 100;
Ok(CuckatooContext {
siphash_keys: [0; 4],
easiness: to_edge!(easiness),
graph: Graph::new(to_edge!(num_edges), max_sols, proof_size)?,
proof_size: proof_size,
edge_mask: to_edge!(num_edges - 1),
})
}
/// Get a siphash key as a hex string (for display convenience)
pub fn sipkey_hex(&self, index: usize) -> Result<String, Error> {
let mut rdr = vec![];
rdr.write_u64::<BigEndian>(self.siphash_keys[index])?;
Ok(util::to_hex(rdr))
}
/// Return number of bytes used by the graph
pub fn byte_count(&self) -> Result<u64, Error> {
self.graph.byte_count()
}
/// Set the header and optional nonce in the last part of the header
pub fn set_header_nonce_impl(
&mut self,
mut header: Vec<u8>,
nonce: Option<u32>,
solve: bool,
) -> Result<(), Error> {
let len = header.len();
header.truncate(len - mem::size_of::<u32>());
if let Some(n) = nonce {
header.write_u32::<LittleEndian>(n)?;
}
self.siphash_keys = common::set_header_nonce(header, nonce)?;
if solve {
self.graph.reset()?;
}
Ok(())
}
/// Return siphash masked for type
fn sipnode(&self, edge: T, uorv: u64) -> Result<T, Error> {
common::sipnode::<T>(&self.siphash_keys, edge, &self.edge_mask, uorv, false)
}
/// Simple implementation of algorithm
pub fn find_cycles_impl(&mut self) -> Result<Vec<Proof>, Error> {
for n in 0..to_u64!(self.easiness) {
let u = self.sipnode(to_edge!(n), 0)?;
let v = self.sipnode(to_edge!(n), 1)?;
self.graph.add_edge(to_edge!(u), to_edge!(v))?;
}
self.graph.solutions.pop();
for s in &mut self.graph.solutions {
s.nonces.sort();
}
for s in &self.graph.solutions {
self.verify_impl(&s)?;
}
if self.graph.solutions.len() == 0 {
Err(ErrorKind::NoSolution)?
} else {
Ok(self.graph.solutions.clone())
}
}
/// Verify that given edges are ascending and form a cycle in a header-generated
/// graph
pub fn verify_impl(&self, proof: &Proof) -> Result<(), Error> {
let nonces = &proof.nonces;
let mut uvs = vec![0u64; 2 * proof.proof_size()];
let mut xor0: u64 = (self.proof_size as u64 / 2) & 1;
let mut xor1: u64 = xor0;
for n in 0..proof.proof_size() {
if nonces[n] > to_u64!(self.edge_mask) {
return Err(ErrorKind::Verification("edge too big".to_owned()))?;
}
if n > 0 && nonces[n] <= nonces[n - 1] {
return Err(ErrorKind::Verification("edges not ascending".to_owned()))?;
}
uvs[2 * n] = to_u64!(self.sipnode(to_edge!(nonces[n]), 0)?);
uvs[2 * n + 1] = to_u64!(self.sipnode(to_edge!(nonces[n]), 1)?);
xor0 ^= uvs[2 * n];
xor1 ^= uvs[2 * n + 1];
}
if xor0 | xor1 != 0 {
return Err(ErrorKind::Verification(
"endpoints don't match up".to_owned(),
))?;
}
let mut n = 0;
let mut i = 0;
let mut j;
loop {
// follow cycle
j = i;
let mut k = j;
loop {
k = (k + 2) % (2 * self.proof_size);
if k == i {
break;
}
if uvs[k] >> 1 == uvs[i] >> 1 {
// find other edge endpoint matching one at i
if j != i {
return Err(ErrorKind::Verification("branch in cycle".to_owned()))?;
}
j = k;
}
}
if j == i || uvs[j] == uvs[i] {
return Err(ErrorKind::Verification("cycle dead ends".to_owned()))?;
}
i = j ^ 1;
n += 1;
if i == 0 {
break;
}
}
if n == self.proof_size {
Ok(())
} else {
Err(ErrorKind::Verification("cycle too short".to_owned()))?
}
}
}
#[cfg(test)]
mod test {
use super::*;
// Cuckatoo 29 Solution for Header [0u8;80] - nonce 20
static V1_29: [u64; 42] = [
0x48a9e2, 0x9cf043, 0x155ca30, 0x18f4783, 0x248f86c, 0x2629a64, 0x5bad752, 0x72e3569,
0x93db760, 0x97d3b37, 0x9e05670, 0xa315d5a, 0xa3571a1, 0xa48db46, 0xa7796b6, 0xac43611,
0xb64912f, 0xbb6c71e, 0xbcc8be1, 0xc38a43a, 0xd4faa99, 0xe018a66, 0xe37e49c, 0xfa975fa,
0x11786035, 0x1243b60a, 0x12892da0, 0x141b5453, 0x1483c3a0, 0x1505525e, 0x1607352c,
0x16181fe3, 0x17e3a1da, 0x180b651e, 0x1899d678, 0x1931b0bb, 0x19606448, 0x1b041655,
0x1b2c20ad, 0x1bd7a83c, 0x1c05d5b0, 0x1c0b9caa,
];
#[test]
fn cuckatoo() {
let ret = basic_solve::<u32>();
if let Err(r) = ret {
panic!("basic_solve u32: Error: {}", r);
}
let ret = basic_solve::<u64>();
if let Err(r) = ret {
panic!("basic_solve u64: Error: {}", r);
}
let ret = validate29_vectors::<u32>();
if let Err(r) = ret {
panic!("validate_29_vectors u32: Error: {}", r);
}
let ret = validate29_vectors::<u64>();
if let Err(r) = ret {
panic!("validate_29_vectors u64: Error: {}", r);
}
let ret = validate_fail::<u32>();
if let Err(r) = ret {
panic!("validate_fail u32: Error: {}", r);
}
let ret = validate_fail::<u64>();
if let Err(r) = ret {
panic!("validate_fail u64: Error: {}", r);
}
}
fn validate29_vectors<T>() -> Result<(), Error>
where
T: EdgeType,
{
let mut ctx = CuckatooContext::<T>::new(29, 42, 50, 10)?;
ctx.set_header_nonce([0u8; 80].to_vec(), Some(20), false)?;
assert!(ctx.verify(&Proof::new(V1_29.to_vec().clone())).is_ok());
Ok(())
}
fn validate_fail<T>() -> Result<(), Error>
where
T: EdgeType,
{
let mut ctx = CuckatooContext::<T>::new(29, 42, 50, 10)?;
let mut header = [0u8; 80];
header[0] = 1u8;
ctx.set_header_nonce(header.to_vec(), Some(20), false)?;
assert!(!ctx.verify(&Proof::new(V1_29.to_vec().clone())).is_ok());
header[0] = 0u8;
ctx.set_header_nonce(header.to_vec(), Some(20), false)?;
assert!(ctx.verify(&Proof::new(V1_29.to_vec().clone())).is_ok());
let mut bad_proof = V1_29.clone();
bad_proof[0] = 0x48a9e1;
assert!(!ctx.verify(&Proof::new(bad_proof.to_vec())).is_ok());
Ok(())
}
fn basic_solve<T>() -> Result<(), Error>
where
T: EdgeType,
{
let easiness_pct = 50;
let nonce = 1546569;
let _range = 1;
let header = [0u8; 80].to_vec();
let proof_size = 42;
let edge_bits = 15;
let max_sols = 4;
println!(
"Looking for {}-cycle on cuckatoo{}(\"{}\",{}) with {}% edges",
proof_size,
edge_bits,
String::from_utf8(header.clone()).unwrap(),
nonce,
easiness_pct
);
let mut ctx_u32 = CuckatooContext::<T>::new(edge_bits, proof_size, easiness_pct, max_sols)?;
let mut bytes = ctx_u32.byte_count()?;
let mut unit = 0;
while bytes >= 10240 {
bytes >>= 10;
unit += 1;
}
println!("Using {}{}B memory", bytes, [' ', 'K', 'M', 'G', 'T'][unit]);
ctx_u32.set_header_nonce(header, Some(nonce), true)?;
println!(
"Nonce {} k0 k1 k2 k3 {} {} {} {}",
nonce,
ctx_u32.sipkey_hex(0)?,
ctx_u32.sipkey_hex(1)?,
ctx_u32.sipkey_hex(2)?,
ctx_u32.sipkey_hex(3)?
);
let sols = ctx_u32.find_cycles()?;
// We know this nonce has 2 solutions
assert_eq!(sols.len(), 2);
for s in sols {
println!("{:?}", s);
}
Ok(())
}
}

View file

@ -17,233 +17,174 @@
//! simple miner is included, mostly for testing purposes. John Tromp's Tomato //! simple miner is included, mostly for testing purposes. John Tromp's Tomato
//! miner will be much faster in almost every environment. //! miner will be much faster in almost every environment.
use pow::common::{self, Edge, EdgeType};
use pow::error::{Error, ErrorKind};
use pow::num::ToPrimitive;
use pow::{PoWContext, Proof};
use std::cmp; use std::cmp;
use std::collections::HashSet; use std::collections::HashSet;
use core::BlockHeader;
use pow::siphash::siphash24;
use pow::Proof;
const MAXPATHLEN: usize = 8192; const MAXPATHLEN: usize = 8192;
/// A cuckoo-cycle related error
#[derive(Debug)]
pub enum Error {
/// Unable to find a short enough path
Path,
/// Unable to find a solution
NoSolution,
}
/// An edge in the Cuckoo graph, simply references two u64 nodes.
#[derive(Debug, Copy, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
struct Edge {
u: u64,
v: u64,
}
/// Cuckoo cycle context /// Cuckoo cycle context
pub struct Cuckoo { pub struct CuckooContext<T>
mask: u64, where
size: u64, T: EdgeType,
v: [u64; 4], {
edge_bits: u8,
siphash_keys: [u64; 4],
easiness: T,
proof_size: usize,
edge_mask: T,
graph: Vec<T>,
_max_sols: u32,
} }
impl Cuckoo { impl<T> PoWContext<T> for CuckooContext<T>
/// Initializes a new Cuckoo Cycle setup, using the provided byte array to where
/// generate a seed. In practice for PoW applications the byte array is a T: EdgeType,
/// serialized block header. {
pub fn from_hash(header_hash: &[u8], sizeshift: u8) -> Cuckoo { fn new(
let size = 1 << sizeshift; edge_bits: u8,
Cuckoo { proof_size: usize,
v: [ easiness_pct: u32,
u8_to_u64(header_hash, 0), max_sols: u32,
u8_to_u64(header_hash, 8), ) -> Result<Box<Self>, Error> {
u8_to_u64(header_hash, 16), Ok(Box::new(CuckooContext::<T>::new_impl(
u8_to_u64(header_hash, 24), edge_bits,
], proof_size,
size: size, easiness_pct,
mask: (1 << sizeshift) / 2 - 1, max_sols,
)?))
} }
fn set_header_nonce(
&mut self,
header: Vec<u8>,
nonce: Option<u32>,
solve: bool,
) -> Result<(), Error> {
self.set_header_nonce_impl(header, nonce, solve)
}
fn find_cycles(&mut self) -> Result<Vec<Proof>, Error> {
self.find_cycles_impl()
}
fn verify(&self, proof: &Proof) -> Result<(), Error> {
self.verify_impl(proof)
}
}
impl<T> CuckooContext<T>
where
T: EdgeType,
{
/// Create a new cuckoo context with given parameters
pub fn new_impl(
edge_bits: u8,
proof_size: usize,
easiness_pct: u32,
max_sols: u32,
) -> Result<CuckooContext<T>, Error> {
let num_edges = 1 << edge_bits;
let easiness = to_u64!(easiness_pct) * num_edges / 100;
Ok(CuckooContext {
edge_bits: edge_bits,
siphash_keys: [0; 4],
easiness: to_edge!(easiness),
proof_size: proof_size,
edge_mask: to_edge!(num_edges / 2 - 1),
graph: vec![T::zero(); num_edges as usize + 1],
_max_sols: max_sols,
})
}
fn reset(&mut self) -> Result<(), Error> {
let num_edges = 1 << self.edge_bits;
self.graph = vec![T::zero(); num_edges + 1];
Ok(())
}
/// Set the header and optional nonce in the last part of the header
/// and create siphash keys
pub fn set_header_nonce_impl(
&mut self,
header: Vec<u8>,
nonce: Option<u32>,
solve: bool,
) -> Result<(), Error> {
self.siphash_keys = common::set_header_nonce(header, nonce)?;
if solve {
self.reset()?;
}
Ok(())
} }
/// Generates a node in the cuckoo graph generated from our seed. A node is /// Generates a node in the cuckoo graph generated from our seed. A node is
/// simply materialized as a u64 from a nonce and an offset (generally 0 or /// simply materialized as a u64 from a nonce and an offset (generally 0 or
/// 1). /// 1).
fn new_node(&self, nonce: u64, uorv: u64) -> u64 { fn new_node(&self, edge: T, uorv: u64) -> Result<T, Error> {
return ((siphash24(self.v, 2 * nonce + uorv) & self.mask) << 1) | uorv; common::sipnode::<T>(&self.siphash_keys, edge, &self.edge_mask, uorv, true)
} }
/// Creates a new edge in the cuckoo graph generated by our seed from a /// Creates a new edge in the cuckoo graph generated by our seed from a
/// nonce. Generates two node coordinates from the nonce and links them /// nonce. Generates two node coordinates from the nonce and links them
/// together. /// together.
fn new_edge(&self, nonce: u64) -> Edge { fn new_edge(&self, nonce: T) -> Result<Edge<T>, Error> {
Edge { Ok(Edge {
u: self.new_node(nonce, 0), u: self.new_node(nonce, 0)?,
v: self.new_node(nonce, 1), v: self.new_node(nonce, 1)?,
} })
} }
/// Assuming increasing nonces all smaller than easiness, verifies the fn path(&self, mut u: T, us: &mut [T]) -> Result<T, Error> {
/// nonces form a cycle in a Cuckoo graph. Each nonce generates an edge, we
/// build the nodes on both side of that edge and count the connections.
pub fn verify(&self, proof: &Proof, ease: u64) -> bool {
let easiness = ease * (self.size as u64) / 100;
let nonces = &proof.nonces;
let mut us = vec![0; proof.proof_size()];
let mut vs = vec![0; proof.proof_size()];
for n in 0..proof.proof_size() {
if nonces[n] >= easiness || (n != 0 && nonces[n] <= nonces[n - 1]) {
return false;
}
us[n] = self.new_node(nonces[n], 0);
vs[n] = self.new_node(nonces[n], 1);
}
let mut i = 0;
let mut count = proof.proof_size();
loop {
let mut j = i;
for k in 0..proof.proof_size() {
// find unique other j with same vs[j]
if k != i && vs[k] == vs[i] {
if j != i {
return false;
}
j = k;
}
}
if j == i {
return false;
}
i = j;
for k in 0..proof.proof_size() {
// find unique other i with same us[i]
if k != j && us[k] == us[j] {
if i != j {
return false;
}
i = k;
}
}
if i == j {
return false;
}
count -= 2;
if i == 0 {
break;
}
}
count == 0
}
}
/// Miner for the Cuckoo Cycle algorithm. While the verifier will work for
/// graph sizes up to a u64, the miner is limited to u32 to be more memory
/// compact (so shift <= 32). Non-optimized for now and and so mostly used for
/// tests, being impractical with sizes greater than 2^22.
pub struct Miner {
easiness: u64,
proof_size: usize,
cuckoo: Cuckoo,
graph: Vec<u32>,
sizeshift: u8,
}
/// What type of cycle we have found?
enum CycleSol {
/// A cycle of the right length is a valid proof.
ValidProof(Vec<u32>),
/// A cycle of the wrong length is great, but not a proof.
InvalidCycle(usize),
/// No cycles have been found.
NoCycle,
}
impl Miner {
/// Creates a new miner for the provided block header
pub fn new(header: &BlockHeader, ease: u32, proof_size: usize, sizeshift: u8) -> Miner {
Miner::from_hash(header.pre_pow_hash().as_ref(), ease, proof_size, sizeshift)
}
/// Creates a new miner directly from a hash
pub fn from_hash(header_hash: &[u8], ease: u32, proof_size: usize, sizeshift: u8) -> Miner {
let cuckoo = Cuckoo::from_hash(header_hash, sizeshift);
let size = 1 << sizeshift;
let graph = vec![0; size + 1];
let easiness = (ease as u64) * (size as u64) / 100;
Miner {
easiness,
cuckoo,
graph,
proof_size,
sizeshift,
}
}
/// Searches for a solution
pub fn mine(&mut self) -> Result<Proof, Error> {
let mut us = [0; MAXPATHLEN];
let mut vs = [0; MAXPATHLEN];
for nonce in 0..self.easiness {
us[0] = self.cuckoo.new_node(nonce, 0) as u32;
vs[0] = self.cuckoo.new_node(nonce, 1) as u32;
let u = self.graph[us[0] as usize];
let v = self.graph[vs[0] as usize];
if us[0] == 0 {
continue; // ignore duplicate edges
}
let nu = self.path(u, &mut us)? as usize;
let nv = self.path(v, &mut vs)? as usize;
let sol = self.find_sol(nu, &us, nv, &vs);
match sol {
CycleSol::ValidProof(res) => {
let mut proof = Proof::new(map_vec!(res.to_vec(), |&n| n as u64));
proof.cuckoo_sizeshift = self.sizeshift;
return Ok(proof);
}
CycleSol::InvalidCycle(_) => continue,
CycleSol::NoCycle => {
self.update_graph(nu, &us, nv, &vs);
}
}
}
Err(Error::NoSolution)
}
fn path(&self, mut u: u32, us: &mut [u32]) -> Result<u32, Error> {
let mut nu = 0; let mut nu = 0;
while u != 0 { while u != T::zero() {
nu += 1; nu += 1;
if nu >= MAXPATHLEN { if nu >= MAXPATHLEN {
while nu != 0 && us[(nu - 1) as usize] != u { while nu != 0 && us[(nu - 1) as usize] != u {
nu -= 1; nu -= 1;
} }
return Err(Error::Path); return Err(ErrorKind::Path)?;
} }
us[nu as usize] = u; us[nu as usize] = u;
u = self.graph[u as usize]; u = self.graph[to_usize!(u)];
} }
Ok(nu as u32) Ok(to_edge!(nu))
} }
fn update_graph(&mut self, mut nu: usize, us: &[u32], mut nv: usize, vs: &[u32]) { fn update_graph(
&mut self,
mut nu: usize,
us: &[T],
mut nv: usize,
vs: &[T],
) -> Result<(), Error> {
if nu < nv { if nu < nv {
while nu != 0 { while nu != 0 {
nu -= 1; nu -= 1;
self.graph[us[nu + 1] as usize] = us[nu]; self.graph[to_usize!(us[nu + 1])] = us[nu];
} }
self.graph[us[0] as usize] = vs[0]; self.graph[to_usize!(us[0])] = vs[0];
} else { } else {
while nv != 0 { while nv != 0 {
nv -= 1; nv -= 1;
self.graph[vs[nv + 1] as usize] = vs[nv]; self.graph[to_usize!(vs[nv + 1])] = vs[nv];
} }
self.graph[vs[0] as usize] = us[0]; self.graph[to_usize!(vs[0])] = us[0];
} }
Ok(())
} }
fn find_sol(&mut self, mut nu: usize, us: &[u32], mut nv: usize, vs: &[u32]) -> CycleSol { fn find_sol(
&mut self,
mut nu: usize,
us: &[T],
mut nv: usize,
vs: &[T],
) -> Result<Vec<T>, Error> {
if us[nu] == vs[nv] { if us[nu] == vs[nv] {
let min = cmp::min(nu, nv); let min = cmp::min(nu, nv);
nu -= min; nu -= min;
@ -253,71 +194,142 @@ impl Miner {
nv += 1; nv += 1;
} }
if nu + nv + 1 == self.proof_size { if nu + nv + 1 == self.proof_size {
self.solution(&us, nu as u32, &vs, nv as u32) self.solution(&us, nu as u64, &vs, nv as u64)
} else { } else {
CycleSol::InvalidCycle(nu + nv + 1) Err(ErrorKind::InvalidCycle(nu + nv + 1))?
} }
} else { } else {
CycleSol::NoCycle Err(ErrorKind::NoCycle)?
} }
} }
fn solution(&mut self, us: &[u32], mut nu: u32, vs: &[u32], mut nv: u32) -> CycleSol { fn solution(&mut self, us: &[T], mut nu: u64, vs: &[T], mut nv: u64) -> Result<Vec<T>, Error> {
let mut cycle = HashSet::new(); let mut cycle = HashSet::new();
cycle.insert(Edge { cycle.insert(Edge { u: us[0], v: vs[0] });
u: us[0] as u64,
v: vs[0] as u64,
});
while nu != 0 { while nu != 0 {
// u's in even position; v's in odd // u's in even position; v's in odd
nu -= 1; nu = nu - 1;
cycle.insert(Edge { cycle.insert(Edge {
u: us[((nu + 1) & !1) as usize] as u64, u: us[((nu + 1) & !1) as usize],
v: us[(nu | 1) as usize] as u64, v: us[(nu | 1) as usize],
}); });
} }
while nv != 0 { while nv != 0 {
// u's in odd position; v's in even // u's in odd position; v's in even
nv -= 1; nv -= 1;
cycle.insert(Edge { cycle.insert(Edge {
u: vs[(nv | 1) as usize] as u64, u: vs[(nv | 1) as usize],
v: vs[((nv + 1) & !1) as usize] as u64, v: vs[((nv + 1) & !1) as usize],
}); });
} }
let mut n = 0; let mut n = 0;
let mut sol = vec![0; self.proof_size]; let mut sol = vec![T::zero(); self.proof_size];
for nonce in 0..self.easiness { for nonce in 0..to_usize!(self.easiness) {
let edge = self.cuckoo.new_edge(nonce); let edge = self.new_edge(to_edge!(nonce))?;
if cycle.contains(&edge) { if cycle.contains(&edge) {
sol[n] = nonce as u32; sol[n] = to_edge!(nonce);
n += 1; n += 1;
cycle.remove(&edge); cycle.remove(&edge);
} }
} }
return if n == self.proof_size { return if n == self.proof_size {
CycleSol::ValidProof(sol) Ok(sol)
} else { } else {
CycleSol::NoCycle Err(ErrorKind::NoCycle)?
}; };
} }
/// Searches for a solution (simple implementation)
pub fn find_cycles_impl(&mut self) -> Result<Vec<Proof>, Error> {
let mut us = [T::zero(); MAXPATHLEN];
let mut vs = [T::zero(); MAXPATHLEN];
for nonce in 0..to_usize!(self.easiness) {
us[0] = self.new_node(to_edge!(nonce), 0)?;
vs[0] = self.new_node(to_edge!(nonce), 1)?;
let u = self.graph[to_usize!(us[0])];
let v = self.graph[to_usize!(vs[0])];
if us[0] == T::zero() {
continue; // ignore duplicate edges
}
let nu = to_usize!(self.path(u, &mut us)?);
let nv = to_usize!(self.path(v, &mut vs)?);
let sol = self.find_sol(nu, &us, nv, &vs);
match sol {
Ok(s) => {
let mut proof = Proof::new(map_vec!(s.to_vec(), |&n| n.to_u64().unwrap_or(0)));
proof.cuckoo_sizeshift = self.edge_bits;
return Ok(vec![proof]);
}
Err(e) => match e.kind() {
ErrorKind::InvalidCycle(_) => continue,
ErrorKind::NoCycle => self.update_graph(nu, &us, nv, &vs)?,
_ => return Err(e),
},
}
}
Err(ErrorKind::NoSolution)?
} }
/// Utility to transform a 8 bytes of a byte array into a u64. /// Assuming increasing nonces all smaller than easiness, verifies the
fn u8_to_u64(p: &[u8], i: usize) -> u64 { /// nonces form a cycle in a Cuckoo graph. Each nonce generates an edge, we
(p[i] as u64) /// build the nodes on both side of that edge and count the connections.
| (p[i + 1] as u64) << 8 pub fn verify_impl(&self, proof: &Proof) -> Result<(), Error> {
| (p[i + 2] as u64) << 16 let easiness = to_u64!(self.easiness);
| (p[i + 3] as u64) << 24 let nonces = &proof.nonces;
| (p[i + 4] as u64) << 32 let mut us = vec![T::zero(); proof.proof_size()];
| (p[i + 5] as u64) << 40 let mut vs = vec![T::zero(); proof.proof_size()];
| (p[i + 6] as u64) << 48 for n in 0..proof.proof_size() {
| (p[i + 7] as u64) << 56 if nonces[n] >= easiness || (n != 0 && nonces[n] <= nonces[n - 1]) {
return Err(ErrorKind::Verification("edge wrong size".to_owned()))?;
}
us[n] = self.new_node(to_edge!(nonces[n]), 0)?;
vs[n] = self.new_node(to_edge!(nonces[n]), 1)?;
}
let mut i = 0;
let mut count = proof.proof_size();
loop {
let mut j = i;
for k in 0..proof.proof_size() {
// find unique other j with same vs[j]
if k != i && vs[k] == vs[i] {
if j != i {
return Err(ErrorKind::Verification("".to_owned()))?;
}
j = k;
}
}
if j == i {
return Err(ErrorKind::Verification("".to_owned()))?;
}
i = j;
for k in 0..proof.proof_size() {
// find unique other i with same us[i]
if k != j && us[k] == us[j] {
if i != j {
return Err(ErrorKind::Verification("".to_owned()))?;
}
i = k;
}
}
if i == j {
return Err(ErrorKind::Verification("".to_owned()))?;
}
count -= 2;
if i == 0 {
break;
}
}
match count == 0 {
true => Ok(()),
false => Err(ErrorKind::Verification("Invalid solution".to_owned()))?,
}
}
} }
#[cfg(test)] #[cfg(test)]
mod test { mod test {
use super::*; use super::*;
use blake2;
static V1: [u64; 42] = [ static V1: [u64; 42] = [
0x3bbd, 0x4e96, 0x1013b, 0x1172b, 0x1371b, 0x13e6a, 0x1aaa6, 0x1b575, 0x1e237, 0x1ee88, 0x3bbd, 0x4e96, 0x1013b, 0x1172b, 0x1371b, 0x13e6a, 0x1aaa6, 0x1b575, 0x1e237, 0x1ee88,
@ -350,99 +362,119 @@ mod test {
0x701f37b, 0x701f37b,
]; ];
fn blake2(data: &[u8]) -> blake2::blake2b::Blake2bResult { #[test]
blake2::blake2b::blake2b(32, &[], data) fn cuckoo_context() {
let ret = mine20_vectors::<u32>();
if let Err(r) = ret {
panic!("mine20_vectors u32: Error: {}", r);
}
let ret = mine20_vectors::<u64>();
if let Err(r) = ret {
panic!("mine20_vectors u64: Error: {}", r);
}
let ret = validate20_vectors::<u32>();
if let Err(r) = ret {
panic!("validate20_vectors u32: Error: {}", r);
}
let ret = validate20_vectors::<u64>();
if let Err(r) = ret {
panic!("validate20_vectors u64: Error: {}", r);
}
let ret = validate_fail::<u32>();
if let Err(r) = ret {
panic!("validate_fail u32: Error: {}", r);
}
let ret = validate_fail::<u64>();
if let Err(r) = ret {
panic!("validate_fail u64: Error: {}", r);
}
let ret = mine16_validate::<u32>();
if let Err(r) = ret {
panic!("mine16_validate u32: Error: {}", r);
}
let ret = mine16_validate::<u64>();
if let Err(r) = ret {
panic!("mine16_validate u64: Error: {}", r);
}
} }
/// Find a 42-cycle on Cuckoo20 at 75% easiness and verify against a few /// Find a 42-cycle on Cuckoo20 at 75% easiness and verify against a few
/// known cycle proofs /// known cycle proofs
/// generated by other implementations. /// generated by other implementations.
#[test] fn mine20_vectors<T>() -> Result<(), Error>
fn mine20_vectors() { where
let nonces1 = Miner::from_hash(blake2(&[49]).as_bytes(), 75, 42, 20) T: EdgeType,
.mine() {
.unwrap(); let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 75, 10)?;
cuckoo_ctx.set_header_nonce([49].to_vec(), None, true)?;
let res = cuckoo_ctx.find_cycles()?;
let mut proof = Proof::new(V1.to_vec()); let mut proof = Proof::new(V1.to_vec());
proof.cuckoo_sizeshift = 20; proof.cuckoo_sizeshift = 20;
assert_eq!(proof, nonces1); assert_eq!(proof, res[0]);
let nonces2 = Miner::from_hash(blake2(&[50]).as_bytes(), 70, 42, 20) let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 70, 10)?;
.mine() cuckoo_ctx.set_header_nonce([50].to_vec(), None, true)?;
.unwrap(); let res = cuckoo_ctx.find_cycles()?;
let mut proof = Proof::new(V2.to_vec()); let mut proof = Proof::new(V2.to_vec());
proof.cuckoo_sizeshift = 20; proof.cuckoo_sizeshift = 20;
assert_eq!(proof, nonces2); assert_eq!(proof, res[0]);
let nonces3 = Miner::from_hash(blake2(&[51]).as_bytes(), 70, 42, 20) //re-use context
.mine() cuckoo_ctx.set_header_nonce([51].to_vec(), None, true)?;
.unwrap(); let res = cuckoo_ctx.find_cycles()?;
let mut proof = Proof::new(V3.to_vec()); let mut proof = Proof::new(V3.to_vec());
proof.cuckoo_sizeshift = 20; proof.cuckoo_sizeshift = 20;
assert_eq!(proof, nonces3); assert_eq!(proof, res[0]);
Ok(())
} }
#[test] fn validate20_vectors<T>() -> Result<(), Error>
fn validate20_vectors() { where
assert!( T: EdgeType,
Cuckoo::from_hash(blake2(&[49]).as_bytes(), 20) {
.verify(&Proof::new(V1.to_vec().clone()), 75) let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 75, 10)?;
); cuckoo_ctx.set_header_nonce([49].to_vec(), None, false)?;
assert!( assert!(cuckoo_ctx.verify(&Proof::new(V1.to_vec().clone())).is_ok());
Cuckoo::from_hash(blake2(&[50]).as_bytes(), 20) let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 70, 10)?;
.verify(&Proof::new(V2.to_vec().clone()), 70) cuckoo_ctx.set_header_nonce([50].to_vec(), None, false)?;
); assert!(cuckoo_ctx.verify(&Proof::new(V2.to_vec().clone())).is_ok());
assert!( cuckoo_ctx.set_header_nonce([51].to_vec(), None, false)?;
Cuckoo::from_hash(blake2(&[51]).as_bytes(), 20) assert!(cuckoo_ctx.verify(&Proof::new(V3.to_vec().clone())).is_ok());
.verify(&Proof::new(V3.to_vec().clone()), 70) Ok(())
);
} }
/// Just going to disable this for now, as it's painful to try and get a fn validate_fail<T>() -> Result<(), Error>
/// valid cuckoo28 vector (TBD: 30 is more relevant now anyhow) where
#[test] T: EdgeType,
#[ignore] {
fn validate28_vectors() {
let mut test_header = [0; 32];
test_header[0] = 24;
assert!(Cuckoo::from_hash(&test_header, 28).verify(&Proof::new(V4.to_vec().clone()), 50));
}
#[test]
fn validate_fail() {
// edge checks // edge checks
assert!( let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 75, 10)?;
!Cuckoo::from_hash(blake2(&[49]).as_bytes(), 20).verify(&Proof::new(vec![0; 42]), 75) cuckoo_ctx.set_header_nonce([49].to_vec(), None, false)?;
); // edge checks
assert!( assert!(!cuckoo_ctx.verify(&Proof::new(vec![0; 42])).is_ok());
!Cuckoo::from_hash(blake2(&[49]).as_bytes(), 20) assert!(!cuckoo_ctx.verify(&Proof::new(vec![0xffff; 42])).is_ok());
.verify(&Proof::new(vec![0xffff; 42]), 75)
);
// wrong data for proof // wrong data for proof
assert!( cuckoo_ctx.set_header_nonce([50].to_vec(), None, false)?;
!Cuckoo::from_hash(blake2(&[50]).as_bytes(), 20) assert!(!cuckoo_ctx.verify(&Proof::new(V1.to_vec().clone())).is_ok());
.verify(&Proof::new(V1.to_vec().clone()), 75)
);
let mut test_header = [0; 32]; let mut test_header = [0; 32];
test_header[0] = 24; test_header[0] = 24;
assert!( let mut cuckoo_ctx = CuckooContext::<T>::new(20, 42, 50, 10)?;
!Cuckoo::from_hash(blake2(&test_header).as_bytes(), 20) cuckoo_ctx.set_header_nonce(test_header.to_vec(), None, false)?;
.verify(&Proof::new(V4.to_vec().clone()), 50) assert!(!cuckoo_ctx.verify(&Proof::new(V4.to_vec().clone())).is_ok());
); Ok(())
} }
#[test] fn mine16_validate<T>() -> Result<(), Error>
fn mine20_validate() { where
// cuckoo20 T: EdgeType,
{
for n in 1..5 { for n in 1..5 {
let h = [n; 32]; let h = [n; 32];
let nonces = Miner::from_hash(&h, 75, 42, 20).mine().unwrap(); let mut cuckoo_ctx = CuckooContext::<T>::new(16, 42, 75, 10)?;
assert!(Cuckoo::from_hash(&h, 20).verify(&nonces, 75)); cuckoo_ctx.set_header_nonce(h.to_vec(), None, false)?;
} let res = cuckoo_ctx.find_cycles()?;
// cuckoo18 assert!(cuckoo_ctx.verify(&res[0]).is_ok())
for n in 1..5 {
let h = [n; 32];
let nonces = Miner::from_hash(&h, 75, 42, 18).mine().unwrap();
assert!(Cuckoo::from_hash(&h, 18).verify(&nonces, 75));
} }
Ok(())
} }
} }

106
core/src/pow/error.rs Normal file
View file

@ -0,0 +1,106 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Cuckatoo specific errors
use failure::{Backtrace, Context, Fail};
use std::fmt::{self, Display};
use std::io;
/// Cuckatoo solver or validation error
#[derive(Debug)]
pub struct Error {
inner: Context<ErrorKind>,
}
#[derive(Clone, Debug, Eq, Fail, PartialEq)]
/// Libwallet error types
pub enum ErrorKind {
/// Verification error
#[fail(display = "Verification Error: {}", _0)]
Verification(String),
/// Failure to cast from/to generic integer type
#[fail(display = "IntegerCast")]
IntegerCast,
/// IO Error
#[fail(display = "IO Error")]
IOError,
/// Unexpected Edge Error
#[fail(display = "Edge Addition Error")]
EdgeAddition,
/// Path Error
#[fail(display = "Path Error")]
Path,
/// Invalid cycle
#[fail(display = "Invalid Cycle length: {}", _0)]
InvalidCycle(usize),
/// No Cycle
#[fail(display = "No Cycle")]
NoCycle,
/// No Solution
#[fail(display = "No Solution")]
NoSolution,
}
impl Fail for Error {
fn cause(&self) -> Option<&Fail> {
self.inner.cause()
}
fn backtrace(&self) -> Option<&Backtrace> {
self.inner.backtrace()
}
}
impl Display for Error {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
Display::fmt(&self.inner, f)
}
}
impl Error {
/// Return errorkind
pub fn kind(&self) -> ErrorKind {
self.inner.get_context().clone()
}
}
impl From<ErrorKind> for Error {
fn from(kind: ErrorKind) -> Error {
Error {
inner: Context::new(kind),
}
}
}
impl From<Context<ErrorKind>> for Error {
fn from(inner: Context<ErrorKind>) -> Error {
Error { inner: inner }
}
}
impl From<fmt::Error> for Error {
fn from(_error: fmt::Error) -> Error {
Error {
inner: Context::new(ErrorKind::IntegerCast),
}
}
}
impl From<io::Error> for Error {
fn from(_error: io::Error) -> Error {
Error {
inner: Context::new(ErrorKind::IOError),
}
}
}

View file

@ -30,12 +30,17 @@
extern crate blake2_rfc as blake2; extern crate blake2_rfc as blake2;
extern crate chrono; extern crate chrono;
extern crate num;
extern crate rand; extern crate rand;
extern crate serde; extern crate serde;
extern crate grin_util as util; extern crate grin_util as util;
#[macro_use]
mod common;
pub mod cuckatoo;
pub mod cuckoo; pub mod cuckoo;
mod error;
mod siphash; mod siphash;
mod types; mod types;
@ -44,15 +49,26 @@ use consensus;
use core::{Block, BlockHeader}; use core::{Block, BlockHeader};
use genesis; use genesis;
use global; use global;
use pow::cuckoo::{Cuckoo, Error};
pub use self::common::EdgeType;
pub use self::types::*; pub use self::types::*;
pub use pow::cuckatoo::CuckatooContext;
pub use pow::cuckoo::CuckooContext;
pub use pow::error::Error;
const MAX_SOLS: u32 = 10;
/// Validates the proof of work of a given header, and that the proof of work /// Validates the proof of work of a given header, and that the proof of work
/// satisfies the requirements of the header. /// satisfies the requirements of the header.
pub fn verify_size(bh: &BlockHeader, cuckoo_sz: u8) -> bool { pub fn verify_size(bh: &BlockHeader, cuckoo_sz: u8) -> Result<(), Error> {
Cuckoo::from_hash(bh.pre_pow_hash().as_ref(), cuckoo_sz) let mut ctx = global::create_pow_context::<u64>(
.verify(&bh.pow.proof, consensus::EASINESS as u64) cuckoo_sz,
bh.pow.proof.nonces.len(),
consensus::EASINESS,
MAX_SOLS,
)?;
ctx.set_header_nonce(bh.pre_pow(), None, false)?;
ctx.verify(&bh.pow.proof)
} }
/// Mines a genesis block using the internal miner /// Mines a genesis block using the internal miner
@ -69,7 +85,7 @@ pub fn mine_genesis_block() -> Result<Block, Error> {
let sz = global::min_sizeshift(); let sz = global::min_sizeshift();
let proof_size = global::proofsize(); let proof_size = global::proofsize();
pow_size(&mut gen.header, genesis_difficulty, proof_size, sz).unwrap(); pow_size(&mut gen.header, genesis_difficulty, proof_size, sz)?;
Ok(gen) Ok(gen)
} }
@ -93,8 +109,11 @@ pub fn pow_size(
loop { loop {
// if we found a cycle (not guaranteed) and the proof hash is higher that the // if we found a cycle (not guaranteed) and the proof hash is higher that the
// diff, we're all good // diff, we're all good
if let Ok(proof) = cuckoo::Miner::new(bh, consensus::EASINESS, proof_size, sz).mine() { let mut ctx =
bh.pow.proof = proof; global::create_pow_context::<u32>(sz, proof_size, consensus::EASINESS, MAX_SOLS)?;
ctx.set_header_nonce(bh.pre_pow(), None, true)?;
if let Ok(proofs) = ctx.find_cycles() {
bh.pow.proof = proofs[0].clone();
if bh.pow.to_difficulty() >= diff { if bh.pow.to_difficulty() >= diff {
return Ok(()); return Ok(());
} }
@ -132,6 +151,6 @@ mod test {
).unwrap(); ).unwrap();
assert!(b.header.pow.nonce != 310); assert!(b.header.pow.nonce != 310);
assert!(b.header.pow.to_difficulty() >= Difficulty::one()); assert!(b.header.pow.to_difficulty() >= Difficulty::one());
assert!(verify_size(&b.header, global::min_sizeshift())); assert!(verify_size(&b.header, global::min_sizeshift()).is_ok());
} }
} }

View file

@ -16,7 +16,7 @@
//! Jean-Philippe Aumasson and Daniel J. Bernstein. //! Jean-Philippe Aumasson and Daniel J. Bernstein.
/// Implements siphash 2-4 specialized for a 4 u64 array key and a u64 nonce /// Implements siphash 2-4 specialized for a 4 u64 array key and a u64 nonce
pub fn siphash24(v: [u64; 4], nonce: u64) -> u64 { pub fn siphash24(v: &[u64; 4], nonce: u64) -> u64 {
let mut v0 = v[0]; let mut v0 = v[0];
let mut v1 = v[1]; let mut v1 = v[1];
let mut v2 = v[2]; let mut v2 = v[2];
@ -73,9 +73,9 @@ mod test {
/// the fact that the Java impl uses a long, aka a signed 64 bits number). /// the fact that the Java impl uses a long, aka a signed 64 bits number).
#[test] #[test]
fn hash_some() { fn hash_some() {
assert_eq!(siphash24([1, 2, 3, 4], 10), 928382149599306901); assert_eq!(siphash24(&[1, 2, 3, 4], 10), 928382149599306901);
assert_eq!(siphash24([1, 2, 3, 4], 111), 10524991083049122233); assert_eq!(siphash24(&[1, 2, 3, 4], 111), 10524991083049122233);
assert_eq!(siphash24([9, 7, 6, 7], 12), 1305683875471634734); assert_eq!(siphash24(&[9, 7, 6, 7], 12), 1305683875471634734);
assert_eq!(siphash24([9, 7, 6, 7], 10), 11589833042187638814); assert_eq!(siphash24(&[9, 7, 6, 7], 10), 11589833042187638814);
} }
} }

View file

@ -26,6 +26,36 @@ use core::hash::Hashed;
use global; use global;
use ser::{self, Readable, Reader, Writeable, Writer}; use ser::{self, Readable, Reader, Writeable, Writer};
use pow::common::EdgeType;
use pow::error::Error;
/// Generic trait for a solver/verifier providing common interface into Cuckoo-family PoW
/// Mostly used for verification, but also for test mining if necessary
pub trait PoWContext<T>
where
T: EdgeType,
{
/// Create new instance of context with appropriate parameters
fn new(
edge_bits: u8,
proof_size: usize,
easiness_pct: u32,
max_sols: u32,
) -> Result<Box<Self>, Error>;
/// Sets the header along with an optional nonce at the end
/// solve: whether to set up structures for a solve (true) or just validate (false)
fn set_header_nonce(
&mut self,
header: Vec<u8>,
nonce: Option<u32>,
solve: bool,
) -> Result<(), Error>;
/// find solutions using the stored parameters and header
fn find_cycles(&mut self) -> Result<Vec<Proof>, Error>;
/// Verify a solution with the stored parameters
fn verify(&self, proof: &Proof) -> Result<(), Error>;
}
/// The difficulty is defined as the maximum target divided by the block hash. /// The difficulty is defined as the maximum target divided by the block hash.
#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)] #[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)]
pub struct Difficulty { pub struct Difficulty {

View file

@ -42,7 +42,7 @@ pub enum Error {
/// Error originating from wallet API. /// Error originating from wallet API.
Wallet(wallet::Error), Wallet(wallet::Error),
/// Error originating from the cuckoo miner /// Error originating from the cuckoo miner
Cuckoo(pow::cuckoo::Error), Cuckoo(pow::Error),
} }
impl From<core::block::Error> for Error { impl From<core::block::Error> for Error {
@ -62,8 +62,8 @@ impl From<p2p::Error> for Error {
} }
} }
impl From<pow::cuckoo::Error> for Error { impl From<pow::Error> for Error {
fn from(e: pow::cuckoo::Error) -> Error { fn from(e: pow::Error) -> Error {
Error::Cuckoo(e) Error::Cuckoo(e)
} }
} }

View file

@ -529,7 +529,7 @@ impl StratumServer {
); );
} else { } else {
// Do some validation but dont submit // Do some validation but dont submit
if !pow::verify_size(&b.header, global::min_sizeshift()) { if !pow::verify_size(&b.header, global::min_sizeshift()).is_ok() {
// Return error status // Return error status
error!( error!(
LOGGER, LOGGER,

View file

@ -26,7 +26,7 @@ use common::types::StratumServerConfig;
use core::core::hash::{Hash, Hashed}; use core::core::hash::{Hash, Hashed};
use core::core::verifier_cache::VerifierCache; use core::core::verifier_cache::VerifierCache;
use core::core::{Block, BlockHeader}; use core::core::{Block, BlockHeader};
use core::pow::cuckoo; use core::pow::PoWContext;
use core::{consensus, global}; use core::{consensus, global};
use mining::mine_block; use mining::mine_block;
use pool; use pool;
@ -96,14 +96,16 @@ impl Miner {
let mut iter_count = 0; let mut iter_count = 0;
while head.hash() == *latest_hash && Utc::now().timestamp() < deadline { while head.hash() == *latest_hash && Utc::now().timestamp() < deadline {
if let Ok(proof) = cuckoo::Miner::new( let mut ctx = global::create_pow_context::<u32>(
&b.header,
consensus::EASINESS,
global::proofsize(),
global::min_sizeshift(), global::min_sizeshift(),
).mine() global::proofsize(),
{ consensus::EASINESS,
b.header.pow.proof = proof; 10,
).unwrap();
ctx.set_header_nonce(b.header.pre_pow(), None, true)
.unwrap();
if let Ok(proofs) = ctx.find_cycles() {
b.header.pow.proof = proofs[0].clone();
let proof_diff = b.header.pow.to_difficulty(); let proof_diff = b.header.pow.to_difficulty();
if proof_diff >= (b.header.total_difficulty() - head.total_difficulty()) { if proof_diff >= (b.header.total_difficulty() - head.total_difficulty()) {
return true; return true;