Wallet LMDB backend (#1151)

* Migrate main node store to LMDB

In preparation to using LMDB as a wallet database, migrate the
node db. There's no point in having 2 key-value stores.
In addition LMDB provides a few advantages as a node db, namely a
much faster build (compared to RocksDb), lesser dependencies and
transactions.

* Migrated p2p store to lmdb, stuff compiles

* More fixes, chain tests starting to pass

* Fixed txhashset rollback messing with block save and general batch delimitation. Chain tests passing.

* rustfmt

* LMDB max map size of 10MB isn't really workable. Half TB seems reasonable.

* Fix wallet tests

* Rather crucial commit was missing

* rustfmt

* Fixing new merged tests following lmdb changes

* rustfmt

* * Make txhashset validation read-only on fast sync to avoid having
a really long open transaction.
* Fix deadlock in new block processing, batch should always be
created within a txhashset lock (when they interact).

* Comment about batch and txhashset interlacing

* Fix store tests to use batch

* Externalize wallet config and seed

* Converted direct read access to file outputs map to an iterator

* Cleaned up and simplified wallet Backend trait:

* No more direct mutable access to internal structures (HashMap)
* Batch interface for all writes
* Remove unneeded read wrapper (read_wallet)

* rustfmt

* First (incomplete) pass at wallet LMDB backend

* Progressing on lmdb backent iml

* Added batch impl for LMDB wallet backend. Pretty much done with it, but not sure how to deal with commit (owned).

* rustfmt

* Wrapping LMDB batch around a refcell to work around borrow rules

* Compilation up to grin chain
This commit is contained in:
Ignotus Peverell 2018-06-22 09:08:06 +01:00 committed by Yeastplume
parent bb95d303f5
commit d189770080
47 changed files with 2374 additions and 2088 deletions

1112
Cargo.lock generated

File diff suppressed because it is too large Load diff

View file

@ -496,21 +496,6 @@ impl Handler for ChainValidationHandler {
} }
} }
/// Temporary - fix header by height index.
/// POST /v1/chain/height-index
pub struct HeaderByHeightHandler {
pub chain: Weak<chain::Chain>,
}
impl Handler for HeaderByHeightHandler {
fn handle(&self, _req: &mut Request) -> IronResult<Response> {
match w(&self.chain).rebuild_header_by_height() {
Ok(_) => Ok(Response::with((status::Ok, ""))),
Err(_) => Ok(Response::with((status::InternalServerError, ""))),
}
}
}
/// Chain compaction handler. Trigger a compaction of the chain state to regain /// Chain compaction handler. Trigger a compaction of the chain state to regain
/// storage space. /// storage space.
/// GET /v1/chain/compact /// GET /v1/chain/compact
@ -733,9 +718,6 @@ pub fn start_rest_apis<T>(
let chain_compact_handler = ChainCompactHandler { let chain_compact_handler = ChainCompactHandler {
chain: chain.clone(), chain: chain.clone(),
}; };
let header_height_handler = HeaderByHeightHandler {
chain: chain.clone(),
};
let chain_validation_handler = ChainValidationHandler { let chain_validation_handler = ChainValidationHandler {
chain: chain.clone(), chain: chain.clone(),
}; };
@ -795,7 +777,6 @@ pub fn start_rest_apis<T>(
chain_compact: get "/chain/compact" => chain_compact_handler, chain_compact: get "/chain/compact" => chain_compact_handler,
chain_validate: get "/chain/validate" => chain_validation_handler, chain_validate: get "/chain/validate" => chain_validation_handler,
chain_outputs: get "/chain/outputs/*" => output_handler, chain_outputs: get "/chain/outputs/*" => output_handler,
header_height: post "/chain/height-index" => header_height_handler,
status: get "/status" => status_handler, status: get "/status" => status_handler,
txhashset_roots: get "/txhashset/*" => txhashset_handler, txhashset_roots: get "/txhashset/*" => txhashset_handler,
pool_info: get "/pool" => pool_info_handler, pool_info: get "/pool" => pool_info_handler,

View file

@ -8,6 +8,7 @@ publish = false
[dependencies] [dependencies]
bitflags = "1" bitflags = "1"
byteorder = "1" byteorder = "1"
lmdb-zero = "0.4.4"
croaring = "0.3" croaring = "0.3"
slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] } slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] }
serde = "1" serde = "1"

View file

