client invalidateheader and resetchainhead (#3618)

* wip - "reset_head" via owner api functionality

* jsonrpc pass hash in as a string

* sort of works

* not a reorg if we simply accept several blocks at once

* remember to reset header MMR separately
as it is readonly when interacting with txhashset extension

* basic client integration
needs error handling etc.

* reset sync status when reset chain head

* track "denylist" (todo) and validate headers against this via the ctx

* track denylist (header hashes) in chain itself

* header denylist in play

* expose invalidateheader as client cmd

* rework reset_chain_head - rewind txhashset then header MMR
This commit is contained in:
Antioch Peverell 2021-04-29 11:05:05 +01:00 committed by GitHub
parent 9e27e6f9d3
commit 89c06ddab7
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
10 changed files with 288 additions and 46 deletions

View file

@ -14,7 +14,7 @@
use super::utils::{get_output, get_output_v2, w};
use crate::chain;
use crate::core::core::hash::Hashed;
use crate::core::core::hash::{Hash, Hashed};
use crate::rest::*;
use crate::router::{Handler, ResponseFuture};
use crate::types::*;
@ -72,6 +72,29 @@ impl Handler for ChainValidationHandler {
}
}
pub struct ChainResetHandler {
pub chain: Weak<chain::Chain>,
pub sync_state: Weak<chain::SyncState>,
}
impl ChainResetHandler {
pub fn reset_chain_head(&self, hash: Hash) -> Result<(), Error> {
let chain = w(&self.chain)?;
let header = chain.get_block_header(&hash)?;
chain.reset_chain_head(&header)?;
// Reset the sync status and clear out any sync error.
w(&self.sync_state)?.reset();
Ok(())
}
pub fn invalidate_header(&self, hash: Hash) -> Result<(), Error> {
let chain = w(&self.chain)?;
chain.invalidate_header(hash)?;
Ok(())
}
}
/// Chain compaction handler. Trigger a compaction of the chain state to regain
/// storage space.
/// POST /v1/chain/compact
@ -81,9 +104,9 @@ pub struct ChainCompactHandler {
impl ChainCompactHandler {
pub fn compact_chain(&self) -> Result<(), Error> {
w(&self.chain)?
.compact()
.map_err(|_| ErrorKind::Internal("chain error".to_owned()).into())
let chain = w(&self.chain)?;
chain.compact()?;
Ok(())
}
}

View file

@ -15,7 +15,8 @@
//! Owner API External Definition
use crate::chain::{Chain, SyncState};
use crate::handlers::chain_api::{ChainCompactHandler, ChainValidationHandler};
use crate::core::core::hash::Hash;
use crate::handlers::chain_api::{ChainCompactHandler, ChainResetHandler, ChainValidationHandler};
use crate::handlers::peers_api::{PeerHandler, PeersConnectedHandler};
use crate::handlers::server_api::StatusHandler;
use crate::p2p::types::PeerInfoDisplay;
@ -107,6 +108,26 @@ impl Owner {
chain_compact_handler.compact_chain()
}
pub fn reset_chain_head(&self, hash: String) -> Result<(), Error> {
let hash = Hash::from_hex(&hash)
.map_err(|_| ErrorKind::RequestError("invalid header hash".into()))?;
let handler = ChainResetHandler {
chain: self.chain.clone(),
sync_state: self.sync_state.clone(),
};
handler.reset_chain_head(hash)
}
pub fn invalidate_header(&self, hash: String) -> Result<(), Error> {
let hash = Hash::from_hex(&hash)
.map_err(|_| ErrorKind::RequestError("invalid header hash".into()))?;
let handler = ChainResetHandler {
chain: self.chain.clone(),
sync_state: self.sync_state.clone(),
};
handler.invalidate_header(hash)
}
/// Retrieves information about stored peers.
/// If `None` is provided, will list all stored peers.
///

View file

@ -132,6 +132,10 @@ pub trait OwnerRpc: Sync + Send {
*/
fn compact_chain(&self) -> Result<(), ErrorKind>;
fn reset_chain_head(&self, hash: String) -> Result<(), ErrorKind>;
fn invalidate_header(&self, hash: String) -> Result<(), ErrorKind>;
/**
Networked version of [Owner::get_peers](struct.Owner.html#method.get_peers).
@ -363,6 +367,14 @@ impl OwnerRpc for Owner {
Owner::validate_chain(self).map_err(|e| e.kind().clone())
}
fn reset_chain_head(&self, hash: String) -> Result<(), ErrorKind> {
Owner::reset_chain_head(self, hash).map_err(|e| e.kind().clone())
}
fn invalidate_header(&self, hash: String) -> Result<(), ErrorKind> {
Owner::invalidate_header(self, hash).map_err(|e| e.kind().clone())
}
fn compact_chain(&self) -> Result<(), ErrorKind> {
Owner::compact_chain(self).map_err(|e| e.kind().clone())
}
@ -391,7 +403,7 @@ macro_rules! doctest_helper_json_rpc_owner_assert_response {
// create temporary grin server, run jsonrpc request on node api, delete server, return
// json response.
{
{
/*use grin_servers::test_framework::framework::run_doctest;
use grin_util as util;
use serde_json;
@ -425,6 +437,6 @@ macro_rules! doctest_helper_json_rpc_owner_assert_response {
serde_json::to_string_pretty(&expected_response).unwrap()
);
}*/
}
}
};
}

