introduce rewindable_kernel_view (readonly but rewindable kernel MMR view) (#1587)

* introduce rewindable_kernel_view
cleanup header in extension

* cleanup and docs/comments

* txhashset does not need to be mutable here

* pull validate_kernel_history out into fn
This commit is contained in:
Antioch Peverell 2018-09-26 09:59:00 +01:00 committed by GitHub
parent e55c3d2ec9
commit acec59e249
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 358 additions and 112 deletions

View file

@ -498,7 +498,7 @@ impl Chain {
// ensure the view is consistent. // ensure the view is consistent.
txhashset::extending_readonly(&mut txhashset, |extension| { txhashset::extending_readonly(&mut txhashset, |extension| {
extension.rewind(&header)?; extension.rewind(&header)?;
extension.validate(&header, skip_rproofs, &NoStatus)?; extension.validate(skip_rproofs, &NoStatus)?;
Ok(()) Ok(())
}) })
} }
@ -534,7 +534,8 @@ impl Chain {
let mut txhashset = self.txhashset.write().unwrap(); let mut txhashset = self.txhashset.write().unwrap();
let merkle_proof = txhashset::extending_readonly(&mut txhashset, |extension| { let merkle_proof = txhashset::extending_readonly(&mut txhashset, |extension| {
extension.merkle_proof(output, block_header) extension.rewind(&block_header)?;
extension.merkle_proof(output)
})?; })?;
Ok(merkle_proof) Ok(merkle_proof)
@ -567,7 +568,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)?; extension.rewind(&header)?;
extension.snapshot(&header)?; extension.snapshot()?;
Ok(()) Ok(())
})?; })?;
} }
@ -581,6 +582,41 @@ impl Chain {
)) ))
} }
// Special handling to make sure the whole kernel set matches each of its
// roots in each block header, without truncation. We go back header by
// header, rewind and check each root. This fixes a potential weakness in
// fast sync where a reorg past the horizon could allow a whole rewrite of
// the kernel set.
fn validate_kernel_history(
&self,
header: &BlockHeader,
txhashset: &txhashset::TxHashSet,
) -> Result<(), Error> {
debug!(
LOGGER,
"chain: validate_kernel_history: rewinding and validating kernel history (readonly)"
);
let mut count = 0;
let mut current = header.clone();
txhashset::rewindable_kernel_view(&txhashset, |view| {
while current.height > 0 {
view.rewind(&current)?;
view.validate_root()?;
current = view.batch().get_block_header(&current.previous)?;
count += 1;
}
Ok(())
})?;
debug!(
LOGGER,
"chain: validate_kernel_history: validated kernel root on {} headers", count,
);
Ok(())
}
/// Writes a reading view on a txhashset state that's been provided to us. /// Writes a reading view on a txhashset state that's been provided to us.
/// If we're willing to accept that new state, the data stream will be /// If we're willing to accept that new state, the data stream will be
/// read as a zip file, unzipped and the resulting state files should be /// read as a zip file, unzipped and the resulting state files should be
@ -604,22 +640,8 @@ impl Chain {
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))?;
// Validate kernel history against a readonly extension first. // Validate the full kernel history (kernel MMR root for every block header).
// Kernel history validation requires a readonly extension self.validate_kernel_history(&header, &txhashset)?;
// due to the internal rewind behavior.
debug!(
LOGGER,
"chain: txhashset_write: rewinding and validating kernel history (readonly)"
);
txhashset::extending_readonly(&mut txhashset, |extension| {
extension.rewind(&header)?;
// Now validate kernel sums at each historical header height
// so we know we can trust the kernel history.
extension.validate_kernel_history(&header)?;
Ok(())
})?;
// all good, prepare a new batch and update all the required records // all good, prepare a new batch and update all the required records
debug!( debug!(
@ -631,7 +653,7 @@ impl Chain {
extension.rewind(&header)?; extension.rewind(&header)?;
// Validate the extension, generating the utxo_sum and kernel_sum. // Validate the extension, generating the utxo_sum and kernel_sum.
let (utxo_sum, kernel_sum) = extension.validate(&header, false, status)?; let (utxo_sum, kernel_sum) = extension.validate(false, status)?;
// Now that we have block_sums the total_kernel_sum on the block_header is redundant. // Now that we have block_sums the total_kernel_sum on the block_header is redundant.
if header.total_kernel_sum != kernel_sum { if header.total_kernel_sum != kernel_sum {
@ -977,7 +999,7 @@ fn setup_head(
let res = txhashset::extending(txhashset, &mut batch, |extension| { let res = txhashset::extending(txhashset, &mut batch, |extension| {
extension.rewind(&header)?; extension.rewind(&header)?;
extension.validate_roots(&header)?; extension.validate_roots()?;
// now check we have the "block sums" for the block in question // now check we have the "block sums" for the block in question
// if we have no sums (migrating an existing node) we need to go // if we have no sums (migrating an existing node) we need to go
@ -993,7 +1015,7 @@ fn setup_head(
// Do a full (and slow) validation of the txhashset extension // Do a full (and slow) validation of the txhashset extension
// to calculate the utxo_sum and kernel_sum at this block height. // to calculate the utxo_sum and kernel_sum at this block height.
let (utxo_sum, kernel_sum) = extension.validate_kernel_sums(&header)?; let (utxo_sum, kernel_sum) = extension.validate_kernel_sums()?;
// Save the block_sums to the db for use later. // Save the block_sums to the db for use later.
extension.batch.save_block_sums( extension.batch.save_block_sums(
@ -1029,9 +1051,11 @@ fn setup_head(
} }
} }
Err(NotFoundErr(_)) => { Err(NotFoundErr(_)) => {
let tip = Tip::from_block(&genesis.header);
batch.save_block(&genesis)?; batch.save_block(&genesis)?;
let tip = Tip::from_block(&genesis.header);
batch.save_head(&tip)?;
batch.setup_height(&genesis.header, &tip)?; batch.setup_height(&genesis.header, &tip)?;
txhashset::extending(txhashset, &mut batch, |extension| { txhashset::extending(txhashset, &mut batch, |extension| {
extension.apply_block(&genesis)?; extension.apply_block(&genesis)?;
@ -1043,8 +1067,6 @@ fn setup_head(
Ok(()) Ok(())
})?; })?;
// saving a new tip based on genesis
batch.save_head(&tip)?;
head = tip; head = tip;
info!(LOGGER, "chain: init: saved genesis: {:?}", genesis.hash()); info!(LOGGER, "chain: init: saved genesis: {:?}", genesis.hash());
} }

View file

@ -46,7 +46,6 @@ mod error;
pub mod pipe; pub mod pipe;
pub mod store; pub mod store;
pub mod txhashset; pub mod txhashset;
pub mod utxo_view;
pub mod types; pub mod types;
// Re-export the base interface // Re-export the base interface

View file

@ -368,7 +368,7 @@ fn check_known_mmr(
// We want to return an error here (block already known) // We want to return an error here (block already known)
// if we *successfully validate the MMR roots and sizes. // if we *successfully validate the MMR roots and sizes.
if extension.validate_roots(header).is_ok() && extension.validate_sizes(header).is_ok() { if extension.validate_roots().is_ok() && extension.validate_sizes().is_ok() {
// TODO - determine if block is more than 50 blocks old // TODO - determine if block is more than 50 blocks old
// and return specific OldBlock error. // and return specific OldBlock error.
// Or pull OldBlock (abusive peer) out into separate processing step. // Or pull OldBlock (abusive peer) out into separate processing step.
@ -563,8 +563,8 @@ fn verify_block_sums(b: &Block, ext: &mut txhashset::Extension) -> Result<(), Er
/// Check both the txhashset roots and sizes are correct after applying the block. /// Check both the txhashset roots and sizes are correct after applying the block.
fn apply_block_to_txhashset(block: &Block, ext: &mut txhashset::Extension) -> Result<(), Error> { fn apply_block_to_txhashset(block: &Block, ext: &mut txhashset::Extension) -> Result<(), Error> {
ext.apply_block(block)?; ext.apply_block(block)?;
ext.validate_roots(&block.header)?; ext.validate_roots()?;
ext.validate_sizes(&block.header)?; ext.validate_sizes()?;
Ok(()) Ok(())
} }

View file

@ -0,0 +1,24 @@
// 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.
//! Utility structs to handle the 3 hashtrees (output, range proof,
//! kernel) more conveniently and transactionally.
mod rewindable_kernel_view;
mod txhashset;
mod utxo_view;
pub use self::rewindable_kernel_view::*;
pub use self::txhashset::*;
pub use self::utxo_view::*;

View file

@ -0,0 +1,80 @@
// 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.
//! Lightweight readonly view into kernel MMR for convenience.
use core::core::pmmr::RewindablePMMR;
use core::core::{BlockHeader, TxKernel};
use error::{Error, ErrorKind};
use grin_store::pmmr::PMMRBackend;
use store::Batch;
/// Rewindable (but readonly) view of the kernel set (based on kernel MMR).
pub struct RewindableKernelView<'a> {
pmmr: RewindablePMMR<'a, TxKernel, PMMRBackend<TxKernel>>,
batch: &'a Batch<'a>,
header: BlockHeader,
}
impl<'a> RewindableKernelView<'a> {
/// Build a new readonly kernel view.
pub fn new(
pmmr: RewindablePMMR<'a, TxKernel, PMMRBackend<TxKernel>>,
batch: &'a Batch,
header: BlockHeader,
) -> RewindableKernelView<'a> {
RewindableKernelView {
pmmr,
batch,
header,
}
}
/// Accessor for the batch used in this view.
/// We will discard this batch (rollback) at the end, so be aware of this.
/// Nothing will get written to the db/index via this view.
pub fn batch(&self) -> &'a Batch {
self.batch
}
/// Rewind this readonly view to a previous block.
/// We accomplish this in a readonly way because we can rewind the PMMR
/// via last_pos, without rewinding the underlying backend files.
pub fn rewind(&mut self, header: &BlockHeader) -> Result<(), Error> {
self.pmmr
.rewind(header.kernel_mmr_size)
.map_err(&ErrorKind::TxHashSetErr)?;
// Update our header to reflect the one we rewound to.
self.header = header.clone();
Ok(())
}
/// Special handling to make sure the whole kernel set matches each of its
/// roots in each block header, without truncation. We go back header by
/// header, rewind and check each root. This fixes a potential weakness in
/// fast sync where a reorg past the horizon could allow a whole rewrite of
/// the kernel set.
pub fn validate_root(&self) -> Result<(), Error> {
if self.pmmr.root() != self.header.kernel_root {
return Err(ErrorKind::InvalidTxHashSet(format!(
"Kernel root at {} does not match",
self.header.height
)).into());
}
Ok(())
}
}

View file

@ -28,7 +28,7 @@ use util::secp::pedersen::{Commitment, RangeProof};
use core::core::committed::Committed; use core::core::committed::Committed;
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::pmmr::{self, ReadonlyPMMR, PMMR}; use core::core::pmmr::{self, ReadonlyPMMR, RewindablePMMR, PMMR};
use core::core::{Block, BlockHeader, Input, Output, OutputFeatures, OutputIdentifier, TxKernel}; use core::core::{Block, BlockHeader, Input, Output, OutputFeatures, OutputIdentifier, TxKernel};
use core::global; use core::global;
use core::ser::{PMMRIndexHashable, PMMRable}; use core::ser::{PMMRIndexHashable, PMMRable};
@ -38,9 +38,9 @@ use grin_store;
use grin_store::pmmr::{PMMRBackend, PMMR_FILES}; use grin_store::pmmr::{PMMRBackend, PMMR_FILES};
use grin_store::types::prune_noop; use grin_store::types::prune_noop;
use store::{Batch, ChainStore}; use store::{Batch, ChainStore};
use txhashset::{RewindableKernelView, UTXOView};
use types::{TxHashSetRoots, TxHashsetWriteStatus}; use types::{TxHashSetRoots, TxHashsetWriteStatus};
use util::{file, secp_static, zip, LOGGER}; use util::{file, secp_static, zip, LOGGER};
use utxo_view::UTXOView;
const TXHASHSET_SUBDIR: &'static str = "txhashset"; const TXHASHSET_SUBDIR: &'static str = "txhashset";
const OUTPUT_SUBDIR: &'static str = "output"; const OUTPUT_SUBDIR: &'static str = "output";
@ -275,8 +275,12 @@ where
let commit_index = trees.commit_index.clone(); let commit_index = trees.commit_index.clone();
let batch = commit_index.batch()?; let batch = commit_index.batch()?;
// We want to use the current head of the most work chain unless
// we explicitly rewind the extension.
let header = batch.head_header()?;
trace!(LOGGER, "Starting new txhashset (readonly) extension."); trace!(LOGGER, "Starting new txhashset (readonly) extension.");
let mut extension = Extension::new(trees, &batch); let mut extension = Extension::new(trees, &batch, header);
extension.force_rollback(); extension.force_rollback();
res = inner(&mut extension); res = inner(&mut extension);
} }
@ -312,6 +316,30 @@ where
res res
} }
/// Rewindable (but still readonly) view on the kernel MMR.
/// The underlying backend is readonly. But we permit the PMMR to be "rewound"
/// via last_pos.
/// We create a new db batch for this view and discard it (rollback)
/// when we are done with the view.
pub fn rewindable_kernel_view<'a, F, T>(trees: &'a TxHashSet, inner: F) -> Result<T, Error>
where
F: FnOnce(&mut RewindableKernelView) -> Result<T, Error>,
{
let res: Result<T, Error>;
{
let kernel_pmmr =
RewindablePMMR::at(&trees.kernel_pmmr_h.backend, trees.kernel_pmmr_h.last_pos);
// Create a new batch here to pass into the kernel_view.
// Discard it (rollback) after we finish with the kernel_view.
let batch = trees.commit_index.batch()?;
let header = batch.head_header()?;
let mut view = RewindableKernelView::new(kernel_pmmr, &batch, header);
res = inner(&mut view);
}
res
}
/// Starts a new unit of work to extend the chain with additional blocks, /// Starts a new unit of work to extend the chain with additional blocks,
/// accepting a closure that will work within that unit of work. The closure /// accepting a closure that will work within that unit of work. The closure
/// has access to an Extension object that allows the addition of blocks to /// has access to an Extension object that allows the addition of blocks to
@ -331,12 +359,16 @@ where
let res: Result<T, Error>; let res: Result<T, Error>;
let rollback: bool; let rollback: bool;
// We want to use the current head of the most work chain unless
// we explicitly rewind the extension.
let header = batch.head_header()?;
// create a child transaction so if the state is rolled back by itself, all // create a child transaction so if the state is rolled back by itself, all
// index saving can be undone // index saving can be undone
let child_batch = batch.child()?; let child_batch = batch.child()?;
{ {
trace!(LOGGER, "Starting new txhashset extension."); trace!(LOGGER, "Starting new txhashset extension.");
let mut extension = Extension::new(trees, &child_batch); let mut extension = Extension::new(trees, &child_batch, header);
res = inner(&mut extension); res = inner(&mut extension);
rollback = extension.rollback; rollback = extension.rollback;
@ -381,6 +413,8 @@ where
/// reversible manner within a unit of work provided by the `extending` /// reversible manner within a unit of work provided by the `extending`
/// function. /// function.
pub struct Extension<'a> { pub struct Extension<'a> {
header: BlockHeader,
output_pmmr: PMMR<'a, OutputIdentifier, PMMRBackend<OutputIdentifier>>, output_pmmr: PMMR<'a, OutputIdentifier, PMMRBackend<OutputIdentifier>>,
rproof_pmmr: PMMR<'a, RangeProof, PMMRBackend<RangeProof>>, rproof_pmmr: PMMR<'a, RangeProof, PMMRBackend<RangeProof>>,
kernel_pmmr: PMMR<'a, TxKernel, PMMRBackend<TxKernel>>, kernel_pmmr: PMMR<'a, TxKernel, PMMRBackend<TxKernel>>,
@ -425,8 +459,9 @@ impl<'a> Committed for Extension<'a> {
} }
impl<'a> Extension<'a> { impl<'a> Extension<'a> {
fn new(trees: &'a mut TxHashSet, batch: &'a Batch) -> Extension<'a> { fn new(trees: &'a mut TxHashSet, batch: &'a Batch, header: BlockHeader) -> Extension<'a> {
Extension { Extension {
header,
output_pmmr: PMMR::at( output_pmmr: PMMR::at(
&mut trees.output_pmmr_h.backend, &mut trees.output_pmmr_h.backend,
trees.output_pmmr_h.last_pos, trees.output_pmmr_h.last_pos,
@ -452,11 +487,7 @@ impl<'a> Extension<'a> {
// TODO - move this into "utxo_view" // TODO - move this into "utxo_view"
/// Verify we are not attempting to spend any coinbase outputs /// Verify we are not attempting to spend any coinbase outputs
/// that have not sufficiently matured. /// that have not sufficiently matured.
pub fn verify_coinbase_maturity( pub fn verify_coinbase_maturity(&self, inputs: &Vec<Input>, height: u64) -> Result<(), Error> {
&mut self,
inputs: &Vec<Input>,
height: u64,
) -> Result<(), Error> {
// Find the greatest output pos of any coinbase // Find the greatest output pos of any coinbase
// outputs we are attempting to spend. // outputs we are attempting to spend.
let pos = inputs let pos = inputs
@ -507,6 +538,9 @@ impl<'a> Extension<'a> {
self.apply_kernel(kernel)?; self.apply_kernel(kernel)?;
} }
// Update the header on the extension to reflect the block we just applied.
self.header = b.header.clone();
Ok(()) Ok(())
} }
@ -597,26 +631,16 @@ impl<'a> Extension<'a> {
} }
/// TODO - move this into "utxo_view" /// TODO - move this into "utxo_view"
/// Build a Merkle proof for the given output and the block by /// Build a Merkle proof for the given output and the block
/// rewinding the MMR to the last pos of the block. /// this extension is currently referencing.
/// Note: this relies on the MMR being stable even after pruning/compaction. /// Note: this relies on the MMR being stable even after pruning/compaction.
/// We need the hash of each sibling pos from the pos up to the peak /// We need the hash of each sibling pos from the pos up to the peak
/// including the sibling leaf node which may have been removed. /// including the sibling leaf node which may have been removed.
pub fn merkle_proof( pub fn merkle_proof(&self, output: &OutputIdentifier) -> Result<MerkleProof, Error> {
&mut self,
output: &OutputIdentifier,
block_header: &BlockHeader,
) -> Result<MerkleProof, Error> {
debug!( debug!(
LOGGER, LOGGER,
"txhashset: merkle_proof: output: {:?}, block: {:?}", "txhashset: merkle_proof: output: {:?}", output.commit,
output.commit,
block_header.hash()
); );
// rewind to the specified block for a consistent view
self.rewind(block_header)?;
// then calculate the Merkle Proof based on the known pos // then calculate the Merkle Proof based on the known pos
let pos = self.batch.get_output_pos(&output.commit)?; let pos = self.batch.get_output_pos(&output.commit)?;
let merkle_proof = self let merkle_proof = self
@ -632,12 +656,12 @@ impl<'a> Extension<'a> {
/// the block hash as filename suffix. /// the block hash as filename suffix.
/// Needed for fast-sync (utxo file needs to be rewound before sending /// Needed for fast-sync (utxo file needs to be rewound before sending
/// across). /// across).
pub fn snapshot(&mut self, header: &BlockHeader) -> Result<(), Error> { pub fn snapshot(&mut self) -> Result<(), Error> {
self.output_pmmr self.output_pmmr
.snapshot(header) .snapshot(&self.header)
.map_err(|e| ErrorKind::Other(e))?; .map_err(|e| ErrorKind::Other(e))?;
self.rproof_pmmr self.rproof_pmmr
.snapshot(header) .snapshot(&self.header)
.map_err(|e| ErrorKind::Other(e))?; .map_err(|e| ErrorKind::Other(e))?;
Ok(()) Ok(())
} }
@ -652,21 +676,24 @@ impl<'a> Extension<'a> {
block_header.hash(), block_header.hash(),
); );
let head_header = self.batch.head_header()?;
// 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
// after the position we are rewinding to (these operations will be // after the position we are rewinding to (these operations will be
// undone during rewind). // undone during rewind).
// Rewound output pos will be removed from the MMR. // Rewound output pos will be removed from the MMR.
// Rewound input (spent) pos will be added back to the MMR. // Rewound input (spent) pos will be added back to the MMR.
let rewind_rm_pos = input_pos_to_rewind(block_header, &head_header, &self.batch)?; let rewind_rm_pos = input_pos_to_rewind(block_header, &self.header, &self.batch)?;
self.rewind_to_pos( self.rewind_to_pos(
block_header.output_mmr_size, block_header.output_mmr_size,
block_header.kernel_mmr_size, block_header.kernel_mmr_size,
&rewind_rm_pos, &rewind_rm_pos,
) )?;
// Update our header to reflect the one we rewound to.
self.header = block_header.clone();
Ok(())
} }
/// Rewinds the MMRs to the provided positions, given the output and /// Rewinds the MMRs to the provided positions, given the output and
@ -707,17 +734,17 @@ impl<'a> Extension<'a> {
} }
/// Validate the various MMR roots against the block header. /// Validate the various MMR roots against the block header.
pub fn validate_roots(&self, header: &BlockHeader) -> Result<(), Error> { pub fn validate_roots(&self) -> Result<(), Error> {
// If we are validating the genesis block then we have no outputs or // If we are validating the genesis block then we have no outputs or
// kernels. So we are done here. // kernels. So we are done here.
if header.height == 0 { if self.header.height == 0 {
return Ok(()); return Ok(());
} }
let roots = self.roots(); let roots = self.roots();
if roots.output_root != header.output_root if roots.output_root != self.header.output_root
|| roots.rproof_root != header.range_proof_root || roots.rproof_root != self.header.range_proof_root
|| roots.kernel_root != header.kernel_root || roots.kernel_root != self.header.kernel_root
{ {
Err(ErrorKind::InvalidRoot.into()) Err(ErrorKind::InvalidRoot.into())
} else { } else {
@ -726,15 +753,19 @@ impl<'a> Extension<'a> {
} }
/// Validate the output and kernel MMR sizes against the block header. /// Validate the output and kernel MMR sizes against the block header.
pub fn validate_sizes(&self, header: &BlockHeader) -> Result<(), Error> { pub fn validate_sizes(&self) -> Result<(), Error> {
// If we are validating the genesis block then we have no outputs or // If we are validating the genesis block then we have no outputs or
// kernels. So we are done here. // kernels. So we are done here.
if header.height == 0 { if self.header.height == 0 {
return Ok(()); return Ok(());
} }
let (output_mmr_size, _, kernel_mmr_size) = self.sizes(); let (output_mmr_size, rproof_mmr_size, kernel_mmr_size) = self.sizes();
if output_mmr_size != header.output_mmr_size || kernel_mmr_size != header.kernel_mmr_size { if output_mmr_size != self.header.output_mmr_size
|| kernel_mmr_size != self.header.kernel_mmr_size
{
Err(ErrorKind::InvalidMMRSize.into())
} else if output_mmr_size != rproof_mmr_size {
Err(ErrorKind::InvalidMMRSize.into()) Err(ErrorKind::InvalidMMRSize.into())
} else { } else {
Ok(()) Ok(())
@ -771,34 +802,32 @@ impl<'a> Extension<'a> {
/// This is an expensive operation as we need to retrieve all the UTXOs and kernels /// This is an expensive operation as we need to retrieve all the UTXOs and kernels
/// from the respective MMRs. /// from the respective MMRs.
/// For a significantly faster way of validating full kernel sums see BlockSums. /// For a significantly faster way of validating full kernel sums see BlockSums.
pub fn validate_kernel_sums( pub fn validate_kernel_sums(&self) -> Result<((Commitment, Commitment)), Error> {
&self, let (utxo_sum, kernel_sum) = self.verify_kernel_sums(
header: &BlockHeader, self.header.total_overage(),
) -> Result<((Commitment, Commitment)), Error> { self.header.total_kernel_offset(),
let (utxo_sum, kernel_sum) = )?;
self.verify_kernel_sums(header.total_overage(), header.total_kernel_offset())?;
Ok((utxo_sum, kernel_sum)) Ok((utxo_sum, kernel_sum))
} }
/// Validate the txhashset state against the provided block header. /// Validate the txhashset state against the provided block header.
pub fn validate( pub fn validate(
&mut self, &self,
header: &BlockHeader,
skip_rproofs: bool, skip_rproofs: bool,
status: &TxHashsetWriteStatus, status: &TxHashsetWriteStatus,
) -> Result<((Commitment, Commitment)), Error> { ) -> Result<((Commitment, Commitment)), Error> {
self.validate_mmrs()?; self.validate_mmrs()?;
self.validate_roots(header)?; self.validate_roots()?;
self.validate_sizes(header)?; self.validate_sizes()?;
if header.height == 0 { if self.header.height == 0 {
let zero_commit = secp_static::commit_to_zero_value(); let zero_commit = secp_static::commit_to_zero_value();
return Ok((zero_commit.clone(), zero_commit.clone())); return Ok((zero_commit.clone(), zero_commit.clone()));
} }
// The real magicking happens here. Sum of kernel excesses should equal // The real magicking happens here. Sum of kernel excesses should equal
// sum of unspent outputs minus total supply. // sum of unspent outputs minus total supply.
let (output_sum, kernel_sum) = self.validate_kernel_sums(header)?; let (output_sum, kernel_sum) = self.validate_kernel_sums()?;
// This is an expensive verification step. // This is an expensive verification step.
self.verify_kernel_signatures(status)?; self.verify_kernel_signatures(status)?;
@ -948,37 +977,6 @@ impl<'a> Extension<'a> {
); );
Ok(()) Ok(())
} }
/// Special handling to make sure the whole kernel set matches each of its
/// roots in each block header, without truncation. We go back header by
/// header, rewind and check each root. This fixes a potential weakness in
/// fast sync where a reorg past the horizon could allow a whole rewrite of
/// the kernel set.
pub fn validate_kernel_history(&mut self, header: &BlockHeader) -> Result<(), Error> {
assert!(
self.rollback,
"verified kernel history on writeable txhashset extension"
);
let mut current = header.clone();
loop {
current = self.batch.get_block_header(&current.previous)?;
if current.height == 0 {
break;
}
// rewinding kernels only further and further back
self.kernel_pmmr
.rewind(current.kernel_mmr_size, &Bitmap::create())
.map_err(&ErrorKind::TxHashSetErr)?;
if self.kernel_pmmr.root() != current.kernel_root {
return Err(ErrorKind::InvalidTxHashSet(format!(
"Kernel root at {} does not match",
current.height
)).into());
}
}
Ok(())
}
} }
/// Packages the txhashset data files into a zip and returns a Read to the /// Packages the txhashset data files into a zip and returns a Read to the

View file

@ -39,7 +39,9 @@
mod backend; mod backend;
mod pmmr; mod pmmr;
mod readonly_pmmr; mod readonly_pmmr;
mod rewindable_pmmr;
pub use self::backend::*; pub use self::backend::*;
pub use self::pmmr::*; pub use self::pmmr::*;
pub use self::readonly_pmmr::*; pub use self::readonly_pmmr::*;
pub use self::rewindable_pmmr::*;

View file

@ -0,0 +1,121 @@
// 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.
//! Rewindable (but still readonly) view of a PMMR.
//! Only supports non-pruneable backends (i.e. kernel MMR backend).
use std::marker;
use core::hash::Hash;
use core::pmmr::{bintree_postorder_height, is_leaf, peaks, Backend};
use ser::{PMMRable, PMMRIndexHashable};
/// Rewindable (but still readonly) view of a PMMR.
pub struct RewindablePMMR<'a, T, B>
where
T: PMMRable,
B: 'a + Backend<T>,
{
/// The last position in the PMMR
last_pos: u64,
/// The backend for this readonly PMMR
backend: &'a B,
// only needed to parameterise Backend
_marker: marker::PhantomData<T>,
}
impl<'a, T, B> RewindablePMMR<'a, T, B>
where
T: PMMRable + ::std::fmt::Debug,
B: 'a + Backend<T>,
{
/// Build a new readonly PMMR.
pub fn new(backend: &'a B) -> RewindablePMMR<T, B> {
RewindablePMMR {
last_pos: 0,
backend: backend,
_marker: marker::PhantomData,
}
}
/// Build a new readonly PMMR pre-initialized to
/// last_pos with the provided backend.
pub fn at(backend: &'a B, last_pos: u64) -> RewindablePMMR<T, B> {
RewindablePMMR {
last_pos: last_pos,
backend: backend,
_marker: marker::PhantomData,
}
}
/// Note: We only rewind the last_pos, we do not rewind the (readonly) backend.
/// Prunable backends are not supported here.
pub fn rewind(&mut self, position: u64) -> Result<(), String> {
// Identify which actual position we should rewind to as the provided
// position is a leaf. We traverse the MMR to include any parent(s) that
// need to be included for the MMR to be valid.
let mut pos = position;
while bintree_postorder_height(pos + 1) > 0 {
pos += 1;
}
self.last_pos = pos;
Ok(())
}
/// Get the data element at provided position in the MMR.
pub fn get_data(&self, pos: u64) -> Option<T> {
if pos > self.last_pos {
// If we are beyond the rhs of the MMR return None.
None
} else if is_leaf(pos) {
// If we are a leaf then get data from the backend.
self.backend.get_data(pos)
} else {
// If we are not a leaf then return None as only leaves have data.
None
}
}
/// Computes the root of the MMR. Find all the peaks in the current
/// tree and "bags" them to get a single peak.
pub fn root(&self) -> Hash {
let mut res = None;
for peak in self.peaks().iter().rev() {
res = match res {
None => Some(*peak),
Some(rhash) => Some((*peak, rhash).hash_with_index(self.unpruned_size())),
}
}
res.expect("no root, invalid tree")
}
/// Returns a vec of the peaks of this MMR.
pub fn peaks(&self) -> Vec<Hash> {
let peaks_pos = peaks(self.last_pos);
peaks_pos
.into_iter()
.filter_map(|pi| {
// here we want to get from underlying hash file
// as the pos *may* have been "removed"
self.backend.get_from_file(pi)
}).collect()
}
/// Total size of the tree, including intermediary nodes and ignoring any
/// pruning.
pub fn unpruned_size(&self) -> u64 {
self.last_pos
}
}