@ -20,17 +20,19 @@ use std::fs::File;
use std::sync::{Arc, Mutex, RwLock}; use std::sync::{Arc, Mutex, RwLock};
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use core::core::Committed; use lmdb;
use core::core::hash::{Hash, Hashed}; use core::core::hash::{Hash, Hashed};
use core::core::merkle_proof::MerkleProof; use core::core::merkle_proof::MerkleProof;
use core::core::target::Difficulty; use core::core::target::Difficulty;
use core::core::{Block, BlockHeader, Output, OutputIdentifier, Transaction, TxKernel}; use core::core::{Block, BlockHeader, Output, OutputIdentifier, Transaction,
TxKernel};
use core::global; use core::global;
use grin_store::Error::NotFoundErr; use grin_store::Error::NotFoundErr;
use pipe; use pipe;
use store; use store;
use txhashset; use txhashset;
use types::{BlockMarker, ChainAdapter, ChainStore, Error, Options, Tip}; use types::{BlockMarker, ChainAdapter, Error, Options, Tip};
use util::LOGGER; use util::LOGGER;
use util::secp::pedersen::{Commitment, RangeProof}; use util::secp::pedersen::{Commitment, RangeProof};
@ -123,7 +125,7 @@ impl OrphanBlockPool {
/// maintains locking for the pipeline to avoid conflicting processing. /// maintains locking for the pipeline to avoid conflicting processing.
pub struct Chain { pub struct Chain {
db_root: String, db_root: String,
store: Arc<ChainStore>, store: Arc<store::ChainStore>,
adapter: Arc<ChainAdapter>, adapter: Arc<ChainAdapter>,
head: Arc<Mutex<Tip>>, head: Arc<Mutex<Tip>>,
@ -139,107 +141,28 @@ unsafe impl Sync for Chain {}
unsafe impl Send for Chain {} unsafe impl Send for Chain {}
impl Chain { impl Chain {
/// Check whether the chain exists. If not, the call to 'init' will
/// expect an already mined genesis block. This keeps the chain free
/// from needing to know about the mining implementation
pub fn chain_exists(db_root: String) -> bool {
let chain_store = store::ChainKVStore::new(db_root).unwrap();
match chain_store.head() {
Ok(_) => true,
Err(NotFoundErr) => false,
Err(_) => false,
}
}
/// Initializes the blockchain and returns a new Chain instance. Does a /// Initializes the blockchain and returns a new Chain instance. Does a
/// check on the current chain head to make sure it exists and creates one /// check on the current chain head to make sure it exists and creates one
/// based on the genesis block if necessary. /// based on the genesis block if necessary.
pub fn init( pub fn init(
db_root: String, db_root: String,
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) -> bool,
) -> Result<Chain, Error> { ) -> Result<Chain, Error> {
let chain_store = store::ChainKVStore::new(db_root.clone())?; let chain_store = store::ChainStore::new(db_env)?;
let store = Arc::new(chain_store); let store = Arc::new(chain_store);
// check if we have a head in store, otherwise the genesis block is it
let head = store.head();
// open the txhashset, creating a new one if necessary // open the txhashset, creating a new one if necessary
let mut txhashset = txhashset::TxHashSet::open(db_root.clone(), store.clone(), None)?; let mut txhashset = txhashset::TxHashSet::open(db_root.clone(), store.clone(), None)?;
match head { setup_head(genesis, store.clone(), &mut txhashset)?;
Ok(head) => {
// TODO - consolidate head vs head_header here.
let head_header = store.head_header()?;
let mut head = head;
loop {
// Use current chain tip if we have one.
// Note: We are rewinding and validating against a writeable extension.
// If validation is successful we will truncate the backend files
// to match the provided block header.
let header = store.get_block_header(&head.last_block_h)?;
let res = txhashset::extending(&mut txhashset, |extension| {
debug!(
LOGGER,
"chain: init: rewinding and validating before we start... {} at {}",
header.hash(),
header.height,
);
extension.rewind(&header, &head_header)?;
extension.validate_roots(&header)?;
Ok(())
});
if res.is_ok() {
break;
} else {
// We may have corrupted the MMR backend files
// last time we stopped the node.
// If this appears to be the case
// revert the head to the previous header and try again
let _ = store.delete_block(&header.hash());
let prev_header = store.get_block_header(&head.prev_block_h)?;
let _ = store.setup_height(&prev_header, &head)?;
head = Tip::from_block(&prev_header);
store.save_head(&head)?;
}
}
}
Err(NotFoundErr) => {
let tip = Tip::from_block(&genesis.header);
store.save_block(&genesis)?;
store.setup_height(&genesis.header, &tip)?;
txhashset::extending(&mut txhashset, |extension| {
extension.apply_block(&genesis)?;
Ok(())
})?;
// saving a new tip based on genesis
store.save_head(&tip)?;
info!(
LOGGER,
"chain: init: saved genesis block: {:?}, nonce: {:?}, pow: {:?}",
genesis.hash(),
genesis.header.nonce,
genesis.header.pow,
);
}
Err(e) => return Err(Error::StoreErr(e, "chain init load head".to_owned())),
};
// Now reload the chain head (either existing head or genesis from above) // Now reload the chain head (either existing head or genesis from above)
let head = store.head()?; let head = store.head()?;
// Initialize header_head and sync_head as necessary for chain init.
store.init_head()?;
debug!( debug!(
LOGGER, LOGGER,
"Chain init: {} @ {} [{}]", "Chain init: {} @ {} [{}]",
@ -291,9 +214,9 @@ impl Chain {
let head = self.store let head = self.store
.head() .head()
.map_err(|e| Error::StoreErr(e, "chain load head".to_owned()))?; .map_err(|e| Error::StoreErr(e, "chain load head".to_owned()))?;
let ctx = self.ctx_from_head(head, opts); let mut ctx = self.ctx_from_head(head, opts)?;
let res = pipe::process_block(&b, ctx); let res = pipe::process_block(&b, &mut ctx);
match res { match res {
Ok(Some(ref tip)) => { Ok(Some(ref tip)) => {
@ -378,8 +301,8 @@ impl Chain {
/// Process a block header received during "header first" propagation. /// Process a block header received during "header first" propagation.
pub fn process_block_header(&self, bh: &BlockHeader, opts: Options) -> Result<(), Error> { pub fn process_block_header(&self, bh: &BlockHeader, opts: Options) -> Result<(), Error> {
let header_head = self.get_header_head()?; let header_head = self.get_header_head()?;
let ctx = self.ctx_from_head(header_head, opts); let mut ctx = self.ctx_from_head(header_head, opts)?;
pipe::process_block_header(bh, ctx) pipe::process_block_header(bh, &mut ctx)
} }
/// Attempt to add a new header to the header chain. /// Attempt to add a new header to the header chain.
@ -387,19 +310,24 @@ impl Chain {
pub fn sync_block_header(&self, bh: &BlockHeader, opts: Options) -> Result<Option<Tip>, Error> { pub fn sync_block_header(&self, bh: &BlockHeader, opts: Options) -> Result<Option<Tip>, Error> {
let sync_head = self.get_sync_head()?; let sync_head = self.get_sync_head()?;
let header_head = self.get_header_head()?; let header_head = self.get_header_head()?;
let sync_ctx = self.ctx_from_head(sync_head, opts); let mut sync_ctx = self.ctx_from_head(sync_head, opts)?;
let header_ctx = self.ctx_from_head(header_head, opts); let mut header_ctx = self.ctx_from_head(header_head, opts)?;
pipe::sync_block_header(bh, sync_ctx, header_ctx) let mut batch = self.store.batch()?;
let res = pipe::sync_block_header(bh, &mut sync_ctx, &mut header_ctx, &mut batch);
if res.is_ok() {
batch.commit()?;
}
res
} }
fn ctx_from_head(&self, head: Tip, opts: Options) -> pipe::BlockContext { fn ctx_from_head<'a>(&self, head: Tip, opts: Options) -> Result<pipe::BlockContext, Error> {
pipe::BlockContext { Ok(pipe::BlockContext {
opts: opts, opts: opts,
store: self.store.clone(), store: self.store.clone(),
head: head, head: head,
pow_verifier: self.pow_verifier, pow_verifier: self.pow_verifier,
txhashset: self.txhashset.clone(), txhashset: self.txhashset.clone(),
} })
} }
/// Check if hash is for a known orphan. /// Check if hash is for a known orphan.
@ -497,39 +425,36 @@ impl Chain {
let mut txhashset = self.txhashset.write().unwrap(); let mut txhashset = self.txhashset.write().unwrap();
// Now create an extension from the txhashset and validate // Now create an extension from the txhashset and validate against the
// against the latest block header. // latest block header. Rewind the extension to the specified header to
// Rewind the extension to the specified header to ensure the view is // ensure the view is consistent.
// consistent.
txhashset::extending_readonly(&mut txhashset, |extension| { txhashset::extending_readonly(&mut txhashset, |extension| {
// TODO - is this rewind guaranteed to be redundant now? // TODO - is this rewind guaranteed to be redundant now?
extension.rewind(&header, &header)?; extension.rewind(&header, &header, true, true, true)?;
extension.validate(&header, skip_rproofs)?; extension.validate(&header, skip_rproofs)?;
Ok(()) Ok(())
}) })
} }
/// Sets the txhashset roots on a brand new block by applying the block on /// Sets the txhashset roots on a brand new block by applying the block on
/// the current txhashset state. /// the current txhashset state.
pub fn set_block_roots(&self, b: &mut Block, is_fork: bool) -> Result<(), Error> { pub fn set_txhashset_roots(&self, b: &mut Block, is_fork: bool) -> Result<(), Error> {
let mut txhashset = self.txhashset.write().unwrap(); let mut txhashset = self.txhashset.write().unwrap();
let store = self.store.clone(); let store = self.store.clone();
let (roots, sizes) = txhashset::extending_readonly(&mut txhashset, |extension| { let roots = txhashset::extending_readonly(&mut txhashset, |extension| {
if is_fork { if is_fork {
pipe::rewind_and_apply_fork(b, store, extension)?; pipe::rewind_and_apply_fork(b, store, extension)?;
} }
extension.apply_block(b)?; extension.apply_block(b)?;
Ok((extension.roots(), extension.sizes())) Ok(extension.roots())
})?; })?;
b.header.output_root = roots.output_root; b.header.output_root = roots.output_root;
b.header.range_proof_root = roots.rproof_root; b.header.range_proof_root = roots.rproof_root;
b.header.kernel_root = roots.kernel_root; b.header.kernel_root = roots.kernel_root;
b.header.output_mmr_size = sizes.0;
b.header.kernel_mmr_size = sizes.2;
Ok(()) Ok(())
} }
/// Return a pre-built Merkle proof for the given commitment from the store. /// Return a pre-built Merkle proof for the given commitment from the store.
pub fn get_merkle_proof( pub fn get_merkle_proof(
@ -580,7 +505,7 @@ impl Chain {
let mut txhashset = self.txhashset.write().unwrap(); let mut txhashset = self.txhashset.write().unwrap();
txhashset::extending_readonly(&mut txhashset, |extension| { txhashset::extending_readonly(&mut txhashset, |extension| {
extension.rewind(&header, &head_header)?; extension.rewind(&header, &head_header, true, true, true)?;
extension.snapshot(&header)?; extension.snapshot(&header)?;
Ok(()) Ok(())
})?; })?;
@ -612,34 +537,29 @@ impl Chain {
let header = self.store.get_block_header(&h)?; let header = self.store.get_block_header(&h)?;
txhashset::zip_write(self.db_root.clone(), txhashset_data)?; txhashset::zip_write(self.db_root.clone(), txhashset_data)?;
{
// write the block marker so we can safely rewind to // write the block marker so we can safely rewind to
// the pos for that block when we validate the extension below // the pos for that block when we validate the extension below
let batch = self.store.batch()?;
let marker = BlockMarker { let marker = BlockMarker {
output_pos: rewind_to_output, output_pos: rewind_to_output,
kernel_pos: rewind_to_kernel, kernel_pos: rewind_to_kernel,
}; };
self.store.save_block_marker(&h, &marker)?; batch.save_block_marker(&h, &marker)?;
batch.commit()?;
debug!( }
LOGGER,
"Going to validate new txhashset, might take some time..."
);
let mut txhashset = let mut txhashset =
txhashset::TxHashSet::open(self.db_root.clone(), self.store.clone(), Some(&header))?; txhashset::TxHashSet::open(self.db_root.clone(), self.store.clone(), Some(&header))?;
// first read-only extension, for validation only // all good, prepare a new batch and update all the required records
txhashset::extending_readonly(&mut txhashset, |extension| { let mut batch = self.store.batch()?;
extension.rewind(&header)?; // Note: we are validataing against a writeable extension.
extension.validate(&header, false)?; txhashset::extending(&mut txhashset, &mut batch, |extension| {
Ok(())
})?;
// second real extension to commit the rewind and indexes
txhashset::extending(&mut txhashset, |extension| {
// TODO do we need to rewind here? We have no blocks to rewind // TODO do we need to rewind here? We have no blocks to rewind
// (and we need them for the pos to unremove) // (and we need them for the pos to unremove)
extension.rewind(&header, &header)?; extension.rewind(&header, &header, true, true, true)?;
extension.validate(&header, false)?; let (_output_sum, _kernel_sum) = extension.validate(&header, false)?;
extension.rebuild_index()?; extension.rebuild_index()?;
Ok(()) Ok(())
})?; })?;
@ -649,18 +569,17 @@ impl Chain {
let mut txhashset_ref = self.txhashset.write().unwrap(); let mut txhashset_ref = self.txhashset.write().unwrap();
*txhashset_ref = txhashset; *txhashset_ref = txhashset;
} }
// setup new head // setup new head
{ {
let mut head = self.head.lock().unwrap(); let mut head = self.head.lock().unwrap();
*head = Tip::from_block(&header); *head = Tip::from_block(&header);
let _ = self.store.save_body_head(&head); batch.save_body_head(&head)?;
self.store.save_header_height(&header)?; batch.save_header_height(&header)?;
self.store.build_by_height_index(&header, true)?; batch.build_by_height_index(&header, true)?;
} }
batch.commit()?;
self.check_orphans(header.height + 1); self.check_orphans(header.height + 1);
Ok(()) Ok(())
} }
@ -710,15 +629,14 @@ impl Chain {
); );
let mut count = 0; let mut count = 0;
let mut current = self.store.get_header_by_height(head.height - horizon - 1)?; let mut current = self.store.get_header_by_height(head.height - horizon - 1)?;
let batch = self.store.batch()?;
loop { loop {
match self.store.get_block(&current.hash()) { match self.store.get_block(&current.hash()) {
Ok(b) => { Ok(b) => {
count += 1; count += 1;
batch.delete_block(&b.hash())?;
// TODO - consider wrapping these up in a single fn call? batch.delete_block_marker(&b.hash())?;
self.store.delete_block(&b.hash())?; batch.delete_block_input_bitmap(&b.hash())?;
self.store.delete_block_marker(&b.hash())?;
self.store.delete_block_input_bitmap(&b.hash())?;
} }
Err(NotFoundErr) => { Err(NotFoundErr) => {
break; break;
@ -734,6 +652,7 @@ impl Chain {
Err(e) => return Err(From::from(e)), Err(e) => return Err(From::from(e)),
} }
} }
batch.commit()?;
debug!(LOGGER, "Compaction removed {} blocks, done.", count); debug!(LOGGER, "Compaction removed {} blocks, done.", count);
Ok(()) Ok(())
} }
@ -799,9 +718,12 @@ impl Chain {
/// Reset header_head and sync_head to head of current body chain /// Reset header_head and sync_head to head of current body chain
pub fn reset_head(&self) -> Result<(), Error> { pub fn reset_head(&self) -> Result<(), Error> {
self.store let batch = self.store.batch()?;
batch
.reset_head() .reset_head()
.map_err(|e| Error::StoreErr(e, "chain reset_head".to_owned())) .map_err(|e| Error::StoreErr(e, "chain reset_head".to_owned()))?;
batch.commit()?;
Ok(())
} }
/// Get the tip that's also the head of the chain /// Get the tip that's also the head of the chain
@ -882,12 +804,75 @@ impl Chain {
.block_exists(&h) .block_exists(&h)
.map_err(|e| Error::StoreErr(e, "chain block exists".to_owned())) .map_err(|e| Error::StoreErr(e, "chain block exists".to_owned()))
} }
}
/// Rebuilds height index. Reachable as endpoint POST /chain/height-index
pub fn rebuild_header_by_height(&self) -> Result<(), Error> { fn setup_head(
let head = self.head_header()?; genesis: Block,
self.store store: Arc<store::ChainStore>,
.build_by_height_index(&head, true) txhashset: &mut txhashset::TxHashSet,
.map_err(|e| Error::StoreErr(e, "rebuild header by height index".to_owned())) ) -> Result<(), Error> {
} // check if we have a head in store, otherwise the genesis block is it
let head_res = store.head();
let mut batch = store.batch()?;
let mut head: Tip;
match head_res {
Ok(h) => {
head = h;
let head_header = store.head_header()?;
loop {
// Use current chain tip if we have one.
// Note: We are rewinding and validating against a writeable extension.
// If validation is successful we will truncate the backend files
// to match the provided block header.
let header = store.get_block_header(&head.last_block_h)?;
let res = txhashset::extending(txhashset, &mut batch, |extension| {
extension.rewind(&header, &head_header, true, true, true)?;
extension.validate_roots(&header)?;
debug!(
LOGGER,
"chain: init: rewinding and validating before we start... {} at {}",
header.hash(),
header.height,
);
Ok(())
});
if res.is_ok() {
break;
} else {
// We may have corrupted the MMR backend files last time we stopped the
// node. If this appears to be the case revert the head to the previous
// header and try again
let prev_header = store.get_block_header(&head.prev_block_h)?;
let _ = batch.delete_block(&header.hash());
let _ = batch.setup_height(&prev_header, &head)?;
head = Tip::from_block(&prev_header);
batch.save_head(&head)?;
}
}
}
Err(NotFoundErr) => {
let tip = Tip::from_block(&genesis.header);
batch.save_block(&genesis)?;
batch.setup_height(&genesis.header, &tip)?;
txhashset::extending(txhashset, &mut batch, |extension| {
extension.apply_block(&genesis)?;
Ok(())
})?;
// saving a new tip based on genesis
batch.save_head(&tip)?;
head = tip;
info!(LOGGER, "chain: init: saved genesis: {:?}", genesis.hash());
}
Err(e) => return Err(Error::StoreErr(e, "chain init load head".to_owned())),
};
// Initialize header_head and sync_head as necessary for chain init.
batch.init_sync_head(&head)?;
batch.commit()?;
Ok(())
} }

View file

@ -23,6 +23,7 @@
#[macro_use] #[macro_use]
extern crate bitflags; extern crate bitflags;
extern crate byteorder; extern crate byteorder;
extern crate lmdb_zero as lmdb;
extern crate croaring; extern crate croaring;
extern crate lru_cache; extern crate lru_cache;
extern crate serde; extern crate serde;
@ -46,4 +47,5 @@ pub mod types;
// Re-export the base interface // Re-export the base interface
pub use chain::{Chain, MAX_ORPHAN_SIZE}; pub use chain::{Chain, MAX_ORPHAN_SIZE};
pub use types::{ChainAdapter, ChainStore, Error, Options, Tip}; pub use store::ChainStore;
pub use types::{BlockSums, ChainAdapter, Error, Options, Tip};

View file

@ -26,7 +26,7 @@ use core::global;
use grin_store; use grin_store;
use store; use store;
use txhashset; use txhashset;
use types::{ChainStore, Error, Options, Tip}; use types::{Error, Options, Tip};
use util::LOGGER; use util::LOGGER;
/// Contextual information required to process a new block and either reject or /// Contextual information required to process a new block and either reject or
@ -35,7 +35,7 @@ pub struct BlockContext {
/// The options /// The options
pub opts: Options, pub opts: Options,
/// The store /// The store
pub store: Arc<ChainStore>, pub store: Arc<store::ChainStore>,
/// The head /// The head
pub head: Tip, pub head: Tip,
/// The POW verification function /// The POW verification function
@ -45,9 +45,9 @@ pub struct BlockContext {
} }
/// Runs the block processing pipeline, including validation and finding a /// Runs the block processing pipeline, including validation and finding a
/// place for the new block in the chain. Returns the new /// place for the new block in the chain. Returns the new chain head if
/// chain head if updated. /// updated.
pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Error> { pub fn process_block(b: &Block, ctx: &mut BlockContext) -> Result<Option<Tip>, Error> {
// TODO should just take a promise for a block with a full header so we don't // TODO should just take a promise for a block with a full header so we don't
// spend resources reading the full block when its header is invalid // spend resources reading the full block when its header is invalid
@ -60,9 +60,9 @@ pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Er
b.outputs.len(), b.outputs.len(),
b.kernels.len(), b.kernels.len(),
); );
check_known(b.hash(), &mut ctx)?; check_known(b.hash(), ctx)?;
validate_header(&b.header, &mut ctx)?; validate_header(&b.header, ctx)?;
// now check we actually have the previous block in the store // now check we actually have the previous block in the store
// not just the header but the block itself // not just the header but the block itself
@ -84,8 +84,8 @@ pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Er
} }
// validate the block itself // validate the block itself
// we can do this now before interact with the txhashset // we can do this now before interacting with the txhashset
validate_block(b, &mut ctx)?; let _sums = validate_block(b, ctx)?;
// header and block both valid, and we have a previous block // header and block both valid, and we have a previous block
// so take the lock on the txhashset // so take the lock on the txhashset
@ -97,9 +97,11 @@ pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Er
.head() .head()
.map_err(|e| Error::StoreErr(e, "pipe reload head".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe reload head".to_owned()))?;
let mut batch = ctx.store.batch()?;
// start a chain extension unit of work dependent on the success of the // start a chain extension unit of work dependent on the success of the
// internal validation and saving operations // internal validation and saving operations
let result = txhashset::extending(&mut txhashset, |mut extension| { let _ = txhashset::extending(&mut txhashset, &mut batch, |mut extension| {
// First we rewind the txhashset extension if necessary // First we rewind the txhashset extension if necessary
// to put it into a consistent state for validating the block. // to put it into a consistent state for validating the block.
// We can skip this step if the previous header is the latest header we saw. // We can skip this step if the previous header is the latest header we saw.
@ -108,30 +110,33 @@ pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Er
} }
validate_block_via_txhashset(b, &mut extension)?; validate_block_via_txhashset(b, &mut extension)?;
if !block_has_more_work(b, &ctx.head) {
extension.force_rollback();
}
Ok(())
});
trace!( trace!(
LOGGER, LOGGER,
"pipe: process_block: {} at {} is valid, save and append.", "pipe: process_block: {} at {} is valid, save and append.",
b.hash(), b.hash(),
b.header.height, b.header.height,
); );
add_block(b, &mut batch)?;
add_block(b, &mut ctx)?; let res = update_head(b, &ctx, &mut batch);
let h = update_head(b, &mut ctx)?; if res.is_ok() {
if h.is_none() { batch.commit()?;
extension.force_rollback();
} }
Ok(h) res
});
result
} }
/// Process the block header. /// Process the block header.
/// This is only ever used during sync and uses a context based on sync_head. /// This is only ever used during sync and uses a context based on sync_head.
pub fn sync_block_header( pub fn sync_block_header(
bh: &BlockHeader, bh: &BlockHeader,
mut sync_ctx: BlockContext, sync_ctx: &mut BlockContext,
mut header_ctx: BlockContext, header_ctx: &mut BlockContext,
batch: &mut store::Batch,
) -> Result<Option<Tip>, Error> { ) -> Result<Option<Tip>, Error> {
debug!( debug!(
LOGGER, LOGGER,
@ -140,20 +145,20 @@ pub fn sync_block_header(
bh.height bh.height
); );
validate_header(&bh, &mut sync_ctx)?; validate_header(&bh, sync_ctx)?;
add_block_header(bh, &mut sync_ctx)?; add_block_header(bh, batch)?;
// now update the header_head (if new header with most work) and the sync_head // now update the header_head (if new header with most work) and the sync_head
// (always) // (always)
update_header_head(bh, &mut header_ctx)?; update_header_head(bh, header_ctx, batch)?;
update_sync_head(bh, &mut sync_ctx) update_sync_head(bh, sync_ctx, batch)
} }
/// Process block header as part of "header first" block propagation. /// Process block header as part of "header first" block propagation.
/// We validate the header but we do not store it or update header head based /// We validate the header but we do not store it or update header head based
/// on this. We will update these once we get the block back after requesting /// on this. We will update these once we get the block back after requesting
/// it. /// it.
pub fn process_block_header(bh: &BlockHeader, mut ctx: BlockContext) -> Result<(), Error> { pub fn process_block_header(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<(), Error> {
debug!( debug!(
LOGGER, LOGGER,
"pipe: process_block_header at {} [{}]", "pipe: process_block_header at {} [{}]",
@ -161,8 +166,8 @@ pub fn process_block_header(bh: &BlockHeader, mut ctx: BlockContext) -> Result<(
bh.hash() bh.hash()
); // keep this ); // keep this
check_header_known(bh.hash(), &mut ctx)?; check_header_known(bh.hash(), ctx)?;
validate_header(&bh, &mut ctx) validate_header(&bh, ctx)
} }
/// Quick in-memory check to fast-reject any block header we've already handled /// Quick in-memory check to fast-reject any block header we've already handled
@ -362,19 +367,19 @@ fn validate_block_via_txhashset(b: &Block, ext: &mut txhashset::Extension) -> Re
} }
/// Officially adds the block to our chain. /// Officially adds the block to our chain.
fn add_block(b: &Block, ctx: &mut BlockContext) -> Result<(), Error> { fn add_block(b: &Block, batch: &store::Batch) -> Result<(), Error> {
ctx.store batch
.save_block(b) .save_block(b)
.map_err(|e| Error::StoreErr(e, "pipe save block".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save block".to_owned()))?;
ctx.store batch
.save_block_input_bitmap(&b) .save_block_input_bitmap(&b)
.map_err(|e| Error::StoreErr(e, "pipe save block input bitmap".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save block input bitmap".to_owned()))?;
Ok(()) Ok(())
} }
/// Officially adds the block header to our header chain. /// Officially adds the block header to our header chain.
fn add_block_header(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<(), Error> { fn add_block_header(bh: &BlockHeader, batch: &mut store::Batch) -> Result<(), Error> {
ctx.store batch
.save_block_header(bh) .save_block_header(bh)
.map_err(|e| Error::StoreErr(e, "pipe save header".to_owned())) .map_err(|e| Error::StoreErr(e, "pipe save header".to_owned()))
} }
@ -382,103 +387,73 @@ fn add_block_header(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<(), Erro
/// Directly updates the head if we've just appended a new block to it or handle /// Directly updates the head if we've just appended a new block to it or handle
/// the situation where we've just added enough work to have a fork with more /// the situation where we've just added enough work to have a fork with more
/// work than the head. /// work than the head.
fn update_head(b: &Block, ctx: &mut BlockContext) -> Result<Option<Tip>, Error> { fn update_head(b: &Block, ctx: &BlockContext, batch: &store::Batch) -> Result<Option<Tip>, Error> {
// if we made a fork with more work than the head (which should also be true // if we made a fork with more work than the head (which should also be true
// when extending the head), update it // when extending the head), update it
let tip = Tip::from_block(&b.header); if block_has_more_work(b, &ctx.head) {
if tip.total_difficulty > ctx.head.total_difficulty {
// update the block height index // update the block height index
ctx.store batch
.setup_height(&b.header, &ctx.head) .setup_height(&b.header, &ctx.head)
.map_err(|e| Error::StoreErr(e, "pipe setup height".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe setup height".to_owned()))?;
// in sync mode, only update the "body chain", otherwise update both the // in sync mode, only update the "body chain", otherwise update both the
// "header chain" and "body chain", updating the header chain in sync resets // "header chain" and "body chain", updating the header chain in sync resets
// all additional "future" headers we've received // all additional "future" headers we've received
let tip = Tip::from_block(&b.header);
if ctx.opts.contains(Options::SYNC) { if ctx.opts.contains(Options::SYNC) {
ctx.store batch
.save_body_head(&tip) .save_body_head(&tip)
.map_err(|e| Error::StoreErr(e, "pipe save body".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save body".to_owned()))?;
} else { } else {
ctx.store batch
.save_head(&tip) .save_head(&tip)
.map_err(|e| Error::StoreErr(e, "pipe save head".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save head".to_owned()))?;
} }
ctx.head = tip.clone();
if b.header.height % 100 == 0 {
info!(
LOGGER,
"pipe: chain head reached {} @ {} [{}]",
b.header.height,
b.header.total_difficulty,
b.hash()
);
} else {
debug!( debug!(
LOGGER, LOGGER,
"pipe: chain head reached {} @ {} [{}]", "pipe: chain head {} @ {}",
b.header.height, b.hash(),
b.header.total_difficulty, b.header.height
b.hash()
); );
}
Ok(Some(tip)) Ok(Some(tip))
} else { } else {
Ok(None) Ok(None)
} }
} }
// Whether the provided block totals more work than the chain tip
fn block_has_more_work(b: &Block, tip: &Tip) -> bool {
let block_tip = Tip::from_block(&b.header);
block_tip.total_difficulty > tip.total_difficulty
}
/// Update the sync head so we can keep syncing from where we left off. /// Update the sync head so we can keep syncing from where we left off.
fn update_sync_head(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<Option<Tip>, Error> { fn update_sync_head(
bh: &BlockHeader,
ctx: &mut BlockContext,
batch: &mut store::Batch,
) -> Result<Option<Tip>, Error> {
let tip = Tip::from_block(bh); let tip = Tip::from_block(bh);
ctx.store batch
.save_sync_head(&tip) .save_sync_head(&tip)
.map_err(|e| Error::StoreErr(e, "pipe save sync head".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save sync head".to_owned()))?;
ctx.head = tip.clone(); ctx.head = tip.clone();
if bh.height % 100 == 0 { debug!(LOGGER, "sync head {} @ {}", bh.hash(), bh.height);
info!(
LOGGER,
"sync head {} @ {} [{}]",
bh.total_difficulty,
bh.height,
bh.hash()
);
} else {
debug!(
LOGGER,
"sync head {} @ {} [{}]",
bh.total_difficulty,
bh.height,
bh.hash()
);
}
Ok(Some(tip)) Ok(Some(tip))
} }
fn update_header_head(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<Option<Tip>, Error> { fn update_header_head(
bh: &BlockHeader,
ctx: &mut BlockContext,
batch: &mut store::Batch,
) -> Result<Option<Tip>, Error> {
let tip = Tip::from_block(bh); let tip = Tip::from_block(bh);
if tip.total_difficulty > ctx.head.total_difficulty { if tip.total_difficulty > ctx.head.total_difficulty {
ctx.store batch
.save_header_head(&tip) .save_header_head(&tip)
.map_err(|e| Error::StoreErr(e, "pipe save header head".to_owned()))?; .map_err(|e| Error::StoreErr(e, "pipe save header head".to_owned()))?;
ctx.head = tip.clone(); ctx.head = tip.clone();
if bh.height % 100 == 0 { debug!(LOGGER, "header head {} @ {}", bh.hash(), bh.height);
info!(
LOGGER,
"header head {} @ {} [{}]",
bh.total_difficulty,
bh.height,
bh.hash()
);
} else {
debug!(
LOGGER,
"header head {} @ {} [{}]",
bh.total_difficulty,
bh.height,
bh.hash()
);
}
Ok(Some(tip)) Ok(Some(tip))
} else { } else {
Ok(None) Ok(None)
@ -491,7 +466,7 @@ fn update_header_head(bh: &BlockHeader, ctx: &mut BlockContext) -> Result<Option
/// the expected state. /// the expected state.
pub fn rewind_and_apply_fork( pub fn rewind_and_apply_fork(
b: &Block, b: &Block,
store: Arc<ChainStore>, store: Arc<store::ChainStore>,
ext: &mut txhashset::Extension, ext: &mut txhashset::Extension,
) -> Result<(), Error> { ) -> Result<(), Error> {
// extending a fork, first identify the block where forking occurred // extending a fork, first identify the block where forking occurred
@ -522,7 +497,7 @@ pub fn rewind_and_apply_fork(
); );
// rewind the sum trees up to the forking block // rewind the sum trees up to the forking block
ext.rewind(&forked_header, &head_header)?; ext.rewind(&forked_header, &head_header, true, true, true)?;
trace!( trace!(
LOGGER, LOGGER,

View file

@ -16,6 +16,7 @@
use std::sync::{Arc, RwLock}; use std::sync::{Arc, RwLock};
use lmdb;
use croaring::Bitmap; use croaring::Bitmap;
use lru_cache::LruCache; use lru_cache::LruCache;
@ -25,8 +26,9 @@ use core::consensus::TargetError;
use core::core::hash::{Hash, Hashed}; use core::core::hash::{Hash, Hashed};
use core::core::target::Difficulty; use core::core::target::Difficulty;
use core::core::{Block, BlockHeader}; use core::core::{Block, BlockHeader};
use grin_store::{self, option_to_not_found, to_key, Error, u64_to_key}; use grin_store as store;
use types::{BlockMarker, ChainStore, Tip}; use grin_store::{option_to_not_found, to_key, Error, u64_to_key};
use types::{BlockMarker, BlockSums, Tip};
const STORE_SUBPATH: &'static str = "chain"; const STORE_SUBPATH: &'static str = "chain";
@ -38,28 +40,254 @@ const SYNC_HEAD_PREFIX: u8 = 's' as u8;
const HEADER_HEIGHT_PREFIX: u8 = '8' as u8; const HEADER_HEIGHT_PREFIX: u8 = '8' as u8;
const COMMIT_POS_PREFIX: u8 = 'c' as u8; const COMMIT_POS_PREFIX: u8 = 'c' as u8;
const BLOCK_MARKER_PREFIX: u8 = 'm' as u8; const BLOCK_MARKER_PREFIX: u8 = 'm' as u8;
const BLOCK_SUMS_PREFIX: u8 = 'M' as u8;
const BLOCK_INPUT_BITMAP_PREFIX: u8 = 'B' as u8; const BLOCK_INPUT_BITMAP_PREFIX: u8 = 'B' as u8;
/// An implementation of the ChainStore trait backed by a simple key-value /// All chain-related database operations
/// store. pub struct ChainStore {
pub struct ChainKVStore { db: store::Store,
db: grin_store::Store,
header_cache: Arc<RwLock<LruCache<Hash, BlockHeader>>>, header_cache: Arc<RwLock<LruCache<Hash, BlockHeader>>>,
block_input_bitmap_cache: Arc<RwLock<LruCache<Hash, Vec<u8>>>>,
} }
impl ChainKVStore { impl ChainStore {
/// Create new chain store /// Create new chain store
pub fn new(root_path: String) -> Result<ChainKVStore, Error> { pub fn new(db_env: Arc<lmdb::Environment>) -> Result<ChainStore, Error> {
let db = grin_store::Store::open(format!("{}/{}", root_path, STORE_SUBPATH).as_str())?; let db = store::Store::open(db_env, STORE_SUBPATH);
Ok(ChainKVStore { Ok(ChainStore {
db, db,
header_cache: Arc::new(RwLock::new(LruCache::new(1_000))), header_cache: Arc::new(RwLock::new(LruCache::new(1_000))),
})
}
}
#[allow(missing_docs)]
impl ChainStore {
pub fn head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![HEAD_PREFIX]))
}
pub fn head_header(&self) -> Result<BlockHeader, Error> {
self.get_block_header(&self.head()?.last_block_h)
}
pub fn get_header_head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![HEADER_HEAD_PREFIX]))
}
pub fn get_sync_head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![SYNC_HEAD_PREFIX]))
}
pub fn get_block(&self, h: &Hash) -> Result<Block, Error> {
option_to_not_found(self.db.get_ser(&to_key(BLOCK_PREFIX, &mut h.to_vec())))
}
pub fn block_exists(&self, h: &Hash) -> Result<bool, Error> {
self.db.exists(&to_key(BLOCK_PREFIX, &mut h.to_vec()))
}
pub fn get_block_header(&self, h: &Hash) -> Result<BlockHeader, Error> {
{
let mut header_cache = self.header_cache.write().unwrap();
// cache hit - return the value from the cache
if let Some(header) = header_cache.get_mut(h) {
return Ok(header.clone());
}
}
let header: Result<BlockHeader, Error> = option_to_not_found(
self.db
.get_ser(&to_key(BLOCK_HEADER_PREFIX, &mut h.to_vec())),
);
// cache miss - so adding to the cache for next time
if let Ok(header) = header {
{
let mut header_cache = self.header_cache.write().unwrap();
header_cache.insert(*h, header.clone());
}
Ok(header)
} else {
header
}
}
// We are on the current chain if -
// * the header by height index matches the header, and
// * we are not ahead of the current head
pub fn is_on_current_chain(&self, header: &BlockHeader) -> Result<(), Error> {
let head = self.head()?;
// check we are not out ahead of the current head
if header.height > head.height {
return Err(Error::NotFoundErr);
}
let header_at_height = self.get_header_by_height(header.height)?;
if header.hash() == header_at_height.hash() {
Ok(())
} else {
Err(Error::NotFoundErr)
}
}
pub fn get_header_by_height(&self, height: u64) -> Result<BlockHeader, Error> {
option_to_not_found(self.db.get_ser(&u64_to_key(HEADER_HEIGHT_PREFIX, height)))
.and_then(|hash| self.get_block_header(&hash))
}
pub fn get_output_pos(&self, commit: &Commitment) -> Result<u64, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(COMMIT_POS_PREFIX, &mut commit.as_ref().to_vec())),
)
}
pub fn get_block_marker(&self, bh: &Hash) -> Result<BlockMarker, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec())),
)
}
pub fn get_block_sums(&self, bh: &Hash) -> Result<BlockSums, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(BLOCK_SUMS_PREFIX, &mut bh.to_vec())),
)
}
/// Builds a new batch to be used with this store.
pub fn batch(&self) -> Result<Batch, Error> {
Ok(Batch {
store: self,
db: self.db.batch()?,
block_input_bitmap_cache: Arc::new(RwLock::new(LruCache::new(1_000))), block_input_bitmap_cache: Arc::new(RwLock::new(LruCache::new(1_000))),
}) })
} }
}
fn get_block_header_db(&self, h: &Hash) -> Result<BlockHeader, Error> { /// An atomic batch in which all changes can be committed all at once or
/// discarded on error.
pub struct Batch<'a> {
store: &'a ChainStore,
db: store::Batch<'a>,
block_input_bitmap_cache: Arc<RwLock<LruCache<Hash, Vec<u8>>>>,
}
#[allow(missing_docs)]
impl<'a> Batch<'a> {
pub fn save_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![HEAD_PREFIX], t)?;
self.db.put_ser(&vec![HEADER_HEAD_PREFIX], t)
}
pub fn save_body_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![HEAD_PREFIX], t)
}
pub fn save_header_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![HEADER_HEAD_PREFIX], t)
}
pub fn save_sync_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![SYNC_HEAD_PREFIX], t)
}
pub fn init_sync_head(&self, t: &Tip) -> Result<(), Error> {
let header_tip = match self.store.get_header_head() {
Ok(hh) => hh,
Err(store::Error::NotFoundErr) => {
self.save_header_head(t)?;
t.clone()
}
Err(e) => return Err(e),
};
self.save_sync_head(&header_tip)
}
// Reset both header_head and sync_head to the current head of the body chain
pub fn reset_head(&self) -> Result<(), Error> {
let tip = self.store.head()?;
self.save_header_head(&tip)?;
self.save_sync_head(&tip)
}
/// get block
fn get_block(&self, h: &Hash) -> Result<Block, Error> {
option_to_not_found(self.db.get_ser(&to_key(BLOCK_PREFIX, &mut h.to_vec())))
}
/// Save the block and its header
pub fn save_block(&self, b: &Block) -> Result<(), Error> {
self.db
.put_ser(&to_key(BLOCK_PREFIX, &mut b.hash().to_vec())[..], b)?;
self.db.put_ser(
&to_key(BLOCK_HEADER_PREFIX, &mut b.hash().to_vec())[..],
&b.header,
)
}
/// Delete a full block. Does not delete any record associated with a block
/// header.
pub fn delete_block(&self, bh: &Hash) -> Result<(), Error> {
self.db.delete(&to_key(BLOCK_PREFIX, &mut bh.to_vec())[..])
}
pub fn save_block_header(&self, bh: &BlockHeader) -> Result<(), Error> {
let hash = bh.hash();
self.db
.put_ser(&to_key(BLOCK_HEADER_PREFIX, &mut hash.to_vec())[..], bh)?;
Ok(())
}
pub fn save_header_height(&self, bh: &BlockHeader) -> Result<(), Error> {
self.db
.put_ser(&u64_to_key(HEADER_HEIGHT_PREFIX, bh.height), &bh.hash())
}
pub fn delete_header_by_height(&self, height: u64) -> Result<(), Error> {
self.db.delete(&u64_to_key(HEADER_HEIGHT_PREFIX, height))
}
pub fn save_output_pos(&self, commit: &Commitment, pos: u64) -> Result<(), Error> {
self.db.put_ser(
&to_key(COMMIT_POS_PREFIX, &mut commit.as_ref().to_vec())[..],
&pos,
)
}
pub fn get_output_pos(&self, commit: &Commitment) -> Result<u64, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(COMMIT_POS_PREFIX, &mut commit.as_ref().to_vec())),
)
}
pub fn delete_output_pos(&self, commit: &[u8]) -> Result<(), Error> {
self.db
.delete(&to_key(COMMIT_POS_PREFIX, &mut commit.to_vec()))
}
pub fn get_block_marker(&self, bh: &Hash) -> Result<BlockMarker, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec())),
)
}
pub fn save_block_marker(&self, bh: &Hash, marker: &BlockMarker) -> Result<(), Error> {
self.db
.put_ser(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec())[..], &marker)
}
pub fn delete_block_marker(&self, bh: &Hash) -> Result<(), Error> {
self.db
.delete(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec()))
}
pub fn get_block_header_db(&self, h: &Hash) -> Result<BlockHeader, Error> {
option_to_not_found( option_to_not_found(
self.db self.db
.get_ser(&to_key(BLOCK_HEADER_PREFIX, &mut h.to_vec())), .get_ser(&to_key(BLOCK_HEADER_PREFIX, &mut h.to_vec())),
@ -93,192 +321,8 @@ impl ChainKVStore {
} }
} }
} }
}
impl ChainStore for ChainKVStore { pub fn get_block_input_bitmap(&self, bh: &Hash) -> Result<Bitmap, Error> {
fn head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![HEAD_PREFIX]))
}
fn head_header(&self) -> Result<BlockHeader, Error> {
self.get_block_header(&self.head()?.last_block_h)
}
fn save_head(&self, t: &Tip) -> Result<(), Error> {
self.db
.batch()
.put_ser(&vec![HEAD_PREFIX], t)?
.put_ser(&vec![HEADER_HEAD_PREFIX], t)?
.write()
}
fn save_body_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![HEAD_PREFIX], t)
}
fn get_header_head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![HEADER_HEAD_PREFIX]))
}
fn save_header_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![HEADER_HEAD_PREFIX], t)
}
fn get_sync_head(&self) -> Result<Tip, Error> {
option_to_not_found(self.db.get_ser(&vec![SYNC_HEAD_PREFIX]))
}
fn save_sync_head(&self, t: &Tip) -> Result<(), Error> {
self.db.put_ser(&vec![SYNC_HEAD_PREFIX], t)
}
fn init_head(&self) -> Result<(), Error> {
if self.get_header_head().is_err() {
let tip = self.head()?;
self.save_header_head(&tip)?;
}
let header_tip = self.get_header_head()?;
self.save_sync_head(&header_tip)
}
// Reset both header_head and sync_head to the current head of the body chain
fn reset_head(&self) -> Result<(), Error> {
let tip = self.head()?;
self.save_header_head(&tip)?;
self.save_sync_head(&tip)
}
fn get_block(&self, h: &Hash) -> Result<Block, Error> {
option_to_not_found(self.db.get_ser(&to_key(BLOCK_PREFIX, &mut h.to_vec())))
}
fn block_exists(&self, h: &Hash) -> Result<bool, Error> {
self.db.exists(&to_key(BLOCK_PREFIX, &mut h.to_vec()))
}
fn get_block_header(&self, h: &Hash) -> Result<BlockHeader, Error> {
{
let mut header_cache = self.header_cache.write().unwrap();
// cache hit - return the value from the cache
if let Some(header) = header_cache.get_mut(h) {
return Ok(header.clone());
}
}
// cache miss - get it from db and cache it for next time (if we found one in
// db)
let res = self.get_block_header_db(h);
if let Ok(header) = res {
let mut header_cache = self.header_cache.write().unwrap();
header_cache.insert(*h, header.clone());
return Ok(header);
}
res
}
/// Save the block and its header
fn save_block(&self, b: &Block) -> Result<(), Error> {
let batch = self.db
.batch()
.put_ser(&to_key(BLOCK_PREFIX, &mut b.hash().to_vec())[..], b)?
.put_ser(
&to_key(BLOCK_HEADER_PREFIX, &mut b.hash().to_vec())[..],
&b.header,
)?;
batch.write()
}
/// Delete a full block. Does not delete any record associated with a block
/// header.
fn delete_block(&self, bh: &Hash) -> Result<(), Error> {
self.db.delete(&to_key(BLOCK_PREFIX, &mut bh.to_vec())[..])
}
// We are on the current chain if -
// * the header by height index matches the header, and
// * we are not ahead of the current head
fn is_on_current_chain(&self, header: &BlockHeader) -> Result<(), Error> {
let head = self.head()?;
// check we are not out ahead of the current head
if header.height > head.height {
return Err(Error::NotFoundErr);
}
let header_at_height = self.get_header_by_height(header.height)?;
if header.hash() == header_at_height.hash() {
Ok(())
} else {
Err(Error::NotFoundErr)
}
}
fn save_block_header(&self, bh: &BlockHeader) -> Result<(), Error> {
let hash = bh.hash();
self.db
.put_ser(&to_key(BLOCK_HEADER_PREFIX, &mut hash.to_vec())[..], bh)?;
// Write the block_header to the cache also.
{
let mut header_cache = self.header_cache.write().unwrap();
header_cache.insert(hash, bh.clone());
}
Ok(())
}
fn get_header_by_height(&self, height: u64) -> Result<BlockHeader, Error> {
option_to_not_found(self.db.get_ser(&u64_to_key(HEADER_HEIGHT_PREFIX, height)))
.and_then(|hash| self.get_block_header(&hash))
}
fn save_header_height(&self, bh: &BlockHeader) -> Result<(), Error> {
self.db
.put_ser(&u64_to_key(HEADER_HEIGHT_PREFIX, bh.height), &bh.hash())
}
fn delete_header_by_height(&self, height: u64) -> Result<(), Error> {
self.db.delete(&u64_to_key(HEADER_HEIGHT_PREFIX, height))
}
fn save_output_pos(&self, commit: &Commitment, pos: u64) -> Result<(), Error> {
self.db.put_ser(
&to_key(COMMIT_POS_PREFIX, &mut commit.as_ref().to_vec())[..],
&pos,
)
}
fn get_output_pos(&self, commit: &Commitment) -> Result<u64, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(COMMIT_POS_PREFIX, &mut commit.as_ref().to_vec())),
)
}
fn delete_output_pos(&self, commit: &[u8]) -> Result<(), Error> {
self.db
.delete(&to_key(COMMIT_POS_PREFIX, &mut commit.to_vec()))
}
fn save_block_marker(&self, bh: &Hash, marker: &BlockMarker) -> Result<(), Error> {
self.db
.put_ser(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec())[..], &marker)
}
fn get_block_marker(&self, bh: &Hash) -> Result<BlockMarker, Error> {
option_to_not_found(
self.db
.get_ser(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec())),
)
}
fn delete_block_marker(&self, bh: &Hash) -> Result<(), Error> {
self.db
.delete(&to_key(BLOCK_MARKER_PREFIX, &mut bh.to_vec()))
}
fn get_block_input_bitmap(&self, bh: &Hash) -> Result<Bitmap, Error> {
{ {
let mut cache = self.block_input_bitmap_cache.write().unwrap(); let mut cache = self.block_input_bitmap_cache.write().unwrap();
@ -299,7 +343,7 @@ impl ChainStore for ChainKVStore {
res res
} }
fn save_block_input_bitmap(&self, block: &Block) -> Result<Bitmap, Error> { pub fn save_block_input_bitmap(&self, block: &Block) -> Result<Bitmap, Error> {
let hash = block.hash(); let hash = block.hash();
let bitmap = self.build_block_input_bitmap(block)?; let bitmap = self.build_block_input_bitmap(block)?;
self.db.put( self.db.put(
@ -313,7 +357,7 @@ impl ChainStore for ChainKVStore {
Ok(bitmap) Ok(bitmap)
} }
fn delete_block_input_bitmap(&self, bh: &Hash) -> Result<(), Error> { pub fn delete_block_input_bitmap(&self, bh: &Hash) -> Result<(), Error> {
self.db self.db
.delete(&to_key(BLOCK_INPUT_BITMAP_PREFIX, &mut bh.to_vec())) .delete(&to_key(BLOCK_INPUT_BITMAP_PREFIX, &mut bh.to_vec()))
} }
@ -325,7 +369,7 @@ impl ChainStore for ChainKVStore {
/// We need to handle the case where we have no index entry for a given /// We need to handle the case where we have no index entry for a given
/// height to account for the case where we just switched to a new fork and /// height to account for the case where we just switched to a new fork and
/// the height jumped beyond current chain height. /// the height jumped beyond current chain height.
fn setup_height(&self, header: &BlockHeader, old_tip: &Tip) -> Result<(), Error> { pub fn setup_height(&self, header: &BlockHeader, old_tip: &Tip) -> Result<(), Error> {
// remove headers ahead if we backtracked // remove headers ahead if we backtracked
for n in header.height..old_tip.height { for n in header.height..old_tip.height {
self.delete_header_by_height(n + 1)?; self.delete_header_by_height(n + 1)?;
@ -333,24 +377,40 @@ impl ChainStore for ChainKVStore {
self.build_by_height_index(header, false) self.build_by_height_index(header, false)
} }
fn build_by_height_index(&self, header: &BlockHeader, force: bool) -> Result<(), Error> { pub fn build_by_height_index(&self, header: &BlockHeader, force: bool) -> Result<(), Error> {
self.save_header_height(&header)?; self.save_header_height(&header)?;
if header.height > 0 { if header.height > 0 {
let mut prev_header = self.get_block_header(&header.previous)?; let mut prev_header = self.store.get_block_header(&header.previous)?;
while prev_header.height > 0 { while prev_header.height > 0 {
if !force { if !force {
if let Ok(_) = self.is_on_current_chain(&prev_header) { if let Ok(_) = self.store.is_on_current_chain(&prev_header) {
break; break;
} }
} }
self.save_header_height(&prev_header)?; self.save_header_height(&prev_header)?;
prev_header = self.get_block_header(&prev_header.previous)?; prev_header = self.store.get_block_header(&prev_header.previous)?;
} }
} }
Ok(()) Ok(())
} }
/// Commits this batch. If it's a child batch, it will be merged with the
/// parent, otherwise the batch is written to db.
pub fn commit(self) -> Result<(), Error> {
self.db.commit()
}
/// Creates a child of this batch. It will be merged with its parent on
/// commit, abandoned otherwise.
pub fn child(&mut self) -> Result<Batch, Error> {
Ok(Batch {
store: self.store,
db: self.db.child()?,
block_input_bitmap_cache: Arc::new(RwLock::new(LruCache::new(1_000))),
})
}
} }
/// An iterator on blocks, from latest to earliest, specialized to return /// An iterator on blocks, from latest to earliest, specialized to return

View file

@ -12,8 +12,8 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
//! Utility structs to handle the 3 hashtrees (output, range proof, kernel) more //! Utility structs to handle the 3 hashtrees (output, range proof,
//! conveniently and transactionally. //! kernel) more conveniently and transactionally.
use std::collections::HashMap; use std::collections::HashMap;
use std::fs; use std::fs;
@ -39,8 +39,9 @@ use core::ser::{PMMRIndexHashable, PMMRable};
use grin_store; use grin_store;
use grin_store::pmmr::PMMRBackend; use grin_store::pmmr::PMMRBackend;
use grin_store::types::prune_noop; use grin_store::types::prune_noop;
use types::{BlockMarker, ChainStore, Error, TxHashSetRoots}; use store::{Batch, ChainStore};
use util::{zip, LOGGER}; use types::{BlockMarker, Error, TxHashSetRoots};
use util::{secp_static, zip, LOGGER};
const TXHASHSET_SUBDIR: &'static str = "txhashset"; const TXHASHSET_SUBDIR: &'static str = "txhashset";
const OUTPUT_SUBDIR: &'static str = "output"; const OUTPUT_SUBDIR: &'static str = "output";
@ -241,9 +242,11 @@ impl TxHashSet {
let rewind_rm_pos = let rewind_rm_pos =
input_pos_to_rewind(self.commit_index.clone(), &horizon_header, &head_header)?; input_pos_to_rewind(self.commit_index.clone(), &horizon_header, &head_header)?;
let batch = self.commit_index.batch()?;
{
let clean_output_index = |commit: &[u8]| { let clean_output_index = |commit: &[u8]| {
// do we care if this fails? // do we care if this fails?
let _ = commit_index.delete_output_pos(commit); let _ = batch.delete_output_pos(commit);
}; };
self.output_pmmr_h.backend.check_compact( self.output_pmmr_h.backend.check_compact(
@ -259,6 +262,8 @@ impl TxHashSet {
&rewind_rm_pos, &rewind_rm_pos,
&prune_noop, &prune_noop,
)?; )?;
}
batch.commit()?;
Ok(()) Ok(())
} }
@ -274,24 +279,19 @@ pub fn extending_readonly<'a, F, T>(trees: &'a mut TxHashSet, inner: F) -> Resul
where where
F: FnOnce(&mut Extension) -> Result<T, Error>, F: FnOnce(&mut Extension) -> Result<T, Error>,
{ {
let sizes: (u64, u64, u64);
let res: Result<T, Error>; let res: Result<T, Error>;
{ {
let commit_index = trees.commit_index.clone(); let commit_index = trees.commit_index.clone();
let commit_index2 = trees.commit_index.clone();
let batch = commit_index.batch()?;
trace!(LOGGER, "Starting new txhashset (readonly) extension."); trace!(LOGGER, "Starting new txhashset (readonly) extension.");
let mut extension = Extension::new(trees, commit_index); let mut extension = Extension::new(trees, &batch, commit_index2);
extension.force_rollback(); extension.force_rollback();
res = inner(&mut extension); res = inner(&mut extension);
sizes = extension.sizes();
} }
trace!( trace!(LOGGER, "Rollbacking txhashset (readonly) extension.");
LOGGER,
"Rollbacking txhashset (readonly) extension. sizes {:?}",
sizes
);
trees.output_pmmr_h.backend.discard(); trees.output_pmmr_h.backend.discard();
trees.rproof_pmmr_h.backend.discard(); trees.rproof_pmmr_h.backend.discard();
@ -309,7 +309,11 @@ where
/// ///
/// If the closure returns an error, modifications are canceled and the unit /// If the closure returns an error, modifications are canceled and the unit
/// of work is abandoned. Otherwise, the unit of work is permanently applied. /// of work is abandoned. Otherwise, the unit of work is permanently applied.
pub fn extending<'a, F, T>(trees: &'a mut TxHashSet, inner: F) -> Result<T, Error> pub fn extending<'a, F, T>(
trees: &'a mut TxHashSet,
batch: &'a mut Batch,
inner: F,
) -> Result<T, Error>
where where
F: FnOnce(&mut Extension) -> Result<T, Error>, F: FnOnce(&mut Extension) -> Result<T, Error>,
{ {
@ -317,11 +321,14 @@ where
let res: Result<T, Error>; let res: Result<T, Error>;
let rollback: bool; let rollback: bool;
// create a child transaction so if the state is rolled back by itself, all
// index saving can be undone
let child_batch = batch.child()?;
{ {
let commit_index = trees.commit_index.clone(); let commit_index = trees.commit_index.clone();
trace!(LOGGER, "Starting new txhashset extension."); trace!(LOGGER, "Starting new txhashset extension.");
let mut extension = Extension::new(trees, commit_index); let mut extension = Extension::new(trees, &child_batch, commit_index);
res = inner(&mut extension); res = inner(&mut extension);
rollback = extension.rollback; rollback = extension.rollback;
@ -347,6 +354,7 @@ where
trees.kernel_pmmr_h.backend.discard(); trees.kernel_pmmr_h.backend.discard();
} else { } else {
trace!(LOGGER, "Committing txhashset extension. sizes {:?}", sizes); trace!(LOGGER, "Committing txhashset extension. sizes {:?}", sizes);
child_batch.commit()?;
trees.output_pmmr_h.backend.sync()?; trees.output_pmmr_h.backend.sync()?;
trees.rproof_pmmr_h.backend.sync()?; trees.rproof_pmmr_h.backend.sync()?;
trees.kernel_pmmr_h.backend.sync()?; trees.kernel_pmmr_h.backend.sync()?;
@ -373,6 +381,11 @@ pub struct Extension<'a> {
new_output_commits: HashMap<Commitment, u64>, new_output_commits: HashMap<Commitment, u64>,
new_block_markers: HashMap<Hash, BlockMarker>, new_block_markers: HashMap<Hash, BlockMarker>,
rollback: bool, rollback: bool,
/// Batch in which the extension occurs, public so it can be used within
/// and `extending` closure. Just be careful using it that way as it will
/// get rolled back with the extension (i.e on a losing fork).
pub batch: &'a Batch<'a>,
} }
impl<'a> Committed for Extension<'a> { impl<'a> Committed for Extension<'a> {
@ -407,7 +420,11 @@ impl<'a> Committed for Extension<'a> {
impl<'a> Extension<'a> { impl<'a> Extension<'a> {
// constructor // constructor
fn new(trees: &'a mut TxHashSet, commit_index: Arc<ChainStore>) -> Extension<'a> { fn new(
trees: &'a mut TxHashSet,
batch: &'a Batch,
commit_index: Arc<ChainStore>,
) -> Extension<'a> {
Extension { Extension {
output_pmmr: PMMR::at( output_pmmr: PMMR::at(
&mut trees.output_pmmr_h.backend, &mut trees.output_pmmr_h.backend,
@ -421,10 +438,11 @@ impl<'a> Extension<'a> {
&mut trees.kernel_pmmr_h.backend, &mut trees.kernel_pmmr_h.backend,
trees.kernel_pmmr_h.last_pos, trees.kernel_pmmr_h.last_pos,
), ),
commit_index: commit_index, commit_index,
new_output_commits: HashMap::new(), new_output_commits: HashMap::new(),
new_block_markers: HashMap::new(), new_block_markers: HashMap::new(),
rollback: false, rollback: false,
batch,
} }
} }
@ -440,7 +458,7 @@ impl<'a> Extension<'a> {
let rewind_add_pos: Bitmap = ((output_pos + 1)..(latest_output_pos + 1)) let rewind_add_pos: Bitmap = ((output_pos + 1)..(latest_output_pos + 1))
.map(|x| x as u32) .map(|x| x as u32)
.collect(); .collect();
self.rewind_to_pos(output_pos, kernel_pos, &rewind_add_pos, rewind_rm_pos)?; self.rewind_to_pos(output_pos, kernel_pos, &rewind_add_pos, rewind_rm_pos, true, true, true)?;
Ok(()) Ok(())
} }
@ -465,8 +483,7 @@ impl<'a> Extension<'a> {
// Build bitmap of output pos spent (as inputs) by this tx for rewind. // Build bitmap of output pos spent (as inputs) by this tx for rewind.
let rewind_rm_pos = tx.inputs let rewind_rm_pos = tx.inputs
.iter() .iter()
.filter_map(|x| self.get_output_pos(&x.commitment()).ok()) .filter_map(|x| self.get_output_pos(&x.commitment()).ok()) .map(|x| x as u32)
.map(|x| x as u32)
.collect(); .collect();
for ref output in &tx.outputs { for ref output in &tx.outputs {
@ -595,6 +612,8 @@ impl<'a> Extension<'a> {
output_pos: self.output_pmmr.unpruned_size(), output_pos: self.output_pmmr.unpruned_size(),
kernel_pos: self.kernel_pmmr.unpruned_size(), kernel_pos: self.kernel_pmmr.unpruned_size(),
}; };
//TODO: This doesn't look right
self.batch.save_block_marker(&b.hash(), &marker)?;
self.new_block_markers.insert(b.hash(), marker); self.new_block_markers.insert(b.hash(), marker);
Ok(()) Ok(())
} }
@ -603,17 +622,17 @@ impl<'a> Extension<'a> {
// Also store any new block_markers. // Also store any new block_markers.
fn save_indexes(&self) -> Result<(), Error> { fn save_indexes(&self) -> Result<(), Error> {
for (commit, pos) in &self.new_output_commits { for (commit, pos) in &self.new_output_commits {
self.commit_index.save_output_pos(commit, *pos)?; self.batch.save_output_pos(commit, *pos)?;
} }
for (bh, marker) in &self.new_block_markers { for (bh, marker) in &self.new_block_markers {
self.commit_index.save_block_marker(bh, marker)?; self.batch.save_block_marker(bh, marker)?;
} }
Ok(()) Ok(())
} }
fn apply_input(&mut self, input: &Input) -> Result<(), Error> { fn apply_input(&mut self, input: &Input) -> Result<(), Error> {
let commit = input.commitment(); let commit = input.commitment();
let pos_res = self.get_output_pos(&commit); let pos_res = self.batch.get_output_pos(&commit);
if let Ok(pos) = pos_res { if let Ok(pos) = pos_res {
let output_id_hash = OutputIdentifier::from_input(input).hash_with_index(pos - 1); let output_id_hash = OutputIdentifier::from_input(input).hash_with_index(pos - 1);
if let Some(read_hash) = self.output_pmmr.get_hash(pos) { if let Some(read_hash) = self.output_pmmr.get_hash(pos) {
@ -649,7 +668,8 @@ impl<'a> Extension<'a> {
fn apply_output(&mut self, out: &Output) -> Result<(), Error> { fn apply_output(&mut self, out: &Output) -> Result<(), Error> {
let commit = out.commitment(); let commit = out.commitment();
if let Ok(pos) = self.get_output_pos(&commit) {
if let Ok(pos) = self.batch.get_output_pos(&commit) {
// we need to check whether the commitment is in the current MMR view // we need to check whether the commitment is in the current MMR view
// as well as the index doesn't support rewind and is non-authoritative // as well as the index doesn't support rewind and is non-authoritative
// (non-historical node will have a much smaller one) // (non-historical node will have a much smaller one)
@ -668,6 +688,7 @@ impl<'a> Extension<'a> {
let pos = self.output_pmmr let pos = self.output_pmmr
.push(OutputIdentifier::from_output(out)) .push(OutputIdentifier::from_output(out))
.map_err(&Error::TxHashSetErr)?; .map_err(&Error::TxHashSetErr)?;
self.batch.save_output_pos(&out.commitment(), pos)?;
self.new_output_commits.insert(out.commitment(), pos); self.new_output_commits.insert(out.commitment(), pos);
// push range proofs in their MMR and file // push range proofs in their MMR and file
@ -705,10 +726,10 @@ impl<'a> Extension<'a> {
// rewind to the specified block for a consistent view // rewind to the specified block for a consistent view
let head_header = self.commit_index.head_header()?; let head_header = self.commit_index.head_header()?;
self.rewind(block_header, &head_header)?; self.rewind(block_header, &head_header, true, true, true)?;
// then calculate the Merkle Proof based on the known pos // then calculate the Merkle Proof based on the known pos
let pos = self.get_output_pos(&output.commit)?; let pos = self.batch.get_output_pos(&output.commit)?;
let merkle_proof = self.output_pmmr let merkle_proof = self.output_pmmr
.merkle_proof(pos) .merkle_proof(pos)
.map_err(&Error::TxHashSetErr)?; .map_err(&Error::TxHashSetErr)?;
@ -737,6 +758,9 @@ impl<'a> Extension<'a> {
&mut self, &mut self,
block_header: &BlockHeader, block_header: &BlockHeader,
head_header: &BlockHeader, head_header: &BlockHeader,
rewind_utxo: bool,
rewind_kernel: bool,
rewind_rangeproof: bool,
) -> Result<(), Error> { ) -> Result<(), Error> {
let hash = block_header.hash(); let hash = block_header.hash();
trace!( trace!(
@ -748,7 +772,7 @@ impl<'a> Extension<'a> {
// Rewind our MMRs to the appropriate positions // Rewind our MMRs to the appropriate positions
// based on the block_marker. // based on the block_marker.
let marker = self.commit_index.get_block_marker(&hash)?; let marker = self.batch.get_block_marker(&hash)?;
// We need to build bitmaps of added and removed output positions // We need to build bitmaps of added and removed output positions
// so we can correctly rewind all operations applied to the output MMR // so we can correctly rewind all operations applied to the output MMR
@ -766,6 +790,9 @@ impl<'a> Extension<'a> {
marker.kernel_pos, marker.kernel_pos,
&rewind_add_pos, &rewind_add_pos,
&rewind_rm_pos, &rewind_rm_pos,
rewind_utxo,
rewind_kernel,
rewind_rangeproof
)?; )?;
Ok(()) Ok(())
@ -779,6 +806,9 @@ impl<'a> Extension<'a> {
kernel_pos: u64, kernel_pos: u64,
rewind_add_pos: &Bitmap, rewind_add_pos: &Bitmap,
rewind_rm_pos: &Bitmap, rewind_rm_pos: &Bitmap,
rewind_utxo: bool,
rewind_kernel: bool,
rewind_rproof: bool,
) -> Result<(), Error> { ) -> Result<(), Error> {
trace!( trace!(
LOGGER, LOGGER,
@ -792,15 +822,21 @@ impl<'a> Extension<'a> {
// been sync'd to disk. // been sync'd to disk.
self.new_output_commits.retain(|_, &mut v| v <= output_pos); self.new_output_commits.retain(|_, &mut v| v <= output_pos);
if rewind_utxo {
self.output_pmmr self.output_pmmr
.rewind(output_pos, rewind_add_pos, rewind_rm_pos) .rewind(output_pos, rewind_add_pos, rewind_rm_pos)
.map_err(&Error::TxHashSetErr)?; .map_err(&Error::TxHashSetErr)?;
}
if rewind_rproof {
self.rproof_pmmr self.rproof_pmmr
.rewind(output_pos, rewind_add_pos, rewind_rm_pos) .rewind(output_pos, rewind_add_pos, rewind_rm_pos)
.map_err(&Error::TxHashSetErr)?; .map_err(&Error::TxHashSetErr)?;
}
if rewind_kernel {
self.kernel_pmmr self.kernel_pmmr
.rewind(kernel_pos, rewind_add_pos, rewind_rm_pos) .rewind(kernel_pos, rewind_add_pos, rewind_rm_pos)
.map_err(&Error::TxHashSetErr)?; .map_err(&Error::TxHashSetErr)?;
}
Ok(()) Ok(())
} }
@ -864,18 +900,26 @@ impl<'a> Extension<'a> {
} }
/// Validate the txhashset state against the provided block header. /// Validate the txhashset state against the provided block header.
pub fn validate(&mut self, header: &BlockHeader, skip_rproofs: bool) -> Result<(), Error> { pub fn validate(
&mut self,
header: &BlockHeader,
skip_rproofs: bool,
) -> Result<((Commitment, Commitment)), Error> {
self.validate_mmrs()?; self.validate_mmrs()?;
self.validate_roots(header)?; self.validate_roots(header)?;
if header.height == 0 { if header.height == 0 {
return Ok(()); let zero_commit = secp_static::commit_to_zero_value();
return Ok((zero_commit.clone(), zero_commit.clone()));
} }
// The real magicking happens here. // The real magicking happens here.
// Sum of kernel excesses should equal sum of // Sum of kernel excesses should equal sum of
// unspent outputs minus total supply. // unspent outputs minus total supply.
self.verify_kernel_sums(header.total_overage(), header.total_kernel_offset())?; let (output_sum, kernel_sum) = self.verify_kernel_sums(
header.total_overage(),
header.total_kernel_offset(),
)?;
// This is an expensive verification step. // This is an expensive verification step.
self.verify_kernel_signatures()?; self.verify_kernel_signatures()?;
@ -890,7 +934,7 @@ impl<'a> Extension<'a> {
// a lot without resetting // a lot without resetting
self.verify_kernel_history(header)?; self.verify_kernel_history(header)?;
Ok(()) Ok((output_sum, kernel_sum))
} }
/// Rebuild the index of MMR positions to the corresponding Output and /// Rebuild the index of MMR positions to the corresponding Output and
@ -901,7 +945,7 @@ impl<'a> Extension<'a> {
// non-pruned leaves only // non-pruned leaves only
if pmmr::bintree_postorder_height(n) == 0 { if pmmr::bintree_postorder_height(n) == 0 {
if let Some(out) = self.output_pmmr.get_data(n) { if let Some(out) = self.output_pmmr.get_data(n) {
self.commit_index.save_output_pos(&out.commit, n)?; self.batch.save_output_pos(&out.commit, n)?;
} }
} }
} }
@ -1015,10 +1059,9 @@ impl<'a> Extension<'a> {
if current.height == 0 { if current.height == 0 {
break; break;
} }
let head_header = self.commit_index.head_header()?;
// rewinding further and further back // rewinding further and further back
self.kernel_pmmr self.rewind(&current, &head_header, false, true, false)?;
.rewind(current.kernel_mmr_size, current.height as u32)
.map_err(&Error::TxHashSetErr)?;
if self.kernel_pmmr.root() != current.kernel_root { if self.kernel_pmmr.root() != current.kernel_root {
return Err(Error::InvalidTxHashSet(format!( return Err(Error::InvalidTxHashSet(format!(
"Kernel root at {} does not match", "Kernel root at {} does not match",
@ -1095,7 +1138,7 @@ fn input_pos_to_rewind(
// I/O should be minimized or eliminated here for most // I/O should be minimized or eliminated here for most
// rewind scenarios. // rewind scenarios.
let current_header = commit_index.get_block_header(&current)?; let current_header = commit_index.get_block_header(&current)?;
let input_bitmap = commit_index.get_block_input_bitmap(&current)?; let input_bitmap = commit_index.batch()?.get_block_input_bitmap(&current)?;
bitmap.or_inplace(&input_bitmap); bitmap.or_inplace(&input_bitmap);
current = current_header.previous; current = current_header.previous;

View file

@ -16,10 +16,9 @@
use std::{error, fmt, io}; use std::{error, fmt, io};
use croaring::Bitmap;
use util::secp; use util::secp;
use util::secp::pedersen::Commitment; use util::secp::pedersen::Commitment;
use util::secp_static;
use core::core::committed; use core::core::committed;
use core::core::hash::{Hash, Hashed}; use core::core::hash::{Hash, Hashed};
@ -256,112 +255,6 @@ impl ser::Readable for Tip {
} }
} }
/// Trait the chain pipeline requires an implementor for in order to process
/// blocks.
pub trait ChainStore: Send + Sync {
/// Get the tip that's also the head of the chain
fn head(&self) -> Result<Tip, store::Error>;
/// Block header for the chain head
fn head_header(&self) -> Result<BlockHeader, store::Error>;
/// Save the provided tip as the current head of our chain
fn save_head(&self, t: &Tip) -> Result<(), store::Error>;
/// Save the provided tip as the current head of the body chain, leaving the
/// header chain alone.
fn save_body_head(&self, t: &Tip) -> Result<(), store::Error>;
/// Gets a block header by hash
fn get_block(&self, h: &Hash) -> Result<Block, store::Error>;
/// Check whether we have a block without reading it
fn block_exists(&self, h: &Hash) -> Result<bool, store::Error>;
/// Gets a block header by hash
fn get_block_header(&self, h: &Hash) -> Result<BlockHeader, store::Error>;
/// Save the provided block in store
fn save_block(&self, b: &Block) -> Result<(), store::Error>;
/// Delete a full block. Does not delete any record associated with a block
/// header.
fn delete_block(&self, bh: &Hash) -> Result<(), store::Error>;
/// Save the provided block header in store
fn save_block_header(&self, bh: &BlockHeader) -> Result<(), store::Error>;
/// Get the tip of the header chain
fn get_header_head(&self) -> Result<Tip, store::Error>;
/// Save the provided tip as the current head of the block header chain
fn save_header_head(&self, t: &Tip) -> Result<(), store::Error>;
/// Get the tip of the current sync header chain
fn get_sync_head(&self) -> Result<Tip, store::Error>;
/// Save the provided tip as the current head of the sync header chain
fn save_sync_head(&self, t: &Tip) -> Result<(), store::Error>;
/// Initialize header_head if necessary and set sync_head to header_head.
fn init_head(&self) -> Result<(), store::Error>;
/// Reset header_head and sync_head to head of current body chain
fn reset_head(&self) -> Result<(), store::Error>;
/// Gets the block header at the provided height
fn get_header_by_height(&self, height: u64) -> Result<BlockHeader, store::Error>;
/// Save a header as associated with its height
fn save_header_height(&self, header: &BlockHeader) -> Result<(), store::Error>;
/// Delete the block header at the height
fn delete_header_by_height(&self, height: u64) -> Result<(), store::Error>;
/// Is the block header on the current chain?
/// Use the header_by_height index to verify the block header is where we
/// think it is.
fn is_on_current_chain(&self, header: &BlockHeader) -> Result<(), store::Error>;
/// Saves the position of an output, represented by its commitment, in the
/// Output MMR. Used as an index for spending and pruning.
fn save_output_pos(&self, commit: &Commitment, pos: u64) -> Result<(), store::Error>;
/// Gets the position of an output, represented by its commitment, in the
/// Output MMR. Used as an index for spending and pruning.
fn get_output_pos(&self, commit: &Commitment) -> Result<u64, store::Error>;
/// Deletes the MMR position of an output.
fn delete_output_pos(&self, commit: &[u8]) -> Result<(), store::Error>;
/// Saves a marker associated with a block recording the MMR positions of
/// its last elements.
fn save_block_marker(&self, bh: &Hash, marker: &BlockMarker) -> Result<(), store::Error>;
/// Retrieves a block marker from a block hash.
fn get_block_marker(&self, bh: &Hash) -> Result<BlockMarker, store::Error>;
/// Deletes a block marker associated with the provided hash
fn delete_block_marker(&self, bh: &Hash) -> Result<(), store::Error>;
/// Get the bitmap representing the inputs for the specified block.
fn get_block_input_bitmap(&self, bh: &Hash) -> Result<Bitmap, store::Error>;
/// Save the bitmap representing the inputs for the specified block.
fn save_block_input_bitmap(&self, b: &Block) -> Result<Bitmap, store::Error>;
/// Delete the bitmap representing the inputs for the specified block.
fn delete_block_input_bitmap(&self, bh: &Hash) -> Result<(), store::Error>;
/// Saves the provided block header at the corresponding height. Also check
/// the consistency of the height chain in store by assuring previous
/// headers are also at their respective heights.
fn setup_height(&self, bh: &BlockHeader, old_tip: &Tip) -> Result<(), store::Error>;
/// Similar to setup_height but without handling fork
fn build_by_height_index(&self, header: &BlockHeader, force: bool) -> Result<(), store::Error>;
}
/// Bridge between the chain pipeline and the rest of the system. Handles /// Bridge between the chain pipeline and the rest of the system. Handles
/// downstream processing of valid blocks by the rest of the system, most /// downstream processing of valid blocks by the rest of the system, most
/// importantly the broadcasting of blocks to our peers. /// importantly the broadcasting of blocks to our peers.
@ -414,3 +307,45 @@ impl Default for BlockMarker {
} }
} }
} }
/// The output_sum and kernel_sum for a given block.
/// This is used to validate the next block being processed by applying
/// the inputs, outputs, kernels and kernel_offset from the new block
/// and checking everything sums correctly.
#[derive(Debug, Clone)]
pub struct BlockSums {
/// The total output sum so far.
pub output_sum: Commitment,
/// The total kernel sum so far.
pub kernel_sum: Commitment,
}
impl Writeable for BlockSums {
fn write<W: Writer>(&self, writer: &mut W) -> Result<(), ser::Error> {
writer.write_fixed_bytes(&self.output_sum)?;
writer.write_fixed_bytes(&self.kernel_sum)?;
Ok(())
}
}
impl Readable for BlockSums {
fn read(reader: &mut Reader) -> Result<BlockSums, ser::Error> {
Ok(BlockSums {
output_sum: Commitment::read(reader)?,
kernel_sum: Commitment::read(reader)?,
})
}
}
impl Default for BlockSums {
fn default() -> BlockSums {
let zero_commit = secp_static::commit_to_zero_value();
BlockSums {
output_sum: zero_commit.clone(),
kernel_sum: zero_commit.clone(),
}
}
}

View file

@ -16,6 +16,7 @@ extern crate env_logger;
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
extern crate rand; extern crate rand;
@ -43,8 +44,10 @@ fn setup(dir_name: &str) -> Chain {
clean_output_dir(dir_name); clean_output_dir(dir_name);
global::set_mining_mode(ChainTypes::AutomatedTesting); global::set_mining_mode(ChainTypes::AutomatedTesting);
let genesis_block = pow::mine_genesis_block().unwrap(); let genesis_block = pow::mine_genesis_block().unwrap();
let db_env = Arc::new(store::new_env(dir_name.to_string()));
chain::Chain::init( chain::Chain::init(
dir_name.to_string(), dir_name.to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis_block, genesis_block,
pow::verify_size, pow::verify_size,
@ -52,8 +55,10 @@ fn setup(dir_name: &str) -> Chain {
} }
fn reload_chain(dir_name: &str) -> Chain { fn reload_chain(dir_name: &str) -> Chain {
let db_env = Arc::new(store::new_env(dir_name.to_string()));
chain::Chain::init( chain::Chain::init(
dir_name.to_string(), dir_name.to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis::genesis_dev(), genesis::genesis_dev(),
pow::verify_size, pow::verify_size,

View file

@ -12,10 +12,10 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
extern crate env_logger;
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
extern crate rand; extern crate rand;
@ -38,25 +38,23 @@ fn clean_output_dir(dir_name: &str) {
let _ = fs::remove_dir_all(dir_name); let _ = fs::remove_dir_all(dir_name);
} }
fn setup(dir_name: &str) -> Chain { fn setup(dir_name: &str, genesis: Block) -> Chain {
match env_logger::try_init() { util::init_test_logger();
Ok(_) => println!("Initializing env logger"),
Err(e) => println!("env logger already initialized: {:?}", e),
};
clean_output_dir(dir_name); clean_output_dir(dir_name);
global::set_mining_mode(ChainTypes::AutomatedTesting); let db_env = Arc::new(store::new_env(dir_name.to_string()));
let genesis_block = pow::mine_genesis_block().unwrap();
chain::Chain::init( chain::Chain::init(
dir_name.to_string(), dir_name.to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis_block, genesis,
pow::verify_size, pow::verify_size,
).unwrap() ).unwrap()
} }
#[test] #[test]
fn mine_empty_chain() { fn mine_empty_chain() {
let chain = setup(".grin"); global::set_mining_mode(ChainTypes::AutomatedTesting);
let chain = setup(".grin", pow::mine_genesis_block().unwrap());
let keychain = ExtKeychain::from_random_seed().unwrap(); let keychain = ExtKeychain::from_random_seed().unwrap();
for n in 1..4 { for n in 1..4 {
@ -105,7 +103,8 @@ fn mine_empty_chain() {
#[test] #[test]
fn mine_forks() { fn mine_forks() {
let chain = setup(".grin2"); global::set_mining_mode(ChainTypes::AutomatedTesting);
let chain = setup(".grin2", pow::mine_genesis_block().unwrap());
let kc = ExtKeychain::from_random_seed().unwrap(); let kc = ExtKeychain::from_random_seed().unwrap();
// add a first block to not fork genesis // add a first block to not fork genesis
@ -147,8 +146,9 @@ fn mine_forks() {
#[test] #[test]
fn mine_losing_fork() { fn mine_losing_fork() {
global::set_mining_mode(ChainTypes::AutomatedTesting);
let kc = ExtKeychain::from_random_seed().unwrap(); let kc = ExtKeychain::from_random_seed().unwrap();
let chain = setup(".grin3"); let chain = setup(".grin3", pow::mine_genesis_block().unwrap());
// add a first block we'll be forking from // add a first block we'll be forking from
let prev = chain.head_header().unwrap(); let prev = chain.head_header().unwrap();
@ -178,12 +178,14 @@ fn mine_losing_fork() {
#[test] #[test]
fn longer_fork() { fn longer_fork() {
global::set_mining_mode(ChainTypes::AutomatedTesting);
let kc = ExtKeychain::from_random_seed().unwrap(); let kc = ExtKeychain::from_random_seed().unwrap();
// to make it easier to compute the txhashset roots in the test, we // to make it easier to compute the txhashset roots in the test, we
// prepare 2 chains, the 2nd will be have the forked blocks we can // prepare 2 chains, the 2nd will be have the forked blocks we can
// then send back on the 1st // then send back on the 1st
let chain = setup(".grin4"); let genesis = pow::mine_genesis_block().unwrap();
let chain_fork = setup(".grin5"); let chain = setup(".grin4", genesis.clone());
let chain_fork = setup(".grin5", genesis);
// add blocks to both chains, 20 on the main one, only the first 5 // add blocks to both chains, 20 on the main one, only the first 5
// for the forked chain // for the forked chain
@ -193,9 +195,8 @@ fn longer_fork() {
let bh = b.header.clone(); let bh = b.header.clone();
if n < 5 { if n < 5 {
let b_fork = b.clone();
chain_fork chain_fork
.process_block(b_fork, chain::Options::SKIP_POW) .process_block(b.clone(), chain::Options::SKIP_POW)
.unwrap(); .unwrap();
} }
@ -227,8 +228,9 @@ fn longer_fork() {
#[test] #[test]
fn spend_in_fork_and_compact() { fn spend_in_fork_and_compact() {
global::set_mining_mode(ChainTypes::AutomatedTesting);
util::init_test_logger(); util::init_test_logger();
let chain = setup(".grin6"); let chain = setup(".grin6", pow::mine_genesis_block().unwrap());
let prev = chain.head_header().unwrap(); let prev = chain.head_header().unwrap();
let kc = ExtKeychain::from_random_seed().unwrap(); let kc = ExtKeychain::from_random_seed().unwrap();
@ -435,8 +437,10 @@ where
fn actual_diff_iter_output() { fn actual_diff_iter_output() {
global::set_mining_mode(ChainTypes::AutomatedTesting); global::set_mining_mode(ChainTypes::AutomatedTesting);
let genesis_block = pow::mine_genesis_block().unwrap(); let genesis_block = pow::mine_genesis_block().unwrap();
let db_env = Arc::new(store::new_env(".grin".to_string()));
let chain = chain::Chain::init( let chain = chain::Chain::init(
"../.grin".to_string(), "../.grin".to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis_block, genesis_block,
pow::verify_size, pow::verify_size,

View file

@ -16,10 +16,12 @@ extern crate env_logger;
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
extern crate rand; extern crate rand;
use std::fs; use std::fs;
use std::sync::Arc;
use chain::{ChainStore, Tip}; use chain::{ChainStore, Tip};
use core::core::hash::Hashed; use core::core::hash::Hashed;
@ -45,27 +47,34 @@ fn test_various_store_indices() {
let keychain = ExtKeychain::from_random_seed().unwrap(); let keychain = ExtKeychain::from_random_seed().unwrap();
let key_id = keychain.derive_key_id(1).unwrap(); let key_id = keychain.derive_key_id(1).unwrap();
let db_env = Arc::new(store::new_env(chain_dir.to_string()));
let chain_store = let chain_store = chain::store::ChainStore::new(db_env).unwrap();
&chain::store::ChainKVStore::new(chain_dir.to_string()).unwrap() as &ChainStore;
global::set_mining_mode(ChainTypes::AutomatedTesting); global::set_mining_mode(ChainTypes::AutomatedTesting);
let genesis = pow::mine_genesis_block().unwrap(); let genesis = pow::mine_genesis_block().unwrap();
chain_store.save_block(&genesis).unwrap();
chain_store
.setup_height(&genesis.header, &Tip::new(genesis.hash()))
.unwrap();
let reward = libtx::reward::output(&keychain, &key_id, 0, 1).unwrap(); let reward = libtx::reward::output(&keychain, &key_id, 0, 1).unwrap();
let block = Block::new(&genesis.header, vec![], Difficulty::one(), reward).unwrap(); let block = Block::new(&genesis.header, vec![], Difficulty::one(), reward).unwrap();
let block_hash = block.hash(); let block_hash = block.hash();
chain_store.save_block(&block).unwrap(); {
chain_store let batch = chain_store.batch().unwrap();
batch.save_block(&genesis).unwrap();
batch
.setup_height(&genesis.header, &Tip::new(genesis.hash()))
.unwrap();
batch.commit().unwrap();
}
{
let batch = chain_store.batch().unwrap();
batch.save_block(&block).unwrap();
batch
.setup_height(&block.header, &Tip::from_block(&block.header)) .setup_height(&block.header, &Tip::from_block(&block.header))
.unwrap(); .unwrap();
batch.commit().unwrap();
}
let block_header = chain_store.get_block_header(&block_hash).unwrap(); let block_header = chain_store.get_block_header(&block_hash).unwrap();
assert_eq!(block_header.hash(), block_hash); assert_eq!(block_header.hash(), block_hash);
@ -82,19 +91,27 @@ fn test_store_header_height() {
let chain_dir = ".grin_idx_2"; let chain_dir = ".grin_idx_2";
clean_output_dir(chain_dir); clean_output_dir(chain_dir);
let chain_store = let db_env = Arc::new(store::new_env(chain_dir.to_string()));
&chain::store::ChainKVStore::new(chain_dir.to_string()).unwrap() as &ChainStore; let chain_store = chain::store::ChainStore::new(db_env).unwrap();
let mut block_header = BlockHeader::default(); let mut block_header = BlockHeader::default();
block_header.height = 1; block_header.height = 1;
chain_store.save_block_header(&block_header).unwrap(); {
chain_store.save_header_height(&block_header).unwrap(); let batch = chain_store.batch().unwrap();
batch.save_block_header(&block_header).unwrap();
batch.save_header_height(&block_header).unwrap();
batch.commit().unwrap();
}
let stored_block_header = chain_store.get_header_by_height(1).unwrap(); let stored_block_header = chain_store.get_header_by_height(1).unwrap();
assert_eq!(block_header.hash(), stored_block_header.hash()); assert_eq!(block_header.hash(), stored_block_header.hash());
chain_store.delete_header_by_height(1).unwrap(); {
let batch = chain_store.batch().unwrap();
batch.delete_header_by_height(1).unwrap();
batch.commit().unwrap();
}
let result = chain_store.get_header_by_height(1); let result = chain_store.get_header_by_height(1);
assert_eq!(result.is_err(), true); assert_eq!(result.is_err(), true);

View file

@ -16,6 +16,7 @@ extern crate env_logger;
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
extern crate rand; extern crate rand;
extern crate time; extern crate time;
@ -43,8 +44,10 @@ fn test_coinbase_maturity() {
let genesis_block = pow::mine_genesis_block().unwrap(); let genesis_block = pow::mine_genesis_block().unwrap();
let db_env = Arc::new(store::new_env(".grin".to_string()));
let chain = chain::Chain::init( let chain = chain::Chain::init(
".grin".to_string(), ".grin".to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis_block, genesis_block,
pow::verify_size, pow::verify_size,

View file

@ -15,13 +15,13 @@
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
use std::fs; use std::fs;
use std::sync::Arc; use std::sync::Arc;
use chain::ChainStore; use chain::store::ChainStore;
use chain::store::ChainKVStore;
use chain::txhashset::{self, TxHashSet}; use chain::txhashset::{self, TxHashSet};
use chain::types::Tip; use chain::types::Tip;
use core::core::merkle_proof::MerkleProof; use core::core::merkle_proof::MerkleProof;
@ -39,6 +39,7 @@ fn test_some_raw_txs() {
let db_root = format!(".grin_txhashset_raw_txs"); let db_root = format!(".grin_txhashset_raw_txs");
clean_output_dir(&db_root); clean_output_dir(&db_root);
let db_env = Arc::new(store::new_env(db_root.clone()));
let store = Arc::new(ChainKVStore::new(db_root.clone()).unwrap()); let store = Arc::new(ChainKVStore::new(db_root.clone()).unwrap());
let mut txhashset = TxHashSet::open(db_root.clone(), store.clone(), None).unwrap(); let mut txhashset = TxHashSet::open(db_root.clone(), store.clone(), None).unwrap();
@ -60,11 +61,15 @@ fn test_some_raw_txs() {
// Note: this results in an output MMR with a single leaf node. // Note: this results in an output MMR with a single leaf node.
// We need to be careful with pruning while processing the txs below // We need to be careful with pruning while processing the txs below
// as we cannot prune a tree with a single node in it (no sibling or parent). // as we cannot prune a tree with a single node in it (no sibling or parent).
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap(); let mut batch = store.batch().unwrap();
txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)
}).unwrap();
// Make sure we setup the head in the store based on block we just accepted. // Make sure we setup the head in the store based on block we just accepted.
let head = Tip::from_block(&block.header); let head = Tip::from_block(&block.header);
store.save_head(&head).unwrap(); batch.save_head(&head).unwrap();
batch.commit().unwrap();
let coinbase_reward = 60_000_000_000; let coinbase_reward = 60_000_000_000;

View file

@ -702,17 +702,17 @@ impl Block {
vec![prev_kernel_offset.clone()], vec![prev_kernel_offset.clone()],
)? )?
}; };
let sum = self.verify_kernel_sums(self.header.overage(), block_kernel_offset)?; let (_utxo_sum, kernel_sum) = self.verify_kernel_sums(self.header.overage(), block_kernel_offset)?;
// check the block header's total kernel sum // check the block header's total kernel sum
let total_sum = committed::sum_commits(vec![sum, prev_kernel_sum.clone()], vec![])?; let total_sum = committed::sum_commits(vec![kernel_sum, prev_kernel_sum.clone()], vec![])?;
if total_sum != self.header.total_kernel_sum { if total_sum != self.header.total_kernel_sum {
return Err(Error::InvalidTotalKernelSum); return Err(Error::InvalidTotalKernelSum);
} }
self.verify_rangeproofs()?; self.verify_rangeproofs()?;
self.verify_kernel_signatures()?; self.verify_kernel_signatures()?;
Ok(sum) Ok(kernel_sum)
} }
fn verify_weight(&self) -> Result<(), Error> { fn verify_weight(&self) -> Result<(), Error> {

View file

@ -117,7 +117,7 @@ pub trait Committed {
&self, &self,
overage: i64, overage: i64,
kernel_offset: BlindingFactor, kernel_offset: BlindingFactor,
) -> Result<(Commitment), Error> { ) -> Result<((Commitment, Commitment)), Error> {
// Sum all input|output|overage commitments. // Sum all input|output|overage commitments.
let utxo_sum = self.sum_commitments(overage)?; let utxo_sum = self.sum_commitments(overage)?;
@ -128,7 +128,7 @@ pub trait Committed {
return Err(Error::KernelSumMismatch); return Err(Error::KernelSumMismatch);
} }
Ok(kernel_sum) Ok((utxo_sum, kernel_sum))
} }
} }

View file

@ -9,6 +9,7 @@ publish = false
bitflags = "1" bitflags = "1"
bytes = "0.4" bytes = "0.4"
enum_primitive = "0.1" enum_primitive = "0.1"
lmdb-zero = "0.4.4"
net2 = "0.2" net2 = "0.2"
num = "0.1" num = "0.1"
rand = "0.3" rand = "0.3"

View file

@ -25,6 +25,7 @@ extern crate bitflags;
extern crate bytes; extern crate bytes;
#[macro_use] #[macro_use]
extern crate enum_primitive; extern crate enum_primitive;
extern crate lmdb_zero as lmdb;
#[macro_use] #[macro_use]
extern crate grin_core as core; extern crate grin_core as core;

View file

@ -19,6 +19,8 @@ use std::sync::{Arc, RwLock};
use std::time::Duration; use std::time::Duration;
use std::{io, thread}; use std::{io, thread};
use lmdb;
use core::core; use core::core;
use core::core::hash::Hash; use core::core::hash::Hash;
use core::core::target::Difficulty; use core::core::target::Difficulty;
@ -46,7 +48,7 @@ unsafe impl Send for Server {}
impl Server { impl Server {
/// Creates a new idle p2p server with no peers /// Creates a new idle p2p server with no peers
pub fn new( pub fn new(
db_root: String, db_env: Arc<lmdb::Environment>,
mut capab: Capabilities, mut capab: Capabilities,
config: P2PConfig, config: P2PConfig,
adapter: Arc<ChainAdapter>, adapter: Arc<ChainAdapter>,
@ -80,7 +82,7 @@ impl Server {
config: config.clone(), config: config.clone(),
capabilities: capab, capabilities: capab,
handshake: Arc::new(Handshake::new(genesis, config.clone())), handshake: Arc::new(Handshake::new(genesis, config.clone())),
peers: Arc::new(Peers::new(PeerStore::new(db_root)?, adapter, config)), peers: Arc::new(Peers::new(PeerStore::new(db_env)?, adapter, config)),
stop: stop, stop: stop,
}) })
} }

View file

@ -17,6 +17,9 @@
use num::FromPrimitive; use num::FromPrimitive;
use rand::{thread_rng, Rng}; use rand::{thread_rng, Rng};
use std::net::SocketAddr; use std::net::SocketAddr;
use std::sync::Arc;
use lmdb;
use core::ser::{self, Readable, Reader, Writeable, Writer}; use core::ser::{self, Readable, Reader, Writeable, Writer};
use grin_store::{self, option_to_not_found, to_key, Error}; use grin_store::{self, option_to_not_found, to_key, Error};
@ -101,15 +104,17 @@ pub struct PeerStore {
impl PeerStore { impl PeerStore {
/// Instantiates a new peer store under the provided root path. /// Instantiates a new peer store under the provided root path.
pub fn new(root_path: String) -> Result<PeerStore, Error> { pub fn new(db_env: Arc<lmdb::Environment>) -> Result<PeerStore, Error> {
let db = grin_store::Store::open(format!("{}/{}", root_path, STORE_SUBPATH).as_str())?; let db = grin_store::Store::open(db_env, STORE_SUBPATH);
Ok(PeerStore { db: db }) Ok(PeerStore { db: db })
} }
pub fn save_peer(&self, p: &PeerData) -> Result<(), Error> { pub fn save_peer(&self, p: &PeerData) -> Result<(), Error> {
debug!(LOGGER, "save_peer: {:?} marked {:?}", p.addr, p.flags); debug!(LOGGER, "save_peer: {:?} marked {:?}", p.addr, p.flags);
self.db.put_ser(&peer_key(p.addr)[..], p) let batch = self.db.batch()?;
batch.put_ser(&peer_key(p.addr)[..], p)?;
batch.commit()
} }
pub fn get_peer(&self, peer_addr: SocketAddr) -> Result<PeerData, Error> { pub fn get_peer(&self, peer_addr: SocketAddr) -> Result<PeerData, Error> {
@ -123,12 +128,15 @@ impl PeerStore {
/// TODO - allow below added to avoid github issue reports /// TODO - allow below added to avoid github issue reports
#[allow(dead_code)] #[allow(dead_code)]
pub fn delete_peer(&self, peer_addr: SocketAddr) -> Result<(), Error> { pub fn delete_peer(&self, peer_addr: SocketAddr) -> Result<(), Error> {
self.db.delete(&peer_key(peer_addr)[..]) let batch = self.db.batch()?;
batch.delete(&peer_key(peer_addr)[..])?;
batch.commit()
} }
pub fn find_peers(&self, state: State, cap: Capabilities, count: usize) -> Vec<PeerData> { pub fn find_peers(&self, state: State, cap: Capabilities, count: usize) -> Vec<PeerData> {
let mut peers = self.db let mut peers = self.db
.iter::<PeerData>(&to_key(PEER_PREFIX, &mut "".to_string().into_bytes())) .iter::<PeerData>(&to_key(PEER_PREFIX, &mut "".to_string().into_bytes()))
.unwrap()
.filter(|p| p.flags == state && p.capabilities.contains(cap)) .filter(|p| p.flags == state && p.capabilities.contains(cap))
.collect::<Vec<_>>(); .collect::<Vec<_>>();
thread_rng().shuffle(&mut peers[..]); thread_rng().shuffle(&mut peers[..]);
@ -138,9 +146,8 @@ impl PeerStore {
/// List all known peers /// List all known peers
/// Used for /v1/peers/all api endpoint /// Used for /v1/peers/all api endpoint
pub fn all_peers(&self) -> Vec<PeerData> { pub fn all_peers(&self) -> Vec<PeerData> {
self.db let key = to_key(PEER_PREFIX, &mut "".to_string().into_bytes());
.iter::<PeerData>(&to_key(PEER_PREFIX, &mut "".to_string().into_bytes())) self.db.iter::<PeerData>(&key).unwrap().collect::<Vec<_>>()
.collect::<Vec<_>>()
} }
/// Convenience method to load a peer data, update its status and save it /// Convenience method to load a peer data, update its status and save it

View file

@ -14,6 +14,8 @@
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_p2p as p2p; extern crate grin_p2p as p2p;
extern crate grin_pool as pool;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
use std::net::{SocketAddr, TcpListener, TcpStream}; use std::net::{SocketAddr, TcpListener, TcpStream};
@ -47,9 +49,10 @@ fn peer_handshake() {
..p2p::P2PConfig::default() ..p2p::P2PConfig::default()
}; };
let net_adapter = Arc::new(p2p::DummyAdapter {}); let net_adapter = Arc::new(p2p::DummyAdapter {});
let db_env = Arc::new(store::new_env(".grin".to_string()));
let server = Arc::new( let server = Arc::new(
p2p::Server::new( p2p::Server::new(
".grin".to_owned(), db_env,
p2p::Capabilities::UNKNOWN, p2p::Capabilities::UNKNOWN,
p2p_config.clone(), p2p_config.clone(),
net_adapter.clone(), net_adapter.clone(),

View file

@ -15,6 +15,7 @@ time = "0.1"
grin_core = { path = "../core" } grin_core = { path = "../core" }
grin_keychain = { path = "../keychain" } grin_keychain = { path = "../keychain" }
grin_store = { path = "../store" }
grin_util = { path = "../util" } grin_util = { path = "../util" }
[dev-dependencies] [dev-dependencies]

View file

@ -55,11 +55,15 @@ fn test_transaction_pool_block_building() {
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap(); let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap(); let mut batch = chain.store.batch().unwrap();
txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)
}).unwrap();
let tip = Tip::from_block(&block.header); let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap(); batch.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap(); batch.save_head(&tip).unwrap();
batch.commit().unwrap();
block.header block.header
}; };
@ -126,11 +130,13 @@ fn test_transaction_pool_block_building() {
}.unwrap(); }.unwrap();
{ {
let mut batch = chain.store.batch().unwrap();
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| { txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)?; extension.apply_block(&block)?;
Ok(()) Ok(())
}).unwrap(); }).unwrap();
batch.commit().unwrap();
} }
// Now reconcile the transaction pool with the new block // Now reconcile the transaction pool with the new block

View file

@ -53,12 +53,16 @@ fn test_transaction_pool_block_reconciliation() {
let reward = libtx::reward::output(&keychain, &key_id, 0, height).unwrap(); let reward = libtx::reward::output(&keychain, &key_id, 0, height).unwrap();
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap(); let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut batch = chain.store.batch().unwrap();
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap(); txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)
}).unwrap();
let tip = Tip::from_block(&block.header); let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap(); batch.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap(); batch.save_head(&tip).unwrap();
batch.commit().unwrap();
block.header block.header
}; };
@ -76,17 +80,19 @@ fn test_transaction_pool_block_reconciliation() {
let reward = libtx::reward::output(&keychain, &key_id, fees, 0).unwrap(); let reward = libtx::reward::output(&keychain, &key_id, fees, 0).unwrap();
let block = Block::new(&header, vec![initial_tx], Difficulty::one(), reward).unwrap(); let block = Block::new(&header, vec![initial_tx], Difficulty::one(), reward).unwrap();
let mut batch = chain.store.batch().unwrap();
{ {
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| { txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)?; extension.apply_block(&block)?;
Ok(()) Ok(())
}).unwrap(); }).unwrap();
} }
let tip = Tip::from_block(&block.header); let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap(); batch.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap(); batch.save_head(&tip).unwrap();
batch.commit().unwrap();
block.header block.header
}; };
@ -174,16 +180,20 @@ fn test_transaction_pool_block_reconciliation() {
let block = Block::new(&header, block_txs, Difficulty::one(), reward).unwrap(); let block = Block::new(&header, block_txs, Difficulty::one(), reward).unwrap();
{ {
let mut batch = chain.store.batch().unwrap();
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| { txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)?; extension.apply_block(&block)?;
Ok(()) Ok(())
}).unwrap(); }).unwrap();
batch.commit().unwrap();
} }
let tip = Tip::from_block(&block.header); let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap(); let batch = chain.store.batch().unwrap();
chain.store.save_head(&tip).unwrap(); batch.save_block_header(&block.header).unwrap();
batch.save_head(&tip).unwrap();
batch.commit().unwrap();
block block
}; };

View file

@ -19,6 +19,7 @@ extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_pool as pool; extern crate grin_pool as pool;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
@ -30,7 +31,7 @@ use std::sync::{Arc, RwLock};
use core::core::{BlockHeader, Transaction}; use core::core::{BlockHeader, Transaction};
use chain::store::ChainKVStore; use chain::store::ChainStore;
use chain::txhashset; use chain::txhashset;
use chain::txhashset::TxHashSet; use chain::txhashset::TxHashSet;
use core::core::hash::Hashed; use core::core::hash::Hashed;
@ -46,14 +47,15 @@ use pool::types::*;
#[derive(Clone)] #[derive(Clone)]
pub struct ChainAdapter { pub struct ChainAdapter {
pub txhashset: Arc<RwLock<TxHashSet>>, pub txhashset: Arc<RwLock<TxHashSet>>,
pub store: Arc<ChainKVStore>, pub store: Arc<ChainStore>,
} }
impl ChainAdapter { impl ChainAdapter {
pub fn init(db_root: String) -> Result<ChainAdapter, String> { pub fn init(db_root: String) -> Result<ChainAdapter, String> {
let target_dir = format!("target/{}", db_root); let target_dir = format!("target/{}", db_root);
let chain_store = ChainKVStore::new(target_dir.clone()) let db_env = Arc::new(store::new_env(target_dir.clone()));
.map_err(|e| format!("failed to init chain_store, {}", e))?; let chain_store =
ChainStore::new(db_env).map_err(|e| format!("failed to init chain_store, {:?}", e))?;
let store = Arc::new(chain_store); let store = Arc::new(chain_store);
let txhashset = TxHashSet::open(target_dir.clone(), store.clone(), None) let txhashset = TxHashSet::open(target_dir.clone(), store.clone(), None)
.map_err(|e| format!("failed to init txhashset, {}", e))?; .map_err(|e| format!("failed to init txhashset, {}", e))?;

View file

@ -54,11 +54,15 @@ fn test_the_transaction_pool() {
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap(); let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut txhashset = chain.txhashset.write().unwrap(); let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap(); let mut batch = chain.store.batch().unwrap();
txhashset::extending(&mut txhashset, &mut batch, |extension| {
extension.apply_block(&block)
}).unwrap();
let tip = Tip::from_block(&block.header); let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap(); batch.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap(); batch.save_head(&tip).unwrap();
batch.commit().unwrap();
block.header block.header
}; };

View file

@ -8,6 +8,7 @@ publish = false
[dependencies] [dependencies]
hyper = "0.10" hyper = "0.10"
itertools = "0.7" itertools = "0.7"
lmdb-zero = "0.4.4"
rand = "0.3" rand = "0.3"
router = "0.6" router = "0.6"
slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] } slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] }

View file

@ -35,6 +35,7 @@ use mining::stratumserver;
use mining::test_miner::Miner; use mining::test_miner::Miner;
use p2p; use p2p;
use pool; use pool;
use store;
use util::LOGGER; use util::LOGGER;
/// Grin server holding internal structures. /// Grin server holding internal structures.
@ -130,8 +131,10 @@ impl Server {
info!(LOGGER, "Starting server, genesis block: {}", genesis.hash()); info!(LOGGER, "Starting server, genesis block: {}", genesis.hash());
let db_env = Arc::new(store::new_env(config.db_root.clone()));
let shared_chain = Arc::new(chain::Chain::init( let shared_chain = Arc::new(chain::Chain::init(
config.db_root.clone(), config.db_root.clone(),
db_env.clone(),
chain_adapter.clone(), chain_adapter.clone(),
genesis.clone(), genesis.clone(),
pow::verify_size, pow::verify_size,
@ -156,7 +159,7 @@ impl Server {
}; };
let p2p_server = Arc::new(p2p::Server::new( let p2p_server = Arc::new(p2p::Server::new(
config.db_root.clone(), db_env,
config.capabilities, config.capabilities,
config.p2p_config.clone(), config.p2p_config.clone(),
net_adapter.clone(), net_adapter.clone(),

View file

@ -25,6 +25,7 @@ extern crate bufstream;
extern crate hyper; extern crate hyper;
extern crate itertools; extern crate itertools;
extern crate jsonrpc_core; extern crate jsonrpc_core;
extern crate lmdb_zero as lmdb;
extern crate rand; extern crate rand;
extern crate serde; extern crate serde;
#[macro_use] #[macro_use]
@ -48,6 +49,6 @@ pub mod common;
mod grin; mod grin;
mod mining; mod mining;
pub use grin::server::Server;
pub use common::types::{Seeding, ServerConfig, StratumServerConfig};
pub use common::stats::{DiffBlock, PeerStats, ServerStats, StratumStats, WorkerStats}; pub use common::stats::{DiffBlock, PeerStats, ServerStats, StratumStats, WorkerStats};
pub use common::types::{Seeding, ServerConfig, StratumServerConfig};
pub use grin::server::Server;

View file

@ -51,7 +51,7 @@ use daemonize::Daemonize;
use config::GlobalConfig; use config::GlobalConfig;
use core::core::amount_to_hr_string; use core::core::amount_to_hr_string;
use core::global; use core::global;
use keychain::ExtKeychain; use keychain::{self, ExtKeychain};
use tui::ui; use tui::ui;
use util::{init_logger, LoggingConfig, LOGGER}; use util::{init_logger, LoggingConfig, LOGGER};
use wallet::{libwallet, FileWallet}; use wallet::{libwallet, FileWallet};
@ -446,8 +446,8 @@ fn server_command(server_args: Option<&ArgMatches>, mut global_config: GlobalCon
let _ = thread::Builder::new() let _ = thread::Builder::new()
.name("wallet_listener".to_string()) .name("wallet_listener".to_string())
.spawn(move || { .spawn(move || {
let wallet: FileWallet<ExtKeychain> = FileWallet::new(wallet_config.clone(), "") let wallet: FileWallet<keychain::ExtKeychain> =
.unwrap_or_else(|e| { FileWallet::new(wallet_config.clone(), "").unwrap_or_else(|e| {
panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config) panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config)
}); });
wallet::controller::foreign_listener(wallet, &wallet_config.api_listen_addr()) wallet::controller::foreign_listener(wallet, &wallet_config.api_listen_addr())
@ -586,8 +586,8 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
// Handle listener startup commands // Handle listener startup commands
{ {
let wallet: FileWallet<ExtKeychain> = FileWallet::new(wallet_config.clone(), passphrase) let wallet: FileWallet<keychain::ExtKeychain> =
.unwrap_or_else(|e| { FileWallet::new(wallet_config.clone(), passphrase).unwrap_or_else(|e| {
panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config) panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config)
}); });
match wallet_args.subcommand() { match wallet_args.subcommand() {
@ -617,7 +617,7 @@ fn wallet_command(wallet_args: &ArgMatches, global_config: GlobalConfig) {
// Handle single-use (command line) owner commands // Handle single-use (command line) owner commands
{ {
let mut wallet: FileWallet<ExtKeychain> = let mut wallet: FileWallet<keychain::ExtKeychain> =
FileWallet::new(wallet_config.clone(), passphrase).unwrap_or_else(|e| { FileWallet::new(wallet_config.clone(), passphrase).unwrap_or_else(|e| {
panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config) panic!("Error creating wallet: {:?} Config: {:?}", e, wallet_config)
}); });

View file

@ -10,9 +10,8 @@ byteorder = "1"
croaring = "0.3" croaring = "0.3"
env_logger = "0.5" env_logger = "0.5"
libc = "0.2" libc = "0.2"
lmdb-zero = "0.4.4"
memmap = { git = "https://github.com/danburkert/memmap-rs", tag = "0.6.2" } memmap = { git = "https://github.com/danburkert/memmap-rs", tag = "0.6.2" }
rand = "0.3"
rocksdb = "0.10"
serde = "1" serde = "1"
serde_derive = "1" serde_derive = "1"
slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] } slog = { version = "~2.2", features = ["max_level_trace", "release_max_level_trace"] }
@ -22,3 +21,4 @@ grin_util = { path = "../util" }
[dev-dependencies] [dev-dependencies]
time = "0.1" time = "0.1"
rand = "0.3"

View file

@ -23,16 +23,18 @@
extern crate byteorder; extern crate byteorder;
extern crate croaring; extern crate croaring;
extern crate env_logger; extern crate env_logger;
#[macro_use]
extern crate grin_core as core;
extern crate grin_util as util;
extern crate libc; extern crate libc;
extern crate lmdb_zero;
extern crate memmap; extern crate memmap;
extern crate rocksdb;
extern crate serde; extern crate serde;
#[macro_use] #[macro_use]
extern crate slog; extern crate slog;
#[macro_use]
extern crate grin_core as core;
extern crate grin_util as util;
mod lmdb;
pub mod leaf_set; pub mod leaf_set;
pub mod pmmr; pub mod pmmr;
pub mod rm_log; pub mod rm_log;
@ -40,209 +42,34 @@ pub mod types;
const SEP: u8 = ':' as u8; const SEP: u8 = ':' as u8;
use std::fmt;
use std::iter::Iterator;
use std::marker;
use std::sync::RwLock;
use byteorder::{BigEndian, WriteBytesExt}; use byteorder::{BigEndian, WriteBytesExt};
use rocksdb::{DBCompactionStyle, DBIterator, Direction, IteratorMode, WriteBatch, DB};
use core::ser; pub use lmdb::*;
/// Main error type for this crate.
#[derive(Debug)]
pub enum Error {
/// Couldn't find what we were looking for
NotFoundErr,
/// Wraps an error originating from RocksDB (which unfortunately returns
/// string errors).
RocksDbErr(String),
/// Wraps a serialization error for Writeable or Readable
SerErr(ser::Error),
}
impl fmt::Display for Error {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
&Error::NotFoundErr => write!(f, "Not Found"),
&Error::RocksDbErr(ref s) => write!(f, "RocksDb Error: {}", s),
&Error::SerErr(ref e) => write!(f, "Serialization Error: {}", e.to_string()),
}
}
}
impl From<rocksdb::Error> for Error {
fn from(e: rocksdb::Error) -> Error {
Error::RocksDbErr(e.to_string())
}
}
/// Thread-safe rocksdb wrapper
pub struct Store {
rdb: RwLock<DB>,
}
unsafe impl Sync for Store {}
unsafe impl Send for Store {}
impl Store {
/// Opens a new RocksDB at the specified location.
pub fn open(path: &str) -> Result<Store, Error> {
let mut opts = rocksdb::Options::default();
opts.create_if_missing(true);
opts.set_compaction_style(DBCompactionStyle::Universal);
opts.set_max_open_files(256);
opts.set_use_fsync(false);
let db = DB::open(&opts, &path)?;
Ok(Store {
rdb: RwLock::new(db),
})
}
/// Writes a single key/value pair to the db
pub fn put(&self, key: &[u8], value: Vec<u8>) -> Result<(), Error> {
let db = self.rdb.write().unwrap();
db.put(key, &value[..]).map_err(&From::from)
}
/// Writes a single key and its `Writeable` value to the db. Encapsulates
/// serialization.
pub fn put_ser<W: ser::Writeable>(&self, key: &[u8], value: &W) -> Result<(), Error> {
let ser_value = ser::ser_vec(value);
match ser_value {
Ok(data) => self.put(key, data),
Err(err) => Err(Error::SerErr(err)),
}
}
/// Gets a value from the db, provided its key
pub fn get(&self, key: &[u8]) -> Result<Option<Vec<u8>>, Error> {
let db = self.rdb.read().unwrap();
db.get(key)
.map(|r| r.map(|o| o.to_vec()))
.map_err(From::from)
}
/// Gets a `Readable` value from the db, provided its key. Encapsulates
/// serialization.
pub fn get_ser<T: ser::Readable>(&self, key: &[u8]) -> Result<Option<T>, Error> {
self.get_ser_limited(key, 0)
}
/// Gets a `Readable` value from the db, provided its key, allowing to
/// extract only partial data. The underlying Readable size must align
/// accordingly. Encapsulates serialization.
pub fn get_ser_limited<T: ser::Readable>(
&self,
key: &[u8],
len: usize,
) -> Result<Option<T>, Error> {
let data = self.get(key)?;
match data {
Some(val) => {
let mut lval = if len > 0 { &val[..len] } else { &val[..] };
let r = ser::deserialize(&mut lval).map_err(Error::SerErr)?;
Ok(Some(r))
}
None => Ok(None),
}
}
/// Whether the provided key exists
pub fn exists(&self, key: &[u8]) -> Result<bool, Error> {
let db = self.rdb.read().unwrap();
db.get(key).map(|r| r.is_some()).map_err(From::from)
}
/// Deletes a key/value pair from the db
pub fn delete(&self, key: &[u8]) -> Result<(), Error> {
let db = self.rdb.write().unwrap();
db.delete(key).map_err(From::from)
}
/// Produces an iterator of `Readable` types moving forward from the
/// provided
/// key.
pub fn iter<T: ser::Readable>(&self, from: &[u8]) -> SerIterator<T> {
let db = self.rdb.read().unwrap();
SerIterator {
iter: db.iterator(IteratorMode::From(from, Direction::Forward)),
_marker: marker::PhantomData,
}
}
/// Builds a new batch to be used with this store.
pub fn batch(&self) -> Batch {
Batch {
store: self,
batch: WriteBatch::default(),
}
}
fn write(&self, batch: WriteBatch) -> Result<(), Error> {
let db = self.rdb.write().unwrap();
db.write(batch).map_err(From::from)
}
}
/// Batch to write multiple Writeables to RocksDb in an atomic manner.
pub struct Batch<'a> {
store: &'a Store,
batch: WriteBatch,
}
impl<'a> Batch<'a> {
/// Writes a single key and its `Writeable` value to the batch. The write
/// function must be called to "commit" the batch to storage.
pub fn put_ser<W: ser::Writeable>(mut self, key: &[u8], value: &W) -> Result<Batch<'a>, Error> {
let ser_value = ser::ser_vec(value);
match ser_value {
Ok(data) => {
self.batch.put(key, &data[..])?;
Ok(self)
}
Err(err) => Err(Error::SerErr(err)),
}
}
/// Delete a single key from the batch. The write function
/// must be called to "commit" the batch to storage.
pub fn delete(mut self, key: &[u8]) -> Result<Batch<'a>, Error> {
self.batch.delete(key)?;
Ok(self)
}
/// Writes the batch to RocksDb.
pub fn write(self) -> Result<(), Error> {
self.store.write(self.batch)
}
}
/// An iterator thad produces Readable instances back. Wraps the lower level /// An iterator thad produces Readable instances back. Wraps the lower level
/// DBIterator and deserializes the returned values. /// DBIterator and deserializes the returned values.
pub struct SerIterator<T> // pub struct SerIterator<T>
where // where
T: ser::Readable, // T: ser::Readable,
{ // {
iter: DBIterator, // iter: DBIterator,
_marker: marker::PhantomData<T>, // _marker: marker::PhantomData<T>,
} // }
//
impl<T> Iterator for SerIterator<T> // impl<T> Iterator for SerIterator<T>
where // where
T: ser::Readable, // T: ser::Readable,
{ // {
type Item = T; // type Item = T;
//
fn next(&mut self) -> Option<T> { // fn next(&mut self) -> Option<T> {
let next = self.iter.next(); // let next = self.iter.next();
next.and_then(|r| { // next.and_then(|r| {
let (_, v) = r; // let (_, v) = r;
ser::deserialize(&mut &v[..]).ok() // ser::deserialize(&mut &v[..]).ok()
}) // })
} // }
} // }
/// Build a db key from a prefix and a byte vector identifier. /// Build a db key from a prefix and a byte vector identifier.
pub fn to_key(prefix: u8, k: &mut Vec<u8>) -> Vec<u8> { pub fn to_key(prefix: u8, k: &mut Vec<u8>) -> Vec<u8> {
@ -261,12 +88,3 @@ pub fn u64_to_key<'a>(prefix: u8, val: u64) -> Vec<u8> {
u64_vec.insert(0, prefix); u64_vec.insert(0, prefix);
u64_vec u64_vec
} }
/// unwraps the inner option by converting the none case to a not found error
pub fn option_to_not_found<T>(res: Result<Option<T>, Error>) -> Result<T, Error> {
match res {
Ok(None) => Err(Error::NotFoundErr),
Ok(Some(o)) => Ok(o),
Err(e) => Err(e),
}
}

270
store/src/lmdb.rs Normal file
View file

@ -0,0 +1,270 @@
// 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.
//! Storage of core types using LMDB.
use std::fs;
use std::marker;
use std::sync::Arc;
use lmdb_zero as lmdb;
use lmdb_zero::LmdbResultExt;
use lmdb_zero::traits::CreateCursor;
use core::ser;
/// Main error type for this lmdb
#[derive(Debug)]
pub enum Error {
/// Couldn't find what we were looking for
NotFoundErr,
/// Wraps an error originating from RocksDB (which unfortunately returns
/// string errors).
LmdbErr(lmdb::error::Error),
/// Wraps a serialization error for Writeable or Readable
SerErr(ser::Error),
}
impl From<lmdb::error::Error> for Error {
fn from(e: lmdb::error::Error) -> Error {
Error::LmdbErr(e)
}
}
/// unwraps the inner option by converting the none case to a not found error
pub fn option_to_not_found<T>(res: Result<Option<T>, Error>) -> Result<T, Error> {
match res {
Ok(None) => Err(Error::NotFoundErr),
Ok(Some(o)) => Ok(o),
Err(e) => Err(e),
}
}
/// Create a new LMDB env under the provided directory to spawn various
/// databases from.
pub fn new_env(path: String) -> lmdb::Environment {
let full_path = path + "/lmdb";
fs::create_dir_all(&full_path).unwrap();
unsafe {
let mut env_builder = lmdb::EnvBuilder::new().unwrap();
env_builder.set_maxdbs(8).unwrap();
// half a TB should give us plenty room, will be an issue on 32 bits
// (which we don't support anyway)
env_builder.set_mapsize(549755813888).unwrap_or_else(|e| {
panic!("Unable to allocate LMDB space: {:?}", e);
});
env_builder
.open(&full_path, lmdb::open::Flags::empty(), 0o600)
.unwrap()
}
}
/// LMDB-backed store facilitating data access and serialization. All writes
/// are done through a Batch abstraction providing atomicity.
pub struct Store {
env: Arc<lmdb::Environment>,
db: Arc<lmdb::Database<'static>>,
}
impl Store {
/// Creates a new store with the provided name under the specified
/// environment
pub fn open(env: Arc<lmdb::Environment>, name: &str) -> Store {
let db = Arc::new(
lmdb::Database::open(
env.clone(),
Some(name),
&lmdb::DatabaseOptions::new(lmdb::db::CREATE),
).unwrap(),
);
Store { env, db }
}
/// Gets a value from the db, provided its key
pub fn get(&self, key: &[u8]) -> Result<Option<Vec<u8>>, Error> {
let txn = lmdb::ReadTransaction::new(self.env.clone())?;
let access = txn.access();
let res = access.get(&self.db, key);
res.map(|res: &[u8]| res.to_vec())
.to_opt()
.map_err(From::from)
}
/// Gets a `Readable` value from the db, provided its key. Encapsulates
/// serialization.
pub fn get_ser<T: ser::Readable>(&self, key: &[u8]) -> Result<Option<T>, Error> {
let txn = lmdb::ReadTransaction::new(self.env.clone())?;
let access = txn.access();
self.get_ser_access(key, &access)
}
fn get_ser_access<T: ser::Readable>(
&self,
key: &[u8],
access: &lmdb::ConstAccessor,
) -> Result<Option<T>, Error> {
let res: lmdb::error::Result<&[u8]> = access.get(&self.db, key);
match res.to_opt() {
Ok(Some(mut res)) => match ser::deserialize(&mut res).map_err(Error::SerErr) {
Ok(res) => Ok(Some(res)),
Err(e) => Err(From::from(e)),
},
Ok(None) => Ok(None),
Err(e) => Err(From::from(e)),
}
}
/// Whether the provided key exists
pub fn exists(&self, key: &[u8]) -> Result<bool, Error> {
let txn = lmdb::ReadTransaction::new(self.env.clone())?;
let access = txn.access();
let res: lmdb::error::Result<&lmdb::Ignore> = access.get(&self.db, key);
res.to_opt().map(|r| r.is_some()).map_err(From::from)
}
/// Produces an iterator of `Readable` types moving forward from the
/// provided key.
pub fn iter<T: ser::Readable>(&self, from: &[u8]) -> Result<SerIterator<T>, Error> {
let txn = Arc::new(lmdb::ReadTransaction::new(self.env.clone())?);
let cursor = Arc::new(txn.cursor(self.db.clone()).unwrap());
Ok(SerIterator {
tx: txn,
cursor: cursor,
seek: false,
prefix: from.to_vec(),
_marker: marker::PhantomData,
})
}
/// Builds a new batch to be used with this store.
pub fn batch(&self) -> Result<Batch, Error> {
let txn = lmdb::WriteTransaction::new(self.env.clone())?;
Ok(Batch {
store: self,
tx: txn,
})
}
}
/// Batch to write multiple Writeables to db in an atomic manner.
pub struct Batch<'a> {
store: &'a Store,
tx: lmdb::WriteTransaction<'a>,
}
impl<'a> Batch<'a> {
/// Writes a single key/value pair to the db
pub fn put(&self, key: &[u8], value: Vec<u8>) -> Result<(), Error> {
self.tx
.access()
.put(&self.store.db, key, &value, lmdb::put::Flags::empty())?;
Ok(())
}
/// Writes a single key and its `Writeable` value to the db. Encapsulates
/// serialization.
pub fn put_ser<W: ser::Writeable>(&self, key: &[u8], value: &W) -> Result<(), Error> {
let ser_value = ser::ser_vec(value);
match ser_value {
Ok(data) => self.put(key, data),
Err(err) => Err(Error::SerErr(err)),
}
}
/// gets a value from the db, provided its key
pub fn get(&self, key: &[u8]) -> Result<Option<Vec<u8>>, Error> {
self.store.get(key)
}
/// Gets a `Readable` value from the db, provided its key, taking the
/// content of the current batch into account.
pub fn get_ser<T: ser::Readable>(&self, key: &[u8]) -> Result<Option<T>, Error> {
let access = self.tx.access();
self.store.get_ser_access(key, &access)
}
/// Deletes a key/value pair from the db
pub fn delete(&self, key: &[u8]) -> Result<(), Error> {
self.tx.access().del_key(&self.store.db, key)?;
Ok(())
}
/// Writes the batch to db
pub fn commit(self) -> Result<(), Error> {
self.tx.commit()?;
Ok(())
}
/// Creates a child of this batch. It will be merged with its parent on
/// commit, abandoned otherwise.
pub fn child(&mut self) -> Result<Batch, Error> {
Ok(Batch {
store: self.store,
tx: self.tx.child_tx()?,
})
}
}
/// An iterator thad produces Readable instances back. Wraps the lower level
/// DBIterator and deserializes the returned values.
pub struct SerIterator<T>
where
T: ser::Readable,
{
tx: Arc<lmdb::ReadTransaction<'static>>,
cursor: Arc<lmdb::Cursor<'static, 'static>>,
seek: bool,
prefix: Vec<u8>,
_marker: marker::PhantomData<T>,
}
impl<T> Iterator for SerIterator<T>
where
T: ser::Readable,
{
type Item = T;
fn next(&mut self) -> Option<T> {
let access = self.tx.access();
let kv = if self.seek {
Arc::get_mut(&mut self.cursor).unwrap().next(&access)
} else {
self.seek = true;
Arc::get_mut(&mut self.cursor)
.unwrap()
.seek_range_k(&access, &self.prefix[..])
};
self.deser_if_prefix_match(kv)
}
}
impl<T> SerIterator<T>
where
T: ser::Readable,
{
fn deser_if_prefix_match(&self, kv: Result<(&[u8], &[u8]), lmdb::Error>) -> Option<T> {
match kv {
Ok((k, v)) => {
let plen = self.prefix.len();
if plen == 0 || k[0..plen] == self.prefix[..] {
ser::deserialize(&mut &v[..]).ok()
} else {
None
}
}
Err(_) => None,
}
}
}

View file

@ -12,12 +12,12 @@
// limitations under the License. // limitations under the License.
//! Logging wrapper to be used throughout all crates in the workspace //! Logging wrapper to be used throughout all crates in the workspace
use std::fs::OpenOptions;
use std::sync::Mutex;
use std::ops::Deref;
use slog::{Discard, Drain, Duplicate, Level, LevelFilter, Logger}; use slog::{Discard, Drain, Duplicate, Level, LevelFilter, Logger};
use slog_term;
use slog_async; use slog_async;
use slog_term;
use std::fs::OpenOptions;
use std::ops::Deref;
use std::sync::Mutex;
use backtrace::Backtrace; use backtrace::Backtrace;
use std::{panic, thread}; use std::{panic, thread};
@ -110,7 +110,6 @@ pub fn init_test_logger() {
let mut config_ref = LOGGING_CONFIG.lock().unwrap(); let mut config_ref = LOGGING_CONFIG.lock().unwrap();
*config_ref = LoggingConfig::default(); *config_ref = LoggingConfig::default();
*was_init_ref = true; *was_init_ref = true;
send_panic_to_log();
} }
/// hook to send panics to logs as well as stderr /// hook to send panics to logs as well as stderr

View file

@ -30,8 +30,10 @@ urlencoded = "0.5"
grin_api = { path = "../api" } grin_api = { path = "../api" }
grin_core = { path = "../core" } grin_core = { path = "../core" }
grin_keychain = { path = "../keychain" } grin_keychain = { path = "../keychain" }
grin_store = { path = "../store" }
grin_util = { path = "../util" } grin_util = { path = "../util" }
[dev-dependencies] [dev-dependencies]
grin_chain = { path = "../chain" } grin_chain = { path = "../chain" }
grin_store = { path = "../store" }
time = "0.1" time = "0.1"

View file

@ -12,10 +12,8 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
use blake2;
use rand::{thread_rng, Rng};
use std::cmp::min;
use std::collections::HashMap; use std::collections::HashMap;
use std::collections::hash_map::Values;
use std::fs::{self, File}; use std::fs::{self, File};
use std::io::{Read, Write}; use std::io::{Read, Write};
use std::path::{Path, MAIN_SEPARATOR}; use std::path::{Path, MAIN_SEPARATOR};
@ -27,9 +25,9 @@ use tokio_retry::strategy::FibonacciBackoff;
use failure::ResultExt; use failure::ResultExt;
use keychain::{self, Keychain}; use keychain::{self, Identifier, Keychain};
use util::LOGGER;
use util::secp::pedersen; use util::secp::pedersen;
use util::{self, LOGGER};
use error::{Error, ErrorKind}; use error::{Error, ErrorKind};
@ -37,7 +35,8 @@ use client;
use libtx::slate::Slate; use libtx::slate::Slate;
use libwallet; use libwallet;
use libwallet::types::{BlockFees, BlockIdentifier, CbData, MerkleProofWrapper, OutputData, use libwallet::types::{BlockFees, BlockIdentifier, CbData, MerkleProofWrapper, OutputData,
TxWrapper, WalletBackend, WalletClient, WalletDetails}; TxWrapper, WalletBackend, WalletClient, WalletDetails, WalletOutputBatch};
use types::{WalletConfig, WalletSeed};
const DETAIL_FILE: &'static str = "wallet.det"; const DETAIL_FILE: &'static str = "wallet.det";
const DET_BCK_FILE: &'static str = "wallet.detbck"; const DET_BCK_FILE: &'static str = "wallet.detbck";
@ -46,123 +45,70 @@ const BCK_FILE: &'static str = "wallet.bck";
const LOCK_FILE: &'static str = "wallet.lock"; const LOCK_FILE: &'static str = "wallet.lock";
const SEED_FILE: &'static str = "wallet.seed"; const SEED_FILE: &'static str = "wallet.seed";
#[derive(Debug, Clone, Serialize, Deserialize)] #[derive(Debug)]
pub struct WalletConfig { struct FileBatch<'a> {
// Right now the decision to run or not a wallet is based on the command. /// List of outputs
// This may change in the near-future. outputs: &'a mut HashMap<String, OutputData>,
// pub enable_wallet: bool, /// Data file path
data_file_path: String,
// The api interface/ip_address that this api server (i.e. this wallet) will run /// lock file path
// by default this is 127.0.0.1 (and will not accept connections from external clients) lock_file_path: String,
pub api_listen_interface: String,
// The port this wallet will run on
pub api_listen_port: u16,
// The api address of a running server node against which transaction inputs
// will be checked during send
pub check_node_api_http_addr: String,
// The directory in which wallet files are stored
pub data_file_dir: String,
} }
impl Default for WalletConfig { impl<'a> WalletOutputBatch for FileBatch<'a> {
fn default() -> WalletConfig { fn save(&mut self, out: OutputData) -> Result<(), libwallet::Error> {
WalletConfig { let _ = self.outputs.insert(out.key_id.to_hex(), out);
// enable_wallet: false, Ok(())
api_listen_interface: "127.0.0.1".to_string(),
api_listen_port: 13415,
check_node_api_http_addr: "http://127.0.0.1:13413".to_string(),
data_file_dir: ".".to_string(),
} }
fn get(&self, id: &Identifier) -> Result<OutputData, libwallet::Error> {
self.outputs
.get(&id.to_hex())
.map(|od| od.clone())
.ok_or(libwallet::ErrorKind::Backend("not found".to_string()).into())
}
fn delete(&mut self, id: &Identifier) -> Result<(), libwallet::Error> {
let _ = self.outputs.remove(&id.to_hex());
Ok(())
}
fn lock_output(&mut self, out: &mut OutputData) -> Result<(), libwallet::Error> {
if let Some(out_to_lock) = self.outputs.get_mut(&out.key_id.to_hex()) {
if out_to_lock.value == out.value {
out_to_lock.lock()
}
}
Ok(())
}
fn commit(&self) -> Result<(), libwallet::Error> {
let mut data_file = File::create(self.data_file_path.clone())
.context(libwallet::ErrorKind::CallbackImpl("Could not create"))?;
let mut outputs = self.outputs.values().collect::<Vec<_>>();
outputs.sort();
let res_json = serde_json::to_vec_pretty(&outputs).context(
libwallet::ErrorKind::CallbackImpl("Error serializing wallet data"),
)?;
data_file
.write_all(res_json.as_slice())
.context(libwallet::ErrorKind::CallbackImpl(
"Error writing wallet file",
))
.map_err(|e| e.into())
} }
} }
impl WalletConfig { impl<'a> Drop for FileBatch<'a> {
pub fn api_listen_addr(&self) -> String { fn drop(&mut self) {
format!("{}:{}", self.api_listen_interface, self.api_listen_port) // delete the lock file
} if let Err(e) = fs::remove_dir(&self.lock_file_path) {
}
#[derive(Clone, PartialEq)]
pub struct WalletSeed([u8; 32]);
impl WalletSeed {
pub fn from_bytes(bytes: &[u8]) -> WalletSeed {
let mut seed = [0; 32];
for i in 0..min(32, bytes.len()) {
seed[i] = bytes[i];
}
WalletSeed(seed)
}
fn from_hex(hex: &str) -> Result<WalletSeed, Error> {
let bytes =
util::from_hex(hex.to_string()).context(ErrorKind::GenericError("Invalid hex"))?;
Ok(WalletSeed::from_bytes(&bytes))
}
pub fn to_hex(&self) -> String {
util::to_hex(self.0.to_vec())
}
pub fn derive_keychain<K: Keychain>(&self, password: &str) -> Result<K, Error> {
let seed = blake2::blake2b::blake2b(64, &password.as_bytes(), &self.0);
let result = K::from_seed(seed.as_bytes())?;
Ok(result)
}
pub fn init_new() -> WalletSeed {
let seed: [u8; 32] = thread_rng().gen();
WalletSeed(seed)
}
pub fn init_file(wallet_config: &WalletConfig) -> Result<WalletSeed, Error> {
// create directory if it doesn't exist
fs::create_dir_all(&wallet_config.data_file_dir).context(ErrorKind::IO)?;
let seed_file_path = &format!(
"{}{}{}",
wallet_config.data_file_dir, MAIN_SEPARATOR, SEED_FILE,
);
debug!(LOGGER, "Generating wallet seed file at: {}", seed_file_path,);
if Path::new(seed_file_path).exists() {
Err(ErrorKind::WalletSeedExists)?
} else {
let seed = WalletSeed::init_new();
let mut file = File::create(seed_file_path).context(ErrorKind::IO)?;
file.write_all(&seed.to_hex().as_bytes())
.context(ErrorKind::IO)?;
Ok(seed)
}
}
pub fn from_file(wallet_config: &WalletConfig) -> Result<WalletSeed, Error> {
// create directory if it doesn't exist
fs::create_dir_all(&wallet_config.data_file_dir).context(ErrorKind::IO)?;
let seed_file_path = &format!(
"{}{}{}",
wallet_config.data_file_dir, MAIN_SEPARATOR, SEED_FILE,
);
debug!(LOGGER, "Using wallet seed file at: {}", seed_file_path,);
if Path::new(seed_file_path).exists() {
let mut file = File::open(seed_file_path).context(ErrorKind::IO)?;
let mut buffer = String::new();
file.read_to_string(&mut buffer).context(ErrorKind::IO)?;
let wallet_seed = WalletSeed::from_hex(&buffer)?;
Ok(wallet_seed)
} else {
error!( error!(
LOGGER, LOGGER,
"wallet seed file {} could not be opened (grin wallet init). \ "Could not remove wallet lock file. Maybe insufficient rights? "
Run \"grin wallet init\" to initialize a new wallet.",
seed_file_path
); );
Err(ErrorKind::WalletSeedDoesntExist)?
} }
info!(LOGGER, "... released wallet lock");
} }
} }
@ -220,113 +166,38 @@ where
self.keychain.as_mut().unwrap() self.keychain.as_mut().unwrap()
} }
/// Return the outputs directly fn iter<'a>(&'a self) -> Box<Iterator<Item = OutputData> + 'a> {
fn outputs(&mut self) -> &mut HashMap<String, OutputData> { Box::new(self.outputs.values().cloned())
&mut self.outputs
} }
/// Allows for reading wallet data (without needing to acquire the write fn get(&self, id: &Identifier) -> Result<OutputData, libwallet::Error> {
/// lock). self.outputs
fn read_wallet<T, F>(&mut self, f: F) -> Result<T, libwallet::Error> .get(&id.to_hex())
where .map(|o| o.clone())
F: FnOnce(&mut Self) -> Result<T, libwallet::Error>, .ok_or(libwallet::ErrorKind::Backend("not found".to_string()).into())
{
self.read_or_create_paths()
.context(libwallet::ErrorKind::CallbackImpl("Error reading wallet"))?;
f(self)
} }
/// Allows the reading and writing of the wallet data within a file lock. fn batch<'a>(&'a mut self) -> Result<Box<WalletOutputBatch + 'a>, libwallet::Error> {
/// Just provide a closure taking a mutable FileWallet. The lock should self.lock()?;
/// be held for as short a period as possible to avoid contention.
/// Note that due to the impossibility to do an actual file lock easily
/// across operating systems, this just creates a lock file with a "should
/// not exist" option.
fn with_wallet<T, F>(&mut self, f: F) -> Result<T, libwallet::Error>
where
F: FnOnce(&mut Self) -> T,
{
// create directory if it doesn't exist
fs::create_dir_all(self.config.data_file_dir.clone()).unwrap_or_else(|why| {
info!(LOGGER, "! {:?}", why.kind());
});
info!(LOGGER, "Acquiring wallet lock ...");
let lock_file_path = self.lock_file_path.clone();
let action = || {
trace!(LOGGER, "making lock file for wallet lock");
fs::create_dir(&lock_file_path)
};
// use tokio_retry to cleanly define some retry logic
let mut core = reactor::Core::new().unwrap();
let retry_strategy = FibonacciBackoff::from_millis(1000).take(10);
let retry_future = Retry::spawn(core.handle(), retry_strategy, action);
let retry_result = core.run(retry_future)
.context(libwallet::ErrorKind::CallbackImpl(
"Failed to acquire lock file",
));
match retry_result {
Ok(_) => {}
Err(e) => {
error!(
LOGGER,
"Failed to acquire wallet lock file (multiple retries)",
);
return Err(e.into());
}
}
// We successfully acquired the lock - so do what needs to be done. // We successfully acquired the lock - so do what needs to be done.
self.read_or_create_paths() self.read_or_create_paths()
.context(libwallet::ErrorKind::CallbackImpl("Lock Error"))?; .context(libwallet::ErrorKind::CallbackImpl("Lock Error"))?;
self.write(&self.backup_file_path, &self.details_bak_path) self.write(&self.backup_file_path, &self.details_bak_path)
.context(libwallet::ErrorKind::CallbackImpl("Write Error"))?; .context(libwallet::ErrorKind::CallbackImpl("Write Error"))?;
let res = f(self);
self.write(&self.data_file_path, &self.details_file_path)
.context(libwallet::ErrorKind::CallbackImpl("Write Error"))?;
// delete the lock file Ok(Box::new(FileBatch {
fs::remove_dir(&self.lock_file_path).context(libwallet::ErrorKind::CallbackImpl( outputs: &mut self.outputs,
&"Could not remove wallet lock file. Maybe insufficient rights? ", data_file_path: self.data_file_path.clone(),
))?; lock_file_path: self.lock_file_path.clone(),
}))
info!(LOGGER, "... released wallet lock");
Ok(res)
}
/// Append a new output data to the wallet data.
/// TODO - we should check for overwriting here - only really valid for
/// unconfirmed coinbase
fn add_output(&mut self, out: OutputData) {
self.outputs.insert(out.key_id.to_hex(), out.clone());
}
// TODO - careful with this, only for Unconfirmed (maybe Locked)?
fn delete_output(&mut self, id: &keychain::Identifier) {
self.outputs.remove(&id.to_hex());
}
/// Lock an output data.
/// TODO - we should track identifier on these outputs (not just n_child)
fn lock_output(&mut self, out: &OutputData) {
if let Some(out_to_lock) = self.outputs.get_mut(&out.key_id.to_hex()) {
if out_to_lock.value == out.value {
out_to_lock.lock()
}
}
}
/// get a single output
fn get_output(&self, key_id: &keychain::Identifier) -> Option<&OutputData> {
self.outputs.get(&key_id.to_hex())
} }
/// Next child index when we want to create a new output. /// Next child index when we want to create a new output.
fn next_child(&mut self, root_key_id: keychain::Identifier) -> u32 { fn next_child<'a>(
&'a mut self,
root_key_id: keychain::Identifier,
) -> Result<u32, libwallet::Error> {
let mut max_n = 0; let mut max_n = 0;
for out in self.outputs.values() { for out in self.outputs.values() {
if max_n < out.n_child && out.root_key_id == root_key_id { if max_n < out.n_child && out.root_key_id == root_key_id {
@ -339,7 +210,7 @@ where
} else { } else {
self.details.last_child_index += 1; self.details.last_child_index += 1;
} }
self.details.last_child_index Ok(self.details.last_child_index)
} }
/// Select spendable coins from the wallet. /// Select spendable coins from the wallet.
@ -530,6 +401,41 @@ where
} }
} }
fn lock(&self) -> Result<(), libwallet::Error> {
// create directory if it doesn't exist
fs::create_dir_all(self.config.data_file_dir.clone()).unwrap_or_else(|why| {
info!(LOGGER, "! {:?}", why.kind());
});
info!(LOGGER, "Acquiring wallet lock ...");
let lock_file_path = self.lock_file_path.clone();
let action = || {
trace!(LOGGER, "making lock file for wallet lock");
fs::create_dir(&lock_file_path)
};
// use tokio_retry to cleanly define some retry logic
let mut core = reactor::Core::new().unwrap();
let retry_strategy = FibonacciBackoff::from_millis(1000).take(10);
let retry_future = Retry::spawn(core.handle(), retry_strategy, action);
let retry_result = core.run(retry_future)
.context(libwallet::ErrorKind::CallbackImpl(
"Failed to acquire lock file",
));
match retry_result {
Ok(_) => Ok(()),
Err(e) => {
error!(
LOGGER,
"Failed to acquire wallet lock file (multiple retries)",
);
Err(e.into())
}
}
}
/// Read the wallet data or create brand files if the data /// Read the wallet data or create brand files if the data
/// files don't yet exist /// files don't yet exist
fn read_or_create_paths(&mut self) -> Result<(), Error> { fn read_or_create_paths(&mut self) -> Result<(), Error> {
@ -570,7 +476,7 @@ where
let outputs = self.read_outputs()?; let outputs = self.read_outputs()?;
self.outputs = HashMap::new(); self.outputs = HashMap::new();
for out in outputs { for out in outputs {
self.add_output(out); self.outputs.insert(out.key_id.to_hex(), out.clone());
} }
Ok(()) Ok(())
} }

View file

@ -44,6 +44,7 @@ extern crate tokio_retry;
extern crate grin_api as api; extern crate grin_api as api;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
mod client; mod client;
@ -52,9 +53,12 @@ mod error;
pub mod file_wallet; pub mod file_wallet;
pub mod libtx; pub mod libtx;
pub mod libwallet; pub mod libwallet;
pub mod lmdb_wallet;
mod types;
pub use client::create_coinbase; pub use client::create_coinbase;
pub use error::{Error, ErrorKind}; pub use error::{Error, ErrorKind};
pub use file_wallet::{FileWallet, WalletConfig, WalletSeed}; pub use file_wallet::FileWallet;
pub use libwallet::controller; pub use libwallet::controller;
pub use libwallet::types::{BlockFees, CbData, WalletInfo}; pub use libwallet::types::{BlockFees, CbData, WalletInfo};
pub use types::{WalletConfig, WalletSeed};

View file

@ -77,6 +77,10 @@ pub enum ErrorKind {
#[fail(display = "Trait Implementation error")] #[fail(display = "Trait Implementation error")]
CallbackImpl(&'static str), CallbackImpl(&'static str),
/// Wallet backend error
#[fail(display = "Wallet store error")]
Backend(String),
/// Callback implementation error conversion /// Callback implementation error conversion
#[fail(display = "Restore Error")] #[fail(display = "Restore Error")]
Restore, Restore,

View file

@ -17,38 +17,29 @@ use keychain::{Identifier, Keychain};
use libwallet::error::Error; use libwallet::error::Error;
use libwallet::types::WalletBackend; use libwallet::types::WalletBackend;
/// Get our next available key
pub fn new_output_key<T, K>(wallet: &mut T) -> Result<(Identifier, u32), Error>
where
T: WalletBackend<K>,
K: Keychain,
{
wallet.with_wallet(|wallet_data| next_available_key(wallet_data))
}
/// Get next available key in the wallet /// Get next available key in the wallet
pub fn next_available_key<T, K>(wallet: &mut T) -> (Identifier, u32) pub fn next_available_key<T, K>(wallet: &mut T) -> Result<(Identifier, u32), Error>
where where
T: WalletBackend<K>, T: WalletBackend<K>,
K: Keychain, K: Keychain,
{ {
let root_key_id = wallet.keychain().root_key_id(); let root_key_id = wallet.keychain().root_key_id();
let derivation = wallet.next_child(root_key_id.clone()); let derivation = wallet.next_child(root_key_id.clone())?;
let key_id = wallet.keychain().derive_key_id(derivation).unwrap(); let key_id = wallet.keychain().derive_key_id(derivation)?;
(key_id, derivation) Ok((key_id, derivation))
} }
/// Retrieve an existing key from a wallet /// Retrieve an existing key from a wallet
pub fn retrieve_existing_key<T, K>(wallet: &T, key_id: Identifier) -> (Identifier, u32) pub fn retrieve_existing_key<T, K>(
wallet: &T,
key_id: Identifier,
) -> Result<(Identifier, u32), Error>
where where
T: WalletBackend<K>, T: WalletBackend<K>,
K: Keychain, K: Keychain,
{ {
if let Some(existing) = wallet.get_output(&key_id) { let existing = wallet.get(&key_id)?;
let key_id = existing.key_id.clone(); let key_id = existing.key_id.clone();
let derivation = existing.n_child; let derivation = existing.n_child;
(key_id, derivation) Ok((key_id, derivation))
} else {
panic!("should never happen");
}
} }

View file

@ -235,9 +235,7 @@ where
K: Keychain, K: Keychain,
{ {
// Don't proceed if wallet.dat has anything in it // Don't proceed if wallet.dat has anything in it
let is_empty = wallet let is_empty = wallet.iter().next().is_none();
.read_wallet(|wallet_data| Ok(wallet_data.outputs().len() == 0))
.context(ErrorKind::FileWallet("could not read wallet"))?;
if !is_empty { if !is_empty {
error!( error!(
LOGGER, LOGGER,
@ -265,17 +263,12 @@ where
output_listing.highest_index output_listing.highest_index
); );
let _ = wallet.with_wallet(|wallet_data| { let root_key_id = wallet.keychain().root_key_id();
let keychain = wallet_data.keychain().clone(); let result_vec =
let result_vec = find_outputs_with_key( find_outputs_with_key(wallet, output_listing.outputs.clone(), &mut found_key_index);
wallet_data, let mut batch = wallet.batch()?;
output_listing.outputs.clone(), for output in result_vec {
&mut found_key_index, batch.save(OutputData {
);
if result_vec.len() > 0 {
for output in result_vec.clone() {
let root_key_id = keychain.root_key_id();
wallet_data.add_output(OutputData {
root_key_id: root_key_id.clone(), root_key_id: root_key_id.clone(),
key_id: output.1.clone(), key_id: output.1.clone(),
n_child: output.2, n_child: output.2,
@ -288,8 +281,8 @@ where
merkle_proof: output.7, merkle_proof: output.7,
}); });
} }
} batch.commit()?;
});
if output_listing.highest_index == output_listing.last_retrieved_index { if output_listing.highest_index == output_listing.last_retrieved_index {
break; break;
} }

View file

@ -86,22 +86,23 @@ where
let lock_inputs = context.get_inputs().clone(); let lock_inputs = context.get_inputs().clone();
let _lock_outputs = context.get_outputs().clone(); let _lock_outputs = context.get_outputs().clone();
let root_key_id = keychain.root_key_id();
// Return a closure to acquire wallet lock and lock the coins being spent // Return a closure to acquire wallet lock and lock the coins being spent
// so we avoid accidental double spend attempt. // so we avoid accidental double spend attempt.
let update_sender_wallet_fn = move |wallet: &mut T| { let update_sender_wallet_fn = move |wallet: &mut T| {
wallet.with_wallet(|wallet_data| { let mut batch = wallet.batch()?;
// Lock the inputs we've selected
for id in lock_inputs { for id in lock_inputs {
let coin = wallet_data.get_output(&id).unwrap().clone(); let mut coin = batch.get(&id).unwrap();
wallet_data.lock_output(&coin); batch.lock_output(&mut coin);
} }
// write the output representing our change
if let Some(d) = change_derivation { if let Some(d) = change_derivation {
// Add our change output to the wallet
let root_key_id = keychain.root_key_id();
let change_id = keychain.derive_key_id(change_derivation.unwrap()).unwrap(); let change_id = keychain.derive_key_id(change_derivation.unwrap()).unwrap();
wallet_data.add_output(OutputData {
root_key_id: root_key_id.clone(), batch.save(OutputData {
key_id: change_id, root_key_id: root_key_id,
key_id: change_id.clone(),
n_child: d, n_child: d,
value: change as u64, value: change as u64,
status: OutputStatus::Unconfirmed, status: OutputStatus::Unconfirmed,
@ -112,7 +113,8 @@ where
merkle_proof: None, merkle_proof: None,
}); });
} }
}) batch.commit()?;
Ok(())
}; };
Ok((slate, context, update_sender_wallet_fn)) Ok((slate, context, update_sender_wallet_fn))
@ -138,7 +140,7 @@ where
K: Keychain, K: Keychain,
{ {
// Create a potential output for this transaction // Create a potential output for this transaction
let (key_id, derivation) = keys::new_output_key(wallet)?; let (key_id, derivation) = keys::next_available_key(wallet).unwrap();
let keychain = wallet.keychain().clone(); let keychain = wallet.keychain().clone();
let root_key_id = keychain.root_key_id(); let root_key_id = keychain.root_key_id();
@ -162,8 +164,8 @@ where
// Create closure that adds the output to recipient's wallet // Create closure that adds the output to recipient's wallet
// (up to the caller to decide when to do) // (up to the caller to decide when to do)
let wallet_add_fn = move |wallet: &mut T| { let wallet_add_fn = move |wallet: &mut T| {
wallet.with_wallet(|wallet_data| { let mut batch = wallet.batch()?;
wallet_data.add_output(OutputData { batch.save(OutputData {
root_key_id: root_key_id, root_key_id: root_key_id,
key_id: key_id_inner, key_id: key_id_inner,
n_child: derivation, n_child: derivation,
@ -175,7 +177,8 @@ where
block: None, block: None,
merkle_proof: None, merkle_proof: None,
}); });
}) batch.commit()?;
Ok(())
}; };
Ok((key_id, context, wallet_add_fn)) Ok((key_id, context, wallet_add_fn))
} }
@ -209,29 +212,25 @@ where
let key_id = wallet.keychain().root_key_id(); let key_id = wallet.keychain().root_key_id();
// select some spendable coins from the wallet // select some spendable coins from the wallet
let mut coins = wallet.read_wallet(|wallet_data| { let mut coins = wallet.select_coins(
Ok(wallet_data.select_coins(
key_id.clone(), key_id.clone(),
amount, amount,
current_height, current_height,
minimum_confirmations, minimum_confirmations,
max_outputs, max_outputs,
selection_strategy_is_use_all, selection_strategy_is_use_all,
)) );
})?;
// Get the maximum number of outputs in the wallet // Get the maximum number of outputs in the wallet
let max_outputs = wallet let max_outputs = wallet
.read_wallet(|wallet_data| { .select_coins(
Ok(wallet_data.select_coins(
key_id.clone(), key_id.clone(),
amount, amount,
current_height, current_height,
minimum_confirmations, minimum_confirmations,
max_outputs, max_outputs,
true, true,
)) )
})?
.len(); .len();
// sender is responsible for setting the fee on the partial tx // sender is responsible for setting the fee on the partial tx
@ -267,16 +266,14 @@ where
} }
// select some spendable coins from the wallet // select some spendable coins from the wallet
coins = wallet.read_wallet(|wallet_data| { coins = wallet.select_coins(
Ok(wallet_data.select_coins(
key_id.clone(), key_id.clone(),
amount_with_fee, amount_with_fee,
current_height, current_height,
minimum_confirmations, minimum_confirmations,
max_outputs, max_outputs,
selection_strategy_is_use_all, selection_strategy_is_use_all,
)) );
})?;
fee = tx_fee(coins.len(), 2, coins_proof_count(&coins), None); fee = tx_fee(coins.len(), 2, coins_proof_count(&coins), None);
total = coins.iter().map(|c| c.value).sum(); total = coins.iter().map(|c| c.value).sum();
amount_with_fee = amount + fee; amount_with_fee = amount + fee;
@ -284,8 +281,7 @@ where
} }
// build transaction skeleton with inputs and change // build transaction skeleton with inputs and change
let (mut parts, change, change_derivation) = let (mut parts, change, change_derivation) = inputs_and_change(&coins, wallet, amount, fee)?;
inputs_and_change(&coins, wallet, current_height, amount, fee)?;
// This is more proof of concept than anything but here we set lock_height // This is more proof of concept than anything but here we set lock_height
// on tx being sent (based on current chain height via api). // on tx being sent (based on current chain height via api).
@ -303,7 +299,6 @@ pub fn coins_proof_count(coins: &Vec<OutputData>) -> usize {
pub fn inputs_and_change<T, K>( pub fn inputs_and_change<T, K>(
coins: &Vec<OutputData>, coins: &Vec<OutputData>,
wallet: &mut T, wallet: &mut T,
height: u64,
amount: u64, amount: u64,
fee: u64, fee: u64,
) -> Result<(Vec<Box<build::Append<K>>>, u64, Option<u32>), Error> ) -> Result<(Vec<Box<build::Append<K>>>, u64, Option<u32>), Error>
@ -341,20 +336,14 @@ where
parts.push(build::input(coin.value, key_id)); parts.push(build::input(coin.value, key_id));
} }
} }
let change_key;
let mut change_derivation = None; let mut change_derivation = None;
if change != 0 { if change != 0 {
// track the output representing our change let keychain = wallet.keychain().clone();
change_key = wallet.with_wallet(|wallet_data| {
let keychain = wallet_data.keychain().clone();
let root_key_id = keychain.root_key_id(); let root_key_id = keychain.root_key_id();
let cd = wallet_data.next_child(root_key_id.clone()); change_derivation = Some(wallet.next_child(root_key_id.clone()).unwrap());
let change_key = keychain.derive_key_id(cd).unwrap(); let change_k = keychain.derive_key_id(change_derivation.unwrap()).unwrap();
change_derivation = Some(cd);
Some(change_key)
})?;
parts.push(build::output(change, change_key.clone().unwrap())); parts.push(build::output(change, change_k.clone()));
} }
Ok((parts, change, change_derivation)) Ok((parts, change, change_derivation))

View file

@ -151,22 +151,20 @@ where
let key_id = keychain.root_key_id(); let key_id = keychain.root_key_id();
// select some spendable coins from the wallet // select some spendable coins from the wallet
let coins = wallet.read_wallet(|wallet_data| { let coins = wallet.select_coins(
Ok(wallet_data.select_coins(
key_id.clone(), key_id.clone(),
amount, amount,
current_height, current_height,
minimum_confirmations, minimum_confirmations,
max_outputs, max_outputs,
false, false,
)) );
})?;
debug!(LOGGER, "selected some coins - {}", coins.len()); debug!(LOGGER, "selected some coins - {}", coins.len());
let fee = tx_fee(coins.len(), 2, selection::coins_proof_count(&coins), None); let fee = tx_fee(coins.len(), 2, selection::coins_proof_count(&coins), None);
let (mut parts, _, _) = let (mut parts, _, _) =
selection::inputs_and_change(&coins, wallet, current_height, amount, fee)?; selection::inputs_and_change(&coins, wallet, amount, fee)?;
//TODO: If we end up using this, create change output here //TODO: If we end up using this, create change output here

View file

@ -40,13 +40,9 @@ where
{ {
let root_key_id = wallet.keychain().clone().root_key_id(); let root_key_id = wallet.keychain().clone().root_key_id();
let mut outputs = vec![];
// just read the wallet here, no need for a write lock // just read the wallet here, no need for a write lock
let _ = wallet.read_wallet(|wallet_data| { let mut outputs = wallet
outputs = wallet_data .iter()
.outputs()
.values()
.filter(|out| out.root_key_id == root_key_id) .filter(|out| out.root_key_id == root_key_id)
.filter(|out| { .filter(|out| {
if show_spent { if show_spent {
@ -55,13 +51,8 @@ where
out.status != OutputStatus::Spent out.status != OutputStatus::Spent
} }
}) })
.collect::<Vec<_>>() .collect::<Vec<_>>();
.iter()
.map(|&o| o.clone())
.collect();
outputs.sort_by_key(|out| out.n_child); outputs.sort_by_key(|out| out.n_child);
Ok(())
});
Ok(outputs) Ok(outputs)
} }
@ -109,21 +100,21 @@ where
// the corresponding api output (if it exists) // the corresponding api output (if it exists)
// and refresh it in-place in the wallet. // and refresh it in-place in the wallet.
// Note: minimizing the time we spend holding the wallet lock. // Note: minimizing the time we spend holding the wallet lock.
wallet.with_wallet(|wallet_data| { let mut batch = wallet.batch()?;
for commit in wallet_outputs.keys() { for (commit, id) in wallet_outputs.iter() {
let id = wallet_outputs.get(&commit).unwrap(); if let Some((height, bid)) = api_blocks.get(&commit) {
if let Entry::Occupied(mut output) = wallet_data.outputs().entry(id.to_hex()) { if let Ok(mut output) = batch.get(id) {
if let Some(b) = api_blocks.get(&commit) { output.height = *height;
let output = output.get_mut(); output.block = Some(bid.clone());
output.height = b.0;
output.block = Some(b.1.clone());
if let Some(merkle_proof) = api_merkle_proofs.get(&commit) { if let Some(merkle_proof) = api_merkle_proofs.get(&commit) {
output.merkle_proof = Some(merkle_proof.clone()); output.merkle_proof = Some(merkle_proof.clone());
} }
batch.save(output);
} }
} }
} }
}) batch.commit()?;
Ok(())
} }
/// build a local map of wallet outputs keyed by commit /// build a local map of wallet outputs keyed by commit
@ -136,19 +127,15 @@ where
K: Keychain, K: Keychain,
{ {
let mut wallet_outputs: HashMap<pedersen::Commitment, Identifier> = HashMap::new(); let mut wallet_outputs: HashMap<pedersen::Commitment, Identifier> = HashMap::new();
let _ = wallet.read_wallet(|wallet_data| { let keychain = wallet.keychain().clone();
let keychain = wallet_data.keychain().clone();
let root_key_id = keychain.root_key_id().clone(); let root_key_id = keychain.root_key_id().clone();
let unspents = wallet_data let unspents = wallet
.outputs() .iter()
.values()
.filter(|x| x.root_key_id == root_key_id && x.status != OutputStatus::Spent); .filter(|x| x.root_key_id == root_key_id && x.status != OutputStatus::Spent);
for out in unspents { for out in unspents {
let commit = keychain.commit_with_key_index(out.value, out.n_child)?; let commit = keychain.commit_with_key_index(out.value, out.n_child)?;
wallet_outputs.insert(commit, out.key_id.clone()); wallet_outputs.insert(commit, out.key_id.clone());
} }
Ok(())
});
Ok(wallet_outputs) Ok(wallet_outputs)
} }
@ -162,9 +149,8 @@ where
K: Keychain, K: Keychain,
{ {
let mut wallet_outputs: HashMap<pedersen::Commitment, Identifier> = HashMap::new(); let mut wallet_outputs: HashMap<pedersen::Commitment, Identifier> = HashMap::new();
let _ = wallet.read_wallet(|wallet_data| { let keychain = wallet.keychain().clone();
let keychain = wallet_data.keychain().clone(); let unspents = wallet.iter().filter(|x| {
let unspents = wallet_data.outputs().values().filter(|x| {
x.root_key_id == keychain.root_key_id() && x.block.is_none() x.root_key_id == keychain.root_key_id() && x.block.is_none()
&& x.status == OutputStatus::Unspent && x.status == OutputStatus::Unspent
}); });
@ -172,8 +158,6 @@ where
let commit = keychain.commit_with_key_index(out.value, out.n_child)?; let commit = keychain.commit_with_key_index(out.value, out.n_child)?;
wallet_outputs.insert(commit, out.key_id.clone()); wallet_outputs.insert(commit, out.key_id.clone());
} }
Ok(())
});
Ok(wallet_outputs) Ok(wallet_outputs)
} }
@ -191,19 +175,21 @@ where
// now for each commit, find the output in the wallet and the corresponding // now for each commit, find the output in the wallet and the corresponding
// api output (if it exists) and refresh it in-place in the wallet. // api output (if it exists) and refresh it in-place in the wallet.
// Note: minimizing the time we spend holding the wallet lock. // Note: minimizing the time we spend holding the wallet lock.
wallet.with_wallet(|wallet_data| { {
for commit in wallet_outputs.keys() { let mut batch = wallet.batch()?;
let id = wallet_outputs.get(&commit).unwrap(); for (commit, id) in wallet_outputs.iter() {
if let Entry::Occupied(mut output) = wallet_data.outputs().entry(id.to_hex()) { if let Ok(mut output) = batch.get(id) {
match api_outputs.get(&commit) { match api_outputs.get(&commit) {
Some(_) => output.get_mut().mark_unspent(), Some(_) => output.mark_unspent(),
None => output.get_mut().mark_spent(), None => output.mark_spent(),
}; };
batch.save(output);
} }
} }
let details = wallet_data.details(); batch.commit()?;
}
let details = wallet.details();
details.last_confirmed_height = height; details.last_confirmed_height = height;
})?;
Ok(()) Ok(())
} }
@ -236,12 +222,18 @@ where
if height < 500 { if height < 500 {
return Ok(()); return Ok(());
} }
wallet.with_wallet(|wallet_data| { let mut ids_to_del = vec![];
wallet_data.outputs().retain(|_, ref mut out| { for out in wallet.iter() {
!(out.status == OutputStatus::Unconfirmed && out.height > 0 if out.status == OutputStatus::Unconfirmed && out.height > 0 && out.height < height - 500 {
&& out.height < height - 500) ids_to_del.push(out.key_id.clone())
}); }
}) }
let mut batch = wallet.batch()?;
for id in ids_to_del {
batch.delete(&id);
}
batch.commit()?;
Ok(())
} }
/// Retrieve summary info about the wallet /// Retrieve summary info about the wallet
@ -251,18 +243,17 @@ where
T: WalletBackend<K> + WalletClient, T: WalletBackend<K> + WalletClient,
K: Keychain, K: Keychain,
{ {
let ret_val = wallet.read_wallet(|wallet_data| { let current_height = wallet.details().last_confirmed_height;
let current_height = wallet_data.details().last_confirmed_height; let keychain = wallet.keychain().clone();
let outputs = wallet
.iter()
.filter(|out| out.root_key_id == keychain.root_key_id());
let mut unspent_total = 0; let mut unspent_total = 0;
let mut immature_total = 0; let mut immature_total = 0;
let mut unconfirmed_total = 0; let mut unconfirmed_total = 0;
let mut locked_total = 0; let mut locked_total = 0;
for out in wallet_data for out in outputs {
.outputs()
.clone()
.values()
.filter(|out| out.root_key_id == wallet_data.keychain().root_key_id())
{
if out.status == OutputStatus::Unspent && out.lock_height <= current_height { if out.status == OutputStatus::Unspent && out.lock_height <= current_height {
unspent_total += out.value; unspent_total += out.value;
} }
@ -278,15 +269,13 @@ where
} }
Ok(WalletInfo { Ok(WalletInfo {
last_confirmed_height: wallet_data.details().last_confirmed_height, last_confirmed_height: current_height,
total: unspent_total + unconfirmed_total + immature_total, total: unspent_total + unconfirmed_total + immature_total,
amount_awaiting_confirmation: unconfirmed_total, amount_awaiting_confirmation: unconfirmed_total,
amount_immature: immature_total, amount_immature: immature_total,
amount_locked: locked_total, amount_locked: locked_total,
amount_currently_spendable: unspent_total, amount_currently_spendable: unspent_total,
}) })
});
ret_val
} }
/// Build a coinbase output and insert into wallet /// Build a coinbase output and insert into wallet
@ -327,17 +316,17 @@ where
let height = block_fees.height; let height = block_fees.height;
let lock_height = height + global::coinbase_maturity(); let lock_height = height + global::coinbase_maturity();
// Now acquire the wallet lock and write the new output.
let (key_id, derivation) = wallet.with_wallet(|wallet_data| {
let key_id = block_fees.key_id(); let key_id = block_fees.key_id();
let (key_id, derivation) = match key_id { let (key_id, derivation) = match key_id {
Some(key_id) => keys::retrieve_existing_key(wallet_data, key_id), Some(key_id) => keys::retrieve_existing_key(wallet, key_id)?,
None => keys::next_available_key(wallet_data), None => keys::next_available_key(wallet)?,
}; };
// track the new output and return the stuff needed for reward {
wallet_data.add_output(OutputData { // Now acquire the wallet lock and write the new output.
let mut batch = wallet.batch()?;
batch.save(OutputData {
root_key_id: root_key_id.clone(), root_key_id: root_key_id.clone(),
key_id: key_id.clone(), key_id: key_id.clone(),
n_child: derivation, n_child: derivation,
@ -349,9 +338,8 @@ where
block: None, block: None,
merkle_proof: None, merkle_proof: None,
}); });
batch.commit()?;
(key_id, derivation) }
})?;
debug!( debug!(
LOGGER, LOGGER,

View file

@ -19,11 +19,13 @@ use std::collections::HashMap;
use std::fmt; use std::fmt;
use serde; use serde;
use serde_json;
use failure::ResultExt; use failure::ResultExt;
use core::core::hash::Hash; use core::core::hash::Hash;
use core::core::merkle_proof::MerkleProof; use core::core::merkle_proof::MerkleProof;
use core::ser;
use keychain::{Identifier, Keychain}; use keychain::{Identifier, Keychain};
@ -49,36 +51,17 @@ where
/// Return the keychain being used /// Return the keychain being used
fn keychain(&mut self) -> &mut K; fn keychain(&mut self) -> &mut K;
/// Return the outputs directly /// Iterate over all output data stored by the backend
fn outputs(&mut self) -> &mut HashMap<String, OutputData>; fn iter<'a>(&'a self) -> Box<Iterator<Item = OutputData> + 'a>;
/// Allows for reading wallet data (read-only) /// Get output data by id
fn read_wallet<T, F>(&mut self, f: F) -> Result<T, Error> fn get(&self, id: &Identifier) -> Result<OutputData, Error>;
where
F: FnOnce(&mut Self) -> Result<T, Error>;
/// Get all outputs from a wallet impl (probably with some sort /// Create a new write batch to update or remove output data
/// of query param), read+write. Implementor should save fn batch<'a>(&'a mut self) -> Result<Box<WalletOutputBatch + 'a>, Error>;
/// any changes to its data and perform any locking needed
fn with_wallet<T, F>(&mut self, f: F) -> Result<T, Error>
where
F: FnOnce(&mut Self) -> T;
/// Add an output
fn add_output(&mut self, out: OutputData);
/// Delete an output
fn delete_output(&mut self, id: &Identifier);
/// Lock an output
fn lock_output(&mut self, out: &OutputData);
/// get a single output
fn get_output(&self, key_id: &Identifier) -> Option<&OutputData>;
/// Next child ID when we want to create a new output /// Next child ID when we want to create a new output
/// Should also increment index fn next_child<'a>(&mut self, root_key_id: Identifier) -> Result<u32, Error>;
fn next_child(&mut self, root_key_id: Identifier) -> u32;
/// Return current details /// Return current details
fn details(&mut self) -> &mut WalletDetails; fn details(&mut self) -> &mut WalletDetails;
@ -98,6 +81,26 @@ where
fn restore(&mut self) -> Result<(), Error>; fn restore(&mut self) -> Result<(), Error>;
} }
/// Batch trait to update the output data backend atomically. Trying to use a
/// batch after commit MAY result in a panic. Due to this being a trait, the
/// commit method can't take ownership.
pub trait WalletOutputBatch {
/// Add or update data about an output to the backend
fn save(&mut self, out: OutputData) -> Result<(), Error>;
/// Gets output data by id
fn get(&self, id: &Identifier) -> Result<OutputData, Error>;
/// Delete data about an output to the backend
fn delete(&mut self, id: &Identifier) -> Result<(), Error>;
/// Save an output as locked in the backend
fn lock_output(&mut self, out: &mut OutputData) -> Result<(), Error>;
/// Write the wallet data to backend file
fn commit(&self) -> Result<(), Error>;
}
/// Encapsulate all communication functions. No functions within libwallet /// Encapsulate all communication functions. No functions within libwallet
/// should care about communication details /// should care about communication details
pub trait WalletClient { pub trait WalletClient {
@ -148,8 +151,8 @@ pub trait WalletClient {
} }
/// Information about an output that's being tracked by the wallet. Must be /// Information about an output that's being tracked by the wallet. Must be
/// enough to reconstruct the commitment associated with the output when the /// enough to reconstruct the commitment associated with the ouput when the
/// root private key is known.*/ /// root private key is known.
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, PartialOrd, Eq, Ord)] #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, PartialOrd, Eq, Ord)]
pub struct OutputData { pub struct OutputData {
@ -175,6 +178,19 @@ pub struct OutputData {
pub merkle_proof: Option<MerkleProofWrapper>, pub merkle_proof: Option<MerkleProofWrapper>,
} }
impl ser::Writeable for OutputData {
fn write<W: ser::Writer>(&self, writer: &mut W) -> Result<(), ser::Error> {
writer.write_bytes(&serde_json::to_vec(self).map_err(|_| ser::Error::CorruptedData)?)
}
}
impl ser::Readable for OutputData {
fn read(reader: &mut ser::Reader) -> Result<OutputData, ser::Error> {
let data = reader.read_vec()?;
serde_json::from_slice(&data[..]).map_err(|_| ser::Error::CorruptedData)
}
}
impl OutputData { impl OutputData {
/// Lock a given output to avoid conflicting use /// Lock a given output to avoid conflicting use
pub fn lock(&mut self) { pub fn lock(&mut self) {

255
wallet/src/lmdb_wallet.rs Normal file
View file

@ -0,0 +1,255 @@
// 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.
use std::cell::RefCell;
use std::collections::HashMap;
use std::collections::hash_map::Values;
use std::ops::Deref;
use std::sync::Arc;
use std::{fs, path};
use failure::{Context, ResultExt};
use keychain::{Identifier, Keychain};
use store::{self, option_to_not_found, to_key, u64_to_key};
use client;
use libtx::slate::Slate;
use libwallet::types::*;
use libwallet::{internal, Error, ErrorKind};
use types::{WalletConfig, WalletSeed};
use util::secp::pedersen;
pub const DB_DIR: &'static str = "wallet";
const OUTPUT_PREFIX: u8 = 'o' as u8;
const DERIV_PREFIX: u8 = 'd' as u8;
impl From<store::Error> for Error {
fn from(error: store::Error) -> Error {
Error::from((ErrorKind::Backend(format!("{:?}", error))))
}
}
pub struct LMDBBackend<K> {
db: store::Store,
config: WalletConfig,
/// passphrase: TODO better ways of dealing with this other than storing
passphrase: String,
/// Keychain
keychain: Option<K>,
}
impl<K> LMDBBackend<K> {
pub fn new(config: WalletConfig, passphrase: &str) -> Result<Self, Error> {
let db_path = path::Path::new(&config.data_file_dir).join(DB_DIR);
fs::create_dir_all(&db_path).expect("Couldn't create wallet backend directory!");
let lmdb_env = Arc::new(store::new_env(db_path.to_str().unwrap().to_string()));
let db = store::Store::open(lmdb_env, DB_DIR);
Ok(LMDBBackend {
db,
config: config.clone(),
passphrase: String::from(passphrase),
keychain: None,
})
}
}
impl<K> WalletBackend<K> for LMDBBackend<K>
where
K: Keychain,
{
/// Initialise with whatever stored credentials we have
fn open_with_credentials(&mut self) -> Result<(), Error> {
let wallet_seed = WalletSeed::from_file(&self.config)
.context(ErrorKind::CallbackImpl("Error opening wallet"))?;
let keychain = wallet_seed.derive_keychain(&self.passphrase);
self.keychain = Some(keychain.context(ErrorKind::CallbackImpl("Error deriving keychain"))?);
// Just blow up password for now after it's been used
self.passphrase = String::from("");
Ok(())
}
/// Close wallet and remove any stored credentials (TBD)
fn close(&mut self) -> Result<(), Error> {
self.keychain = None;
Ok(())
}
/// Return the keychain being used
fn keychain(&mut self) -> &mut K {
self.keychain.as_mut().unwrap()
}
fn get(&self, id: &Identifier) -> Result<OutputData, Error> {
let key = to_key(OUTPUT_PREFIX, &mut id.to_bytes().to_vec());
option_to_not_found(self.db.get_ser(&key)).map_err(|e| e.into())
}
fn iter<'a>(&'a self) -> Box<Iterator<Item = OutputData> + 'a> {
Box::new(self.db.iter(&[OUTPUT_PREFIX]).unwrap())
}
fn batch<'a>(&'a mut self) -> Result<Box<WalletOutputBatch + 'a>, Error> {
Ok(Box::new(Batch {
store: self,
db: RefCell::new(Some(self.db.batch()?)),
}))
}
fn next_child<'a>(&mut self, root_key_id: Identifier) -> Result<u32, Error> {
let batch = self.db.batch()?;
// a simple counter, only one batch per db guarantees atomicity
let deriv_key = to_key(DERIV_PREFIX, &mut root_key_id.to_bytes().to_vec());
let deriv_idx = match batch.get_ser(&deriv_key)? {
Some(idx) => idx,
None => 0,
};
batch.put_ser(&deriv_key, &(deriv_idx + 1))?;
batch.commit()?;
Ok(deriv_idx + 1)
}
fn select_coins(
&self,
root_key_id: Identifier,
amount: u64,
current_height: u64,
minimum_confirmations: u64,
max_outputs: usize,
select_all: bool,
) -> Vec<OutputData> {
unimplemented!()
}
fn details(&mut self) -> &mut WalletDetails {
unimplemented!()
}
fn restore(&mut self) -> Result<(), Error> {
internal::restore::restore(self).context(ErrorKind::Restore)?;
Ok(())
}
}
/// An atomic batch in which all changes can be committed all at once or
/// discarded on error.
pub struct Batch<'a, K: 'a> {
store: &'a LMDBBackend<K>,
db: RefCell<Option<store::Batch<'a>>>,
}
#[allow(missing_docs)]
impl<'a, K> WalletOutputBatch for Batch<'a, K> {
fn save(&mut self, out: OutputData) -> Result<(), Error> {
let key = to_key(OUTPUT_PREFIX, &mut out.key_id.to_bytes().to_vec());
self.db.borrow().as_ref().unwrap().put_ser(&key, &out)?;
Ok(())
}
fn get(&self, id: &Identifier) -> Result<OutputData, Error> {
let key = to_key(OUTPUT_PREFIX, &mut id.to_bytes().to_vec());
option_to_not_found(self.db.borrow().as_ref().unwrap().get_ser(&key)).map_err(|e| e.into())
}
fn delete(&mut self, id: &Identifier) -> Result<(), Error> {
let key = to_key(OUTPUT_PREFIX, &mut id.to_bytes().to_vec());
self.db.borrow().as_ref().unwrap().delete(&key)?;
Ok(())
}
fn lock_output(&mut self, out: &mut OutputData) -> Result<(), Error> {
out.lock();
self.save(out.clone())
}
fn commit(&self) -> Result<(), Error> {
let db = self.db.replace(None);
db.unwrap().commit()?;
Ok(())
}
}
impl<K> WalletClient for LMDBBackend<K> {
/// Return URL for check node
fn node_url(&self) -> &str {
&self.config.check_node_api_http_addr
}
/// Call the wallet API to create a coinbase transaction
fn create_coinbase(&self, dest: &str, block_fees: &BlockFees) -> Result<CbData, Error> {
let res = client::create_coinbase(dest, block_fees)
.context(ErrorKind::WalletComms(format!("Creating Coinbase")))?;
Ok(res)
}
/// Send a transaction slate to another listening wallet and return result
fn send_tx_slate(&self, dest: &str, slate: &Slate) -> Result<Slate, Error> {
let res = client::send_tx_slate(dest, slate)
.context(ErrorKind::WalletComms(format!("Sending transaction")))?;
Ok(res)
}
/// Posts a tranaction to a grin node
fn post_tx(&self, dest: &str, tx: &TxWrapper, fluff: bool) -> Result<(), Error> {
let res = client::post_tx(dest, tx, fluff).context(ErrorKind::Node)?;
Ok(res)
}
/// retrieves the current tip from the specified grin node
fn get_chain_height(&self, addr: &str) -> Result<u64, Error> {
let res = client::get_chain_height(addr).context(ErrorKind::Node)?;
Ok(res)
}
/// retrieve a list of outputs from the specified grin node
/// need "by_height" and "by_id" variants
fn get_outputs_from_node(
&self,
addr: &str,
wallet_outputs: Vec<pedersen::Commitment>,
) -> Result<HashMap<pedersen::Commitment, String>, Error> {
let res = client::get_outputs_from_node(addr, wallet_outputs).context(ErrorKind::Node)?;
Ok(res)
}
/// Get any missing block hashes from node
fn get_missing_block_hashes_from_node(
&self,
addr: &str,
height: u64,
wallet_outputs: Vec<pedersen::Commitment>,
) -> Result<
(
HashMap<pedersen::Commitment, (u64, BlockIdentifier)>,
HashMap<pedersen::Commitment, MerkleProofWrapper>,
),
Error,
> {
let res = client::get_missing_block_hashes_from_node(addr, height, wallet_outputs)
.context(ErrorKind::Node)?;
Ok(res)
}
/// retrieve merkle proof for a commit from a node
fn get_merkle_proof_for_commit(
&self,
addr: &str,
commit: &str,
) -> Result<MerkleProofWrapper, Error> {
Err(ErrorKind::GenericError("Not Implemented"))?
}
}

150
wallet/src/types.rs Normal file
View file

@ -0,0 +1,150 @@
// 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.
use std::cmp::min;
use std::fs::{self, File};
use std::io::{Read, Write};
use std::path::MAIN_SEPARATOR;
use std::path::Path;
use blake2;
use rand::{thread_rng, Rng};
use error::{Error, ErrorKind};
use failure::ResultExt;
use keychain::Keychain;
use util;
use util::LOGGER;
pub const SEED_FILE: &'static str = "wallet.seed";
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct WalletConfig {
// Right now the decision to run or not a wallet is based on the command.
// This may change in the near-future.
// pub enable_wallet: bool,
// The api interface/ip_address that this api server (i.e. this wallet) will run
// by default this is 127.0.0.1 (and will not accept connections from external clients)
pub api_listen_interface: String,
// The port this wallet will run on
pub api_listen_port: u16,
// The api address of a running server node against which transaction inputs
// will be checked during send
pub check_node_api_http_addr: String,
// The directory in which wallet files are stored
pub data_file_dir: String,
}
impl Default for WalletConfig {
fn default() -> WalletConfig {
WalletConfig {
// enable_wallet: false,
api_listen_interface: "127.0.0.1".to_string(),
api_listen_port: 13415,
check_node_api_http_addr: "http://127.0.0.1:13413".to_string(),
data_file_dir: ".".to_string(),
}
}
}
impl WalletConfig {
pub fn api_listen_addr(&self) -> String {
format!("{}:{}", self.api_listen_interface, self.api_listen_port)
}
}
#[derive(Clone, PartialEq)]
pub struct WalletSeed([u8; 32]);
impl WalletSeed {
pub fn from_bytes(bytes: &[u8]) -> WalletSeed {
let mut seed = [0; 32];
for i in 0..min(32, bytes.len()) {
seed[i] = bytes[i];
}
WalletSeed(seed)
}
fn from_hex(hex: &str) -> Result<WalletSeed, Error> {
let bytes =
util::from_hex(hex.to_string()).context(ErrorKind::GenericError("Invalid hex"))?;
Ok(WalletSeed::from_bytes(&bytes))
}
pub fn to_hex(&self) -> String {
util::to_hex(self.0.to_vec())
}
pub fn derive_keychain<K: Keychain>(&self, password: &str) -> Result<K, Error> {
let seed = blake2::blake2b::blake2b(64, &password.as_bytes(), &self.0);
let result = K::from_seed(seed.as_bytes())?;
Ok(result)
}
pub fn init_new() -> WalletSeed {
let seed: [u8; 32] = thread_rng().gen();
WalletSeed(seed)
}
pub fn init_file(wallet_config: &WalletConfig) -> Result<WalletSeed, Error> {
// create directory if it doesn't exist
fs::create_dir_all(&wallet_config.data_file_dir).context(ErrorKind::IO)?;
let seed_file_path = &format!(
"{}{}{}",
wallet_config.data_file_dir, MAIN_SEPARATOR, SEED_FILE,
);
debug!(LOGGER, "Generating wallet seed file at: {}", seed_file_path);
if Path::new(seed_file_path).exists() {
Err(ErrorKind::WalletSeedExists)?
} else {
let seed = WalletSeed::init_new();
let mut file = File::create(seed_file_path).context(ErrorKind::IO)?;
file.write_all(&seed.to_hex().as_bytes())
.context(ErrorKind::IO)?;
Ok(seed)
}
}
pub fn from_file(wallet_config: &WalletConfig) -> Result<WalletSeed, Error> {
// create directory if it doesn't exist
fs::create_dir_all(&wallet_config.data_file_dir).context(ErrorKind::IO)?;
let seed_file_path = &format!(
"{}{}{}",
wallet_config.data_file_dir, MAIN_SEPARATOR, SEED_FILE,
);
debug!(LOGGER, "Using wallet seed file at: {}", seed_file_path,);
if Path::new(seed_file_path).exists() {
let mut file = File::open(seed_file_path).context(ErrorKind::IO)?;
let mut buffer = String::new();
file.read_to_string(&mut buffer).context(ErrorKind::IO)?;
let wallet_seed = WalletSeed::from_hex(&buffer)?;
Ok(wallet_seed)
} else {
error!(
LOGGER,
"wallet seed file {} could not be opened (grin wallet init). \
Run \"grin wallet init\" to initialize a new wallet.",
seed_file_path
);
Err(ErrorKind::WalletSeedDoesntExist)?
}
}
}

View file

@ -15,6 +15,7 @@
extern crate grin_chain as chain; extern crate grin_chain as chain;
extern crate grin_core as core; extern crate grin_core as core;
extern crate grin_keychain as keychain; extern crate grin_keychain as keychain;
extern crate grin_store as store;
extern crate grin_util as util; extern crate grin_util as util;
extern crate grin_wallet as wallet; extern crate grin_wallet as wallet;
extern crate rand; extern crate rand;
@ -46,8 +47,10 @@ fn setup(test_dir: &str, chain_dir: &str) -> Chain {
global::set_mining_mode(ChainTypes::AutomatedTesting); global::set_mining_mode(ChainTypes::AutomatedTesting);
let genesis_block = pow::mine_genesis_block().unwrap(); let genesis_block = pow::mine_genesis_block().unwrap();
let dir_name = format!("{}/{}", test_dir, chain_dir); let dir_name = format!("{}/{}", test_dir, chain_dir);
let db_env = Arc::new(store::new_env(dir_name.to_string()));
chain::Chain::init( chain::Chain::init(
dir_name.to_string(), dir_name.to_string(),
db_env,
Arc::new(NoopAdapter {}), Arc::new(NoopAdapter {}),
genesis_block, genesis_block,
pow::verify_size, pow::verify_size,