View file

@ -15,7 +15,6 @@
//! Facade and handler for the rest of the blockchain implementation
//! and mostly the chain pipeline.
use crate::core::core::hash::{Hash, Hashed};
use crate::core::core::merkle_proof::MerkleProof;
use crate::core::core::{
Block, BlockHeader, BlockSums, Committed, Inputs, KernelFeatures, Output, OutputIdentifier,
@ -34,6 +33,11 @@ use crate::types::{
};
use crate::util::secp::pedersen::{Commitment, RangeProof};
use crate::util::RwLock;
use crate::{
core::core::hash::{Hash, Hashed},
store::Batch,
txhashset::{ExtensionPair, HeaderExtension},
};
use grin_store::Error::NotFoundErr;
use std::collections::HashMap;
use std::fs::{self, File};
@ -151,6 +155,7 @@ pub struct Chain {
pibd_segmenter: Arc<RwLock<Option<Segmenter>>>,
// POW verification function
pow_verifier: fn(&BlockHeader) -> Result<(), pow::Error>,
denylist: Arc<RwLock<Vec<Hash>>>,
archive_mode: bool,
genesis: BlockHeader,
}
@ -198,6 +203,7 @@ impl Chain {
header_pmmr: Arc::new(RwLock::new(header_pmmr)),
pibd_segmenter: Arc::new(RwLock::new(None)),
pow_verifier,
denylist: Arc::new(RwLock::new(vec![])),
archive_mode,
genesis: genesis.header,
};
@ -222,6 +228,51 @@ impl Chain {
Ok(chain)
}
/// Add provided header hash to our "denylist".
/// The header corresponding to any "denied" hash will be rejected
/// and the peer subsequently banned.
pub fn invalidate_header(&self, hash: Hash) -> Result<(), Error> {
self.denylist.write().push(hash);
Ok(())
}
/// Reset both head and header_head to the provided header.
/// Handles simple rewind and more complex fork scenarios.
/// Used by the reset_chain_head owner api endpoint.
pub fn reset_chain_head<T: Into<Tip>>(&self, head: T) -> Result<(), Error> {
let head = head.into();
let mut header_pmmr = self.header_pmmr.write();
let mut txhashset = self.txhashset.write();
let mut batch = self.store.batch()?;
let header = batch.get_block_header(&head.hash())?;
// Rewind and reapply blocks to reset the output/rangeproof/kernel MMR.
txhashset::extending(
&mut header_pmmr,
&mut txhashset,
&mut batch,
|ext, batch| {
self.rewind_and_apply_fork(&header, ext, batch)?;
batch.save_body_head(&head)?;
Ok(())
},
)?;
// If the rewind of full blocks was successful then we can rewind the header MMR.
// Rewind and reapply headers to reset the header MMR.
txhashset::header_extending(&mut header_pmmr, &mut batch, |ext, batch| {
self.rewind_and_apply_header_fork(&header, ext, batch)?;
batch.save_header_head(&head)?;
Ok(())
})?;
batch.commit()?;
Ok(())
}
/// Are we running with archive_mode enabled?
pub fn archive_mode(&self) -> bool {
self.archive_mode
@ -278,7 +329,7 @@ impl Chain {
// If head is updated then we are either "next" block or we just experienced a "reorg" to new head.
// Otherwise this is a "fork" off the main chain.
if let Some(head) = head {
if head.prev_block_h == prev_head.last_block_h {
if self.is_on_current_chain(prev_head, head).is_ok() {
BlockStatus::Next { prev }
} else {
BlockStatus::Reorg {
@ -297,7 +348,8 @@ impl Chain {
}
/// Quick check for "known" duplicate block up to and including current chain head.
fn is_known(&self, header: &BlockHeader) -> Result<(), Error> {
/// Returns an error if this block is "known".
pub fn is_known(&self, header: &BlockHeader) -> Result<(), Error> {
let head = self.head()?;
if head.hash() == header.hash() {
return Err(ErrorKind::Unfit("duplicate block".into()).into());
@ -347,14 +399,14 @@ impl Chain {
/// Returns true if it has been added to the longest chain
/// or false if it has added to a fork (or orphan?).
fn process_block_single(&self, b: Block, opts: Options) -> Result<Option<Tip>, Error> {
// Check if we already know about this block.
self.is_known(&b.header)?;
// Process the header first.
// If invalid then fail early.
// If valid then continue with block processing with header_head committed to db etc.
self.process_block_header(&b.header, opts)?;
// Check if we already know about this full block.
self.is_known(&b.header)?;
// Check if this block is an orphan.
// Only do this once we know the header PoW is valid.
self.check_orphan(&b, opts)?;
@ -431,9 +483,13 @@ impl Chain {
header_pmmr: &'a mut txhashset::PMMRHandle<BlockHeader>,
txhashset: &'a mut txhashset::TxHashSet,
) -> Result<pipe::BlockContext<'a>, Error> {
let denylist = self.denylist.read().clone();
Ok(pipe::BlockContext {
opts,
pow_verifier: self.pow_verifier,
header_allowed: Box::new(move |header| {
pipe::validate_header_denylist(header, &denylist)
}),
header_pmmr,
txhashset,
batch,
@ -621,7 +677,7 @@ impl Chain {
// latest block header. Rewind the extension to the specified header to
// ensure the view is consistent.
txhashset::extending_readonly(&mut header_pmmr, &mut txhashset, |ext, batch| {
pipe::rewind_and_apply_fork(&header, ext, batch)?;
self.rewind_and_apply_fork(&header, ext, batch)?;
ext.extension
.validate(&self.genesis, fast_validation, &NoStatus, &header)?;
Ok(())
@ -634,7 +690,7 @@ impl Chain {
let prev_root =
txhashset::header_extending_readonly(&mut header_pmmr, &self.store(), |ext, batch| {
let prev_header = batch.get_previous_header(header)?;
pipe::rewind_and_apply_header_fork(&prev_header, ext, batch)?;
self.rewind_and_apply_header_fork(&prev_header, ext, batch)?;
ext.root()
})?;
@ -653,7 +709,7 @@ impl Chain {
let (prev_root, roots, sizes) =
txhashset::extending_readonly(&mut header_pmmr, &mut txhashset, |ext, batch| {
let previous_header = batch.get_previous_header(&b.header)?;
pipe::rewind_and_apply_fork(&previous_header, ext, batch)?;
self.rewind_and_apply_fork(&previous_header, ext, batch)?;
let extension = &mut ext.extension;
let header_extension = &mut ext.header_extension;
@ -698,7 +754,7 @@ impl Chain {
let mut txhashset = self.txhashset.write();
let merkle_proof =
txhashset::extending_readonly(&mut header_pmmr, &mut txhashset, |ext, batch| {
pipe::rewind_and_apply_fork(&header, ext, batch)?;
self.rewind_and_apply_fork(&header, ext, batch)?;
ext.extension.merkle_proof(out_id, batch)
})?;
@ -712,6 +768,34 @@ impl Chain {
txhashset.merkle_proof(commit)
}
/// Rewind and apply fork with the chain specific header validation (denylist) rules.
/// If we rewind and re-apply a "denied" block then validation will fail.
fn rewind_and_apply_fork(
&self,
header: &BlockHeader,
ext: &mut ExtensionPair,
batch: &Batch,
) -> Result<BlockHeader, Error> {
let denylist = self.denylist.read().clone();
pipe::rewind_and_apply_fork(header, ext, batch, &|header| {
pipe::validate_header_denylist(header, &denylist)
})
}
/// Rewind and apply fork with the chain specific header validation (denylist) rules.
/// If we rewind and re-apply a "denied" header then validation will fail.
fn rewind_and_apply_header_fork(
&self,
header: &BlockHeader,
ext: &mut HeaderExtension,
batch: &Batch,
) -> Result<(), Error> {
let denylist = self.denylist.read().clone();
pipe::rewind_and_apply_header_fork(header, ext, batch, &|header| {
pipe::validate_header_denylist(header, &denylist)
})
}
/// Provides a reading view into the current txhashset state as well as
/// the required indexes for a consumer to rewind to a consistent state
/// at the provided block hash.
@ -725,8 +809,9 @@ impl Chain {
let mut header_pmmr = self.header_pmmr.write();
let mut txhashset = self.txhashset.write();
txhashset::extending_readonly(&mut header_pmmr, &mut txhashset, |ext, batch| {
pipe::rewind_and_apply_fork(&header, ext, batch)?;
self.rewind_and_apply_fork(&header, ext, batch)?;
ext.extension.snapshot(batch)?;
// prepare the zip
@ -853,7 +938,7 @@ impl Chain {
pub fn fork_point(&self) -> Result<BlockHeader, Error> {
let body_head = self.head()?;
let mut current = self.get_block_header(&body_head.hash())?;
while !self.is_on_current_chain(&current).is_ok() {
while !self.is_on_current_chain(&current, body_head).is_ok() {
current = self.get_previous_header(&current)?;
}
Ok(current)
@ -1404,9 +1489,13 @@ impl Chain {
/// Verifies the given block header is actually on the current chain.
/// Checks the header_by_height index to verify the header is where we say
/// it is
pub fn is_on_current_chain(&self, header: &BlockHeader) -> Result<(), Error> {
let chain_header = self.get_header_by_height(header.height)?;
if chain_header.hash() == header.hash() {
fn is_on_current_chain<T: Into<Tip>>(&self, x: T, head: Tip) -> Result<(), Error> {
let x: Tip = x.into();
if x.height > head.height {
return Err(ErrorKind::Other("not on current chain".to_string()).into());
}
if x.hash() == self.get_header_hash_by_height(x.height)? {
Ok(())
} else {
Err(ErrorKind::Other("not on current chain".to_string()).into())
@ -1419,7 +1508,7 @@ impl Chain {
let mut header_pmmr = self.header_pmmr.write();
txhashset::header_extending_readonly(&mut header_pmmr, &self.store(), |ext, batch| {
let header = batch.get_block_header(&sync_head.hash())?;
pipe::rewind_and_apply_header_fork(&header, ext, batch)?;
self.rewind_and_apply_header_fork(&header, ext, batch)?;
let hashes = heights
.iter()
@ -1496,7 +1585,7 @@ fn setup_head(
let header = batch.get_block_header(&head.last_block_h)?;
let res = txhashset::extending(header_pmmr, txhashset, &mut batch, |ext, batch| {
pipe::rewind_and_apply_fork(&header, ext, batch)?;
pipe::rewind_and_apply_fork(&header, ext, batch, &|_| Ok(()))?;
let extension = &mut ext.extension;
@ -1544,7 +1633,7 @@ fn setup_head(
let prev_header = batch.get_block_header(&head.prev_block_h)?;
txhashset::extending(header_pmmr, txhashset, &mut batch, |ext, batch| {
pipe::rewind_and_apply_fork(&prev_header, ext, batch)
pipe::rewind_and_apply_fork(&prev_header, ext, batch, &|_| Ok(()))
})?;
// Now "undo" the latest block and forget it ever existed.

View file

@ -15,7 +15,7 @@
//! Implementation of the chain block acceptance (or refusal) pipeline.
use crate::core::consensus;
use crate::core::core::hash::Hashed;
use crate::core::core::hash::{Hash, Hashed};
use crate::core::core::Committed;
use crate::core::core::{
block, Block, BlockHeader, BlockSums, HeaderVersion, OutputIdentifier, TransactionBody,
@ -34,6 +34,8 @@ pub struct BlockContext<'a> {
pub opts: Options,
/// The pow verifier to use when processing a block.
pub pow_verifier: fn(&BlockHeader) -> Result<(), pow::Error>,
/// Custom fn allowing arbitrary header validation rules (denylist) to be applied.
pub header_allowed: Box<dyn Fn(&BlockHeader) -> Result<(), Error>>,
/// The active txhashset (rewindable MMRs) to use for block processing.
pub txhashset: &'a mut txhashset::TxHashSet,
/// The active header MMR handle.
@ -119,8 +121,9 @@ pub fn process_block(
let header_pmmr = &mut ctx.header_pmmr;
let txhashset = &mut ctx.txhashset;
let batch = &mut ctx.batch;
let ctx_specific_validation = &ctx.header_allowed;
let fork_point = txhashset::extending(header_pmmr, txhashset, batch, |ext, batch| {
let fork_point = rewind_and_apply_fork(&prev, ext, batch)?;
let fork_point = rewind_and_apply_fork(&prev, ext, batch, ctx_specific_validation)?;
// Check any coinbase being spent have matured sufficiently.
// This needs to be done within the context of a potentially
@ -198,9 +201,11 @@ pub fn process_block_headers(
add_block_header(header, &ctx.batch)?;
}
let ctx_specific_validation = &ctx.header_allowed;
// Now apply this entire chunk of headers to the header MMR.
txhashset::header_extending(&mut ctx.header_pmmr, &mut ctx.batch, |ext, batch| {
rewind_and_apply_header_fork(&last_header, ext, batch)?;
rewind_and_apply_header_fork(&last_header, ext, batch, ctx_specific_validation)?;
// If previous sync_head is not on the "current" chain then
// these headers are on an alternative fork to sync_head.
@ -255,10 +260,12 @@ pub fn process_block_header(header: &BlockHeader, ctx: &mut BlockContext<'_>) ->
// We want to validate this individual header before applying it to our header PMMR.
validate_header(header, ctx)?;
let ctx_specific_validation = &ctx.header_allowed;
// Apply the header to the header PMMR, making sure we put the extension in the correct state
// based on previous header first.
txhashset::header_extending(&mut ctx.header_pmmr, &mut ctx.batch, |ext, batch| {
rewind_and_apply_header_fork(&prev_header, ext, batch)?;
rewind_and_apply_header_fork(&prev_header, ext, batch, ctx_specific_validation)?;
ext.validate_root(header)?;
ext.apply_header(header)?;
if !has_more_work(&header, &header_head) {
@ -322,10 +329,42 @@ fn prev_header_store(
Ok(prev)
}
/// Apply any "header_invalidated" (aka denylist) rules provided as part of the context.
fn validate_header_ctx(header: &BlockHeader, ctx: &mut BlockContext<'_>) -> Result<(), Error> {
// Apply any custom header validation rules via the context.
(ctx.header_allowed)(header)
}
/// Validate header against an explicit "denylist" of header hashes.
/// Returns a "Block" error which is "bad_data" and will result in peer being banned.
pub fn validate_header_denylist(header: &BlockHeader, denylist: &[Hash]) -> Result<(), Error> {
if denylist.is_empty() {
return Ok(());
}
// Assume our denylist is a manageable size for now.
// Log it here to occasionally remind us.
debug!(
"validate_header_denylist: {} at {}, denylist: {:?}",
header.hash(),
header.height,
denylist
);
if denylist.contains(&header.hash()) {
return Err(ErrorKind::Block(block::Error::Other("header hash denied".into())).into());
} else {
return Ok(());
}
}
/// First level of block validation that only needs to act on the block header
/// to make it as cheap as possible. The different validations are also
/// arranged by order of cost to have as little DoS surface as possible.
fn validate_header(header: &BlockHeader, ctx: &mut BlockContext<'_>) -> Result<(), Error> {
// Apply any ctx specific header validation (denylist) rules.
validate_header_ctx(header, ctx)?;
// First I/O cost, delayed as late as possible.
let prev = prev_header_store(header, &mut ctx.batch)?;
@ -537,6 +576,7 @@ pub fn rewind_and_apply_header_fork(
header: &BlockHeader,
ext: &mut txhashset::HeaderExtension<'_>,
batch: &store::Batch<'_>,
ctx_specific_validation: &dyn Fn(&BlockHeader) -> Result<(), Error>,
) -> Result<(), Error> {
let mut fork_hashes = vec![];
let mut current = header.clone();
@ -556,6 +596,11 @@ pub fn rewind_and_apply_header_fork(
let header = batch
.get_block_header(&h)
.map_err(|e| ErrorKind::StoreErr(e, "getting forked headers".to_string()))?;
// Re-validate every header being re-applied.
// This makes it possible to check all header hashes against the ctx specific "denylist".
(ctx_specific_validation)(&header)?;
ext.validate_root(&header)?;
ext.apply_header(&header)?;
}
@ -572,12 +617,13 @@ pub fn rewind_and_apply_fork(
header: &BlockHeader,
ext: &mut txhashset::ExtensionPair<'_>,
batch: &store::Batch<'_>,
ctx_specific_validation: &dyn Fn(&BlockHeader) -> Result<(), Error>,
) -> Result<BlockHeader, Error> {
let extension = &mut ext.extension;
let header_extension = &mut ext.header_extension;
// Prepare the header MMR.
rewind_and_apply_header_fork(header, header_extension, batch)?;
rewind_and_apply_header_fork(header, header_extension, batch, ctx_specific_validation)?;
// Rewind the txhashset extension back to common ancestor based on header MMR.
let mut current = batch.head_header()?;

View file

@ -807,8 +807,6 @@ where
{
let batch = store.batch()?;
// Note: Extending either the sync_head or header_head MMR here.
// Use underlying MMR to determine the "head".
let head = match handle.head_hash() {
Ok(hash) => {
let header = batch.get_block_header(&hash)?;
@ -845,8 +843,6 @@ where
// index saving can be undone
let child_batch = batch.child()?;
// Note: Extending either the sync_head or header_head MMR here.
// Use underlying MMR to determine the "head".
let head = match handle.head_hash() {
Ok(hash) => {
let header = child_batch.get_block_header(&hash)?;

View file

@ -134,6 +134,12 @@ impl SyncState {
}
}
/// Reset sync status to NoSync.
pub fn reset(&self) {
self.clear_sync_error();
self.update(SyncStatus::NoSync);
}
/// Whether the current state matches any active syncing operation.
/// Note: This includes our "initial" state.
pub fn is_syncing(&self) -> bool {
@ -363,6 +369,23 @@ impl Tip {
}
}
impl From<BlockHeader> for Tip {
fn from(header: BlockHeader) -> Self {
Self::from(&header)
}
}
impl From<&BlockHeader> for Tip {
fn from(header: &BlockHeader) -> Self {
Tip {
height: header.height,
last_block_h: header.hash(),
prev_block_h: header.prev_hash,
total_difficulty: header.total_difficulty(),
}
}
}
impl Hashed for Tip {
/// The hash of the underlying block.
fn hash(&self) -> Hash {
@ -380,16 +403,6 @@ impl Default for Tip {
}
}
}
impl From<&BlockHeader> for Tip {
fn from(header: &BlockHeader) -> Tip {
Tip {
height: header.height,
last_block_h: header.hash(),
prev_block_h: header.prev_hash,
total_difficulty: header.total_difficulty(),
}
}
}
/// Serialization of a tip, required to save to datastore.
impl ser::Writeable for Tip {

View file

@ -139,9 +139,10 @@ where
peer_info: &PeerInfo,
opts: chain::Options,
) -> Result<bool, chain::Error> {
if self.chain().block_exists(b.hash())? {
if self.chain().is_known(&b.header).is_err() {
return Ok(true);
}
debug!(
"Received block {} at {} from {} [in/out/kern: {}/{}/{}] going to process.",
b.hash(),
@ -160,9 +161,10 @@ where
peer_info: &PeerInfo,
) -> Result<bool, chain::Error> {
// No need to process this compact block if we have previously accepted the _full block_.
if self.chain().block_exists(cb.hash())? {
if self.chain().is_known(&cb.header).is_err() {
return Ok(true);
}
let bhash = cb.hash();
debug!(
"Received compact_block {} at {} from {} [out/kern/kern_ids: {}/{}/{}] going to process.",

View file

@ -129,6 +129,26 @@ impl HTTPNodeClient {
e.reset().unwrap();
}
pub fn reset_chain_head(&self, hash: String) {
let mut e = term::stdout().unwrap();
let params = json!([hash]);
match self.send_json_request::<()>("reset_chain_head", &params) {
Ok(_) => writeln!(e, "Successfully reset chain head {}", hash).unwrap(),
Err(_) => writeln!(e, "Failed to reset chain head {}", hash).unwrap(),
}
e.reset().unwrap();
}
pub fn invalidate_header(&self, hash: String) {
let mut e = term::stdout().unwrap();
let params = json!([hash]);
match self.send_json_request::<()>("invalidate_header", &params) {
Ok(_) => writeln!(e, "Successfully invalidated header: {}", hash).unwrap(),
Err(_) => writeln!(e, "Failed to invalidate header: {}", hash).unwrap(),
}
e.reset().unwrap();
}
pub fn ban_peer(&self, peer_addr: &SocketAddr) {
let mut e = term::stdout().unwrap();
let params = json!([peer_addr]);
@ -163,6 +183,14 @@ pub fn client_command(client_args: &ArgMatches<'_>, global_config: GlobalConfig)
("listconnectedpeers", Some(_)) => {
node_client.list_connected_peers();
}
("resetchainhead", Some(args)) => {
let hash = args.value_of("hash").unwrap();
node_client.reset_chain_head(hash.to_string());
}
("invalidateheader", Some(args)) => {
let hash = args.value_of("hash").unwrap();
node_client.invalidate_header(hash.to_string());
}
("ban", Some(peer_args)) => {
let peer = peer_args.value_of("peer").unwrap();

View file

@ -72,3 +72,15 @@ subcommands:
long: peer
required: true
takes_value: true
- resetchainhead:
about: Resets the local chain head
args:
- hash:
help: The header hash to reset to
required: true
- invalidateheader:
about: Adds header hash to denylist
args:
- hash:
help: The header hash to invalidate
required: true