[WIP] Aggsig Transactions (#530)

* First steps converting transaction workflow to be aggsig-enable

* integrating updated version of aggsig, which gives greater control over the contents of e

* added wallet transaction test to testing framework to enable testing the whole thing, completed interaction as far as inital response from recipient

* more aggsig work, final signature is produced now

* Construction of aggsig transaction now working to the point of the signature being built

* aggsig transactions working end-to-end in the nominal case

* refactor aggsig verify from commit and fix some tests

* more cleanup and test fixing

* cleaning up automated tests

* test+formatting fix
This commit is contained in:
Yeastplume 2018-01-10 19:36:27 +00:00 committed by GitHub
parent 6a9a584c43
commit 1199ed2cc1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
23 changed files with 721 additions and 206 deletions

View file

@ -335,7 +335,10 @@ fn prepare_fork_block_tx(kc: &Keychain, prev: &BlockHeader, chain: &Chain, diff:
fn prepare_block_nosum(kc: &Keychain, prev: &BlockHeader, diff: u64, txs: Vec<&Transaction>) -> Block {
let key_id = kc.derive_key_id(diff as u32).unwrap();
let mut b = core::core::Block::new(prev, txs, kc, &key_id).unwrap();
let mut b = match core::core::Block::new(prev, txs, kc, &key_id) {
Err(e) => panic!("{:?}",e),
Ok(b) => b
};
b.header.timestamp = prev.timestamp + time::Duration::seconds(60);
b.header.total_difficulty = Difficulty::from_num(diff);
b

View file

@ -341,7 +341,6 @@ impl Block {
kernels.sort();
// calculate the overall Merkle tree and fees (todo?)
Ok(
Block {
header: BlockHeader {
@ -585,7 +584,7 @@ impl Block {
let excess = secp.commit_sum(vec![out_commit], vec![over_commit])?;
let msg = util::secp::Message::from_slice(&[0; secp::constants::MESSAGE_SIZE])?;
let sig = keychain.sign(&msg, &key_id)?;
let sig = keychain.aggsig_sign_from_key_id(&msg, &key_id).unwrap();
let excess_sig = sig.serialize_der(&secp);

View file

@ -25,10 +25,9 @@
//! build::transaction(vec![input_rand(75), output_rand(42), output_rand(32),
//! with_fee(1)])
use util::{secp, static_secp_instance};
use util::{secp, static_secp_instance, kernel_sig_msg};
use core::{Input, Output, SwitchCommitHash, Transaction, DEFAULT_OUTPUT};
use core::transaction::kernel_sig_msg;
use util::LOGGER;
use keychain;
use keychain::{BlindSum, BlindingFactor, Identifier, Keychain};
@ -138,7 +137,7 @@ pub fn transaction(
);
let blind_sum = ctx.keychain.blind_sum(&sum)?;
let msg = secp::Message::from_slice(&kernel_sig_msg(tx.fee, tx.lock_height))?;
let sig = ctx.keychain.sign_with_blinding(&msg, &blind_sum)?;
let sig = Keychain::aggsig_sign_with_blinding(&keychain.secp(), &msg, &blind_sum)?;
let secp = static_secp_instance();
let secp = secp.lock().unwrap();

View file

@ -13,11 +13,9 @@
// limitations under the License.
//! Transactions
use byteorder::{BigEndian, ByteOrder};
use blake2::blake2b::blake2b;
use util::secp::{self, Message, Signature};
use util::static_secp_instance;
use util::{static_secp_instance, kernel_sig_msg};
use util::secp::pedersen::{Commitment, RangeProof};
use std::cmp::Ordering;
use std::ops;
@ -92,14 +90,6 @@ impl From<consensus::Error> for Error {
}
}
/// Construct msg bytes from tx fee and lock_height
pub fn kernel_sig_msg(fee: u64, lock_height: u64) -> [u8; 32] {
let mut bytes = [0; 32];
BigEndian::write_u64(&mut bytes[16..24], fee);
BigEndian::write_u64(&mut bytes[24..], lock_height);
bytes
}
/// A proof that a transaction sums to zero. Includes both the transaction's
/// Pedersen commitment and the signature, that guarantees that the commitments
/// amount to zero.
@ -166,7 +156,11 @@ impl TxKernel {
let secp = static_secp_instance();
let secp = secp.lock().unwrap();
let sig = try!(Signature::from_der(&secp, &self.excess_sig));
secp.verify_from_commit(&msg, &sig, &self.excess)
let valid = Keychain::aggsig_verify_single_from_commit(&secp, &sig, &msg, &self.excess);
if !valid{
return Err(secp::Error::IncorrectSignature);
}
Ok(())
}
}
@ -330,16 +324,12 @@ impl Transaction {
let secp = static_secp_instance();
let secp = secp.lock().unwrap();
let sig = Signature::from_der(&secp, &self.excess_sig)?;
// pretend the sum is a public key (which it is, being of the form r.G) and
// verify the transaction sig with it
//
// we originally converted the commitment to a key_id here (commitment to zero)
// and then passed the key_id to secp.verify()
// the secp api no longer allows us to do this so we have wrapped the complexity
// of generating a public key from a commitment behind verify_from_commit
secp.verify_from_commit(&msg, &sig, &rsum)?;
let valid = Keychain::aggsig_verify_single_from_commit(&secp, &sig, &msg, &rsum);
if !valid{
return Err(secp::Error::IncorrectSignature);
}
Ok(rsum)
}

View file

@ -32,3 +32,4 @@ itertools = "~0.6.0"
[dev_dependencies]
blake2-rfc = "~0.2.17"
grin_config = { path = "../config" }

View file

@ -38,7 +38,6 @@ extern crate tokio_timer;
extern crate grin_api as api;
extern crate grin_chain as chain;
#[macro_use]
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_p2p as p2p;

View file

@ -161,10 +161,16 @@ impl Server {
_ => {}
}
let skip_sync_wait = match config.skip_sync_wait {
None => false,
Some(b) => b,
};
sync::run_sync(
currently_syncing.clone(),
p2p_server.peers.clone(),
shared_chain.clone(),
skip_sync_wait,
);
evt_handle.spawn(p2p_server.start(evt_handle.clone()).map_err(|_| ()));

View file

@ -30,6 +30,7 @@ pub fn run_sync(
currently_syncing: Arc<AtomicBool>,
peers: p2p::Peers,
chain: Arc<chain::Chain>,
skip_sync_wait: bool,
) {
let chain = chain.clone();
@ -40,7 +41,9 @@ pub fn run_sync(
let mut prev_header_sync = prev_body_sync.clone();
// initial sleep to give us time to peer with some nodes
thread::sleep(Duration::from_secs(30));
if !skip_sync_wait {
thread::sleep(Duration::from_secs(30));
}
loop {
let syncing = needs_syncing(

View file

@ -136,6 +136,10 @@ pub struct ServerConfig {
/// Transaction pool configuration
#[serde(default)]
pub pool_config: pool::PoolConfig,
/// Whether to skip the sync timeout on startup
/// (To assist testing on solo chains)
pub skip_sync_wait: Option<bool>,
}
impl Default for ServerConfig {
@ -150,6 +154,7 @@ impl Default for ServerConfig {
mining_config: Some(pow::types::MinerConfig::default()),
chain_type: ChainTypes::default(),
pool_config: pool::PoolConfig::default(),
skip_sync_wait: Some(true),
}
}
}

View file

@ -34,12 +34,9 @@ use std::default::Default;
use std::fs;
use std::sync::{Arc, Mutex};
use tokio_core::reactor;
use tokio_timer::Timer;
use self::tokio_core::reactor;
use self::tokio_timer::Timer;
use util::secp::Secp256k1;
// TODO - why does this need self here? Missing something somewhere.
use self::keychain::Keychain;
use wallet::WalletConfig;
/// Just removes all results from previous runs
@ -158,7 +155,10 @@ pub struct LocalServerContainer {
pub peer_list: Vec<String>,
// base directory for the server instance
working_dir: String,
pub working_dir: String,
// Wallet configuration
pub wallet_config:WalletConfig,
}
impl LocalServerContainer {
@ -168,6 +168,11 @@ impl LocalServerContainer {
pub fn new(config: LocalServerContainerConfig) -> Result<LocalServerContainer, Error> {
let working_dir = format!("target/test_servers/{}", config.name);
let mut wallet_config = WalletConfig::default();
wallet_config.api_listen_port = format!("{}", config.wallet_port);
wallet_config.check_node_api_http_addr = config.wallet_validating_node_url.clone();
wallet_config.data_file_dir = working_dir.clone();
Ok(
(LocalServerContainer {
config: config,
@ -178,6 +183,7 @@ impl LocalServerContainer {
wallet_is_running: false,
working_dir: working_dir,
peer_list: Vec::new(),
wallet_config:wallet_config,
}),
)
}
@ -206,6 +212,7 @@ impl LocalServerContainer {
seeds: Some(seeds),
seeding_type: seeding_type,
chain_type: core::global::ChainTypes::AutomatedTesting,
skip_sync_wait:Some(true),
..Default::default()
},
&event_loop.handle(),
@ -260,52 +267,93 @@ impl LocalServerContainer {
/// Starts a wallet daemon to receive and returns the
/// listening server url
pub fn run_wallet(&mut self, _duration_in_seconds: u64) {
pub fn run_wallet(&mut self, _duration_in_mills: u64) {
// URL on which to start the wallet listener (i.e. api server)
let url = format!("{}:{}", self.config.base_addr, self.config.wallet_port);
let _url = format!("{}:{}", self.config.base_addr, self.config.wallet_port);
// Just use the name of the server for a seed for now
let seed = format!("{}", self.config.name);
let seed = blake2::blake2b::blake2b(32, &[], seed.as_bytes());
let _seed = blake2::blake2b::blake2b(32, &[], seed.as_bytes());
// TODO - just use from_random_seed here?
let keychain =
Keychain::from_seed(seed.as_bytes()).expect("Error initializing keychain from seed");
println!(
"Starting the Grin wallet receiving daemon on {} ",
self.config.wallet_port
);
let mut wallet_config = WalletConfig::default();
self.wallet_config = WalletConfig::default();
wallet_config.api_listen_port = format!("{}", self.config.wallet_port);
wallet_config.check_node_api_http_addr = self.config.wallet_validating_node_url.clone();
wallet_config.data_file_dir = self.working_dir.clone();
self.wallet_config.api_listen_port = format!("{}", self.config.wallet_port);
self.wallet_config.check_node_api_http_addr = self.config.wallet_validating_node_url.clone();
self.wallet_config.data_file_dir = self.working_dir.clone();
let receive_tx_handler = wallet::WalletReceiver {
config: wallet_config.clone(),
keychain: keychain.clone(),
};
let router = router!(
receive_tx: get "/receive/transaction" => receive_tx_handler,
let _=fs::create_dir_all(self.wallet_config.clone().data_file_dir);
wallet::WalletSeed::init_file(&self.wallet_config);
let wallet_seed =
wallet::WalletSeed::from_file(&self.wallet_config).expect("Failed to read wallet seed file.");
let keychain = wallet_seed.derive_keychain("grin_test").expect(
"Failed to derive keychain from seed file and passphrase.",
);
let mut api_server = api::ApiServer::new("/v1".to_string());
api_server.register_handler(router);
api_server.start(url).unwrap_or_else(|e| {
println!("Failed to start Grin wallet receiver: {}.", e);
});
self.api_server = Some(api_server);
wallet::server::start_rest_apis(self.wallet_config.clone(), keychain);
self.wallet_is_running = true;
}
/// Stops the running wallet server
pub fn send_amount_to(config: &WalletConfig,
amount:&str,
minimum_confirmations: u64,
selection_strategy:&str,
dest: &str){
let amount = core::core::amount_from_hr_string(amount).expect(
"Could not parse amount as a number with optional decimal point.",
);
let wallet_seed =
wallet::WalletSeed::from_file(config).expect("Failed to read wallet seed file.");
let mut keychain = wallet_seed.derive_keychain("grin_test").expect(
"Failed to derive keychain from seed file and passphrase.",
);
let max_outputs = 500;
let result = wallet::issue_send_tx(
config,
&mut keychain,
amount,
minimum_confirmations,
dest.to_string(),
max_outputs,
(selection_strategy == "all"),
);
match result {
Ok(_) => {
println!(
"Tx sent: {} grin to {} (strategy '{}')",
core::core::amount_to_hr_string(amount),
dest,
selection_strategy,
)
}
Err(wallet::Error::NotEnoughFunds(available)) => {
println!(
"Tx not sent: insufficient funds (max: {})",
core::core::amount_to_hr_string(available),
);
}
Err(e) => {
println!("Tx not sent to {}: {:?}", dest, e);
}
};
}
/// Stops the running wallet server
pub fn stop_wallet(&mut self) {
let mut api_server = self.api_server.as_mut().unwrap();
println!("Stop wallet!");
let api_server = self.api_server.as_mut().unwrap();
api_server.stop();
}
@ -497,8 +545,8 @@ impl LocalServerContainerPool {
}
pub fn connect_all_peers(&mut self) {
/// just pull out all currently active servers, build a list,
/// and feed into all servers
// just pull out all currently active servers, build a list,
// and feed into all servers
let mut server_addresses: Vec<String> = Vec::new();
for s in &self.server_containers {
let server_address = format!("{}:{}", s.config.base_addr, s.config.p2p_server_port);

View file

@ -12,9 +12,6 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#[macro_use]
extern crate router;
extern crate grin_api as api;
extern crate grin_chain as chain;
extern crate grin_core as core;
@ -37,20 +34,18 @@ use std::default::Default;
use futures::{Async, Future, Poll};
use futures::task::current;
use tokio_core::reactor;
use tokio_timer::Timer;
use core::consensus;
use core::global;
use core::global::ChainTypes;
use wallet::WalletConfig;
use framework::{LocalServerContainer, LocalServerContainerConfig, LocalServerContainerPool,
use framework::{LocalServerContainerConfig, LocalServerContainerPool,
LocalServerContainerPoolConfig};
/// Testing the frameworks by starting a fresh server, creating a genesis
/// Block and mining into a wallet for a bit
#[test]
fn basic_genesis_mine() {
util::init_test_logger();
global::set_mining_mode(ChainTypes::AutomatedTesting);
let test_name_dir = "genesis_mine";
@ -70,7 +65,8 @@ fn basic_genesis_mine() {
// Create a server to add into the pool
let mut server_config = LocalServerContainerConfig::default();
server_config.start_miner = true;
server_config.start_wallet = true;
server_config.start_wallet = false;
server_config.burn_mining_rewards = true;
pool.create_server(&mut server_config);
pool.run_all_servers();
@ -100,7 +96,8 @@ fn simulate_seeding() {
// Create a first seed server to add into the pool
let mut server_config = LocalServerContainerConfig::default();
// server_config.start_miner = true;
server_config.start_wallet = true;
server_config.start_wallet = false;
server_config.burn_mining_rewards = true;
server_config.is_seeding = true;
pool.create_server(&mut server_config);
@ -279,7 +276,7 @@ fn simulate_full_sync() {
// instantiates 2 servers on different ports
let mut servers = vec![];
for n in 0..2 {
let mut config = grin::ServerConfig {
let config = grin::ServerConfig {
api_http_addr: format!("127.0.0.1:{}", 19000 + n),
db_root: format!("target/{}/grin-sync-{}", test_name_dir, n),
p2p_config: Some(p2p::P2PConfig {

113
grin/tests/wallet.rs Normal file
View file

@ -0,0 +1,113 @@
// Copyright 2017 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.
#[macro_use]
extern crate router;
#[macro_use]
extern crate slog;
extern crate grin_api as api;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_grin as grin;
extern crate grin_p2p as p2p;
extern crate grin_pow as pow;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate grin_config as config;
mod framework;
use std::{thread, time};
use std::sync::{Arc, Mutex};
use framework::{LocalServerContainer,
LocalServerContainerConfig,
LocalServerContainerPoolConfig};
use util::{init_logger, LOGGER};
/// Start 1 node mining and two wallets, then send a few
/// transactions from one to the other
#[test]
fn basic_wallet_transactions() {
let test_name_dir = "test_servers";
core::global::set_mining_mode(core::global::ChainTypes::AutomatedTesting);
framework::clean_all_output(test_name_dir);
let mut log_config = util::LoggingConfig::default();
//log_config.stdout_log_level = util::LogLevel::Trace;
log_config.stdout_log_level = util::LogLevel::Info;
//init_logger(Some(log_config));
util::init_test_logger();
// Run a separate coinbase wallet for coinbase transactions
let mut coinbase_config = LocalServerContainerConfig::default();
coinbase_config.name = String::from("coinbase_wallet");
coinbase_config.wallet_validating_node_url=String::from("http://127.0.0.1:30001");
coinbase_config.wallet_port = 10002;
let coinbase_wallet = Arc::new(Mutex::new(LocalServerContainer::new(coinbase_config).unwrap()));
let coinbase_wallet_config = {
coinbase_wallet.lock().unwrap().wallet_config.clone()
};
let _ = thread::spawn(move || {
let mut w = coinbase_wallet.lock().unwrap();
w.run_wallet(0);
});
let mut recp_config = LocalServerContainerConfig::default();
recp_config.name = String::from("target_wallet");
recp_config.wallet_validating_node_url=String::from("http://127.0.0.1:30001");
recp_config.wallet_port = 20002;
let target_wallet = Arc::new(Mutex::new(LocalServerContainer::new(recp_config).unwrap()));
let target_wallet_cloned = target_wallet.clone();
let recp_wallet_config = {
target_wallet.lock().unwrap().wallet_config.clone()
};
//Start up a second wallet, to receive
let _ = thread::spawn(move || {
let mut w = target_wallet_cloned.lock().unwrap();
w.run_wallet(0);
});
// Spawn server and let it run for a bit
let _ = thread::spawn(move || {
let mut server_config = LocalServerContainerConfig::default();
server_config.name = String::from("server_one");
server_config.p2p_server_port = 30000;
server_config.api_server_port = 30001;
server_config.start_miner = true;
server_config.start_wallet = false;
server_config.coinbase_wallet_address = String::from(format!(
"http://{}:{}",
server_config.base_addr,
10002
));
let mut server_one = LocalServerContainer::new(server_config).unwrap();
server_one.run_server(120);
});
//Wait for chain to build
thread::sleep(time::Duration::from_millis(5000));
warn!(LOGGER, "Sending 50 Grins to recipient wallet");
LocalServerContainer::send_amount_to(&coinbase_wallet_config, "50.00", 1, "all", "http://127.0.0.1:20002");
//let some more mining happen, make sure nothing pukes
thread::sleep(time::Duration::from_millis(5000));
//send some cash right back
LocalServerContainer::send_amount_to(&recp_wallet_config, "25.00", 1, "all", "http://127.0.0.1:10002");
thread::sleep(time::Duration::from_millis(5000));
}

View file

@ -18,14 +18,15 @@ use std::sync::{Arc, RwLock};
use util::secp;
use util::secp::{Message, Secp256k1, Signature};
use util::secp::key::SecretKey;
use util::secp::key::{SecretKey, PublicKey};
use util::secp::pedersen::{Commitment, ProofMessage, ProofInfo, RangeProof};
use util::secp::aggsig;
use util::logger::LOGGER;
use util::kernel_sig_msg;
use blake2;
use blind::{BlindSum, BlindingFactor};
use extkey::{self, Identifier};
#[derive(PartialEq, Eq, Clone, Debug)]
pub enum Error {
ExtendedKey(extkey::Error),
@ -45,10 +46,23 @@ impl From<extkey::Error> for Error {
}
}
/// Holds internal information about an aggsig operation
#[derive(Clone, Debug)]
pub struct AggSigTxContext {
// Secret key (of which public is shared)
pub sec_key: SecretKey,
// Secret nonce (of which public is shared)
// (basically a SecretKey)
pub sec_nonce: SecretKey,
// If I'm the recipient, store my outputs between invocations (that I need to sum)
pub output_ids: Vec<Identifier>,
}
#[derive(Clone, Debug)]
pub struct Keychain {
secp: Secp256k1,
extkey: extkey::ExtendedKey,
pub aggsig_context: Arc<RwLock<Option<AggSigTxContext>>>,
key_overrides: HashMap<Identifier, SecretKey>,
key_derivation_cache: Arc<RwLock<HashMap<Identifier, u32>>>,
}
@ -77,6 +91,7 @@ impl Keychain {
let keychain = Keychain {
secp: secp,
extkey: extkey,
aggsig_context: Arc::new(RwLock::new(None)),
key_overrides: HashMap::new(),
key_derivation_cache: Arc::new(RwLock::new(HashMap::new())),
};
@ -232,6 +247,138 @@ impl Keychain {
Ok(BlindingFactor::new(blinding))
}
pub fn aggsig_create_context(&self, sec_key:SecretKey) {
let mut context = self.aggsig_context.write().unwrap();
*context = Some(AggSigTxContext{
sec_key: sec_key,
sec_nonce: aggsig::export_secnonce_single(&self.secp).unwrap(),
output_ids: vec![],
});
}
/// Tracks an output contributing to my excess value (if it needs to
/// be kept between invocations
pub fn aggsig_add_output(&self, id: &Identifier){
let mut agg_context=self.aggsig_context.write().unwrap();
let agg_context_write=agg_context.as_mut().unwrap();
agg_context_write.output_ids.push(id.clone());
}
/// Returns all stored outputs
pub fn aggsig_get_outputs(&self) -> Vec<Identifier> {
let context = self.aggsig_context.clone();
let context_read=context.read().unwrap();
let agg_context=context_read.as_ref().unwrap();
agg_context.output_ids.clone()
}
/// Returns private key, private nonce
pub fn aggsig_get_private_keys(&self) -> (SecretKey, SecretKey) {
let context = self.aggsig_context.clone();
let context_read=context.read().unwrap();
let agg_context=context_read.as_ref().unwrap();
(agg_context.sec_key.clone(),
agg_context.sec_nonce.clone())
}
/// Returns public key, public nonce
pub fn aggsig_get_public_keys(&self) -> (PublicKey, PublicKey) {
let context = self.aggsig_context.clone();
let context_read=context.read().unwrap();
let agg_context=context_read.as_ref().unwrap();
(PublicKey::from_secret_key(&self.secp, &agg_context.sec_key).unwrap(),
PublicKey::from_secret_key(&self.secp, &agg_context.sec_nonce).unwrap())
}
/// Note 'secnonce' here is used to perform the signature, while 'pubnonce' just allows you to
/// provide a custom public nonce to include while calculating e
/// nonce_sum is the sum used to decide whether secnonce should be inverted during sig time
pub fn aggsig_sign_single(&self, msg: &Message, secnonce:Option<&SecretKey>, pubnonce: Option<&PublicKey>, nonce_sum: Option<&PublicKey>) -> Result<Signature, Error> {
let context = self.aggsig_context.clone();
let context_read=context.read().unwrap();
let agg_context=context_read.as_ref().unwrap();
let sig = aggsig::sign_single(&self.secp, msg, &agg_context.sec_key, secnonce, pubnonce, nonce_sum)?;
Ok(sig)
}
//Verifies an aggsig signature
pub fn aggsig_verify_single(&self, sig: &Signature, msg: &Message, pubnonce:Option<&PublicKey>, pubkey:&PublicKey, is_partial:bool) -> bool {
aggsig::verify_single(&self.secp, sig, msg, pubnonce, pubkey, is_partial)
}
//Verifies other final sig corresponds with what we're expecting
pub fn aggsig_verify_final_sig_build_msg(&self, sig: &Signature, pubkey: &PublicKey, fee: u64, lock_height:u64) -> bool {
let msg = secp::Message::from_slice(&kernel_sig_msg(fee, lock_height)).unwrap();
self.aggsig_verify_single(sig, &msg, None, pubkey, true)
}
//Verifies other party's sig corresponds with what we're expecting
pub fn aggsig_verify_partial_sig(&self, sig: &Signature, other_pub_nonce:&PublicKey, pubkey:&PublicKey, fee: u64, lock_height:u64) -> bool {
let (_, sec_nonce) = self.aggsig_get_private_keys();
let mut nonce_sum = other_pub_nonce.clone();
let _ = nonce_sum.add_exp_assign(&self.secp, &sec_nonce);
let msg = secp::Message::from_slice(&kernel_sig_msg(fee, lock_height)).unwrap();
self.aggsig_verify_single(sig, &msg, Some(&nonce_sum), pubkey, true)
}
pub fn aggsig_calculate_partial_sig(&self, other_pub_nonce:&PublicKey, fee:u64, lock_height:u64) -> Result<Signature, Error>{
// Add public nonces kR*G + kS*G
let (_, sec_nonce) = self.aggsig_get_private_keys();
let mut nonce_sum = other_pub_nonce.clone();
let _ = nonce_sum.add_exp_assign(&self.secp, &sec_nonce);
let msg = secp::Message::from_slice(&kernel_sig_msg(fee, lock_height))?;
//Now calculate signature using message M=fee, nonce in e=nonce_sum
self.aggsig_sign_single(&msg, Some(&sec_nonce), Some(&nonce_sum), Some(&nonce_sum))
}
/// Helper function to calculate final singature
pub fn aggsig_calculate_final_sig(&self, their_sig: &Signature, our_sig: &Signature, their_pub_nonce: &PublicKey) -> Result<Signature, Error> {
// Add public nonces kR*G + kS*G
let (_, sec_nonce) = self.aggsig_get_private_keys();
let mut nonce_sum = their_pub_nonce.clone();
let _ = nonce_sum.add_exp_assign(&self.secp, &sec_nonce);
let sig = aggsig::add_signatures_single(&self.secp, their_sig, our_sig, &nonce_sum)?;
Ok(sig)
}
/// Helper function to calculate final public key
pub fn aggsig_calculate_final_pubkey(&self, their_public_key: &PublicKey) -> Result<PublicKey, Error> {
let (our_sec_key, _) = self.aggsig_get_private_keys();
let mut pk_sum = their_public_key.clone();
let _ = pk_sum.add_exp_assign(&self.secp, &our_sec_key);
Ok(pk_sum)
}
/// Just a simple sig, creates its own nonce, etc
pub fn aggsig_sign_from_key_id(&self, msg: &Message, key_id: &Identifier) -> Result<Signature, Error> {
let skey = self.derived_key(key_id)?;
let sig = aggsig::sign_single(&self.secp, &msg, &skey, None, None, None)?;
Ok(sig)
}
/// Verifies a sig given a commitment
pub fn aggsig_verify_single_from_commit(secp:&Secp256k1, sig: &Signature, msg: &Message, commit:&Commitment) -> bool {
// Extract the pubkey, unfortunately we need this hack for now, (we just hope one is valid)
// TODO: Create better secp256k1 API to do this
let pubkeys = commit.to_two_pubkeys(secp);
let mut valid=false;
for i in 0..pubkeys.len() {
valid=aggsig::verify_single(secp, &sig, &msg, None, &pubkeys[i], false);
if valid {
break;
}
}
valid
}
/// Just a simple sig, creates its own nonce, etc
pub fn aggsig_sign_with_blinding(secp:&Secp256k1, msg: &Message, blinding:&BlindingFactor) -> Result<Signature, Error> {
let sig = aggsig::sign_single(secp, &msg, &blinding.secret_key(), None, None, None)?;
Ok(sig)
}
pub fn sign(&self, msg: &Message, key_id: &Identifier) -> Result<Signature, Error> {
let skey = self.derived_key(key_id)?;
let sig = self.secp.sign(msg, &skey)?;

View file

@ -31,4 +31,4 @@ mod extkey;
pub use blind::{BlindSum, BlindingFactor};
pub use extkey::{ExtendedKey, Identifier, IDENTIFIER_SIZE};
pub mod keychain;
pub use keychain::{Error, Keychain};
pub use keychain::{Error, Keychain, AggSigTxContext};

View file

@ -737,7 +737,7 @@ mod tests {
// a valid transaction.
let valid_transaction = test_transaction(vec![5, 6], vec![9]);
match write_pool.add_to_memory_pool(test_source(), valid_transaction) {
match write_pool.add_to_memory_pool(test_source(), valid_transaction.clone()) {
Ok(_) => {}
Err(x) => panic!("Unexpected error while adding a valid transaction: {:?}", x),
};
@ -764,9 +764,15 @@ mod tests {
}
};
let already_in_pool = test_transaction(vec![5, 6], vec![9]);
// Note, this used to work as expected, but after aggsig implementation
// creating another transaction with the same inputs/outputs doesn't create
// the same hash ID due to the random nonces in an aggsig. This
// will instead throw a (correct as well) Already spent error. An AlreadyInPool
// error can only come up in the case of the exact same transaction being
// added
//let already_in_pool = test_transaction(vec![5, 6], vec![9]);
match write_pool.add_to_memory_pool(test_source(), already_in_pool) {
match write_pool.add_to_memory_pool(test_source(), valid_transaction) {
Ok(_) => panic!("Expected error when adding already in pool, got Ok"),
Err(x) => {
match x {

View file

@ -33,8 +33,6 @@ extern crate grin_wallet as wallet;
mod client;
use std::thread;
use std::io::Read;
use std::fs::File;
use std::time::Duration;
use std::env::current_dir;
@ -211,10 +209,8 @@ fn main() {
.takes_value(true)))
.subcommand(SubCommand::with_name("send")
.about("Builds a transaction to send someone some coins. By default, \
the transaction will just be printed to stdout. If a destination is \
provided, the command will attempt to contact the receiver at that \
address and send the transaction directly.")
.about("Builds a transaction to send coins and sends it to the specified \
listener directly.")
.arg(Arg::with_name("amount")
.help("Number of coins to send with optional fraction, e.g. 12.423")
.index(1))
@ -417,7 +413,7 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
let passphrase = wallet_args.value_of("pass").expect(
"Failed to read passphrase.",
);
let keychain = wallet_seed.derive_keychain(&passphrase).expect(
let mut keychain = wallet_seed.derive_keychain(&passphrase).expect(
"Failed to derive keychain from seed file and passphrase.",
);
@ -428,7 +424,9 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
}
wallet::server::start_rest_apis(wallet_config, keychain);
}
("receive", Some(receive_args)) => {
// The following is gone for now, as a result of aggsig transactions
// being implemented
/*("receive", Some(receive_args)) => {
let input = receive_args.value_of("input").expect("Input file required");
let mut file = File::open(input).expect("Unable to open transaction file.");
let mut contents = String::new();
@ -444,7 +442,7 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
println!(" * your node isn't running or can't be reached");
println!("\nDetailed error: {:?}", e);
}
}
}*/
("send", Some(send_args)) => {
let amount = send_args.value_of("amount").expect(
"Amount to send required",
@ -461,14 +459,13 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
let selection_strategy = send_args.value_of("selection_strategy").expect(
"Selection strategy required",
);
let mut dest = "stdout";
if let Some(d) = send_args.value_of("dest") {
dest = d;
}
let dest = send_args.value_of("dest").expect(
"Destination wallet address required",
);
let max_outputs = 500;
let result = wallet::issue_send_tx(
&wallet_config,
&keychain,
&mut keychain,
amount,
minimum_confirmations,
dest.to_string(),

View file

@ -10,8 +10,10 @@ slog = { version = "^2.0.12", features = ["max_level_trace", "release_max_level_
slog-term = "^2.2.0"
slog-async = "^2.1.0"
lazy_static = "~0.2.8"
byteorder = "^0.5"
rand = "0.3"
serde = "~1.0.8"
serde_derive = "~1.0.8"
secp256k1zkp = { git = "https://github.com/mimblewimble/rust-secp256k1-zkp", tag="grin_integration_2" }
secp256k1zkp = { git = "https://github.com/mimblewimble/rust-secp256k1-zkp", tag="grin_integration_6" }
#secp256k1zkp = { path = "../../rust-secp256k1-zkp" }

View file

@ -25,7 +25,7 @@
extern crate slog;
extern crate slog_async;
extern crate slog_term;
extern crate byteorder;
extern crate rand;
#[macro_use]
@ -48,12 +48,13 @@ pub mod secp_static;
pub use secp_static::static_secp_instance;
pub mod types;
pub use types::LoggingConfig;
pub use types::{LoggingConfig, LogLevel};
// other utils
use std::cell::{Ref, RefCell};
#[allow(unused_imports)]
use std::ops::Deref;
use byteorder::{BigEndian, ByteOrder};
mod hex;
pub use hex::*;
@ -98,3 +99,11 @@ impl<T> OneTime<T> {
Ref::map(self.inner.borrow(), |o| o.as_ref().unwrap())
}
}
/// Construct msg bytes from tx fee and lock_height
pub fn kernel_sig_msg(fee: u64, lock_height: u64) -> [u8; 32] {
let mut bytes = [0; 32];
BigEndian::write_u64(&mut bytes[16..24], fee);
BigEndian::write_u64(&mut bytes[24..], lock_height);
bytes
}

View file

@ -56,11 +56,11 @@ where
Ok(res)
}
pub fn send_partial_tx(url: &str, partial_tx: &PartialTx) -> Result<(), Error> {
pub fn send_partial_tx(url: &str, partial_tx: &PartialTx) -> Result<PartialTx, Error> {
single_send_partial_tx(url, partial_tx)
}
fn single_send_partial_tx(url: &str, partial_tx: &PartialTx) -> Result<(), Error> {
fn single_send_partial_tx(url: &str, partial_tx: &PartialTx) -> Result<PartialTx, Error> {
let mut core = reactor::Core::new()?;
let client = hyper::Client::new(&core.handle());
@ -69,21 +69,15 @@ fn single_send_partial_tx(url: &str, partial_tx: &PartialTx) -> Result<(), Error
let json = serde_json::to_string(&partial_tx)?;
req.set_body(json);
let work = client.request(req);
let _ = core.run(work).and_then(|res| {
if res.status() == hyper::StatusCode::Ok {
info!(LOGGER, "Transaction sent successfully");
} else {
error!(
LOGGER,
"Error sending transaction - status: {}",
res.status()
);
return Err(hyper::Error::Status);
}
Ok(())
})?;
Ok(())
let work = client.request(req).and_then(|res| {
res.body().concat2().and_then(move |body| {
let partial_tx: PartialTx =
serde_json::from_slice(&body).map_err(|e| io::Error::new(io::ErrorKind::Other, e))?;
Ok(partial_tx)
})
});
let res = core.run(work)?;
Ok(res)
}
/// Makes a single request to the wallet API to create a new coinbase output.

View file

@ -54,7 +54,7 @@ pub mod server;
pub use outputs::show_outputs;
pub use info::show_info;
pub use receiver::{receive_json_tx, receive_json_tx_str, WalletReceiver};
pub use receiver::{WalletReceiver};
pub use sender::{issue_burn_tx, issue_send_tx};
pub use types::{BlockFees, CbData, Error, WalletConfig, WalletReceiveRequest, WalletSeed};
pub use restore::restore;

View file

@ -26,10 +26,9 @@ use api;
use core::consensus::reward;
use core::core::{build, Block, Output, Transaction, TxKernel};
use core::ser;
use keychain::{BlindingFactor, Identifier, Keychain};
use keychain::{Identifier, Keychain};
use types::*;
use util;
use util::LOGGER;
use util::{LOGGER, to_hex, secp};
/// Dummy wrapper for the hex-encoded serialized transaction.
#[derive(Serialize, Deserialize)]
@ -37,41 +36,136 @@ pub struct TxWrapper {
pub tx_hex: String,
}
pub fn receive_json_tx_str(
/// Receive Part 1 of interactive transactions from sender, Sender Initiation
/// Return result of part 2, Recipient Initation, to sender
/// -Receiver receives inputs, outputs xS * G and kS * G
/// -Receiver picks random blinding factors for all outputs being received, computes total blinding
/// excess xR
/// -Receiver picks random nonce kR
/// -Receiver computes Schnorr challenge e = H(M | kR * G + kS * G)
/// -Receiver computes their part of signature, sR = kR + e * xR
/// -Receiver responds with sR, blinding excess xR * G, public nonce kR * G
fn handle_sender_initiation(
config: &WalletConfig,
keychain: &Keychain,
json_tx: &str,
) -> Result<(), Error> {
let partial_tx = serde_json::from_str(json_tx).unwrap();
receive_json_tx(config, keychain, &partial_tx)
partial_tx: &PartialTx
) -> Result<PartialTx, Error> {
let (amount, _sender_pub_blinding, sender_pub_nonce, _sig, tx) = read_partial_tx(keychain, partial_tx)?;
let root_key_id = keychain.root_key_id();
// double check the fee amount included in the partial tx
// we don't necessarily want to just trust the sender
// we could just overwrite the fee here (but we won't) due to the ecdsa sig
let fee = tx_fee(tx.inputs.len(), tx.outputs.len() + 1, None);
if fee != tx.fee {
return Err(Error::FeeDispute {
sender_fee: tx.fee,
recipient_fee: fee,
});
}
let out_amount = amount - fee;
//First step is just to get the excess sum of the outputs we're participating in
//Output and key needs to be stored until transaction finalisation time, somehow
let key_id = WalletData::with_wallet(&config.data_file_dir, |wallet_data| {
let (key_id, derivation) = next_available_key(&wallet_data, keychain);
wallet_data.add_output(OutputData {
root_key_id: root_key_id.clone(),
key_id: key_id.clone(),
n_child: derivation,
value: out_amount,
status: OutputStatus::Unconfirmed,
height: 0,
lock_height: 0,
is_coinbase: false,
});
key_id
})?;
// Still handy for getting the blinding sum
let (_, blind_sum) = build::transaction(
vec![
build::output(out_amount, key_id.clone()),
],
keychain,
)?;
warn!(LOGGER, "Creating new aggsig context");
// Create a new aggsig context
// this will create a new blinding sum and nonce, and store them
keychain.aggsig_create_context(blind_sum.secret_key());
keychain.aggsig_add_output(&key_id);
let sig_part=keychain.aggsig_calculate_partial_sig(&sender_pub_nonce, fee, tx.lock_height).unwrap();
// Build the response, which should contain sR, blinding excess xR * G, public nonce kR * G
let mut partial_tx = build_partial_tx(keychain, amount, Some(sig_part), tx);
partial_tx.phase = PartialTxPhase::ReceiverInitiation;
Ok(partial_tx)
}
/// Receive an already well formed JSON transaction issuance and finalize the
/// transaction, adding our receiving output, to broadcast to the rest of the
/// network.
pub fn receive_json_tx(
/// Receive Part 3 of interactive transactions from sender, Sender Confirmation
/// Return Ok/Error
/// -Receiver receives sS
/// -Receiver verifies sender's sig, by verifying that kS * G + e *xS * G = sS * G
/// -Receiver calculates final sig as s=(sS+sR, kS * G+kR * G)
/// -Receiver puts into TX kernel:
///
/// Signature S
/// pubkey xR * G+xS * G
/// fee (= M)
/// -Receiver sends completed TX to mempool. responds OK to sender
fn handle_sender_confirmation(
config: &WalletConfig,
keychain: &Keychain,
partial_tx: &PartialTx,
) -> Result<(), Error> {
partial_tx: &PartialTx
) -> Result<PartialTx, Error> {
let (amount, sender_pub_blinding, sender_pub_nonce, sender_sig_part, tx) = read_partial_tx(keychain, partial_tx)?;
let sender_sig_part=sender_sig_part.unwrap();
let res = keychain.aggsig_verify_partial_sig(&sender_sig_part, &sender_pub_nonce, &sender_pub_blinding, tx.fee, tx.lock_height);
// reading the partial transaction and finalizing it, adding our output
let (amount, blinding, tx) = read_partial_tx(keychain, partial_tx)?;
let (final_tx, key_id) = receive_transaction(config, keychain, amount, blinding, tx)?;
let tx_hex = util::to_hex(ser::ser_vec(&final_tx).unwrap());
// pushing to the transaction pool, in case of error the output that was
// reserved needs to be removed
let url = format!("{}/v1/pool/push", config.check_node_api_http_addr.as_str());
let res = api::client::post(url.as_str(), &TxWrapper { tx_hex: tx_hex });
if let Err(e) = res {
WalletData::with_wallet(&config.data_file_dir, |wallet_data| {
wallet_data.delete_output(&key_id);
})?;
Err(Error::Node(e))
} else {
Ok(())
if !res {
error!(LOGGER, "Partial Sig from sender invalid.");
return Err(Error::Signature(String::from("Partial Sig from sender invalid.")));
}
//Just calculate our sig part again instead of storing
let our_sig_part=keychain.aggsig_calculate_partial_sig(&sender_pub_nonce, tx.fee, tx.lock_height).unwrap();
// And the final signature
let final_sig=keychain.aggsig_calculate_final_sig(&sender_sig_part, &our_sig_part, &sender_pub_nonce).unwrap();
// Calculate the final public key (for our own sanity check)
let final_pubkey=keychain.aggsig_calculate_final_pubkey(&sender_pub_blinding).unwrap();
//Check our final sig verifies
let res = keychain.aggsig_verify_final_sig_build_msg(&final_sig, &final_pubkey, tx.fee, tx.lock_height);
if !res {
error!(LOGGER, "Final aggregated signature invalid.");
return Err(Error::Signature(String::from("Final aggregated signature invalid.")));
}
let final_tx = build_final_transaction(config, keychain, amount, &final_sig, tx.clone())?;
let tx_hex = to_hex(ser::ser_vec(&final_tx).unwrap());
let url = format!("{}/v1/pool/push", config.check_node_api_http_addr.as_str());
api::client::post(url.as_str(), &TxWrapper { tx_hex: tx_hex })
.map_err(|e| Error::Node(e))?;
// Return what we've actually posted
let mut partial_tx = build_partial_tx(keychain, amount, Some(final_sig), tx);
partial_tx.phase = PartialTxPhase::ReceiverConfirmation;
Ok(partial_tx)
}
/// Component used to receive coins, implements all the receiving end of the
@ -87,14 +181,34 @@ impl Handler for WalletReceiver {
let struct_body = req.get::<bodyparser::Struct<PartialTx>>();
if let Ok(Some(partial_tx)) = struct_body {
receive_json_tx(&self.config, &self.keychain, &partial_tx)
.map_err(|e| {
error!(LOGGER, "Problematic partial tx, looks like this: {:?}", partial_tx);
api::Error::Internal(
format!("Error processing partial transaction: {:?}", e),
)})
.unwrap();
Ok(Response::with(status::Ok))
match partial_tx.phase {
PartialTxPhase::SenderInitiation => {
let resp_tx=handle_sender_initiation(&self.config, &self.keychain, &partial_tx)
.map_err(|e| {
error!(LOGGER, "Phase 1 Sender Initiation -> Problematic partial tx, looks like this: {:?}", partial_tx);
api::Error::Internal(
format!("Error processing partial transaction: {:?}", e),
)})
.unwrap();
let json = serde_json::to_string(&resp_tx).unwrap();
Ok(Response::with((status::Ok, json)))
},
PartialTxPhase::SenderConfirmation => {
let resp_tx=handle_sender_confirmation(&self.config, &self.keychain, &partial_tx)
.map_err(|e| {
error!(LOGGER, "Phase 3 Sender Confirmation -> Problematic partial tx, looks like this: {:?}", partial_tx);
api::Error::Internal(
format!("Error processing partial transaction: {:?}", e),
)})
.unwrap();
let json = serde_json::to_string(&resp_tx).unwrap();
Ok(Response::with((status::Ok, json)))
},
_=> {
error!(LOGGER, "Unhandled Phase: {:?}", partial_tx);
Ok(Response::with((status::BadRequest, "Unhandled Phase")))
}
}
} else {
Ok(Response::with((status::BadRequest, "")))
}
@ -174,33 +288,37 @@ pub fn receive_coinbase(
Ok((out, kern, block_fees))
}
/// Builds a full transaction from the partial one sent to us for transfer
fn receive_transaction(
/// builds a final transaction after the aggregated sig exchange
fn build_final_transaction(
config: &WalletConfig,
keychain: &Keychain,
amount: u64,
blinding: BlindingFactor,
partial: Transaction,
) -> Result<(Transaction, Identifier), Error> {
excess_sig: &secp::Signature,
tx: Transaction,
) -> Result<Transaction, Error> {
let root_key_id = keychain.root_key_id();
// double check the fee amount included in the partial tx
// we don't necessarily want to just trust the sender
// we could just overwrite the fee here (but we won't) due to the ecdsa sig
let fee = tx_fee(partial.inputs.len(), partial.outputs.len() + 1, None);
if fee != partial.fee {
let fee = tx_fee(tx.inputs.len(), tx.outputs.len() + 1, None);
if fee != tx.fee {
return Err(Error::FeeDispute {
sender_fee: partial.fee,
sender_fee: tx.fee,
recipient_fee: fee,
});
}
let out_amount = amount - fee;
// Get output we created in earlier step
// TODO: will just be one for now, support multiple later
let output_vec = keychain.aggsig_get_outputs();
// operate within a lock on wallet data
let (key_id, derivation) = WalletData::with_wallet(&config.data_file_dir, |wallet_data| {
let (key_id, derivation) = next_available_key(&wallet_data, keychain);
let (key_id, derivation) = retrieve_existing_key(&wallet_data, output_vec[0].clone());
wallet_data.add_output(OutputData {
root_key_id: root_key_id.clone(),
@ -216,27 +334,29 @@ fn receive_transaction(
(key_id, derivation)
})?;
let (tx_final, _) = build::transaction(
// Build final transaction, the sum of which should
// be the same as the exchanged excess values
let (mut final_tx, _) = build::transaction(
vec![
build::initial_tx(partial),
build::with_excess(blinding),
build::initial_tx(tx),
build::output(out_amount, key_id.clone()),
// build::with_fee(fee_amount),
],
keychain,
)?;
final_tx.excess_sig = excess_sig.serialize_der(&keychain.secp());
// make sure the resulting transaction is valid (could have been lied to on
// excess).
tx_final.validate()?;
let _ = final_tx.validate()?;
debug!(
LOGGER,
"Received txn and built output - {:?}, {:?}, {}",
"Finalized transaction and built output - {:?}, {:?}, {}",
root_key_id.clone(),
key_id.clone(),
derivation,
);
Ok((tx_final, key_id))
Ok(final_tx)
}

View file

@ -1,4 +1,4 @@
// Copyright 2016 The Grin Developers
// 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.
@ -12,8 +12,6 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use serde_json;
use api;
use client;
use checker;
@ -27,7 +25,7 @@ use util;
/// Issue a new transaction to the provided sender by spending some of our
/// wallet
/// UTXOs. The destination can be "stdout" (for command line) or a URL to the
/// UTXOs. The destination can be "stdout" (for command line) (currently disabled) or a URL to the
/// recipients wallet receiver (to be implemented).
pub fn issue_send_tx(
@ -57,8 +55,16 @@ pub fn issue_send_tx(
max_outputs,
selection_strategy,
)?;
/*
* -Sender picks random blinding factors for all outputs it participates in, computes total blinding excess xS
* -Sender picks random nonce kS
* -Sender posts inputs, outputs, Message M=fee, xS * G and kS * G to Receiver
*/
let partial_tx = build_partial_tx(amount, blind_sum, tx);
// Create a new aggsig context
keychain.aggsig_create_context(blind_sum.secret_key());
let partial_tx = build_partial_tx(keychain, amount, None, tx);
// Closure to acquire wallet lock and lock the coins being spent
// so we avoid accidental double spend attempt.
@ -74,27 +80,56 @@ pub fn issue_send_tx(
wallet_data.delete_output(&change_key);
});
if dest == "stdout" {
// TODO: stdout option removed for now, as it won't work very will with this version of
// aggsig exchange
/*if dest == "stdout" {
let json_tx = serde_json::to_string_pretty(&partial_tx).unwrap();
update_wallet()?;
println!("{}", json_tx);
} else if &dest[..4] == "http" {
let url = format!("{}/v1/receive/transaction", &dest);
debug!(LOGGER, "Posting partial transaction to {}", url);
let res = client::send_partial_tx(&url, &partial_tx);
match res {
Err(_) => {
error!(LOGGER, "Communication with receiver failed. Aborting transaction");
rollback_wallet()?;
return res;
}
Ok(_) => {
update_wallet()?;
}
}
} else {
} else */
if &dest[..4] != "http" {
panic!("dest formatted as {} but send -d expected stdout or http://IP:port", dest);
}
let url = format!("{}/v1/receive/transaction", &dest);
debug!(LOGGER, "Posting partial transaction to {}", url);
let res = client::send_partial_tx(&url, &partial_tx);
if let Err(e) = res {
error!(LOGGER, "Communication with receiver failed on SenderInitiation send. Aborting transaction");
rollback_wallet()?;
return Err(e);
}
/* -Sender receives xR * G, kR * G, sR
* -Sender computes Schnorr challenge e = H(M | kR * G + kS * G)
* -Sender verifies receivers sig, by verifying that kR * G + e * xR * G = sR * G·
* -Sender computes their part of signature, sS = kS + e * xS
* -Sender posts sS to receiver
*/
let (_amount, recp_pub_blinding, recp_pub_nonce, sig, tx) = read_partial_tx(keychain, &res.unwrap())?;
let res = keychain.aggsig_verify_partial_sig(&sig.unwrap(), &recp_pub_nonce, &recp_pub_blinding, tx.fee, lock_height);
if !res {
error!(LOGGER, "Partial Sig from recipient invalid.");
return Err(Error::Signature(String::from("Partial Sig from recipient invalid.")));
}
let sig_part=keychain.aggsig_calculate_partial_sig(&recp_pub_nonce, tx.fee, tx.lock_height).unwrap();
// Build the next stage, containing sS (and our pubkeys again, for the recipient's convenience)
let mut partial_tx = build_partial_tx(keychain, amount, Some(sig_part), tx);
partial_tx.phase = PartialTxPhase::SenderConfirmation;
// And send again
let res = client::send_partial_tx(&url, &partial_tx);
if let Err(e) = res {
error!(LOGGER, "Communication with receiver failed on SenderConfirmation send. Aborting transaction");
rollback_wallet()?;
return Err(e);
}
//All good so
update_wallet()?;
Ok(())
}
@ -199,15 +234,15 @@ fn inputs_and_change(
}
// sender is responsible for setting the fee on the partial tx
// recipient should double check the fee calculation and not blindly trust the
// sender
// recipient should double check the fee calculation and not blindly trust the
// sender
let fee = tx_fee(coins.len(), 2, None);
parts.push(build::with_fee(fee));
// if we are spending 10,000 coins to send 1,000 then our change will be 9,000
// the fee will come out of the amount itself
// if the fee is 80 then the recipient will only receive 920
// but our change will still be 9,000
// the fee will come out of the amount itself
// if the fee is 80 then the recipient will only receive 920
// but our change will still be 9,000
let change = total - amount;
// build inputs using the appropriate derived key_ids

View file

@ -25,7 +25,6 @@ use std::cmp::min;
use hyper;
use serde_json;
use util::secp;
use tokio_core::reactor;
use tokio_retry::Retry;
use tokio_retry::strategy::FibonacciBackoff;
@ -37,6 +36,9 @@ use core::core::{transaction, Transaction};
use core::ser;
use keychain;
use util;
use util::secp;
use util::secp::Signature;
use util::secp::key::PublicKey;
use util::LOGGER;
const DAT_FILE: &'static str = "wallet.dat";
@ -78,6 +80,8 @@ pub enum Error {
Hyper(hyper::Error),
/// Error originating from hyper uri parsing.
Uri(hyper::error::UriError),
/// Error with signatures during exchange
Signature(String),
GenericError(String,)
}
@ -601,24 +605,55 @@ impl WalletData {
}
}
/// Helper in serializing the information a receiver requires to build a
/// transaction.
/// Define the stages of a transaction
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct PartialTx {
amount: u64,
blind_sum: String,
tx: String,
pub enum PartialTxPhase {
SenderInitiation,
ReceiverInitiation,
SenderConfirmation,
ReceiverConfirmation
}
/// Builds a PartialTx from data sent by a sender (not yet completed by the receiver).
/// Helper in serializing the information required during an interactive aggsig
/// transaction
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct PartialTx {
pub phase: PartialTxPhase,
pub amount: u64,
pub public_blind_excess: String,
pub public_nonce: String,
pub part_sig: String,
pub tx: String,
}
/// Builds a PartialTx
/// aggsig_tx_context should contain the private key/nonce pair
/// the resulting partial tx will contain the corresponding public keys
pub fn build_partial_tx(
keychain: &keychain::Keychain,
receive_amount: u64,
blind_sum: keychain::BlindingFactor,
part_sig: Option<secp::Signature>,
tx: Transaction,
) -> PartialTx {
let (pub_excess, pub_nonce) = keychain.aggsig_get_public_keys();
let mut pub_excess = pub_excess.serialize_vec(keychain.secp(), true).clone();
let len = pub_excess.clone().len();
let pub_excess: Vec<_> = pub_excess.drain(0..len).collect();
let mut pub_nonce = pub_nonce.serialize_vec(keychain.secp(), true);
let len = pub_nonce.clone().len();
let pub_nonce: Vec<_> = pub_nonce.drain(0..len).collect();
PartialTx {
phase: PartialTxPhase::SenderInitiation,
amount: receive_amount,
blind_sum: util::to_hex(blind_sum.secret_key().as_ref().to_vec()),
public_blind_excess: util::to_hex(pub_excess),
public_nonce: util::to_hex(pub_nonce),
part_sig: match part_sig {
None => String::from("00"),
Some(p) => util::to_hex(p.serialize_der(&keychain.secp())),
},
tx: util::to_hex(ser::ser_vec(&tx).unwrap()),
}
}
@ -628,14 +663,21 @@ pub fn build_partial_tx(
pub fn read_partial_tx(
keychain: &keychain::Keychain,
partial_tx: &PartialTx,
) -> Result<(u64, keychain::BlindingFactor, Transaction), Error> {
let blind_bin = util::from_hex(partial_tx.blind_sum.clone())?;
let blinding = keychain::BlindingFactor::from_slice(keychain.secp(), &blind_bin[..])?;
) -> Result<(u64, PublicKey, PublicKey, Option<Signature>, Transaction), Error> {
let blind_bin = util::from_hex(partial_tx.public_blind_excess.clone())?;
let blinding = PublicKey::from_slice(keychain.secp(), &blind_bin[..])?;
let nonce_bin = util::from_hex(partial_tx.public_nonce.clone())?;
let nonce = PublicKey::from_slice(keychain.secp(), &nonce_bin[..])?;
let sig_bin = util::from_hex(partial_tx.part_sig.clone())?;
let sig = match sig_bin.len() {
1 => None,
_ => Some(Signature::from_der(keychain.secp(), &sig_bin[..])?),
};
let tx_bin = util::from_hex(partial_tx.tx.clone())?;
let tx = ser::deserialize(&mut &tx_bin[..]).map_err(|_| {
Error::Format("Could not deserialize transaction, invalid format.".to_string())
})?;
Ok((partial_tx.amount, blinding, tx))
Ok((partial_tx.amount, blinding, nonce, sig, tx))
}
/// Amount in request to build a coinbase output.