saito_core/core/consensus/
blockchain.rs

1use std::cmp::max;
2use std::collections::VecDeque;
3use std::fmt::Debug;
4use std::io::Error;
5use std::sync::Arc;
6
7use ahash::{AHashMap, HashMap};
8use log::{debug, error, info, trace, warn};
9use rayon::prelude::*;
10use tokio::sync::mpsc::Sender;
11use tokio::sync::RwLock;
12
13use crate::core::consensus::block::{Block, BlockType};
14use crate::core::consensus::blockring::BlockRing;
15use crate::core::consensus::mempool::Mempool;
16use crate::core::consensus::slip::{Slip, SlipType};
17use crate::core::consensus::transaction::{Transaction, TransactionType};
18use crate::core::consensus::wallet::{Wallet, WalletUpdateStatus, WALLET_NOT_UPDATED};
19use crate::core::defs::{
20    BlockHash, BlockId, Currency, ForkId, PrintForLog, SaitoHash, SaitoPublicKey, SaitoUTXOSetKey,
21    Timestamp, UtxoSet, MIN_GOLDEN_TICKETS_DENOMINATOR, MIN_GOLDEN_TICKETS_NUMERATOR,
22    PROJECT_PUBLIC_KEY,
23};
24use crate::core::io::interface_io::InterfaceEvent;
25use crate::core::io::network::Network;
26use crate::core::io::storage::Storage;
27use crate::core::mining_thread::MiningEvent;
28use crate::core::routing_thread::RoutingEvent;
29use crate::core::util::balance_snapshot::BalanceSnapshot;
30use crate::core::util::configuration::Configuration;
31use crate::{drain, iterate};
32
33pub fn bit_pack(top: u32, bottom: u32) -> u64 {
34    ((top as u64) << 32) + (bottom as u64)
35}
36
37pub fn bit_unpack(packed: u64) -> (u32, u32) {
38    // Casting from a larger integer to a smaller integer (e.g. u32 -> u8) will truncate, no need to mask this
39    let bottom = packed as u32;
40    let top = (packed >> 32) as u32;
41    (top, bottom)
42}
43
44const FORK_ID_WEIGHTS: [u64; 16] = [
45    0, 10, 10, 10, 10, 10, 25, 25, 100, 300, 500, 4000, 10000, 20000, 50000, 100000,
46];
47
48// pub const DEFAULT_SOCIAL_STAKE: Currency = 2_000_000 * NOLAN_PER_SAITO;
49// pub const DEFAULT_SOCIAL_STAKE: Currency = 0;
50
51// pub const DEFAULT_SOCIAL_STAKE_PERIOD: BlockId = 60;
52
53#[derive(Debug)]
54pub enum AddBlockResult {
55    BlockAddedSuccessfully(
56        BlockHash,
57        bool, /*is in the longest chain ?*/
58        WalletUpdateStatus,
59    ),
60    BlockAlreadyExists,
61    FailedButRetry(
62        Block,
63        bool, /*fetch previous block*/
64        bool, /*fetch whole blockchain*/
65    ),
66    FailedNotValid,
67}
68
69#[derive(Debug)]
70pub enum WindingResult {
71    Wind(usize, bool, WalletUpdateStatus),
72    Unwind(usize, bool, Vec<SaitoHash>, WalletUpdateStatus),
73    FinishWithSuccess(WalletUpdateStatus),
74    FinishWithFailure,
75}
76
77#[derive(Debug)]
78pub struct Blockchain {
79    pub utxoset: UtxoSet,
80    pub blockring: BlockRing,
81    pub blocks: AHashMap<SaitoHash, Block>,
82    pub wallet_lock: Arc<RwLock<Wallet>>,
83    pub genesis_block_id: u64,
84    pub fork_id: Option<SaitoHash>,
85    pub last_block_hash: SaitoHash,
86    pub last_block_id: u64,
87    pub last_timestamp: u64,
88    pub last_burnfee: Currency,
89
90    pub genesis_timestamp: u64,
91    genesis_block_hash: SaitoHash,
92    pub lowest_acceptable_timestamp: u64,
93    pub lowest_acceptable_block_hash: SaitoHash,
94    pub lowest_acceptable_block_id: u64,
95
96    pub social_stake_requirement: Currency,
97    pub social_stake_period: u64,
98    pub genesis_period: BlockId,
99
100    pub checkpoint_found: bool,
101    pub initial_token_supply: Currency,
102    pub last_issuance_written_on: BlockId,
103}
104
105impl Blockchain {
106    #[allow(clippy::new_without_default)]
107    pub fn new(
108        wallet_lock: Arc<RwLock<Wallet>>,
109        genesis_period: BlockId,
110        social_stake: Currency,
111        social_stake_period: BlockId,
112    ) -> Self {
113        info!("initializing blockchain with genesis period : {:?}, social_stake : {:?}, social_stake_period : {:?}", genesis_period,social_stake,social_stake_period);
114        Blockchain {
115            utxoset: AHashMap::new(),
116            blockring: BlockRing::new(genesis_period),
117            blocks: AHashMap::new(),
118            wallet_lock,
119            genesis_block_id: 0,
120            fork_id: None,
121            last_block_hash: [0; 32],
122            last_block_id: 0,
123            last_timestamp: 0,
124            last_burnfee: 0,
125            genesis_timestamp: 0,
126            genesis_block_hash: [0; 32],
127            lowest_acceptable_timestamp: 0,
128            lowest_acceptable_block_hash: [0; 32],
129            lowest_acceptable_block_id: 0,
130            // blocks_fetching: Default::default(),
131            social_stake_requirement: social_stake,
132            social_stake_period,
133            genesis_period,
134            checkpoint_found: false,
135            initial_token_supply: 0,
136            last_issuance_written_on: 0,
137        }
138    }
139    pub fn init(&mut self) -> Result<(), Error> {
140        Ok(())
141    }
142
143    pub fn set_fork_id(&mut self, fork_id: SaitoHash) {
144        debug!("setting fork id as : {:?}", fork_id.to_hex());
145        self.fork_id = Some(fork_id);
146    }
147
148    // #[async_recursion]
149    pub async fn add_block(
150        &mut self,
151        mut block: Block,
152        storage: &mut Storage,
153        mempool: &mut Mempool,
154        configs: &(dyn Configuration + Send + Sync),
155    ) -> AddBlockResult {
156        if block.generate().is_err() {
157            error!(
158                "block metadata generation failed. not adding block : {:?}",
159                block.hash.to_hex()
160            );
161            return AddBlockResult::FailedNotValid;
162        }
163
164        debug!(
165            "adding block {:?} of type : {:?} with id : {:?} with latest id : {:?} with tx count (gt/spv/total) : {:?}/{:?}/{:?}",
166            block.hash.to_hex(),
167            block.block_type,
168            block.id,
169            self.get_latest_block_id(),
170            block
171            .transactions
172            .iter()
173            .filter(|tx| tx.transaction_type == TransactionType::GoldenTicket)
174            .count(),
175            block
176            .transactions
177            .iter()
178            .filter(|tx| tx.transaction_type == TransactionType::SPV)
179            .count(),
180            block.transactions.len()
181        );
182
183        // start by extracting some variables that we will use
184        // repeatedly in the course of adding this block to the
185        // blockchain and our various indices.
186        let block_hash = block.hash;
187        let block_id = block.id;
188        let latest_block_hash = self.blockring.get_latest_block_hash();
189
190        // sanity checks
191        if self.blocks.contains_key(&block_hash) {
192            error!(
193                "block : {:?}-{:?} already exists in blockchain. not adding",
194                block.id,
195                block.hash.to_hex()
196            );
197            return AddBlockResult::BlockAlreadyExists;
198        }
199
200        // get missing block
201        if !self.blockring.is_empty() && self.get_block(&block.previous_block_hash).is_none() {
202            if block.previous_block_hash == [0; 32] {
203                info!(
204                    "hash is empty for parent of block : {:?}",
205                    block.hash.to_hex()
206                );
207            } else if configs.get_blockchain_configs().initial_loading_completed
208                || self.checkpoint_found
209            {
210                let previous_block_fetched = iterate!(mempool.blocks_queue, 100)
211                    .any(|b| block.previous_block_hash == b.hash);
212                let genesis_period = configs.get_consensus_config().unwrap().genesis_period;
213
214                return if !previous_block_fetched {
215                    if block.id > max(1, self.get_latest_block_id().saturating_sub(genesis_period))
216                    {
217                        let block_diff_before_fetching_chain: BlockId =
218                            std::cmp::min(1000, genesis_period);
219                        if block.id.abs_diff(self.get_latest_block_id())
220                            < block_diff_before_fetching_chain
221                        {
222                            debug!(
223                                "need to fetch previous block : {:?}-{:?}",
224                                block.id - 1,
225                                block.previous_block_hash.to_hex()
226                            );
227
228                            AddBlockResult::FailedButRetry(block, true, false)
229                        } else {
230                            info!("block : {:?}-{:?} is too distant with the current latest block : id={:?}. so need to fetch the whole blockchain from the peer to make sure this is not an attack",
231                            block.id,block.hash.to_hex(),self.get_latest_block_id());
232                            AddBlockResult::FailedButRetry(block, false, true)
233                        }
234                    } else {
235                        debug!(
236                            "block : {:?}-{:?} is too old to be added to the blockchain",
237                            block.id,
238                            block.hash.to_hex()
239                        );
240                        AddBlockResult::FailedNotValid
241                    }
242                } else {
243                    debug!(
244                        "previous block : {:?} is in the mempool. not fetching",
245                        block.previous_block_hash.to_hex()
246                    );
247                    AddBlockResult::FailedButRetry(block, false, false)
248                };
249            }
250        }
251
252        // pre-validation
253        //
254        // this would be a great place to put in a pre-validation check
255        // once we are finished implementing Saito Classic. Goal would
256        // be a fast form of lite-validation just to determine that it
257        // is worth going through the more general effort of evaluating
258        // this block for consensus.
259        //
260
261        // save block to disk
262        //
263        // we have traditionally saved blocks to disk AFTER validating them
264        // but this can slow down block propagation. So it may be sensible
265        // to start a save earlier-on in the process so that we can relay
266        // the block faster serving it off-disk instead of fetching it
267        // repeatedly from memory. Exactly when to do this is left as an
268        // optimization exercise.
269
270        // insert block into hashmap and index
271        //
272        // the blockring is a BlockRing which lets us know which blocks (at which depth)
273        // form part of the longest-chain. We also use the BlockRing to track information
274        // on network congestion (how many block candidates exist at various depths and
275        // in the future potentially the amount of work on each viable fork chain.
276        //
277        // we are going to transfer ownership of the block into the HashMap that stores
278        // the block next, so we insert it into our BlockRing first as that will avoid
279        // needing to borrow the value back for insertion into the BlockRing.
280        // TODO : check if this "if" condition can be moved to an assert
281        if !self
282            .blockring
283            .contains_block_hash_at_block_id(block_id, block_hash)
284        {
285            self.blockring.add_block(&block);
286        }
287
288        // blocks are stored in a hashmap indexed by the block_hash. we expect all
289        // all block_hashes to be unique, so simply insert blocks one-by-one on
290        // arrival if they do not exist.
291
292        if !self.blocks.contains_key(&block_hash) {
293            self.blocks.insert(block_hash, block);
294        } else {
295            error!(
296                "BLOCK IS ALREADY IN THE BLOCKCHAIN, WHY ARE WE ADDING IT????? {:?}",
297                block.hash.to_hex()
298            );
299            return AddBlockResult::BlockAlreadyExists;
300        }
301
302        // find shared ancestor of new_block with old_chain
303        let mut old_chain: Vec<[u8; 32]> = Vec::new();
304        let mut am_i_the_longest_chain = false;
305
306        let (shared_ancestor_found, shared_block_hash, mut new_chain) =
307            self.calculate_new_chain_for_add_block(block_hash);
308
309        // and get existing current chain for comparison
310        if shared_ancestor_found {
311            old_chain =
312                self.calculate_old_chain_for_add_block(latest_block_hash, shared_block_hash);
313        } else {
314            debug!(
315                "block without parent. block : {}-{:?}, latest : {:?}",
316                block_id,
317                block_hash.to_hex(),
318                latest_block_hash.to_hex()
319            );
320
321            // we have a block without a parent.
322            //
323            // if this is our first block, the blockring will have no entry yet
324            // and block_ring_lc_pos (longest_chain_position) will be pointing
325            // at None. We use this to determine if we are a new chain instead
326            // of creating a separate variable to manually track entries.
327            if self.blockring.is_empty() {
328
329                // no need for action as fall-through will result in proper default
330                // behavior. we have the comparison here to separate expected from
331                // unexpected / edge-case issues around block receipt.
332            } else {
333                // if this not our first block, handle edge-case around receiving
334                // block 503 before block 453 when block 453 is our expected proper
335                // next block and we are getting blocks out-of-order because of
336                // connection or network issues.
337                if latest_block_hash != [0; 32]
338                    && latest_block_hash == self.get_latest_block_hash()
339                    && (block_id
340                        > self
341                            .get_latest_block_id()
342                            .saturating_sub(self.genesis_period))
343                {
344                    info!("blocks received out-of-order issue. handling edge case...");
345
346                    let disconnected_block_id = self.get_latest_block_id();
347                    debug!("disconnected id : {:?}", disconnected_block_id);
348                    debug!(
349                        "disconnecting blocks from : {:?} to : {:?}",
350                        block_id + 1,
351                        disconnected_block_id
352                    );
353
354                    for i in block_id + 1..=disconnected_block_id {
355                        if let Some(disconnected_block_hash) =
356                            self.blockring.get_longest_chain_block_hash_at_block_id(i)
357                        {
358                            if disconnected_block_hash != [0; 32] {
359                                self.blockring.on_chain_reorganization(
360                                    i,
361                                    disconnected_block_hash,
362                                    false,
363                                );
364                                trace!("checking block id : {:?}", i);
365                                let disconnected_block =
366                                    self.get_mut_block(&disconnected_block_hash);
367                                if let Some(disconnected_block) = disconnected_block {
368                                    trace!("in longest chain set to false");
369                                    disconnected_block.in_longest_chain = false;
370                                }
371                            }
372                        }
373                    }
374
375                    // new_chain.clear();
376                    // new_chain.push(block_hash);
377                    am_i_the_longest_chain = false;
378                }
379            }
380            old_chain =
381                self.calculate_old_chain_upto_length(latest_block_hash, new_chain.len() as BlockId);
382        }
383
384        // at this point we should have a shared ancestor or not
385        // find out whether this new block is claiming to require chain-validation
386        if !am_i_the_longest_chain
387            && (block_id
388                > self
389                    .get_latest_block_id()
390                    .saturating_sub(self.genesis_period))
391            && self.is_new_chain_the_longest_chain(&new_chain, &old_chain)
392        {
393            debug!(
394                "new chain is the longest chain. changing am I the longest chain? {:?}. current block id : {} latest block id : {} genesis_period : {}",
395                block_hash.to_hex(),
396                block_id,
397                self.get_latest_block_id(),
398                self.genesis_period
399            );
400            am_i_the_longest_chain = true;
401        }
402
403        // now update blockring so it is not empty
404        //
405        // we do this down here instead of automatically on
406        // adding a block, as we want to have the above check
407        // for handling the edge-case of blocks received in the
408        // wrong order. the longest_chain check also requires a
409        // first-block-received check that is conducted against
410        // the blockring.
411        //
412        self.blockring.empty = false;
413
414        // validate
415        //
416        // blockchain validate "validates" the new_chain by unwinding the old
417        // and winding the new, which calling validate on any new previously-
418        // unvalidated blocks. When the longest-chain status of blocks changes
419        // the function on_chain_reorganization is triggered in blocks and
420        // with the BlockRing. We fail if the newly-preferred chain is not
421        // viable.
422        if am_i_the_longest_chain {
423            debug!(
424                "this is the longest chain, adding block : {:?}",
425                block_hash.to_hex()
426            );
427            self.blocks.get_mut(&block_hash).unwrap().in_longest_chain = true;
428
429            // debug!(
430            //     "Full block count before= {:?}",
431            //     self.blocks
432            //         .iter()
433            //         .filter(|(_, block)| matches!(block.block_type, BlockType::Full))
434            //         .count()
435            // );
436
437            let (does_new_chain_validate, wallet_updated) = self
438                .validate(new_chain.as_slice(), old_chain.as_slice(), storage, configs)
439                .await;
440
441            // debug!(
442            //     "Full block count after= {:?} wallet_updated= {:?}",
443            //     self.blocks
444            //         .iter()
445            //         .filter(|(_, block)| matches!(block.block_type, BlockType::Full))
446            //         .count(),
447            //     wallet_updated
448            // );
449
450            if does_new_chain_validate {
451                // crash if total supply has changed
452                self.check_total_supply(configs).await;
453
454                self.add_block_success(block_hash, storage, mempool, configs)
455                    .await;
456
457                AddBlockResult::BlockAddedSuccessfully(block_hash, true, wallet_updated)
458            } else {
459                warn!(
460                    "new chain doesn't validate with hash : {:?}",
461                    block_hash.to_hex()
462                );
463                self.blocks.get_mut(&block_hash).unwrap().in_longest_chain = false;
464                self.add_block_failure(&block_hash, mempool).await;
465                AddBlockResult::FailedNotValid
466            }
467        } else {
468            debug!("this is not the longest chain");
469            self.add_block_success(block_hash, storage, mempool, configs)
470                .await;
471            AddBlockResult::BlockAddedSuccessfully(
472                block_hash,
473                false, /*not in longest_chain*/
474                WALLET_NOT_UPDATED,
475            )
476        }
477    }
478
479    fn calculate_old_chain_for_add_block(
480        &mut self,
481        latest_block_hash: SaitoHash,
482        shared_block_hash: SaitoHash,
483    ) -> Vec<SaitoHash> {
484        let mut old_chain: Vec<[u8; 32]> = Vec::new();
485        let mut old_chain_hash = latest_block_hash;
486
487        while shared_block_hash != old_chain_hash {
488            if self.blocks.contains_key(&old_chain_hash) {
489                old_chain.push(old_chain_hash);
490                old_chain_hash = self
491                    .blocks
492                    .get(&old_chain_hash)
493                    .unwrap()
494                    .previous_block_hash;
495                if old_chain_hash == [0; 32] {
496                    break;
497                }
498            } else {
499                break;
500            }
501        }
502
503        old_chain
504    }
505
506    fn calculate_old_chain_upto_length(
507        &mut self,
508        latest_block_hash: SaitoHash,
509        length: BlockId,
510    ) -> Vec<SaitoHash> {
511        let mut old_chain: Vec<[u8; 32]> = Vec::new();
512        let mut old_chain_hash = latest_block_hash;
513
514        while old_chain.len() <= length as usize {
515            if self.blocks.contains_key(&old_chain_hash) {
516                old_chain.push(old_chain_hash);
517                old_chain_hash = self
518                    .blocks
519                    .get(&old_chain_hash)
520                    .unwrap()
521                    .previous_block_hash;
522                if old_chain_hash == [0; 32] {
523                    break;
524                }
525            } else {
526                break;
527            }
528        }
529
530        old_chain
531    }
532
533    fn calculate_new_chain_for_add_block(
534        &mut self,
535        block_hash: SaitoHash,
536    ) -> (bool, SaitoHash, Vec<SaitoHash>) {
537        let mut new_chain: Vec<SaitoHash> = Vec::new();
538        let mut shared_ancestor_found = false;
539        let mut new_chain_hash = block_hash;
540
541        while !shared_ancestor_found {
542            if let Some(block) = self.blocks.get(&new_chain_hash) {
543                if block.in_longest_chain {
544                    shared_ancestor_found = true;
545                    trace!(
546                        "shared ancestor found : {:?} at id : {:?}",
547                        new_chain_hash.to_hex(),
548                        block.id
549                    );
550                    break;
551                } else if new_chain_hash == [0; 32] {
552                    break;
553                }
554                new_chain.push(new_chain_hash);
555                new_chain_hash = block.previous_block_hash;
556            } else {
557                break;
558            }
559        }
560
561        (shared_ancestor_found, new_chain_hash, new_chain)
562    }
563
564    async fn add_block_success(
565        &mut self,
566        block_hash: SaitoHash,
567        storage: &mut Storage,
568        mempool: &mut Mempool,
569        configs: &(dyn Configuration + Send + Sync),
570    ) {
571        debug!("add_block_success : {:?}", block_hash.to_hex());
572
573        let block_id;
574        let block_type;
575        let tx_count;
576        // save to disk
577        {
578            let block = self.get_block(&block_hash).unwrap();
579            block_id = block.id;
580            block_type = block.block_type;
581            tx_count = block.transactions.len();
582            if block.block_type != BlockType::Header
583                && !configs.is_browser()
584                && !configs.is_spv_mode()
585            {
586                // TODO : this will have an impact when the block sizes are getting large or there are many forks. need to handle this
587                storage.write_block_to_disk(block).await;
588
589                let writing_interval = configs
590                    .get_blockchain_configs()
591                    .issuance_writing_block_interval;
592
593                if writing_interval > 0
594                    && block_id >= self.last_issuance_written_on + writing_interval
595                {
596                    debug!("writing interval : {:?} last issuance written on : {:?}, writing for current block : {}", writing_interval, self.last_issuance_written_on, block_id);
597                    self.write_issuance_file(0, "", storage).await;
598                    self.last_issuance_written_on = block_id;
599                }
600            } else if block.block_type == BlockType::Header {
601                debug!(
602                    "block : {:?} not written to disk as type : {:?}",
603                    block.hash.to_hex(),
604                    block.block_type
605                );
606            }
607
608            if let Some(fork_id) = self.generate_fork_id(block_id) {
609                if fork_id != [0; 32] {
610                    self.set_fork_id(fork_id);
611                }
612            }
613
614            self.set_safe_to_prune_transaction(block_id);
615        }
616
617        // TODO: clean up mempool - I think we shouldn't cleanup mempool here.
618        //  because that's already happening in send_blocks_to_blockchain
619        //  So who is in charge here?
620        //  is send_blocks_to_blockchain calling add_block or
621        //  is blockchain calling mempool.on_chain_reorganization?
622        self.remove_block_transactions(&block_hash, mempool);
623
624        // ensure pruning of next block OK will have the right CVs
625        self.prune_blocks_after_add_block(storage, configs).await;
626        debug!(
627            "block {:?} added successfully. type : {:?} tx count = {:?}",
628            block_hash.to_hex(),
629            block_type,
630            tx_count
631        );
632    }
633
634    pub async fn write_issuance_file(
635        &self,
636        threshold: Currency,
637        issuance_file_path: &str,
638        storage: &mut Storage,
639    ) {
640        info!("utxo size : {:?}", self.utxoset.len());
641
642        let data = self.get_utxoset_data();
643
644        info!("{:?} entries in utxo to write to file", data.len());
645        let latest_block = self.get_latest_block().unwrap();
646        let issuance_path: String;
647
648        if issuance_file_path.is_empty() {
649            issuance_path = format!(
650                "./data/issuance/archive/block_{}_{}_{}.issuance",
651                latest_block.timestamp,
652                latest_block.hash.to_hex(),
653                latest_block.id
654            );
655        } else {
656            issuance_path = issuance_file_path.to_string();
657        }
658
659        info!("opening file : {:?}", issuance_path);
660
661        let mut buffer: Vec<u8> = vec![];
662        let slip_type = "Normal";
663        let mut aggregated_value = 0;
664        let mut total_written_lines = 0;
665        for (key, value) in &data {
666            if value < &threshold {
667                aggregated_value += value;
668            } else {
669                total_written_lines += 1;
670                let key_base58 = key.to_base58();
671
672                let s = format!("{}\t{}\t{}\n", value, key_base58, slip_type);
673                let buf = s.as_bytes();
674                buffer.extend(buf);
675            };
676        }
677
678        // add remaining value
679        if aggregated_value > 0 {
680            total_written_lines += 1;
681            let s = format!(
682                "{}\t{}\t{}\n",
683                aggregated_value,
684                PROJECT_PUBLIC_KEY.to_string(),
685                slip_type
686            );
687            let buf = s.as_bytes();
688            buffer.extend(buf);
689        }
690
691        storage
692            .io_interface
693            .ensure_block_directory_exists("./data/issuance/archive");
694
695        storage
696            .io_interface
697            .write_value(issuance_path.as_str(), buffer.as_slice())
698            .await
699            .expect("issuance file should be written");
700
701        info!("total written lines : {:?}", total_written_lines);
702    }
703
704    fn remove_block_transactions(&self, block_hash: &SaitoHash, mempool: &mut Mempool) {
705        mempool
706            .transactions
707            .retain(|_, tx| tx.validate_against_utxoset(&self.utxoset));
708        let block = self.get_block(block_hash).unwrap();
709        // we call delete_tx after removing invalidated txs, to make sure routing work is calculated after removing all the txs
710        mempool.delete_transactions(&block.transactions);
711    }
712
713    async fn prune_blocks_after_add_block(
714        &mut self,
715        storage: &mut Storage,
716        configs: &(dyn Configuration + Send + Sync),
717    ) {
718        if self.get_latest_block_id() > configs.get_consensus_config().unwrap().genesis_period {
719            if let Some(pruned_block_hash) =
720                self.blockring.get_longest_chain_block_hash_at_block_id(
721                    self.get_latest_block_id()
722                        - configs.get_consensus_config().unwrap().genesis_period,
723                )
724            {
725                let block = self.get_mut_block(&pruned_block_hash).unwrap();
726
727                block
728                    .upgrade_block_to_block_type(BlockType::Pruned, storage, configs.is_spv_mode())
729                    .await;
730            }
731        }
732    }
733
734    async fn add_block_failure(&mut self, block_hash: &SaitoHash, mempool: &mut Mempool) {
735        info!("add block failed : {:?}", block_hash.to_hex());
736
737        mempool.delete_block(block_hash);
738        let block = self.blocks.remove(block_hash);
739
740        if block.is_none() {
741            error!(
742                "block : {:?} is not found in blocks collection. couldn't handle block failure.",
743                block_hash.to_hex()
744            );
745            return;
746        }
747
748        let mut block = block.unwrap();
749        self.blockring.delete_block(block.id, block.hash);
750        self.add_block_transactions_back(mempool, &mut block).await;
751    }
752
753    async fn add_block_transactions_back(&mut self, mempool: &mut Mempool, block: &mut Block) {
754        let wallet = mempool.wallet_lock.read().await;
755        let public_key = wallet.public_key;
756        if block.creator == public_key {
757            let transactions = &mut block.transactions;
758            let prev_count = transactions.len();
759
760            let transactions: Vec<Transaction> = drain!(transactions, 10)
761                .filter(|tx| {
762                    // TODO : what other types should be added back to the mempool
763                    if tx.transaction_type == TransactionType::Normal {
764                        // TODO : is there a way to not validate these again ?
765                        return tx.validate(&self.utxoset, self, true);
766                    }
767                    false
768                })
769                .collect();
770            // transactions.retain(|tx| tx.validate(&self.utxoset));
771            info!(
772                "adding {:?} transactions back to mempool. dropped {:?} invalid transactions",
773                transactions.len(),
774                (prev_count - transactions.len())
775            );
776            for tx in transactions {
777                mempool.transactions.insert(tx.signature, tx);
778            }
779            mempool.new_tx_added = true;
780        }
781    }
782
783    pub fn generate_fork_id(&self, block_id: u64) -> Option<ForkId> {
784        let mut fork_id: ForkId = [0; 32];
785        let mut current_block_id = block_id;
786
787        // roll back to last even 10 blocks
788        current_block_id = current_block_id - (current_block_id % 10);
789        trace!(
790            "generate_fork_id : {:?} -> {:?}",
791            block_id,
792            current_block_id
793        );
794
795        // loop backwards through blockchain
796        for (i, weight) in FORK_ID_WEIGHTS.iter().enumerate() {
797            if current_block_id <= *weight {
798                debug!(
799                    "generating fork id for block : {:?}. current_id : {:?} is less than weight : {:?}",
800                    block_id,
801                    current_block_id, weight
802                );
803                break;
804            }
805            current_block_id -= weight;
806
807            // index to update
808
809            let index = 2 * i;
810            if let Some(block_hash) = self
811                .blockring
812                .get_longest_chain_block_hash_at_block_id(current_block_id)
813            {
814                fork_id[index] = block_hash[index];
815                fork_id[index + 1] = block_hash[index + 1];
816            } else {
817                debug!(
818                    "no block at block id : {:?} in the longest chain",
819                    current_block_id
820                );
821                break;
822                // return None;
823            }
824        }
825
826        Some(fork_id)
827    }
828
829    pub fn generate_last_shared_ancestor(
830        &self,
831        peer_latest_block_id: u64,
832        fork_id: SaitoHash,
833    ) -> u64 {
834        let my_latest_block_id = self.get_latest_block_id();
835
836        debug!(
837            "generate last shared ancestor : peer_latest_id : {:?}, fork_id : {:?} my_latest_id : {:?}",
838            peer_latest_block_id,
839            fork_id.to_hex(),
840            my_latest_block_id
841        );
842
843        if peer_latest_block_id >= my_latest_block_id {
844            if let Some(value) = self.generate_last_shared_ancestor_when_peer_ahead(
845                peer_latest_block_id,
846                fork_id,
847                my_latest_block_id,
848            ) {
849                return value;
850            }
851        } else if let Some(value) = self.generate_last_shared_ancestor_when_peer_behind(
852            peer_latest_block_id,
853            fork_id,
854            my_latest_block_id,
855        ) {
856            return value;
857        }
858
859        debug!("no shared ancestor found. returning 0");
860        // no match? return 0 -- no shared ancestor
861        0
862    }
863
864    fn generate_last_shared_ancestor_when_peer_behind(
865        &self,
866        peer_latest_block_id: u64,
867        fork_id: SaitoHash,
868        my_latest_block_id: u64,
869    ) -> Option<u64> {
870        let mut block_id = peer_latest_block_id;
871        block_id = block_id - (block_id % 10);
872
873        debug!(
874            "generate_last_shared_ancestor_when_peer_behind peer_block_id : {:?}, my_block_id : {:?}",
875            peer_latest_block_id,
876            my_latest_block_id
877        );
878        for (index, weight) in FORK_ID_WEIGHTS.iter().enumerate() {
879            if block_id < *weight {
880                trace!(
881                    "my_block_id : {:?} is less than weight : {:?}. returning 0",
882                    block_id,
883                    weight
884                );
885                return Some(0);
886            }
887            block_id -= weight;
888            trace!(
889                "block_id : {:?} , weight : {:?} , index : {:?}",
890                block_id,
891                weight,
892                index
893            );
894
895            // index in fork_id hash
896            let index = 2 * index;
897
898            // compare input hash to my hash
899            if let Some(block_hash) = self
900                .blockring
901                .get_longest_chain_block_hash_at_block_id(block_id)
902            {
903                trace!(
904                    "comparing {:?} vs {:?} at block_id : {}",
905                    hex::encode(&fork_id[index..=index + 1]),
906                    hex::encode(&block_hash[index..=index + 1]),
907                    block_id
908                );
909                if fork_id[index] == block_hash[index]
910                    && fork_id[index + 1] == block_hash[index + 1]
911                {
912                    return Some(block_id);
913                }
914            } else {
915                trace!("cannot find longest chain hash for : {:?}", block_id);
916            }
917        }
918        None
919    }
920
921    fn generate_last_shared_ancestor_when_peer_ahead(
922        &self,
923        peer_latest_block_id: u64,
924        fork_id: SaitoHash,
925        my_latest_block_id: u64,
926    ) -> Option<u64> {
927        let mut block_id = my_latest_block_id;
928        // roll back to last even 10 blocks
929        block_id = block_id - (block_id % 10);
930        debug!(
931            "generate_last_shared_ancestor_when_peer_ahead peer_block_id : {:?}, my_block_id : {:?}",
932            peer_latest_block_id,
933            my_latest_block_id
934        );
935
936        // their fork id
937        for (index, weight) in FORK_ID_WEIGHTS.iter().enumerate() {
938            if block_id < *weight {
939                trace!(
940                    "peer_block_id : {:?} is less than weight : {:?}. returning 0",
941                    block_id,
942                    weight
943                );
944                return Some(0);
945            }
946            block_id -= weight;
947            trace!(
948                "block_id : {:?} , weight : {:?} , index : {:?}",
949                block_id,
950                weight,
951                index
952            );
953
954            // index in fork_id hash
955            let index = 2 * index;
956
957            // compare input hash to my hash
958            if let Some(block_hash) = self
959                .blockring
960                .get_longest_chain_block_hash_at_block_id(block_id)
961            {
962                trace!(
963                    "comparing {:?} vs {:?} at block_id : {}",
964                    hex::encode(&fork_id[index..=index + 1]),
965                    hex::encode(&block_hash[index..=index + 1]),
966                    block_id
967                );
968                if fork_id[index] == block_hash[index]
969                    && fork_id[index + 1] == block_hash[index + 1]
970                {
971                    return Some(block_id);
972                }
973            } else {
974                trace!("cannot find longest chain hash for : {:?}", block_id);
975            }
976        }
977        None
978    }
979    fn print(&self, count: u64, configs: &(dyn Configuration + Send + Sync)) {
980        let latest_block_id = self.get_latest_block_id();
981        let mut current_id = latest_block_id;
982
983        let mut min_id = 0;
984        if latest_block_id > count {
985            min_id = latest_block_id - count;
986        }
987        debug!("------------------------------------------------------");
988        while current_id > 0 && current_id >= min_id {
989            if current_id < (2 * configs.get_consensus_config().unwrap().genesis_period) {
990                break;
991            }
992
993            if let Some(hash) = self
994                .blockring
995                .get_longest_chain_block_hash_at_block_id(current_id)
996            {
997                if hash == [0; 32] {
998                    break;
999                }
1000                debug!("{} - {:?}", current_id, hash.to_hex());
1001                current_id -= 1;
1002            } else {
1003                break;
1004            }
1005        }
1006        debug!("------------------------------------------------------");
1007    }
1008
1009    pub fn get_latest_block(&self) -> Option<&Block> {
1010        let block_hash = self.blockring.get_latest_block_hash();
1011        self.blocks.get(&block_hash)
1012    }
1013
1014    pub fn get_latest_block_hash(&self) -> SaitoHash {
1015        self.blockring.get_latest_block_hash()
1016    }
1017
1018    pub fn get_latest_block_id(&self) -> BlockId {
1019        self.blockring.get_latest_block_id()
1020    }
1021
1022    pub fn get_latest_unlocked_stake_block_id(&self) -> BlockId {
1023        // if we have any time to unlock slips
1024        if self.get_latest_block_id() > self.social_stake_period {
1025            // we check for next block's id
1026            self.get_latest_block_id() + 1 - self.social_stake_period
1027        } else {
1028            0
1029        }
1030    }
1031
1032    pub fn get_block_sync(&self, block_hash: &SaitoHash) -> Option<&Block> {
1033        self.blocks.get(block_hash)
1034    }
1035
1036    pub fn get_block(&self, block_hash: &SaitoHash) -> Option<&Block> {
1037        self.blocks.get(block_hash)
1038    }
1039
1040    pub fn get_mut_block(&mut self, block_hash: &SaitoHash) -> Option<&mut Block> {
1041        self.blocks.get_mut(block_hash)
1042    }
1043
1044    pub fn is_block_indexed(&self, block_hash: SaitoHash) -> bool {
1045        self.blocks.contains_key(&block_hash)
1046    }
1047
1048    pub fn contains_block_hash_at_block_id(&self, block_id: u64, block_hash: SaitoHash) -> bool {
1049        self.blockring
1050            .contains_block_hash_at_block_id(block_id, block_hash)
1051    }
1052
1053    fn is_new_chain_the_longest_chain(
1054        &self,
1055        new_chain: &[SaitoHash],
1056        old_chain: &[SaitoHash],
1057    ) -> bool {
1058        trace!("checking for longest chain");
1059        if self.blockring.is_empty() {
1060            return true;
1061        }
1062        if old_chain.len() > new_chain.len() {
1063            warn!(
1064                "WARN: old chain length : {:?} is greater than new chain length : {:?}",
1065                old_chain.len(),
1066                new_chain.len()
1067            );
1068            return false;
1069        }
1070
1071        if self.blockring.get_latest_block_id() >= self.blocks.get(&new_chain[0]).unwrap().id {
1072            debug!(
1073                "blockring latest : {:?} >= new chain block id : {:?}",
1074                self.blockring.get_latest_block_id(),
1075                self.blocks.get(&new_chain[0]).unwrap().id
1076            );
1077            return false;
1078        }
1079
1080        let mut old_bf: Currency = 0;
1081        let mut new_bf: Currency = 0;
1082
1083        for hash in old_chain.iter() {
1084            old_bf += self.blocks.get(hash).unwrap().burnfee;
1085        }
1086        for hash in new_chain.iter() {
1087            if let Some(x) = self.blocks.get(hash) {
1088                new_bf += x.burnfee;
1089            } else {
1090                trace!(
1091                    "block : {:?} in the new chain cannot be found",
1092                    hash.to_hex()
1093                );
1094                return false;
1095            }
1096            //new_bf += self.blocks.get(hash).unwrap().get_burnfee();
1097        }
1098        trace!(
1099            "old chain len : {:?} new chain len : {:?} old_bf : {:?} new_bf : {:?}",
1100            old_chain.len(),
1101            new_chain.len(),
1102            old_bf,
1103            new_bf
1104        );
1105
1106        // new chain must have more accumulated work AND be longer
1107        old_chain.len() < new_chain.len() && old_bf <= new_bf
1108    }
1109
1110    /// when new_chain and old_chain are generated the block_hashes are added
1111    /// to their vectors from tip-to-shared-ancestors. if the shared ancestors
1112    /// is at position [0] in our blockchain for instance, we may receive:
1113    ///
1114    /// new_chain --> adds the hashes in this order
1115    /// [5] [4] [3] [2] [1]
1116    ///
1117    /// old_chain --> adds the hashes in this order
1118    /// [4] [3] [2] [1]
1119    ///
1120    /// unwinding requires starting from the BEGINNING of the vector, while
1121    /// winding requires starting from th END of the vector. the loops move
1122    /// in opposite directions.
1123    async fn validate(
1124        &mut self,
1125        new_chain: &[SaitoHash],
1126        old_chain: &[SaitoHash],
1127        storage: &Storage,
1128        configs: &(dyn Configuration + Send + Sync),
1129    ) -> (bool, WalletUpdateStatus) {
1130        debug!(
1131            "validating chains. latest : {:?} new_chain_len : {:?} old_chain_len : {:?}",
1132            self.get_latest_block_id(),
1133            new_chain.len(),
1134            old_chain.len()
1135        );
1136
1137        let previous_block_hash;
1138        let mut wallet_update_status = WALLET_NOT_UPDATED;
1139        let has_gt;
1140        {
1141            let block = self.blocks.get(new_chain[0].as_ref()).unwrap();
1142            previous_block_hash = block.previous_block_hash;
1143            has_gt = block.has_golden_ticket;
1144        }
1145
1146        // ensure new chain has adequate mining support to be considered as
1147        // a viable chain. we handle this check here as opposed to handling
1148        // it in wind_chain as we only need to check once for the entire chain
1149        if !self.is_golden_ticket_count_valid(
1150            previous_block_hash,
1151            has_gt,
1152            configs.is_browser(),
1153            configs.is_spv_mode(),
1154        ) {
1155            debug!("gt count is not valid");
1156            return (false, WALLET_NOT_UPDATED);
1157        }
1158
1159        if old_chain.is_empty() {
1160            let mut result: WindingResult =
1161                WindingResult::Wind(new_chain.len() - 1, false, WALLET_NOT_UPDATED);
1162            loop {
1163                match result {
1164                    WindingResult::Wind(current_wind_index, wind_failure, wallet_status) => {
1165                        wallet_update_status |= wallet_status;
1166
1167                        result = self
1168                            .wind_chain(
1169                                new_chain,
1170                                old_chain,
1171                                current_wind_index,
1172                                wind_failure,
1173                                storage,
1174                                configs,
1175                            )
1176                            .await;
1177                    }
1178                    WindingResult::Unwind(
1179                        current_unwind_index,
1180                        wind_failure,
1181                        old_chain,
1182                        wallet_status,
1183                    ) => {
1184                        wallet_update_status |= wallet_status;
1185                        result = self
1186                            .unwind_chain(
1187                                new_chain,
1188                                old_chain.as_slice(),
1189                                current_unwind_index,
1190                                wind_failure,
1191                                storage,
1192                                configs,
1193                            )
1194                            .await;
1195                    }
1196                    WindingResult::FinishWithSuccess(wallet_updated) => {
1197                        return (true, wallet_update_status | wallet_updated)
1198                    }
1199                    WindingResult::FinishWithFailure => return (false, wallet_update_status),
1200                }
1201            }
1202        } else if !new_chain.is_empty() {
1203            let mut result = WindingResult::Unwind(0, true, old_chain.to_vec(), WALLET_NOT_UPDATED);
1204            loop {
1205                match result {
1206                    WindingResult::Wind(current_wind_index, wind_failure, wallet_status) => {
1207                        wallet_update_status |= wallet_status;
1208                        result = self
1209                            .wind_chain(
1210                                new_chain,
1211                                old_chain,
1212                                current_wind_index,
1213                                wind_failure,
1214                                storage,
1215                                configs,
1216                            )
1217                            .await;
1218                    }
1219                    WindingResult::Unwind(
1220                        current_wind_index,
1221                        wind_failure,
1222                        old_chain,
1223                        wallet_status,
1224                    ) => {
1225                        wallet_update_status |= wallet_status;
1226                        result = self
1227                            .unwind_chain(
1228                                new_chain,
1229                                old_chain.as_slice(),
1230                                current_wind_index,
1231                                wind_failure,
1232                                storage,
1233                                configs,
1234                            )
1235                            .await;
1236                    }
1237                    WindingResult::FinishWithSuccess(wallet_updated) => {
1238                        return (true, wallet_update_status | wallet_updated);
1239                    }
1240                    WindingResult::FinishWithFailure => {
1241                        return (false, wallet_update_status);
1242                    }
1243                }
1244            }
1245        } else {
1246            warn!("lengths are inappropriate");
1247            (false, wallet_update_status)
1248        }
1249    }
1250
1251    pub fn is_golden_ticket_count_valid(
1252        &self,
1253        previous_block_hash: SaitoHash,
1254        current_block_has_golden_ticket: bool,
1255        is_browser: bool,
1256        is_spv: bool,
1257    ) -> bool {
1258        is_golden_ticket_count_valid_(
1259            previous_block_hash,
1260            current_block_has_golden_ticket,
1261            is_browser || is_spv,
1262            |hash| self.get_block_sync(&hash),
1263        )
1264    }
1265
1266    // when new_chain and old_chain are generated the block_hashes are added
1267    // to their vectors from tip-to-shared-ancestors. if the shared ancestors
1268    // is at position [0] for instance, we may receive:
1269    //
1270    // new_chain --> adds the hashes in this order
1271    //   [5] [4] [3] [2] [1]
1272    //
1273    // old_chain --> adds the hashes in this order
1274    //   [4] [3] [2] [1]
1275    //
1276    // unwinding requires starting from the BEGINNING of the vector, while
1277    // winding requires starting from the END of the vector. the loops move
1278    // in opposite directions. the argument current_wind_index is the
1279    // position in the vector NOT the ordinal number of the block_hash
1280    // being processed. we start winding with current_wind_index 4 not 0.
1281    async fn wind_chain(
1282        &mut self,
1283        new_chain: &[SaitoHash],
1284        old_chain: &[SaitoHash],
1285        current_wind_index: usize,
1286        wind_failure: bool,
1287        storage: &Storage,
1288        configs: &(dyn Configuration + Send + Sync),
1289    ) -> WindingResult {
1290        // trace!(" ... blockchain.wind_chain strt: {:?}", create_timestamp());
1291
1292        debug!(
1293            "wind_chain: current_wind_index : {:?} new_chain_len: {:?} old_chain_len: {:?} failed : {:?}",
1294            current_wind_index,new_chain.len(),old_chain.len(), wind_failure
1295        );
1296
1297        // if we are winding a non-existent chain with a wind_failure it
1298        // means our wind attempt failed, and we should move directly into
1299        // add_block_failure() by returning false.
1300        if wind_failure && new_chain.is_empty() {
1301            return WindingResult::FinishWithFailure;
1302        }
1303
1304        // winding the chain requires us to have certain data associated
1305        // with the block and the transactions, particularly the tx hashes
1306        // that we need to generate the slip UUIDs and create the tx sigs.
1307        //
1308        // we fetch the block mutably first in order to update these vars.
1309        // we cannot just send the block mutably into our regular validate()
1310        // function because of limitatins imposed by Rust on mutable data
1311        // structures. So validation is "read-only" and our "write" actions
1312        // happen first.
1313        let block_hash = new_chain.get(current_wind_index).unwrap();
1314
1315        debug!("winding hash: {:?}", block_hash.to_hex());
1316        self.upgrade_blocks_for_wind_chain(storage, configs, block_hash)
1317            .await;
1318
1319        let block = self.blocks.get(block_hash).unwrap();
1320        if block.has_checkpoint {
1321            info!("block has checkpoint. cannot wind over this block");
1322            return WindingResult::FinishWithFailure;
1323        }
1324        let does_block_validate;
1325        {
1326            debug!("winding hash validates: {:?}", block_hash.to_hex());
1327            let genesis_period = configs.get_consensus_config().unwrap().genesis_period;
1328            let validate_against_utxo = self.has_total_supply_loaded(genesis_period);
1329
1330            does_block_validate = block
1331                .validate(self, &self.utxoset, configs, storage, validate_against_utxo)
1332                .await;
1333
1334            if !does_block_validate {
1335                debug!("latest_block_id = {:?}", self.get_latest_block_id());
1336                debug!("genesis_block_id = {:?}", self.genesis_block_id);
1337                debug!(
1338                    "genesis_period = {:?}",
1339                    configs.get_consensus_config().unwrap().genesis_period
1340                );
1341            }
1342        }
1343
1344        let mut wallet_updated = WALLET_NOT_UPDATED;
1345
1346        if does_block_validate {
1347            // blockring update
1348            self.blockring
1349                .on_chain_reorganization(block.id, block.hash, true);
1350
1351            // TODO - wallet update should be optional, as core routing nodes
1352            //  will not want to do the work of scrolling through the block and
1353            //  updating their wallets by default. wallet processing can be
1354            //  more efficiently handled by lite-nodes.
1355            {
1356                let mut wallet = self.wallet_lock.write().await;
1357
1358                wallet_updated |= wallet.on_chain_reorganization(
1359                    block,
1360                    true,
1361                    configs.get_consensus_config().unwrap().genesis_period,
1362                );
1363            }
1364            let block_id = block.id;
1365
1366            // utxoset update
1367            {
1368                let block = self.blocks.get_mut(block_hash).unwrap();
1369                block.on_chain_reorganization(&mut self.utxoset, true);
1370            }
1371
1372            wallet_updated |= self
1373                .on_chain_reorganization(block_id, *block_hash, true, storage, configs)
1374                .await;
1375
1376            // we have received the first entry in new_blocks() which means we
1377            // have added the latest tip. if the variable wind_failure is set
1378            // that indicates that we ran into an issue when winding the new_chain
1379            // and what we have just processed is the old_chain (being rewound)
1380            // so we should exit with failure.
1381            //
1382            // otherwise we have successfully wound the new chain, and exit with
1383            // success.
1384            if current_wind_index == 0 {
1385                if wind_failure {
1386                    return WindingResult::FinishWithFailure;
1387                }
1388                return WindingResult::FinishWithSuccess(wallet_updated);
1389            }
1390
1391            WindingResult::Wind(current_wind_index - 1, false, wallet_updated)
1392        } else {
1393            // we have had an error while winding the chain. this requires us to
1394            // unwind any blocks we have already wound, and rewind any blocks we
1395            // have unwound.
1396            //
1397            // we set wind_failure to "true" so that when we reach the end of
1398            // the process of rewinding the old-chain, our wind_chain function
1399            // will know it has rewound the old chain successfully instead of
1400            // successfully added the new chain.
1401            error!(
1402                "ERROR: this block : {:?} : {:?} does not validate!",
1403                block.id,
1404                block.hash.to_hex()
1405            );
1406            if current_wind_index == new_chain.len() - 1 {
1407                // this is the first block we have tried to add
1408                // and so we can just roll out the older chain
1409                // again as it is known good.
1410                //
1411                // note that old and new hashes are swapped
1412                // and the old chain is set as null because
1413                // we won't move back to it. we also set the
1414                // resetting_flag to 1 so we know to fork
1415                // into addBlockToBlockchainFailure
1416                //
1417                // true -> force -> we had issues, is failure
1418                //
1419                // new_chain --> hashes are still in this order
1420                //   [5] [4] [3] [2] [1]
1421                //
1422                // we are at the beginning of our own vector so we have nothing
1423                // to unwind. Because of this, we start WINDING the old chain back
1424                // which requires us to start at the END of the new chain vector.
1425                if !old_chain.is_empty() {
1426                    debug!("old chain len: {}", old_chain.len());
1427                    WindingResult::Wind(old_chain.len() - 1, true, wallet_updated)
1428                } else {
1429                    debug!("old chain is empty. finishing with failure");
1430                    WindingResult::FinishWithFailure
1431                }
1432            } else {
1433                let mut chain_to_unwind: Vec<SaitoHash> = vec![];
1434
1435                // if we run into a problem winding our chain after we have
1436                // wound any blocks, we take the subset of the blocks we have
1437                // already pushed through on_chain_reorganization (i.e. not
1438                // including this block!) and put them onto a new vector we
1439                // will unwind in turn.
1440                for i in current_wind_index + 1..new_chain.len() {
1441                    chain_to_unwind.push(new_chain[i]);
1442                }
1443
1444                // chain to unwind is now something like this...
1445                //
1446                //  [3] [2] [1]
1447                //
1448                // unwinding starts from the BEGINNING of the vector
1449                WindingResult::Unwind(0, true, chain_to_unwind, wallet_updated)
1450            }
1451        }
1452    }
1453
1454    ///  ensure previous blocks that may be needed to calculate the staking
1455    ///  tables or the nolan that are potentially falling off the chain have
1456    ///  full access to their transaction data.
1457    ///
1458    /// # Arguments
1459    ///
1460    /// * `storage`:
1461    /// * `configs`:
1462    /// * `block_hash`:
1463    ///
1464    /// returns: ()
1465    ///
1466    /// # Examples
1467    ///
1468    /// ```
1469    ///
1470    /// ```
1471    async fn upgrade_blocks_for_wind_chain(
1472        &mut self,
1473        storage: &Storage,
1474        configs: &(dyn Configuration + Send + Sync),
1475        block_hash: &SaitoHash,
1476    ) {
1477        debug!(
1478            "upgrading blocks for wind chain... : {:?}",
1479            block_hash.to_hex()
1480        );
1481        let block = self.get_mut_block(block_hash).unwrap();
1482
1483        block
1484            .upgrade_block_to_block_type(BlockType::Full, storage, configs.is_spv_mode())
1485            .await;
1486
1487        let latest_block_id: BlockId = block.id;
1488        for i in 1..configs
1489            .get_consensus_config()
1490            .unwrap()
1491            .max_staker_recursions
1492        {
1493            if i >= latest_block_id {
1494                break;
1495            }
1496            let bid = latest_block_id - i;
1497            if let Some(previous_block_hash) =
1498                self.blockring.get_longest_chain_block_hash_at_block_id(bid)
1499            {
1500                if self.is_block_indexed(previous_block_hash) {
1501                    let block = self.get_mut_block(&previous_block_hash).unwrap();
1502                    block
1503                        .upgrade_block_to_block_type(
1504                            BlockType::Full,
1505                            storage,
1506                            configs.is_spv_mode(),
1507                        )
1508                        .await;
1509                }
1510            }
1511        }
1512    }
1513
1514    fn has_total_supply_loaded(&self, genesis_period: BlockId) -> bool {
1515        let has_genesis_block = self
1516            .blockring
1517            .get_longest_chain_block_hash_at_block_id(1)
1518            .is_some();
1519        let latest_block_id = self.get_latest_block_id();
1520        let mut has_genesis_period_of_blocks = false;
1521        if latest_block_id > genesis_period {
1522            let result = self
1523                .blockring
1524                .get_longest_chain_block_hash_at_block_id(latest_block_id - genesis_period);
1525            has_genesis_period_of_blocks = result.is_some();
1526        }
1527        has_genesis_block || has_genesis_period_of_blocks
1528    }
1529
1530    // when new_chain and old_chain are generated the block_hashes are pushed
1531    // to their vectors from tip-to-shared-ancestors. if the shared ancestors
1532    // is at position [0] for instance, we may receive:
1533    //
1534    // new_chain --> adds the hashes in this order
1535    //   [5] [4] [3] [2] [1]
1536    //
1537    // old_chain --> adds the hashes in this order
1538    //   [4] [3] [2] [1]
1539    //
1540    // unwinding requires starting from the BEGINNING of the vector, while
1541    // winding requires starting from the END of the vector. the first
1542    // block we have to remove in the old_chain is thus at position 0, and
1543    // walking up the vector from there until we reach the end.
1544    //
1545    async fn unwind_chain(
1546        &mut self,
1547        new_chain: &[SaitoHash],
1548        old_chain: &[SaitoHash],
1549        current_unwind_index: usize,
1550        wind_failure: bool,
1551        storage: &Storage,
1552        configs: &(dyn Configuration + Send + Sync),
1553    ) -> WindingResult {
1554        debug!(
1555            "unwind_chain: current_wind_index : {:?} new_chain_len: {:?} old_chain_len: {:?} failed : {:?}",
1556            current_unwind_index,new_chain.len(),old_chain.len(), wind_failure
1557        );
1558        let block_id;
1559        let block_hash;
1560        let mut wallet_updated = WALLET_NOT_UPDATED;
1561        {
1562            let block = self
1563                .blocks
1564                .get_mut(&old_chain[current_unwind_index])
1565                .unwrap();
1566            if block.has_checkpoint {
1567                info!("block has checkpoint. cannot unwind over this block");
1568                return WindingResult::FinishWithFailure;
1569            }
1570            block
1571                .upgrade_block_to_block_type(BlockType::Full, storage, configs.is_spv_mode())
1572                .await;
1573            block_id = block.id;
1574            block_hash = block.hash;
1575
1576            debug!(
1577                "unwinding hash: {:?} w/id {:?}",
1578                block_hash.to_hex(),
1579                block_id
1580            );
1581            // utxoset update
1582            block.on_chain_reorganization(&mut self.utxoset, false);
1583
1584            // blockring update
1585            self.blockring
1586                .on_chain_reorganization(block.id, block.hash, false);
1587
1588            // wallet update
1589            let mut wallet = self.wallet_lock.write().await;
1590            wallet_updated |= wallet.on_chain_reorganization(
1591                block,
1592                false,
1593                configs.get_consensus_config().unwrap().genesis_period,
1594            );
1595        }
1596        wallet_updated |= self
1597            .on_chain_reorganization(block_id, block_hash, false, storage, configs)
1598            .await;
1599        if current_unwind_index == old_chain.len() - 1 {
1600            // start winding new chain
1601            //
1602            // new_chain --> adds the hashes in this order
1603            //   [5] [4] [3] [2] [1]
1604            //
1605            // old_chain --> adds the hashes in this order
1606            //   [4] [3] [2] [1]
1607            //
1608            // winding requires starting at the END of the vector and rolling
1609            // backwards until we have added block #5, etc.
1610            WindingResult::Wind(new_chain.len() - 1, wind_failure, wallet_updated)
1611        } else {
1612            // continue unwinding,, which means
1613            //
1614            // unwinding requires moving FORWARD in our vector (and backwards in
1615            // the blockchain). So we increment our unwind index.
1616            WindingResult::Unwind(
1617                current_unwind_index + 1,
1618                wind_failure,
1619                old_chain.to_vec(),
1620                wallet_updated,
1621            )
1622        }
1623    }
1624
1625    /// keeps any blockchain variables like fork_id or genesis_period
1626    /// tracking variables updated as the chain gets new blocks. also
1627    /// pre-loads any blocks needed to improve performance.
1628    pub async fn on_chain_reorganization(
1629        &mut self,
1630        block_id: u64,
1631        block_hash: SaitoHash,
1632        longest_chain: bool,
1633        storage: &Storage,
1634        configs: &(dyn Configuration + Send + Sync),
1635    ) -> WalletUpdateStatus {
1636        debug!(
1637            "blockchain.on_chain_reorganization : block_id = {:?} block_hash = {:?}",
1638            block_id,
1639            block_hash.to_hex()
1640        );
1641
1642        let mut wallet_updated: WalletUpdateStatus = WALLET_NOT_UPDATED;
1643        // skip out if earlier than we need to be vis-à-vis last_block_id
1644        if self.last_block_id >= block_id {
1645            debug!(
1646                "last block id : {:?} is later than this block id : {:?}. skipping reorg",
1647                self.last_block_id, block_id
1648            );
1649            self.downgrade_blockchain_data(configs).await;
1650            return true;
1651        }
1652
1653        if longest_chain {
1654            let block = self.blocks.get(&block_hash);
1655            if let Some(block) = block {
1656                self.last_block_id = block_id;
1657                self.last_block_hash = block.hash;
1658                self.last_timestamp = block.timestamp;
1659                self.last_burnfee = block.burnfee;
1660
1661                if self.lowest_acceptable_timestamp == 0 {
1662                    self.lowest_acceptable_block_id = block_id;
1663                    self.lowest_acceptable_block_hash = block.hash;
1664                    self.lowest_acceptable_timestamp = block.timestamp;
1665                }
1666            } else {
1667                warn!("block not found for hash : {:?}", block_hash.to_hex());
1668            }
1669
1670            // update genesis period, purge old data
1671            wallet_updated |= self.update_genesis_period(storage, configs).await;
1672
1673            // generate fork_id
1674            if let Some(fork_id) = self.generate_fork_id(block_id) {
1675                if fork_id != [0; 32] {
1676                    self.set_fork_id(fork_id);
1677                }
1678            } else {
1679                debug!(
1680                    "cannot set fork id as fork id cannot be generated for block id : {:?}",
1681                    block_id
1682                );
1683            }
1684        }
1685
1686        self.downgrade_blockchain_data(configs).await;
1687
1688        wallet_updated
1689    }
1690
1691    async fn update_genesis_period(
1692        &mut self,
1693        storage: &Storage,
1694        configs: &(dyn Configuration + Send + Sync),
1695    ) -> WalletUpdateStatus {
1696        // we need to make sure this is not a random block that is disconnected
1697        // from our previous genesis_id. If there is no connection between it
1698        // and us, then we cannot delete anything as otherwise the provision of
1699        // the block may be an attack on us intended to force us to discard
1700        // actually useful data.
1701        //
1702        // so we check that our block is the head of the longest-chain and only
1703        // update the genesis period when that is the case.
1704        let latest_block_id = self.get_latest_block_id();
1705        let block_limit = configs.get_consensus_config().unwrap().genesis_period * 2 + 1;
1706        debug!(
1707            "latest block id : {:?} block limit : {:?}. upgrading genesis_period. : {:?}",
1708            latest_block_id,
1709            block_limit,
1710            latest_block_id >= block_limit
1711        );
1712        if latest_block_id >= block_limit {
1713            // prune blocks
1714            let purge_bid =
1715                latest_block_id - (configs.get_consensus_config().unwrap().genesis_period * 2);
1716            self.genesis_block_id =
1717                latest_block_id - configs.get_consensus_config().unwrap().genesis_period;
1718            debug!("genesis block id set as : {:?}", self.genesis_block_id);
1719
1720            // in either case, we are OK to throw out everything below the
1721            // lowest_block_id that we have found. we use the purge_id to
1722            // handle purges.
1723            if purge_bid > 0 {
1724                return self.delete_blocks(purge_bid, storage).await;
1725            }
1726        }
1727
1728        WALLET_NOT_UPDATED
1729        //TODO: we already had in update_genesis_period() in self method - maybe no need to call here?
1730        // self.downgrade_blockchain_data().await;
1731    }
1732
1733    /// deletes all blocks at a single block_id
1734    async fn delete_blocks(
1735        &mut self,
1736        delete_block_id: u64,
1737        storage: &Storage,
1738    ) -> WalletUpdateStatus {
1739        info!("removing blocks from disk at id {}", delete_block_id);
1740
1741        let mut block_hashes_copy: Vec<SaitoHash> = vec![];
1742
1743        {
1744            let block_hashes = self.blockring.get_block_hashes_at_block_id(delete_block_id);
1745            for hash in block_hashes {
1746                block_hashes_copy.push(hash);
1747            }
1748        }
1749
1750        trace!("number of hashes to remove {}", block_hashes_copy.len());
1751
1752        let mut wallet_update_status = WALLET_NOT_UPDATED;
1753        for hash in block_hashes_copy {
1754            let status = self.delete_block(delete_block_id, hash, storage).await;
1755            wallet_update_status |= status;
1756        }
1757        wallet_update_status
1758    }
1759
1760    /// deletes a single block
1761    async fn delete_block(
1762        &mut self,
1763        delete_block_id: u64,
1764        delete_block_hash: SaitoHash,
1765        storage: &Storage,
1766    ) -> WalletUpdateStatus {
1767        let wallet_update_status;
1768        // ask block to delete itself / utxo-wise
1769        {
1770            let block = self.blocks.get(&delete_block_hash).unwrap();
1771            let block_filename = storage.generate_block_filepath(block);
1772
1773            // remove slips from wallet
1774            {
1775                let mut wallet = self.wallet_lock.write().await;
1776
1777                wallet_update_status = wallet.delete_block(block);
1778            }
1779            // removes utxoset data
1780            block.delete(&mut self.utxoset).await;
1781
1782            // deletes block from disk
1783            storage
1784                .delete_block_from_disk(block_filename.as_str())
1785                .await;
1786        }
1787
1788        // ask blockring to remove
1789        self.blockring
1790            .delete_block(delete_block_id, delete_block_hash);
1791
1792        // remove from block index
1793        if self.blocks.contains_key(&delete_block_hash) {
1794            self.blocks.remove_entry(&delete_block_hash);
1795        }
1796
1797        wallet_update_status
1798    }
1799
1800    async fn downgrade_blockchain_data(&mut self, configs: &(dyn Configuration + Send + Sync)) {
1801        // downgrade blocks still on the chain
1802        if configs.get_consensus_config().unwrap().prune_after_blocks > self.get_latest_block_id() {
1803            return;
1804        }
1805        let prune_blocks_at_block_id =
1806            self.get_latest_block_id() - configs.get_consensus_config().unwrap().prune_after_blocks;
1807        let mut block_hashes_copy: Vec<SaitoHash> = vec![];
1808        debug!(
1809            "downgrading blockchain data. latest block id : {:?}. prune blocks at : {:?}",
1810            self.get_latest_block_id(),
1811            prune_blocks_at_block_id
1812        );
1813
1814        {
1815            let block_hashes = self
1816                .blockring
1817                .get_block_hashes_at_block_id(prune_blocks_at_block_id);
1818            for hash in block_hashes {
1819                block_hashes_copy.push(hash);
1820            }
1821        }
1822
1823        for hash in block_hashes_copy {
1824            // ask the block to remove its transactions
1825            {
1826                let block = self.get_mut_block(&hash);
1827                if let Some(block) = block {
1828                    if block.safe_to_prune_transactions {
1829                        block
1830                            .downgrade_block_to_block_type(BlockType::Pruned, configs.is_spv_mode())
1831                            .await;
1832                    }
1833                } else {
1834                    warn!("block : {:?} not found to downgrade", hash.to_hex());
1835                }
1836            }
1837        }
1838    }
1839    pub async fn add_blocks_from_mempool(
1840        &mut self,
1841        mempool_lock: Arc<RwLock<Mempool>>,
1842        network: Option<&Network>,
1843        storage: &mut Storage,
1844        sender_to_miner: Option<Sender<MiningEvent>>,
1845        sender_to_router: Option<Sender<RoutingEvent>>,
1846        configs: &(dyn Configuration + Send + Sync),
1847    ) {
1848        debug!("adding blocks from mempool to blockchain");
1849        let mut blocks: VecDeque<Block>;
1850        {
1851            let mut mempool = mempool_lock.write().await;
1852
1853            blocks = mempool.blocks_queue.drain(..).collect();
1854            blocks.make_contiguous().sort_by(|a, b| a.id.cmp(&b.id));
1855
1856            debug!("blocks to add : {:?}", blocks.len());
1857            while let Some(block) = blocks.pop_front() {
1858                let peer_index = block.routed_from_peer;
1859                let block_id = block.id;
1860                let result = self.add_block(block, storage, &mut mempool, configs).await;
1861                match result {
1862                    AddBlockResult::BlockAddedSuccessfully(
1863                        block_hash,
1864                        in_longest_chain,
1865                        wallet_updated,
1866                    ) => {
1867                        let sender_to_miner = if blocks.is_empty() {
1868                            sender_to_miner.clone()
1869                        } else {
1870                            None
1871                        };
1872
1873                        // check for any checkpoint data and process them
1874                        if let Some(checkpoints) =
1875                            storage.load_checkpoint_file(&block_hash, block_id).await
1876                        {
1877                            let mut wallet = self.wallet_lock.write().await;
1878                            for key in checkpoints {
1879                                if let Some((key, _)) = self.utxoset.remove_entry(&key) {
1880                                    if let Ok(slip) = Slip::parse_slip_from_utxokey(&key) {
1881                                        wallet.delete_slip(&slip, None);
1882                                        let block = self.blocks.get_mut(&block_hash).unwrap();
1883                                        block.graveyard += slip.amount;
1884                                        block.has_checkpoint = true;
1885                                        self.checkpoint_found = true;
1886                                        info!("skipping slip : {} according to the checkpoint file : {}-{}",
1887                                            slip,block_id,block_hash.to_hex());
1888                                    } else {
1889                                        error!("Key : {:?} in checkpoint file : {}-{} cannot be parsed to a slip", key.to_hex(),block_id,block_hash.to_hex());
1890                                        panic!("cannot continue loading blocks");
1891                                    }
1892                                }
1893                            }
1894                        }
1895
1896                        // TODO : to fix blocks being pruned before js processing them, pass a parameter in add_block to not prune and then prune manually after adding all.
1897                        //  need to do that in batches to make sure too much memory is not being used.
1898                        self.handle_successful_block_addition(
1899                            network,
1900                            sender_to_miner,
1901                            sender_to_router.clone(),
1902                            configs.is_spv_mode(),
1903                            block_hash,
1904                            in_longest_chain,
1905                            wallet_updated,
1906                        )
1907                        .await;
1908                    }
1909                    AddBlockResult::BlockAlreadyExists => {}
1910                    AddBlockResult::FailedButRetry(block, fetch_prev_block, fetch_blockchain) => {
1911                        Self::handle_failed_block_to_be_retried(
1912                            sender_to_router.clone(),
1913                            &mut mempool,
1914                            block,
1915                            fetch_prev_block,
1916                            fetch_blockchain,
1917                        )
1918                        .await;
1919                    }
1920                    AddBlockResult::FailedNotValid => {
1921                        if let Some(peer_index) = peer_index {
1922                            let mut peers = network.unwrap().peer_lock.write().await;
1923                            if let Some(peer) = peers.find_peer_by_index_mut(peer_index) {
1924                                peer.invalid_block_limiter.increase();
1925                            }
1926                        }
1927                    }
1928                }
1929            }
1930
1931            if sender_to_miner.is_some() {
1932                self.print(10, configs);
1933            }
1934
1935            debug!(
1936                "added blocks to blockchain. added back : {:?}",
1937                mempool.blocks_queue.len()
1938            );
1939        }
1940
1941        let mut wallet = self.wallet_lock.write().await;
1942        Wallet::save(&mut wallet, storage.io_interface.as_ref()).await;
1943    }
1944
1945    async fn handle_successful_block_addition(
1946        &mut self,
1947        network: Option<&Network>,
1948        sender_to_miner: Option<Sender<MiningEvent>>,
1949        sender_to_router: Option<Sender<RoutingEvent>>,
1950        is_spv_mode: bool,
1951        block_hash: BlockHash,
1952        in_longest_chain: bool,
1953        wallet_updated: WalletUpdateStatus,
1954    ) {
1955        let block = self
1956            .blocks
1957            .get(&block_hash)
1958            .expect("block should be here since it was added successfully");
1959
1960        if sender_to_miner.is_some() && in_longest_chain {
1961            debug!("sending longest chain block added event to miner : hash : {:?} difficulty : {:?} channel_capacity : {:?}",
1962                block_hash.to_hex(), block.difficulty, sender_to_miner.as_ref().unwrap().capacity());
1963            sender_to_miner
1964                .unwrap()
1965                .send(MiningEvent::LongestChainBlockAdded {
1966                    hash: block_hash,
1967                    difficulty: block.difficulty,
1968                    block_id: block.id,
1969                })
1970                .await
1971                .unwrap();
1972        }
1973
1974        if let Some(network) = network {
1975            network
1976                .io_interface
1977                .send_interface_event(InterfaceEvent::BlockAddSuccess(block_hash, block.id));
1978
1979            if wallet_updated {
1980                network
1981                    .io_interface
1982                    .send_interface_event(InterfaceEvent::WalletUpdate());
1983            } else {
1984                debug!("not updating wallet for block : {:?}", block_hash.to_hex());
1985            }
1986
1987            if !is_spv_mode {
1988                network.propagate_block(block).await;
1989            }
1990        }
1991
1992        if let Some(sender) = sender_to_router {
1993            debug!("sending blockchain updated event to router. channel_capacity : {:?} block_hash : {:?}", sender.capacity(),block_hash.to_hex());
1994            sender
1995                .send(RoutingEvent::BlockchainUpdated(block_hash))
1996                .await
1997                .unwrap();
1998        }
1999    }
2000
2001    async fn handle_failed_block_to_be_retried(
2002        sender_to_router: Option<Sender<RoutingEvent>>,
2003        mempool: &mut Mempool,
2004        block: Block,
2005        fetch_prev_block: bool,
2006        fetch_blockchain: bool,
2007    ) {
2008        debug!("adding block : {:?} back to mempool so it can be processed again after the previous block : {:?} is added",
2009                                    block.hash.to_hex(),
2010                                    block.previous_block_hash.to_hex());
2011
2012        if let Some(sender) = sender_to_router.as_ref() {
2013            if fetch_blockchain {
2014                sender
2015                    .send(RoutingEvent::BlockchainRequest(
2016                        block.routed_from_peer.unwrap(),
2017                    ))
2018                    .await
2019                    .expect("sending blockchain request failed");
2020            } else if fetch_prev_block {
2021                sender
2022                    .send(RoutingEvent::BlockFetchRequest(
2023                        block.routed_from_peer.unwrap_or(0),
2024                        block.previous_block_hash,
2025                        block.id - 1,
2026                    ))
2027                    .await
2028                    .expect("sending block fetch request failed");
2029            }
2030        }
2031
2032        mempool.add_block(block);
2033    }
2034
2035    pub fn add_ghost_block(
2036        &mut self,
2037        id: u64,
2038        previous_block_hash: SaitoHash,
2039        ts: Timestamp,
2040        pre_hash: SaitoHash,
2041        gt: bool,
2042        hash: SaitoHash,
2043    ) {
2044        debug!(
2045            "adding ghost block : {:?}-{:?} prev_block : {:?} ts : {:?}",
2046            id,
2047            hash.to_hex(),
2048            previous_block_hash.to_hex(),
2049            ts
2050        );
2051        let ring_buffer_size = self.blockring.get_ring_buffer_size();
2052        let mut block = Block::new();
2053        block.id = id;
2054        block.previous_block_hash = previous_block_hash;
2055        block.timestamp = ts;
2056        block.has_golden_ticket = gt;
2057        block.pre_hash = pre_hash;
2058        block.hash = hash;
2059        block.block_type = BlockType::Ghost;
2060
2061        if self.is_block_indexed(hash) {
2062            warn!("block :{:?} exists in blockchain", hash.to_hex());
2063            return;
2064        }
2065        if !self.blockring.contains_block_hash_at_block_id(id, hash) {
2066            self.blockring.add_block(&block);
2067            self.blockring.lc_pos = Some((id % ring_buffer_size) as usize);
2068            self.blockring.ring[(id % ring_buffer_size) as usize].lc_pos = Some(0);
2069        } else {
2070            debug!("didn't add ghost block : {:?}-{:?}", id, hash.to_hex());
2071        }
2072        self.blocks.insert(hash, block);
2073    }
2074
2075    pub async fn reset(&mut self) {
2076        self.last_burnfee = 0;
2077        self.last_timestamp = 0;
2078        self.last_block_id = 0;
2079        self.last_block_hash = [0; 32];
2080        self.genesis_timestamp = 0;
2081        self.genesis_block_hash = [0; 32];
2082        self.genesis_block_id = 0;
2083        self.lowest_acceptable_block_id = 0;
2084        self.lowest_acceptable_timestamp = 0;
2085        self.lowest_acceptable_block_hash = [0; 32];
2086        self.fork_id = Some([0; 32]);
2087        self.save().await;
2088    }
2089
2090    pub async fn save(&self) {
2091        // TODO : what should be done here in rust code?
2092    }
2093    pub fn get_utxoset_data(&self) -> HashMap<SaitoPublicKey, Currency> {
2094        let mut data: HashMap<SaitoPublicKey, Currency> = Default::default();
2095        self.utxoset.iter().for_each(|(key, value)| {
2096            if !value {
2097                return;
2098            }
2099            let slip = Slip::parse_slip_from_utxokey(key).unwrap();
2100            *data.entry(slip.public_key).or_default() += slip.amount;
2101        });
2102        data
2103    }
2104    pub fn get_slips_for(&self, public_key: SaitoPublicKey) -> Vec<Slip> {
2105        let mut slips: Vec<Slip> = Default::default();
2106        self.utxoset
2107            .iter()
2108            .filter(|(_, value)| **value)
2109            .for_each(|(key, _)| {
2110                let slip = Slip::parse_slip_from_utxokey(key).unwrap();
2111                if slip.public_key == public_key {
2112                    slips.push(slip);
2113                }
2114            });
2115        slips
2116    }
2117    pub fn get_balance_snapshot(
2118        &self,
2119        keys: Vec<SaitoPublicKey>,
2120        configs: &(dyn Configuration + Send + Sync),
2121    ) -> BalanceSnapshot {
2122        let latest_block_id = self.get_latest_block_id();
2123        let genesis_period = configs.get_consensus_config().unwrap().genesis_period;
2124
2125        let mut snapshot = BalanceSnapshot {
2126            latest_block_id: latest_block_id,
2127            latest_block_hash: self.get_latest_block_hash(),
2128            timestamp: self.last_timestamp,
2129            slips: vec![],
2130        };
2131        // TODO : calling this will be a huge performance hit for the node. so need to refactor the design.
2132        self.utxoset
2133            .iter()
2134            .filter(|(_, value)| **value)
2135            .for_each(|(key, _)| {
2136                let slip = Slip::parse_slip_from_utxokey(key).unwrap();
2137
2138                //
2139                // Skip any Bound‐type slips
2140                //
2141                if slip.slip_type == SlipType::Bound {
2142                    return;
2143                }
2144
2145                //
2146                // Check if UTXO is valid (not from an off-chain block)
2147                //
2148                if slip.block_id < latest_block_id.saturating_sub(genesis_period) {
2149                    return;
2150                }
2151
2152                //
2153                // if no keys provided we get the full picture
2154                //
2155                if keys.is_empty() || keys.contains(&slip.public_key) {
2156                    snapshot.slips.push(slip);
2157                }
2158            });
2159
2160        snapshot
2161    }
2162    pub fn is_slip_unlocked(&self, utxo_key: &SaitoUTXOSetKey) -> bool {
2163        let latest_unlocked_block_id = self.get_latest_unlocked_stake_block_id();
2164        let result = Slip::parse_slip_from_utxokey(utxo_key);
2165        if result.is_err() {
2166            warn!("cannot parse utxo key  : {:?}", utxo_key.to_hex());
2167            return false;
2168        }
2169        let slip = result.unwrap();
2170        let result = self.utxoset.get(utxo_key);
2171        if result.is_none() {
2172            warn!(
2173                "slip not found. : {:?}-{:?}-{:?} type: {:?} amount : {:?}",
2174                slip.block_id, slip.tx_ordinal, slip.slip_index, slip.slip_type, slip.amount
2175            );
2176            return false;
2177        }
2178        let spendable = result.unwrap();
2179        if !spendable {
2180            return false;
2181        }
2182        if let SlipType::BlockStake = slip.slip_type {
2183            if slip.block_id > latest_unlocked_block_id {
2184                return false;
2185            }
2186        }
2187
2188        true
2189    }
2190
2191    pub fn set_safe_to_prune_transaction(&mut self, block_id: u64) {
2192        let block_hash_option = self.blockring.get_block_hash_by_block_id(block_id);
2193
2194        if let Some(block_hash) = block_hash_option {
2195            if let Some(block) = self.blocks.get_mut(&block_hash) {
2196                block.safe_to_prune_transactions = true;
2197            }
2198        }
2199    }
2200    pub fn calculate_current_supply(&self) -> Currency {
2201        let mut current_supply = Currency::default();
2202        self.utxoset.iter().for_each(|(key, value)| {
2203            if !value {
2204                return;
2205            }
2206            let slip = Slip::parse_slip_from_utxokey(key).unwrap();
2207            current_supply += slip.amount;
2208        });
2209
2210        if let Some(latest_block) = self.get_latest_block() {
2211            current_supply += latest_block.graveyard;
2212            current_supply += latest_block.treasury;
2213            current_supply += latest_block.previous_block_unpaid;
2214            current_supply += latest_block.total_fees;
2215        } else {
2216            debug!("latest block not found in blockchain to calculate total supply");
2217            return 0;
2218        }
2219        current_supply
2220    }
2221    pub async fn check_total_supply(&mut self, configs: &(dyn Configuration + Send + Sync)) {
2222        let genesis_period = configs.get_consensus_config().unwrap().genesis_period;
2223
2224        if !self.has_total_supply_loaded(genesis_period) {
2225            debug!("total supply not loaded yet. skipping check");
2226            return;
2227        }
2228
2229        if configs.is_browser() || configs.is_spv_mode() {
2230            debug!("skipping total supply check in spv mode");
2231            return;
2232        }
2233
2234        let latest_block = self
2235            .get_latest_block()
2236            .expect("There should be a latest block in blockchain");
2237
2238        let mut current_supply = 0;
2239        let amount_in_utxo = self
2240            .utxoset
2241            .iter()
2242            .filter(|(_, &spent)| spent)
2243            .filter_map(|(key, &spent)| {
2244                let slip = Slip::parse_slip_from_utxokey(key).ok()?;
2245
2246                //
2247                // Skip any Bound‐type slips
2248                //
2249                if slip.slip_type == SlipType::Bound {
2250                    return None;
2251                }
2252
2253                //
2254                // skip old UTXOs outside the genesis window
2255                if slip.block_id < latest_block.id.saturating_sub(genesis_period) {
2256                    return None;
2257                }
2258
2259                trace!(
2260                    "Utxo : {:?} : {} : {:?}, block : {}-{}-{}, valid : {}",
2261                    slip.public_key.to_base58(),
2262                    slip.amount,
2263                    slip.slip_type,
2264                    slip.block_id,
2265                    slip.tx_ordinal,
2266                    slip.slip_index,
2267                    spent
2268                );
2269
2270                Some(slip.amount)
2271            })
2272            .sum::<Currency>();
2273
2274        current_supply += amount_in_utxo;
2275
2276        current_supply += latest_block.graveyard;
2277        current_supply += latest_block.treasury;
2278        current_supply += latest_block.previous_block_unpaid;
2279        current_supply += latest_block.total_fees;
2280
2281        if self.initial_token_supply == 0 {
2282            info!(
2283                "initial token supply is not set. setting it to current supply : {}",
2284                current_supply
2285            );
2286            self.initial_token_supply = current_supply;
2287        }
2288
2289        if current_supply != self.initial_token_supply {
2290            let latest_block = self.get_latest_block().unwrap();
2291            warn!(
2292                "diff : {}",
2293                self.initial_token_supply as i64 - current_supply as i64
2294            );
2295            warn!("Current supply is {}", current_supply);
2296            warn!("Initial token supply is {}", self.initial_token_supply);
2297            warn!(
2298                "Social Stake Requirement is {}",
2299                self.social_stake_requirement
2300            );
2301            warn!("Graveyard is {}", latest_block.graveyard);
2302            warn!("Treasury is {}", latest_block.treasury);
2303            warn!("Unpaid fees is {}", latest_block.previous_block_unpaid);
2304            warn!("Total Fees ATR is {}", latest_block.total_fees_atr);
2305            warn!("Total Fees New is {}", latest_block.total_fees_new);
2306            warn!("Total Fee is {}", latest_block.total_fees);
2307            warn!("Amount in utxo {}", amount_in_utxo);
2308
2309            warn!("latest block : {}", latest_block);
2310            warn!(
2311                "current supply : {:?} doesn't equal to initial supply : {:?}",
2312                current_supply, self.initial_token_supply
2313            );
2314            latest_block.print_all();
2315            panic!("cannot continue with invalid total supply");
2316        }
2317        debug!(
2318            "total supply check passed. current supply : {:?} initial supply : {:?}",
2319            current_supply, self.initial_token_supply
2320        );
2321    }
2322}
2323
2324pub fn generate_fork_id_weights(genesis_period: BlockId) -> [u64; 16] {
2325    const LENGTH: BlockId = 100_000;
2326    [
2327        0,
2328        max((10 * genesis_period) / LENGTH, 1),
2329        max((10 * genesis_period) / LENGTH, 1),
2330        max((10 * genesis_period) / LENGTH, 1),
2331        max((10 * genesis_period) / LENGTH, 1),
2332        max((10 * genesis_period) / LENGTH, 1),
2333        max((25 * genesis_period) / LENGTH, 1),
2334        max((25 * genesis_period) / LENGTH, 1),
2335        max((100 * genesis_period) / LENGTH, 1),
2336        max((300 * genesis_period) / LENGTH, 1),
2337        max((500 * genesis_period) / LENGTH, 1),
2338        max((4000 * genesis_period) / LENGTH, 1),
2339        max((10000 * genesis_period) / LENGTH, 1),
2340        max((20000 * genesis_period) / LENGTH, 1),
2341        max((50000 * genesis_period) / LENGTH, 1),
2342        genesis_period,
2343    ]
2344}
2345
2346fn is_golden_ticket_count_valid_<'a, F: Fn(SaitoHash) -> Option<&'a Block>>(
2347    previous_block_hash: SaitoHash,
2348    current_block_has_golden_ticket: bool,
2349    bypass: bool,
2350    get_block: F,
2351) -> bool {
2352    let mut golden_tickets_found = 0;
2353    let mut required_tickets = 0;
2354    let mut search_depth_index = 0;
2355    let mut latest_block_hash = previous_block_hash;
2356
2357    for _ in 0..MIN_GOLDEN_TICKETS_DENOMINATOR - 1 {
2358        if let Some(block) = get_block(latest_block_hash) {
2359            search_depth_index += 1;
2360            debug!(
2361                "searching for golden tickets : block id : {:?} hash : {:?} has_golden_ticket : {:?} search_depth : {:?}",
2362                block.id, block.hash.to_hex(), block.has_golden_ticket,search_depth_index
2363            );
2364            // the latest block will not have has_golden_ticket set yet
2365            // so it is possible we undercount the latest block. this
2366            // is dealt with by manually checking for the existence of
2367            // a golden ticket if we only have 1 golden ticket below.
2368            if block.has_golden_ticket {
2369                golden_tickets_found += 1;
2370            }
2371            // if i == 0 {
2372            //     // 3/7 => [1,2,3,4,5,6 + 7(current)]. but we only check the first 6 since we start from previous block.
2373            //     // if we only have 5 blocks, we need 1 golden ticket. if we have 6 blocks, we need 2 golden tickets. if current block has golden ticket, we need 3 golden tickets.
2374            //     required_tickets = MIN_GOLDEN_TICKETS_NUMERATOR
2375            //         .saturating_sub(MIN_GOLDEN_TICKETS_DENOMINATOR.saturating_sub(block.id));
2376            // }
2377            latest_block_hash = block.previous_block_hash;
2378        } else {
2379            break;
2380        }
2381    }
2382    // 2/6 => [1,2,3,4,5 + 6(current)]. but we only check the first 5 since we start from previous block.
2383    // if we only have 4 blocks, we need 1 golden ticket. if we have 5 blocks, we need 2 golden tickets (including the gt at hand). because we calculate only upto the previous block and then consider the current block's gt
2384    required_tickets = MIN_GOLDEN_TICKETS_NUMERATOR
2385        .saturating_sub(MIN_GOLDEN_TICKETS_DENOMINATOR.saturating_sub(search_depth_index + 1));
2386
2387    if current_block_has_golden_ticket {
2388        golden_tickets_found += 1;
2389    }
2390    // if search_depth_index < MIN_GOLDEN_TICKETS_DENOMINATOR - MIN_GOLDEN_TICKETS_NUMERATOR {
2391    //     // we need to check for search_depth here because we need to have at least MIN_GOLDEN_TICKETS_DENOMINATOR blocks loaded before we can check for golden tickets
2392    //     debug!(
2393    //         "returning true since search depth {} is not enough. current_block_has_golden_ticket : {:?} golden_tickets_found : {:?} required_tickets : {}",
2394    //         search_depth_index, current_block_has_golden_ticket, golden_tickets_found, required_tickets
2395    //     );
2396    //     return true;
2397    // }
2398    // else if search_depth_index == MIN_GOLDEN_TICKETS_DENOMINATOR - 1 {
2399    //     // we only have enough blocks to check for a single golden ticket
2400    //     debug!(
2401    //         "search depth is exactly MIN_GOLDEN_TICKETS_DENOMINATOR - 1. current_block_has_golden_ticket : {:?} golden_tickets_found : {:?} required_tickets : {}",
2402    //         current_block_has_golden_ticket, golden_tickets_found, required_tickets
2403    //     );
2404    //     required_tickets = 1;
2405    // }
2406    // TODO : uncomment above after fixing and remove this
2407    if search_depth_index < MIN_GOLDEN_TICKETS_DENOMINATOR - MIN_GOLDEN_TICKETS_NUMERATOR {
2408        // we need to check for search_depth here because we need to have at least MIN_GOLDEN_TICKETS_DENOMINATOR blocks loaded before we can check for golden tickets
2409        debug!(
2410            "returning true since search depth {} is not enough. current_block_has_golden_ticket : {:?} golden_tickets_found : {:?} required_tickets : {}",
2411            search_depth_index, current_block_has_golden_ticket, golden_tickets_found, required_tickets
2412        );
2413        return true;
2414    }
2415
2416    if golden_tickets_found < required_tickets {
2417        info!(
2418            "not enough golden tickets : found = {:?} depth = {:?} current_block_has_golden_ticket : {:?} required_tickets : {}",
2419            golden_tickets_found, search_depth_index, current_block_has_golden_ticket, required_tickets
2420        );
2421        // TODO : browsers might want to implement this check somehow
2422        if !bypass {
2423            return false;
2424        }
2425    }
2426    debug!(
2427        "found enough golden tickets : found = {:?} depth = {:?} current_block_has_golden_ticket : {:?} required_tickets : {}",
2428        golden_tickets_found, search_depth_index, current_block_has_golden_ticket, required_tickets
2429    );
2430    true
2431}
2432
2433#[cfg(test)]
2434mod tests {
2435    use crate::core::consensus::block::Block;
2436    use crate::core::consensus::blockchain::{
2437        bit_pack, bit_unpack, is_golden_ticket_count_valid_, AddBlockResult, Blockchain,
2438    };
2439    use crate::core::consensus::slip::Slip;
2440    use crate::core::consensus::wallet::Wallet;
2441    use crate::core::defs::{ForkId, PrintForLog, SaitoHash, SaitoPublicKey, NOLAN_PER_SAITO};
2442    use crate::core::io::storage::Storage;
2443    use crate::core::util::crypto::{generate_keys, hash};
2444    use crate::core::util::test::node_tester::test::NodeTester;
2445    use crate::core::util::test::test_manager::test::TestManager;
2446    use ahash::HashMap;
2447    use log::{debug, error, info};
2448    use std::fs;
2449    use std::ops::Deref;
2450    use std::sync::Arc;
2451
2452    use tokio::sync::RwLock;
2453
2454    // fn init_testlog() {
2455    //     let _ = pretty_env_logger::try_init();
2456    // }
2457
2458    #[tokio::test]
2459    async fn test_blockchain_init() {
2460        let keys = generate_keys();
2461
2462        let wallet = Arc::new(RwLock::new(Wallet::new(keys.1, keys.0)));
2463        let blockchain = Blockchain::new(wallet, 1_000, 0, 60);
2464
2465        assert_eq!(blockchain.fork_id, None);
2466        assert_eq!(blockchain.genesis_block_id, 0);
2467    }
2468
2469    #[tokio::test]
2470    async fn test_add_block() {
2471        let keys = generate_keys();
2472        let wallet = Arc::new(RwLock::new(Wallet::new(keys.1, keys.0)));
2473        let blockchain = Blockchain::new(wallet, 1_000, 0, 60);
2474
2475        assert_eq!(blockchain.fork_id, None);
2476        assert_eq!(blockchain.genesis_block_id, 0);
2477    }
2478
2479    #[test]
2480    //
2481    // code that packs/unpacks two 32-bit values into one 64-bit variable
2482    //
2483    fn bit_pack_test() {
2484        let top = 157171715;
2485        let bottom = 11661612;
2486        let packed = bit_pack(top, bottom);
2487        assert_eq!(packed, 157171715 * (u64::pow(2, 32)) + 11661612);
2488        let (new_top, new_bottom) = bit_unpack(packed);
2489        assert_eq!(top, new_top);
2490        assert_eq!(bottom, new_bottom);
2491
2492        let top = u32::MAX;
2493        let bottom = u32::MAX;
2494        let packed = bit_pack(top, bottom);
2495        let (new_top, new_bottom) = bit_unpack(packed);
2496        assert_eq!(top, new_top);
2497        assert_eq!(bottom, new_bottom);
2498
2499        let top = 0;
2500        let bottom = 1;
2501        let packed = bit_pack(top, bottom);
2502        let (new_top, new_bottom) = bit_unpack(packed);
2503        assert_eq!(top, new_top);
2504        assert_eq!(bottom, new_bottom);
2505    }
2506
2507    #[tokio::test]
2508    #[serial_test::serial]
2509    async fn initialize_blockchain_test() {
2510        let mut t = TestManager::default();
2511
2512        // create first block, with 100 VIP txs with 1_000_000_000 NOLAN each
2513        t.initialize(100, 1_000_000_000).await;
2514        // t.wait_for_mining_event().await;
2515
2516        {
2517            let blockchain = t.blockchain_lock.read().await;
2518            assert_eq!(1, blockchain.get_latest_block_id());
2519        }
2520        t.check_blockchain().await;
2521        t.check_utxoset().await;
2522        t.check_token_supply().await;
2523    }
2524
2525    #[tokio::test]
2526    #[serial_test::serial]
2527    //
2528    // test we can produce five blocks in a row
2529    //
2530    async fn add_five_good_blocks() {
2531        // let filter = tracing_subscriber::EnvFilter::from_default_env();
2532        // let fmt_layer = tracing_subscriber::fmt::Layer::default().with_filter(filter);
2533        //
2534        // tracing_subscriber::registry().with(fmt_layer).init();
2535
2536        let mut t = TestManager::default();
2537        let block1;
2538        let block1_id;
2539        let block1_hash;
2540        let ts;
2541
2542        //
2543        // block 1
2544        //
2545        t.initialize(100, 200_000_000_000_000).await;
2546
2547        {
2548            let blockchain = t.blockchain_lock.write().await;
2549            block1 = blockchain.get_latest_block().unwrap();
2550            block1_id = block1.id;
2551            block1_hash = block1.hash;
2552            ts = block1.timestamp;
2553
2554            assert_eq!(blockchain.get_latest_block_hash(), block1_hash);
2555            assert_eq!(blockchain.get_latest_block_id(), block1_id);
2556            assert_eq!(blockchain.get_latest_block_id(), 1);
2557        }
2558
2559        //
2560        // block 2
2561        //
2562        let mut block2 = t
2563            .create_block(
2564                block1_hash, // hash of parent block
2565                ts + 120000, // timestamp
2566                0,           // num transactions
2567                0,           // amount
2568                0,           // fee
2569                true,        // mine golden ticket
2570            )
2571            .await;
2572        block2.generate().unwrap(); // generate hashes
2573
2574        let block2_hash = block2.hash;
2575        let block2_id = block2.id;
2576
2577        t.add_block(block2).await;
2578
2579        {
2580            let blockchain = t.blockchain_lock.write().await;
2581
2582            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2583            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2584            assert_eq!(blockchain.get_latest_block_hash(), block2_hash);
2585            assert_eq!(blockchain.get_latest_block_id(), block2_id);
2586            assert_eq!(blockchain.get_latest_block_id(), 2);
2587        }
2588
2589        //
2590        // block 3
2591        //
2592        let mut block3 = t
2593            .create_block(
2594                block2_hash, // hash of parent block
2595                ts + 240000, // timestamp
2596                0,           // num transactions
2597                0,           // amount
2598                0,           // fee
2599                true,        // mine golden ticket
2600            )
2601            .await;
2602        block3.generate().unwrap(); // generate hashes
2603
2604        let block3_hash = block3.hash;
2605        let block3_id = block3.id;
2606
2607        t.add_block(block3).await;
2608
2609        {
2610            let blockchain = t.blockchain_lock.write().await;
2611
2612            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2613            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2614            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2615            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2616            assert_eq!(blockchain.get_latest_block_hash(), block3_hash);
2617            assert_eq!(blockchain.get_latest_block_id(), block3_id);
2618            assert_eq!(blockchain.get_latest_block_id(), 3);
2619        }
2620
2621        //
2622        // block 4
2623        //
2624        let mut block4 = t
2625            .create_block(
2626                block3_hash, // hash of parent block
2627                ts + 360000, // timestamp
2628                0,           // num transactions
2629                0,           // amount
2630                0,           // fee
2631                true,        // mine golden ticket
2632            )
2633            .await;
2634        block4.generate().unwrap(); // generate hashes
2635
2636        let block4_hash = block4.hash;
2637        let block4_id = block4.id;
2638
2639        t.add_block(block4).await;
2640
2641        {
2642            let blockchain = t.blockchain_lock.write().await;
2643
2644            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2645            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2646            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2647            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2648            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
2649            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2650            assert_eq!(blockchain.get_latest_block_hash(), block4_hash);
2651            assert_eq!(blockchain.get_latest_block_id(), block4_id);
2652            assert_eq!(blockchain.get_latest_block_id(), 4);
2653        }
2654
2655        //
2656        // block 5
2657        //
2658        let mut block5 = t
2659            .create_block(
2660                block4_hash, // hash of parent block
2661                ts + 480000, // timestamp
2662                0,           // num transactions
2663                0,           // amount
2664                0,           // fee
2665                true,        // mine golden ticket
2666            )
2667            .await;
2668        block5.generate().unwrap(); // generate hashes
2669
2670        let block5_hash = block5.hash;
2671        let block5_id = block5.id;
2672
2673        t.add_block(block5).await;
2674
2675        {
2676            let blockchain = t.blockchain_lock.write().await;
2677
2678            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2679            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2680            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2681            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2682            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
2683            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2684            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
2685            assert_ne!(blockchain.get_latest_block_id(), block4_id);
2686            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
2687            assert_eq!(blockchain.get_latest_block_id(), block5_id);
2688            assert_eq!(blockchain.get_latest_block_id(), 5);
2689        }
2690
2691        t.check_blockchain().await;
2692        t.check_utxoset().await;
2693        t.check_token_supply().await;
2694
2695        {
2696            let wallet = t.wallet_lock.read().await;
2697            let count = wallet.get_unspent_slip_count();
2698            assert_ne!(count, 0);
2699            let balance = wallet.get_available_balance();
2700            assert_ne!(balance, 0);
2701        }
2702    }
2703
2704    #[tokio::test]
2705    #[serial_test::serial]
2706    //
2707    // test we do not add blocks because of insufficient mining
2708    //
2709    async fn insufficient_golden_tickets_test() {
2710        // let filter = tracing_subscriber::EnvFilter::from_default_env();
2711        // let fmt_layer = tracing_subscriber::fmt::Layer::default().with_filter(filter);
2712        //
2713        // tracing_subscriber::registry().with(fmt_layer).init();
2714        // pretty_env_logger::init();
2715
2716        let mut t = TestManager::default();
2717        let block1;
2718        let block1_id;
2719        let block1_hash;
2720        let ts;
2721
2722        //
2723        // block 1
2724        //
2725        t.initialize(100, 200_000_000_000_000).await;
2726
2727        {
2728            let blockchain = t.blockchain_lock.read().await;
2729
2730            block1 = blockchain.get_latest_block().unwrap();
2731            block1_id = block1.id;
2732            block1_hash = block1.hash;
2733            ts = block1.timestamp;
2734
2735            assert_eq!(blockchain.get_latest_block_hash(), block1_hash);
2736            assert_eq!(blockchain.get_latest_block_id(), block1_id);
2737            assert_eq!(blockchain.get_latest_block_id(), 1);
2738        }
2739
2740        //
2741        // block 2
2742        //
2743        let mut block2 = t
2744            .create_block(
2745                block1_hash, // hash of parent block
2746                ts + 120000, // timestamp
2747                1,           // num transactions
2748                0,           // amount
2749                0,           // fee
2750                true,        // mine golden ticket
2751            )
2752            .await;
2753        block2.generate().unwrap(); // generate hashes
2754
2755        let block2_hash = block2.hash;
2756        let block2_id = block2.id;
2757
2758        t.add_block(block2).await;
2759
2760        {
2761            let blockchain = t.blockchain_lock.write().await;
2762
2763            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2764            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2765            assert_eq!(blockchain.get_latest_block_hash(), block2_hash);
2766            assert_eq!(blockchain.get_latest_block_id(), block2_id);
2767            assert_eq!(blockchain.get_latest_block_id(), 2);
2768        }
2769
2770        //
2771        // block 3
2772        //
2773        let mut block3 = t
2774            .create_block(
2775                block2_hash, // hash of parent block
2776                ts + 240000, // timestamp
2777                1,           // num transactions
2778                0,           // amount
2779                0,           // fee
2780                false,       // mine golden ticket
2781            )
2782            .await;
2783        block3.generate().unwrap(); // generate hashes
2784
2785        let block3_hash = block3.hash;
2786        let block3_id = block3.id;
2787
2788        t.add_block(block3).await;
2789
2790        {
2791            let blockchain = t.blockchain_lock.write().await;
2792
2793            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2794            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2795            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2796            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2797            assert_eq!(blockchain.get_latest_block_hash(), block3_hash);
2798            assert_eq!(blockchain.get_latest_block_id(), block3_id);
2799            assert_eq!(blockchain.get_latest_block_id(), 3);
2800        }
2801
2802        //
2803        // block 4
2804        //
2805        let mut block4 = t
2806            .create_block(
2807                block3_hash, // hash of parent block
2808                ts + 360000, // timestamp
2809                1,           // num transactions
2810                0,           // amount
2811                0,           // fee
2812                false,       // mine golden ticket
2813            )
2814            .await;
2815        block4.generate().unwrap(); // generate hashes
2816
2817        let block4_hash = block4.hash;
2818        let block4_id = block4.id;
2819
2820        t.add_block(block4).await;
2821
2822        {
2823            let blockchain = t.blockchain_lock.write().await;
2824
2825            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2826            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2827            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2828            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2829            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
2830            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2831            assert_eq!(blockchain.get_latest_block_hash(), block4_hash);
2832            assert_eq!(blockchain.get_latest_block_id(), block4_id);
2833            assert_eq!(blockchain.get_latest_block_id(), 4);
2834        }
2835
2836        //
2837        // block 5
2838        //
2839        let mut block5 = t
2840            .create_block(
2841                block4_hash, // hash of parent block
2842                ts + 480000, // timestamp
2843                1,           // num transactions
2844                0,           // amount
2845                0,           // fee
2846                false,       // mine golden ticket
2847            )
2848            .await;
2849        block5.generate().unwrap(); // generate hashes
2850
2851        let block5_hash = block5.hash;
2852        let block5_id = block5.id;
2853
2854        t.add_block(block5).await;
2855
2856        {
2857            let blockchain = t.blockchain_lock.write().await;
2858
2859            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2860            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2861            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2862            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2863            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
2864            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2865            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
2866            assert_ne!(blockchain.get_latest_block_id(), block4_id);
2867            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
2868            assert_eq!(blockchain.get_latest_block_id(), block5_id);
2869            assert_eq!(blockchain.get_latest_block_id(), 5);
2870        }
2871
2872        //
2873        // block 6
2874        //
2875        let mut block6 = t
2876            .create_block(
2877                block5_hash, // hash of parent block
2878                ts + 600000, // timestamp
2879                1,           // num transactions
2880                0,           // amount
2881                0,           // fee
2882                false,       // mine golden ticket
2883            )
2884            .await;
2885        block6.generate().unwrap(); // generate hashes
2886
2887        let block6_hash = block6.hash;
2888        let block6_id = block6.id;
2889
2890        t.add_block(block6).await;
2891
2892        {
2893            let blockchain = t.blockchain_lock.write().await;
2894
2895            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2896            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2897            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2898            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2899            assert_ne!(
2900                blockchain.get_latest_block_hash().to_hex(),
2901                block3_hash.to_hex()
2902            );
2903            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2904            assert_ne!(
2905                blockchain.get_latest_block_hash().to_hex(),
2906                block4_hash.to_hex()
2907            );
2908            assert_ne!(blockchain.get_latest_block_id(), block4_id);
2909            assert_eq!(blockchain.get_latest_block_id(), block5_id);
2910            assert_eq!(
2911                blockchain.get_latest_block_hash().to_hex(),
2912                block5_hash.to_hex()
2913            );
2914            assert_ne!(blockchain.get_latest_block_hash(), block6_hash);
2915            assert_ne!(blockchain.get_latest_block_id(), block6_id);
2916            assert_eq!(blockchain.get_latest_block_id(), 5);
2917        }
2918
2919        //
2920        // block 7
2921        //
2922        let mut block7 = t
2923            .create_block(
2924                block5_hash, // hash of parent block
2925                ts + 720000, // timestamp
2926                1,           // num transactions
2927                0,           // amount
2928                0,           // fee
2929                true,        // mine golden ticket
2930            )
2931            .await;
2932        block7.generate().unwrap(); // generate hashes
2933
2934        let block7_hash = block7.hash;
2935        let block7_id = block7.id;
2936
2937        t.add_block(block7).await;
2938
2939        {
2940            let blockchain = t.blockchain_lock.write().await;
2941
2942            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
2943            assert_ne!(blockchain.get_latest_block_id(), block1_id);
2944            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
2945            assert_ne!(blockchain.get_latest_block_id(), block2_id);
2946            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
2947            assert_ne!(blockchain.get_latest_block_id(), block3_id);
2948            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
2949            assert_ne!(blockchain.get_latest_block_id(), block4_id);
2950            assert_ne!(blockchain.get_latest_block_hash(), block5_hash);
2951            assert_ne!(blockchain.get_latest_block_id(), block5_id);
2952            assert_ne!(blockchain.get_latest_block_hash(), block6_hash);
2953            // assert_ne!(blockchain.get_latest_block_id(), block6_id);
2954            assert_eq!(blockchain.get_latest_block_hash(), block7_hash);
2955            assert_eq!(blockchain.get_latest_block_id(), block7_id);
2956            assert_eq!(blockchain.get_latest_block_id(), 6);
2957        }
2958
2959        t.check_blockchain().await;
2960        t.check_utxoset().await;
2961        t.check_token_supply().await;
2962    }
2963
2964    // tests if utxo hashmap persists after a blockchain reset
2965    #[tokio::test]
2966    #[serial_test::serial]
2967    async fn balance_hashmap_persists_after_blockchain_reset_test() {
2968        // pretty_env_logger::init();
2969        let mut t: TestManager = TestManager::default();
2970        let file_path = t.issuance_path;
2971        let slips = t
2972            .storage
2973            .get_token_supply_slips_from_disk_path(file_path)
2974            .await;
2975
2976        // start blockchain with existing issuance and some value to my public key
2977        t.initialize_from_slips_and_value(slips.clone(), 200_000_000_000_000)
2978            .await;
2979
2980        // add a few transactions
2981        let public_keys = [
2982            "s8oFPjBX97NC2vbm9E5Kd2oHWUShuSTUuZwSB1U4wsPR",
2983            // "s9adoFPjBX972vbm9E5Kd2oHWUShuSTUuZwSB1U4wsPR",
2984            // "s223oFPjBX97NC2bmE5Kd2oHWUShuSTUuZwSB1U4wsPR",
2985        ];
2986
2987        let mut last_param = 120000;
2988        for &public_key_string in &public_keys {
2989            let public_key = Storage::decode_str(public_key_string).unwrap();
2990            let mut to_public_key: SaitoPublicKey = [0u8; 33];
2991            to_public_key.copy_from_slice(&public_key);
2992            t.transfer_value_to_public_key(to_public_key, 500, last_param)
2993                .await
2994                .unwrap();
2995            last_param += 120000;
2996        }
2997
2998        // save utxo balance map on issuance file
2999        let balance_map = t.balance_map().await;
3000        match t
3001            .storage
3002            .write_utxoset_to_disk_path(balance_map.clone(), 1, file_path)
3003            .await
3004        {
3005            Ok(_) => {
3006                debug!("store file ok");
3007            }
3008            Err(e) => {
3009                error!("Error: {:?}", e);
3010            }
3011        }
3012
3013        // reset blockchain
3014        let mut t: TestManager = TestManager::default();
3015        let slips = t
3016            .storage
3017            .get_token_supply_slips_from_disk_path(t.issuance_path)
3018            .await;
3019
3020        let issuance_hashmap = t.convert_issuance_to_hashmap(t.issuance_path).await;
3021
3022        // initialize from existing slips
3023        t.initialize_from_slips(slips.clone()).await;
3024
3025        let balance_map_after_reset = t.balance_map().await;
3026
3027        assert_eq!(issuance_hashmap, balance_map_after_reset);
3028    }
3029
3030    // test we do not add blocks because of insufficient mining
3031    #[tokio::test]
3032    #[serial_test::serial]
3033    async fn seven_blocks_with_sufficient_golden_tickets_test() {
3034        // pretty_env_logger::init();
3035        let mut t = TestManager::default();
3036        let block1;
3037        let block1_id;
3038        let block1_hash;
3039        let ts;
3040
3041        // block 1
3042        t.initialize(100, 200_000_000_000_000).await;
3043
3044        {
3045            let blockchain = t.blockchain_lock.write().await;
3046
3047            block1 = blockchain.get_latest_block().unwrap();
3048            block1_hash = block1.hash;
3049            block1_id = block1.id;
3050            ts = block1.timestamp;
3051
3052            assert_eq!(blockchain.get_latest_block_hash(), block1_hash);
3053            assert_eq!(blockchain.get_latest_block_id(), block1_id);
3054            assert_eq!(blockchain.get_latest_block_id(), 1);
3055            assert_eq!(block1.transactions.len(), 100);
3056        }
3057
3058        //
3059        // block 2
3060        //
3061        let mut block2 = t
3062            .create_block(
3063                block1_hash, // hash of parent block
3064                ts + 120000, // timestamp
3065                0,           // num transactions
3066                0,           // amount
3067                0,           // fee
3068                true,        // mine golden ticket
3069            )
3070            .await;
3071        block2.generate().unwrap(); // generate hashes
3072
3073        let block2_hash = block2.hash;
3074        let block2_id = block2.id;
3075
3076        t.add_block(block2).await;
3077
3078        {
3079            let blockchain = t.blockchain_lock.write().await;
3080
3081            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3082            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3083            assert_eq!(blockchain.get_latest_block_hash(), block2_hash);
3084            assert_eq!(blockchain.get_latest_block_id(), block2_id);
3085            assert_eq!(blockchain.get_latest_block_id(), 2);
3086        }
3087
3088        //
3089        // block 3
3090        //
3091        let mut block3 = t
3092            .create_block(
3093                block2_hash, // hash of parent block
3094                ts + 240000, // timestamp
3095                1,           // num transactions
3096                0,           // amount
3097                0,           // fee
3098                false,       // mine golden ticket
3099            )
3100            .await;
3101        block3.generate().unwrap(); // generate hashes
3102
3103        let block3_hash = block3.hash;
3104        let block3_id = block3.id;
3105
3106        t.add_block(block3).await;
3107
3108        {
3109            let blockchain = t.blockchain_lock.write().await;
3110
3111            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3112            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3113            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
3114            assert_ne!(blockchain.get_latest_block_id(), block2_id);
3115            assert_eq!(blockchain.get_latest_block_hash(), block3_hash);
3116            assert_eq!(blockchain.get_latest_block_id(), block3_id);
3117            assert_eq!(blockchain.get_latest_block_id(), 3);
3118        }
3119
3120        //
3121        // block 4
3122        //
3123        let mut block4 = t
3124            .create_block(
3125                block3_hash, // hash of parent block
3126                ts + 360000, // timestamp
3127                0,           // num transactions
3128                0,           // amount
3129                0,           // fee
3130                true,        // mine golden ticket
3131            )
3132            .await;
3133        block4.generate().unwrap(); // generate hashes
3134
3135        let block4_hash = block4.hash;
3136        let block4_id = block4.id;
3137
3138        t.add_block(block4).await;
3139
3140        {
3141            let blockchain = t.blockchain_lock.write().await;
3142
3143            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3144            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3145            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
3146            assert_ne!(blockchain.get_latest_block_id(), block2_id);
3147            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
3148            assert_ne!(blockchain.get_latest_block_id(), block3_id);
3149            assert_eq!(blockchain.get_latest_block_hash(), block4_hash);
3150            assert_eq!(blockchain.get_latest_block_id(), block4_id);
3151            assert_eq!(blockchain.get_latest_block_id(), 4);
3152        }
3153
3154        //
3155        // block 5
3156        //
3157        let mut block5 = t
3158            .create_block(
3159                block4_hash, // hash of parent block
3160                ts + 480000, // timestamp
3161                1,           // num transactions
3162                0,           // amount
3163                0,           // fee
3164                false,       // mine golden ticket
3165            )
3166            .await;
3167        block5.generate().unwrap(); // generate hashes
3168
3169        let block5_hash = block5.hash;
3170        let block5_id = block5.id;
3171
3172        t.add_block(block5).await;
3173
3174        {
3175            let blockchain = t.blockchain_lock.write().await;
3176
3177            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3178            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3179            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
3180            assert_ne!(blockchain.get_latest_block_id(), block2_id);
3181            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
3182            assert_ne!(blockchain.get_latest_block_id(), block3_id);
3183            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
3184            assert_ne!(blockchain.get_latest_block_id(), block4_id);
3185            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
3186            assert_eq!(blockchain.get_latest_block_id(), block5_id);
3187            assert_eq!(blockchain.get_latest_block_id(), 5);
3188        }
3189
3190        //
3191        // block 6
3192        //
3193        let mut block6 = t
3194            .create_block(
3195                block5_hash, // hash of parent block
3196                ts + 600000, // timestamp
3197                0,           // num transactions
3198                0,           // amount
3199                0,           // fee
3200                true,        // mine golden ticket
3201            )
3202            .await;
3203        block6.generate().unwrap(); // generate hashes
3204
3205        let block6_hash = block6.hash;
3206        let block6_id = block6.id;
3207
3208        t.add_block(block6).await;
3209
3210        {
3211            let blockchain = t.blockchain_lock.write().await;
3212
3213            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3214            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3215            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
3216            assert_ne!(blockchain.get_latest_block_id(), block2_id);
3217            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
3218            assert_ne!(blockchain.get_latest_block_id(), block3_id);
3219            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
3220            assert_ne!(blockchain.get_latest_block_id(), block4_id);
3221            assert_ne!(blockchain.get_latest_block_hash(), block5_hash);
3222            assert_ne!(blockchain.get_latest_block_id(), block5_id);
3223            assert_eq!(blockchain.get_latest_block_hash(), block6_hash);
3224            assert_eq!(blockchain.get_latest_block_id(), block6_id);
3225            assert_eq!(blockchain.get_latest_block_id(), 6);
3226        }
3227
3228        //
3229        // block 7
3230        //
3231        let mut block7 = t
3232            .create_block(
3233                block6_hash, // hash of parent block
3234                ts + 720000, // timestamp
3235                1,           // num transactions
3236                0,           // amount
3237                0,           // fee
3238                false,       // mine golden ticket
3239            )
3240            .await;
3241        block7.generate().unwrap(); // generate hashes
3242
3243        let block7_hash = block7.hash;
3244        let block7_id = block7.id;
3245
3246        t.add_block(block7).await;
3247
3248        {
3249            let blockchain = t.blockchain_lock.write().await;
3250
3251            assert_ne!(blockchain.get_latest_block_hash(), block1_hash);
3252            assert_ne!(blockchain.get_latest_block_id(), block1_id);
3253            assert_ne!(blockchain.get_latest_block_hash(), block2_hash);
3254            assert_ne!(blockchain.get_latest_block_id(), block2_id);
3255            assert_ne!(blockchain.get_latest_block_hash(), block3_hash);
3256            assert_ne!(blockchain.get_latest_block_id(), block3_id);
3257            assert_ne!(blockchain.get_latest_block_hash(), block4_hash);
3258            assert_ne!(blockchain.get_latest_block_id(), block4_id);
3259            assert_ne!(blockchain.get_latest_block_hash(), block5_hash);
3260            assert_ne!(blockchain.get_latest_block_id(), block5_id);
3261            assert_ne!(blockchain.get_latest_block_hash(), block6_hash);
3262            assert_ne!(blockchain.get_latest_block_id(), block6_id);
3263            assert_eq!(blockchain.get_latest_block_hash(), block7_hash);
3264            assert_eq!(blockchain.get_latest_block_id(), block7_id);
3265            assert_eq!(blockchain.get_latest_block_id(), 7);
3266        }
3267
3268        t.check_blockchain().await;
3269        t.check_utxoset().await;
3270        t.check_token_supply().await;
3271    }
3272
3273    #[tokio::test]
3274    #[serial_test::serial]
3275    async fn block_add_test_staking() {
3276        // pretty_env_logger::init();
3277
3278        debug!("testing block_add_test_staking");
3279
3280        let mut t = TestManager::default();
3281        t.enable_staking(2_000_000 * NOLAN_PER_SAITO).await;
3282        let block1;
3283        let block1_hash;
3284        let ts;
3285
3286        t.initialize(100, 200_000_000_000_000).await;
3287        t.enable_staking(2_000_000 * NOLAN_PER_SAITO).await;
3288
3289        {
3290            let blockchain = t.blockchain_lock.read().await;
3291
3292            if blockchain.social_stake_requirement == 0 {
3293                // this test won't pass if staking is not available
3294                return;
3295            }
3296
3297            assert_eq!(blockchain.blocks.len(), 1);
3298
3299            block1 = blockchain.get_latest_block().unwrap();
3300            block1_hash = block1.hash;
3301            ts = block1.timestamp;
3302        }
3303
3304        // block 2
3305        let mut block2 = t
3306            .create_block_with_staking(
3307                block1_hash, // hash of parent block
3308                ts + 120000, // timestamp
3309                0,           // num transactions
3310                0,           // amount
3311                0,           // fee
3312                true,        // mine golden ticket
3313                false,
3314            )
3315            .await;
3316
3317        block2.generate().unwrap();
3318
3319        let block2_hash = block2.hash;
3320        assert!(!block2.has_staking_transaction);
3321
3322        let result = t.add_block(block2).await;
3323        assert!(matches!(result, AddBlockResult::FailedNotValid));
3324
3325        {
3326            let blockchain = t.blockchain_lock.read().await;
3327            assert_eq!(blockchain.blocks.len(), 1);
3328
3329            assert_eq!(blockchain.get_latest_block_hash(), block1_hash);
3330            assert_eq!(blockchain.get_latest_block_id(), 1);
3331        }
3332
3333        let mut block2 = t
3334            .create_block(
3335                block1_hash, // hash of parent block
3336                ts + 120000, // timestamp
3337                0,           // num transactions
3338                0,           // amount
3339                0,           // fee
3340                true,        // mine golden ticket
3341            )
3342            .await;
3343
3344        block2.generate().unwrap(); // generate hashes
3345
3346        assert!(block2.has_staking_transaction);
3347
3348        let block2_hash = block2.hash;
3349        let result = t.add_block(block2).await;
3350        assert!(matches!(
3351            result,
3352            AddBlockResult::BlockAddedSuccessfully(_, _, _)
3353        ));
3354
3355        {
3356            let blockchain = t.blockchain_lock.read().await;
3357            assert_eq!(blockchain.blocks.len(), 2);
3358
3359            assert_eq!(blockchain.get_latest_block_hash(), block2_hash);
3360            assert_eq!(blockchain.get_latest_block_id(), 2);
3361        }
3362    }
3363
3364    #[tokio::test]
3365    #[serial_test::serial]
3366    // add 6 blocks including 4 block reorg
3367    async fn basic_longest_chain_reorg_test() {
3368        // pretty_env_logger::init();
3369
3370        let mut t = TestManager::default();
3371        let block1;
3372        let block1_hash;
3373        let ts;
3374
3375        // block 1
3376        t.initialize(100, 200_000_000_000_000).await;
3377
3378        {
3379            let blockchain = t.blockchain_lock.read().await;
3380
3381            block1 = blockchain.get_latest_block().unwrap();
3382            block1_hash = block1.hash;
3383            ts = block1.timestamp;
3384        }
3385
3386        // block 2
3387        let mut block2 = t
3388            .create_block(
3389                block1_hash, // hash of parent block
3390                ts + 120000, // timestamp
3391                0,           // num transactions
3392                0,           // amount
3393                0,           // fee
3394                true,        // mine golden ticket
3395            )
3396            .await;
3397
3398        block2.generate().unwrap(); // generate hashes
3399
3400        let block2_hash = block2.hash;
3401        let block2_id = block2.id;
3402
3403        t.add_block(block2).await;
3404
3405        {
3406            let blockchain = t.blockchain_lock.read().await;
3407
3408            assert_eq!(blockchain.get_latest_block_hash(), block2_hash);
3409            assert_eq!(blockchain.get_latest_block_id(), block2_id);
3410        }
3411
3412        // block 3
3413        let mut block3 = t
3414            .create_block(
3415                block2_hash, // hash of parent block
3416                ts + 240000, // timestamp
3417                1,           // num transactions
3418                0,           // amount
3419                0,           // fee
3420                false,       // mine golden ticket
3421            )
3422            .await;
3423        block3.generate().unwrap(); // generate hashes
3424        let block3_hash = block3.hash;
3425        let _block3_id = block3.id;
3426        t.add_block(block3).await;
3427
3428        // block 4
3429        let mut block4 = t
3430            .create_block(
3431                block3_hash, // hash of parent block
3432                ts + 360000, // timestamp
3433                0,           // num transactions
3434                0,           // amount
3435                0,           // fee
3436                true,        // mine golden ticket
3437            )
3438            .await;
3439        block4.generate().unwrap(); // generate hashes
3440        let block4_hash = block4.hash;
3441        let _block4_id = block4.id;
3442        t.add_block(block4).await;
3443
3444        // block 5
3445        let mut block5 = t
3446            .create_block(
3447                block4_hash, // hash of parent block
3448                ts + 480000, // timestamp
3449                1,           // num transactions
3450                0,           // amount
3451                0,           // fee
3452                false,       // mine golden ticket
3453            )
3454            .await;
3455        block5.generate().unwrap(); // generate hashes
3456        let block5_hash = block5.hash;
3457        let block5_id = block5.id;
3458        t.add_block(block5).await;
3459
3460        {
3461            let blockchain = t.blockchain_lock.read().await;
3462
3463            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
3464            assert_eq!(blockchain.get_latest_block_id(), block5_id);
3465        }
3466
3467        //  block3-2
3468        let mut block3_2 = t
3469            .create_block(
3470                block2_hash, // hash of parent block
3471                ts + 240000, // timestamp
3472                0,           // num transactions
3473                0,           // amount
3474                0,           // fee
3475                true,        // mine golden ticket
3476            )
3477            .await;
3478        block3_2.generate().unwrap(); // generate hashes
3479        let block3_2_hash = block3_2.hash;
3480        let _block3_2_id = block3_2.id;
3481        t.add_block(block3_2).await;
3482
3483        {
3484            let blockchain = t.blockchain_lock.read().await;
3485
3486            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
3487            assert_eq!(blockchain.get_latest_block_id(), block5_id);
3488        }
3489
3490        //  block4-2
3491        let mut block4_2 = t
3492            .create_block(
3493                block3_2_hash, // hash of parent block
3494                ts + 360000,   // timestamp
3495                0,             // num transactions
3496                0,             // amount
3497                0,             // fee
3498                true,          // mine golden ticket
3499            )
3500            .await;
3501        block4_2.generate().unwrap(); // generate hashes
3502        let block4_2_hash = block4_2.hash;
3503        let _block4_2_id = block4_2.id;
3504        t.add_block(block4_2).await;
3505
3506        {
3507            let blockchain = t.blockchain_lock.read().await;
3508
3509            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
3510            assert_eq!(blockchain.get_latest_block_id(), block5_id);
3511        }
3512
3513        //  block5-2
3514        let mut block5_2 = t
3515            .create_block(
3516                block4_2_hash, // hash of parent block
3517                ts + 480000,   // timestamp
3518                1,             // num transactions
3519                0,             // amount
3520                0,             // fee
3521                false,         // mine golden ticket
3522            )
3523            .await;
3524        block5_2.generate().unwrap(); // generate hashes
3525        let block5_2_hash = block5_2.hash;
3526        let _block5_2_id = block5_2.id;
3527        t.add_block(block5_2).await;
3528
3529        {
3530            let blockchain = t.blockchain_lock.read().await;
3531
3532            assert_eq!(blockchain.get_latest_block_hash(), block5_hash);
3533            assert_eq!(blockchain.get_latest_block_id(), block5_id);
3534        }
3535
3536        //  block6_2
3537        let mut block6_2 = t
3538            .create_block(
3539                block5_2_hash, // hash of parent block
3540                ts + 600000,   // timestamp
3541                0,             // num transactions
3542                0,             // amount
3543                0,             // fee
3544                true,          // mine golden ticket
3545            )
3546            .await;
3547        block6_2.generate().unwrap(); // generate hashes
3548        let block6_2_hash = block6_2.hash;
3549        let block6_2_id = block6_2.id;
3550        t.add_block(block6_2).await;
3551
3552        {
3553            let blockchain = t.blockchain_lock.read().await;
3554
3555            assert_eq!(blockchain.get_latest_block_hash(), block6_2_hash);
3556            assert_eq!(blockchain.get_latest_block_id(), block6_2_id);
3557            assert_eq!(blockchain.get_latest_block_id(), 6);
3558        }
3559
3560        t.check_blockchain().await;
3561        t.check_utxoset().await;
3562        t.check_token_supply().await;
3563    }
3564
3565    /// Loading blocks into a blockchain which were created from another blockchain instance
3566    #[tokio::test]
3567    #[serial_test::serial]
3568    async fn load_blocks_from_another_blockchain_test() {
3569        // pretty_env_logger::init();
3570        let mut t = TestManager::default();
3571        let mut t2 = TestManager::default();
3572        let block1;
3573        let block1_id;
3574        let block1_hash;
3575        let ts;
3576
3577        // block 1
3578        t.initialize(100, 1_000_000_000).await;
3579        t2.disable_staking().await;
3580
3581        {
3582            let blockchain = t.blockchain_lock.write().await;
3583
3584            block1 = blockchain.get_latest_block().unwrap();
3585            block1_id = block1.id;
3586            block1_hash = block1.hash;
3587            ts = block1.timestamp;
3588        }
3589
3590        // block 2
3591        let mut block2 = t
3592            .create_block(
3593                block1_hash, // hash of parent block
3594                ts + 120000, // timestamp
3595                0,           // num transactions
3596                0,           // amount
3597                0,           // fee
3598                true,        // mine golden ticket
3599            )
3600            .await;
3601        block2.generate().unwrap(); // generate hashes
3602
3603        let block2_hash = block2.hash;
3604        let _block2_id = block2.id;
3605
3606        t.add_block(block2).await;
3607
3608        let list = t2.storage.load_block_name_list().await.unwrap();
3609        t2.storage
3610            .load_blocks_from_disk(list.as_slice(), t2.mempool_lock.clone())
3611            .await;
3612        {
3613            let configs = t2.config_lock.read().await;
3614            let mut blockchain2 = t2.blockchain_lock.write().await;
3615
3616            blockchain2
3617                .add_blocks_from_mempool(
3618                    t2.mempool_lock.clone(),
3619                    Some(&t2.network),
3620                    &mut t2.storage,
3621                    Some(t2.sender_to_miner.clone()),
3622                    None,
3623                    configs.deref(),
3624                )
3625                .await;
3626        }
3627
3628        {
3629            let blockchain1 = t.blockchain_lock.read().await;
3630            let blockchain2 = t2.blockchain_lock.read().await;
3631
3632            assert_eq!(blockchain1.blocks.len(), 2);
3633            assert_eq!(blockchain2.blocks.len(), 2);
3634
3635            let block1_chain1 = blockchain1.get_block(&block1_hash).unwrap();
3636            let block1_chain2 = blockchain2.get_block(&block1_hash).unwrap();
3637
3638            let block2_chain1 = blockchain1.get_block(&block2_hash).unwrap();
3639            let block2_chain2 = blockchain2.get_block(&block2_hash).unwrap();
3640
3641            for (block_new, block_old) in [
3642                (block1_chain2, block1_chain1),
3643                (block2_chain2, block2_chain1),
3644            ] {
3645                assert_eq!(block_new.hash, block_old.hash);
3646                assert_eq!(block_new.has_golden_ticket, block_old.has_golden_ticket);
3647                assert_eq!(block_new.previous_block_hash, block_old.previous_block_hash);
3648                assert_eq!(block_new.block_type, block_old.block_type);
3649                assert_eq!(block_new.signature, block_old.signature);
3650            }
3651        }
3652    }
3653
3654    #[tokio::test]
3655    #[serial_test::serial]
3656    async fn fork_id_test() {
3657        // pretty_env_logger::init();
3658
3659        let mut t = TestManager::default();
3660        let mut block1;
3661        let mut block1_id;
3662        let mut block1_hash;
3663        let mut ts;
3664
3665        t.initialize_with_timestamp(100, 200_000_000_000_000, 10_000_000)
3666            .await;
3667
3668        for _i in (0..20).step_by(1) {
3669            {
3670                let blockchain = t.blockchain_lock.read().await;
3671
3672                block1 = blockchain.get_latest_block().unwrap();
3673                block1_hash = block1.hash;
3674                block1_id = block1.id;
3675                ts = block1.timestamp;
3676            }
3677
3678            let mut block = t
3679                .create_block(
3680                    block1_hash, // hash of parent block
3681                    ts + 120000, // timestamp
3682                    0,           // num transactions
3683                    0,           // amount
3684                    0,           // fee
3685                    true,        // mine golden ticket
3686                )
3687                .await;
3688            block.generate().unwrap(); // generate hashes
3689
3690            let _block_hash = block.hash;
3691            let _block_id = block.id;
3692
3693            t.add_block(block).await;
3694
3695            let _result = t.receiver_in_miner.try_recv();
3696        }
3697
3698        {
3699            let blockchain = t.blockchain_lock.read().await;
3700
3701            let fork_id = blockchain.generate_fork_id(15).unwrap();
3702            assert_eq!(fork_id[2..], [0; 30]);
3703
3704            let fork_id = blockchain.generate_fork_id(20).unwrap();
3705            assert_eq!(fork_id[4..], [0; 28]);
3706        }
3707    }
3708
3709    //create a test genesis block and test store state and reload from the same file
3710    #[tokio::test]
3711    #[serial_test::serial]
3712    async fn test_genesis_inout() {
3713        //init_testlog();
3714
3715        let mut t = TestManager::default();
3716        //generate a test genesis block
3717        t.create_test_gen_block(1000).await;
3718        {
3719            let blockchain = t.blockchain_lock.read().await;
3720
3721            let block1 = blockchain.get_latest_block().unwrap();
3722            assert_eq!(block1.id, 1);
3723            assert!(block1.timestamp > 1687867265673);
3724            assert_eq!(block1.transactions.len(), 1);
3725        }
3726
3727        //create the balance map
3728        let bmap = t.balance_map().await;
3729
3730        //store it
3731        let filepath = "./utxoset_test";
3732
3733        match t
3734            .storage
3735            .write_utxoset_to_disk_path(bmap, 1, filepath)
3736            .await
3737        {
3738            Ok(_) => {
3739                debug!("store file ok");
3740            }
3741            Err(e) => {
3742                error!("Error: {:?}", e);
3743            }
3744        }
3745
3746        //now assume the stored map is issued as issued, pass it in
3747
3748        //convert_issuance_into_slip
3749        let slips: Vec<Slip> = t
3750            .storage
3751            .get_token_supply_slips_from_disk_path(filepath)
3752            .await;
3753        assert_eq!(slips.len(), 1);
3754
3755        //TODO more tests on slips
3756
3757        //clean up the testing file
3758        let _ = fs::remove_file(filepath);
3759    }
3760
3761    #[tokio::test]
3762    #[serial_test::serial]
3763    async fn ghost_chain_hash_test() {
3764        // pretty_env_logger::init();
3765        let mut t = TestManager::default();
3766        let block1;
3767        let parent_block_hash;
3768        let parent_block_id;
3769        let mut ts;
3770
3771        // block 1
3772        t.initialize(100, 200_000_000_000_000).await;
3773
3774        {
3775            let blockchain = t.blockchain_lock.write().await;
3776
3777            block1 = blockchain.get_latest_block().unwrap();
3778            parent_block_hash = block1.hash;
3779            parent_block_id = block1.id;
3780            ts = block1.timestamp;
3781        }
3782
3783        for _i in 0..10 {
3784            let mut block2 = t
3785                .create_block(
3786                    parent_block_hash, // hash of parent block
3787                    ts + 120000,       // timestamp
3788                    10,                // num transactions
3789                    0,                 // amount
3790                    0,                 // fee
3791                    false,             // mine golden ticket
3792                )
3793                .await;
3794            block2.id = parent_block_id + 1;
3795            info!("block generate : {:?}", block2.id);
3796            block2.generate().unwrap(); // generate hashes
3797            block2.sign(&t.wallet_lock.read().await.private_key);
3798            ts = block2.timestamp;
3799
3800            let buf = [
3801                block2.previous_block_hash.as_slice(),
3802                block2.pre_hash.as_slice(),
3803            ]
3804            .concat();
3805            let calculate_hash = hash(&buf);
3806            assert_eq!(block2.hash, calculate_hash);
3807        }
3808        // TODO : check ghost chain data here
3809    }
3810
3811    #[tokio::test]
3812    #[serial_test::serial]
3813    async fn ghost_chain_content_test() {
3814        NodeTester::delete_data().await.unwrap();
3815        let mut tester = NodeTester::default();
3816        tester
3817            .init_with_staking(2_000_000 * NOLAN_PER_SAITO, 60, 100_000 * NOLAN_PER_SAITO)
3818            .await
3819            .unwrap();
3820
3821        let fork_id_1: ForkId = tester.get_fork_id(1).await;
3822        tester.wait_till_block_id_with_txs(10, 10, 0).await.unwrap();
3823        let fork_id_1_after: ForkId = tester.get_fork_id(1).await;
3824        assert_eq!(fork_id_1, fork_id_1_after);
3825
3826        // only testing if the fork id is not changing when the blockchain is updated
3827        let fork_id = tester.get_fork_id(10).await;
3828        tester
3829            .wait_till_block_id_with_txs(100, 10, 0)
3830            .await
3831            .unwrap();
3832        let fork_id_after = tester.get_fork_id(10).await;
3833        assert_eq!(fork_id, fork_id_after);
3834
3835        {}
3836    }
3837
3838    #[tokio::test]
3839    #[serial_test::serial]
3840    async fn test_fork_id_difference() {
3841        // pretty_env_logger::init()
3842        NodeTester::delete_data().await.unwrap();
3843        let mut tester = NodeTester::default();
3844        tester
3845            .init_with_staking(2_000_000 * NOLAN_PER_SAITO, 60, 100_000 * NOLAN_PER_SAITO)
3846            .await
3847            .unwrap();
3848
3849        let fork_id_1 = tester.get_fork_id(1).await;
3850
3851        tester.wait_till_block_id_with_txs(10, 10, 0).await.unwrap();
3852
3853        let fork_id_10 = tester.get_fork_id(10).await;
3854        assert_ne!(fork_id_1.to_hex(), fork_id_10.to_hex());
3855
3856        tester
3857            .wait_till_block_id_with_txs(100, 10, 10)
3858            .await
3859            .unwrap();
3860
3861        let fork_id_100 = tester.get_fork_id(100).await;
3862        assert_ne!(fork_id_10.to_hex(), fork_id_100.to_hex());
3863    }
3864
3865    #[tokio::test]
3866    #[serial_test::serial]
3867    async fn test_block_generation_without_fees() {
3868        // pretty_env_logger::init();
3869        NodeTester::delete_data().await.unwrap();
3870        let mut tester = NodeTester::default();
3871        tester
3872            .init_with_staking(0, 60, 100_000 * NOLAN_PER_SAITO)
3873            .await
3874            .unwrap();
3875
3876        tester.wait_till_block_id_with_txs(100, 0, 0).await.unwrap()
3877    }
3878    #[tokio::test]
3879    #[serial_test::serial]
3880    async fn test_block_generation_with_fees() {
3881        // pretty_env_logger::init();
3882        NodeTester::delete_data().await.unwrap();
3883        let mut tester = NodeTester::default();
3884        tester
3885            .init_with_staking(0, 60, 100_000 * NOLAN_PER_SAITO)
3886            .await
3887            .unwrap();
3888
3889        tester.wait_till_block_id_with_txs(5, 0, 10).await.unwrap()
3890    }
3891
3892    #[tokio::test]
3893    #[serial_test::serial]
3894    async fn is_golden_ticket_count_valid_test_all_gts() {
3895        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
3896        let mut block = Block::new();
3897        block.id = 10;
3898        block.has_golden_ticket = true;
3899        block.hash = [2; 32];
3900        block.previous_block_hash = [1; 32];
3901        blocks.insert(block.hash, block);
3902
3903        let mut block = Block::new();
3904        block.id = 11;
3905        block.has_golden_ticket = true;
3906        block.hash = [3; 32];
3907        block.previous_block_hash = [2; 32];
3908        blocks.insert(block.hash, block);
3909
3910        let mut block = Block::new();
3911        block.id = 12;
3912        block.has_golden_ticket = true;
3913        block.hash = [4; 32];
3914        block.previous_block_hash = [3; 32];
3915        blocks.insert(block.hash, block);
3916
3917        let mut block = Block::new();
3918        block.id = 13;
3919        block.has_golden_ticket = true;
3920        block.hash = [5; 32];
3921        block.previous_block_hash = [4; 32];
3922        blocks.insert(block.hash, block);
3923
3924        let mut block = Block::new();
3925        block.id = 14;
3926        block.has_golden_ticket = true;
3927        block.hash = [6; 32];
3928        block.previous_block_hash = [5; 32];
3929        blocks.insert(block.hash, block);
3930
3931        let mut block = Block::new();
3932        block.id = 15;
3933        block.has_golden_ticket = true;
3934        block.hash = [7; 32];
3935        block.previous_block_hash = [6; 32];
3936        blocks.insert(block.hash, block);
3937
3938        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
3939            blocks.get(&block_hash)
3940        });
3941        assert!(result);
3942    }
3943
3944    #[tokio::test]
3945    #[serial_test::serial]
3946    async fn is_golden_ticket_count_valid_test_no_gts() {
3947        // pretty_env_logger::init();
3948        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
3949        let mut block = Block::new();
3950        block.id = 10;
3951        block.has_golden_ticket = false;
3952        block.hash = [2; 32];
3953        block.previous_block_hash = [1; 32];
3954        blocks.insert(block.hash, block);
3955
3956        let mut block = Block::new();
3957        block.id = 11;
3958        block.has_golden_ticket = false;
3959        block.hash = [3; 32];
3960        block.previous_block_hash = [2; 32];
3961        blocks.insert(block.hash, block);
3962
3963        let mut block = Block::new();
3964        block.id = 12;
3965        block.has_golden_ticket = false;
3966        block.hash = [4; 32];
3967        block.previous_block_hash = [3; 32];
3968        blocks.insert(block.hash, block);
3969
3970        let mut block = Block::new();
3971        block.id = 13;
3972        block.has_golden_ticket = false;
3973        block.hash = [5; 32];
3974        block.previous_block_hash = [4; 32];
3975        blocks.insert(block.hash, block);
3976
3977        let mut block = Block::new();
3978        block.id = 14;
3979        block.has_golden_ticket = false;
3980        block.hash = [6; 32];
3981        block.previous_block_hash = [5; 32];
3982        blocks.insert(block.hash, block);
3983
3984        let mut block = Block::new();
3985        block.id = 15;
3986        block.has_golden_ticket = false;
3987        block.hash = [7; 32];
3988        block.previous_block_hash = [6; 32];
3989        blocks.insert(block.hash, block);
3990
3991        let mut block = Block::new();
3992        block.id = 16;
3993        block.has_golden_ticket = false;
3994        block.hash = [8; 32];
3995        block.previous_block_hash = [7; 32];
3996        blocks.insert(block.hash, block);
3997
3998        // let result = is_golden_ticket_count_valid_([6; 32], false, false, |block_hash| {
3999        //     blocks.get(&block_hash)
4000        // });
4001        // assert!(result);
4002        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4003            blocks.get(&block_hash)
4004        });
4005        assert!(!result);
4006
4007        let result = is_golden_ticket_count_valid_([8; 32], false, false, |block_hash| {
4008            blocks.get(&block_hash)
4009        });
4010        assert!(!result);
4011    }
4012    #[tokio::test]
4013    #[serial_test::serial]
4014    async fn is_golden_ticket_count_valid_test_gt_in_block_and_in_hand() {
4015        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4016        let mut block = Block::new();
4017        block.id = 10;
4018        block.has_golden_ticket = false;
4019        block.hash = [2; 32];
4020        block.previous_block_hash = [1; 32];
4021        blocks.insert(block.hash, block);
4022
4023        let mut block = Block::new();
4024        block.id = 11;
4025        block.has_golden_ticket = false;
4026        block.hash = [3; 32];
4027        block.previous_block_hash = [2; 32];
4028        blocks.insert(block.hash, block);
4029
4030        let mut block = Block::new();
4031        block.id = 12;
4032        block.has_golden_ticket = false;
4033        block.hash = [4; 32];
4034        block.previous_block_hash = [3; 32];
4035        blocks.insert(block.hash, block);
4036
4037        let mut block = Block::new();
4038        block.id = 13;
4039        block.has_golden_ticket = false;
4040        block.hash = [5; 32];
4041        block.previous_block_hash = [4; 32];
4042        blocks.insert(block.hash, block);
4043
4044        let mut block = Block::new();
4045        block.id = 14;
4046        block.has_golden_ticket = false;
4047        block.hash = [6; 32];
4048        block.previous_block_hash = [5; 32];
4049        blocks.insert(block.hash, block);
4050
4051        let mut block = Block::new();
4052        block.id = 15;
4053        block.has_golden_ticket = true;
4054        block.hash = [7; 32];
4055        block.previous_block_hash = [6; 32];
4056        blocks.insert(block.hash, block);
4057
4058        let result = is_golden_ticket_count_valid_([7; 32], true, false, |block_hash| {
4059            blocks.get(&block_hash)
4060        });
4061        assert!(result);
4062    }
4063    #[tokio::test]
4064    #[serial_test::serial]
4065    async fn is_golden_ticket_count_valid_test_one_gt() {
4066        // pretty_env_logger::init();
4067        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4068        let mut block = Block::new();
4069        block.id = 10;
4070        block.has_golden_ticket = false;
4071        block.hash = [2; 32];
4072        block.previous_block_hash = [1; 32];
4073        blocks.insert(block.hash, block);
4074
4075        let mut block = Block::new();
4076        block.id = 11;
4077        block.has_golden_ticket = false;
4078        block.hash = [3; 32];
4079        block.previous_block_hash = [2; 32];
4080        blocks.insert(block.hash, block);
4081
4082        let mut block = Block::new();
4083        block.id = 12;
4084        block.has_golden_ticket = false;
4085        block.hash = [4; 32];
4086        block.previous_block_hash = [3; 32];
4087        blocks.insert(block.hash, block);
4088
4089        let mut block = Block::new();
4090        block.id = 13;
4091        block.has_golden_ticket = false;
4092        block.hash = [5; 32];
4093        block.previous_block_hash = [4; 32];
4094        blocks.insert(block.hash, block);
4095
4096        let mut block = Block::new();
4097        block.id = 14;
4098        block.has_golden_ticket = false;
4099        block.hash = [6; 32];
4100        block.previous_block_hash = [5; 32];
4101        blocks.insert(block.hash, block);
4102
4103        let mut block = Block::new();
4104        block.id = 15;
4105        block.has_golden_ticket = true;
4106        block.hash = [7; 32];
4107        block.previous_block_hash = [6; 32];
4108        blocks.insert(block.hash, block);
4109
4110        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4111            blocks.get(&block_hash)
4112        });
4113        assert!(!result);
4114    }
4115    #[tokio::test]
4116    #[serial_test::serial]
4117    async fn is_golden_ticket_count_valid_test_one_gt_in_hand() {
4118        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4119        let mut block = Block::new();
4120        block.id = 10;
4121        block.has_golden_ticket = false;
4122        block.hash = [2; 32];
4123        block.previous_block_hash = [1; 32];
4124        blocks.insert(block.hash, block);
4125
4126        let mut block = Block::new();
4127        block.id = 11;
4128        block.has_golden_ticket = false;
4129        block.hash = [3; 32];
4130        block.previous_block_hash = [2; 32];
4131        blocks.insert(block.hash, block);
4132
4133        let mut block = Block::new();
4134        block.id = 12;
4135        block.has_golden_ticket = false;
4136        block.hash = [4; 32];
4137        block.previous_block_hash = [3; 32];
4138        blocks.insert(block.hash, block);
4139
4140        let mut block = Block::new();
4141        block.id = 13;
4142        block.has_golden_ticket = false;
4143        block.hash = [5; 32];
4144        block.previous_block_hash = [4; 32];
4145        blocks.insert(block.hash, block);
4146
4147        let mut block = Block::new();
4148        block.id = 14;
4149        block.has_golden_ticket = false;
4150        block.hash = [6; 32];
4151        block.previous_block_hash = [5; 32];
4152        blocks.insert(block.hash, block);
4153
4154        let mut block = Block::new();
4155        block.id = 15;
4156        block.has_golden_ticket = false;
4157        block.hash = [7; 32];
4158        block.previous_block_hash = [6; 32];
4159        blocks.insert(block.hash, block);
4160
4161        let result = is_golden_ticket_count_valid_([7; 32], true, false, |block_hash| {
4162            blocks.get(&block_hash)
4163        });
4164        assert!(!result);
4165    }
4166    #[tokio::test]
4167    #[serial_test::serial]
4168    async fn is_golden_ticket_count_valid_test_2_gt() {
4169        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4170        let mut block = Block::new();
4171        block.id = 10;
4172        block.has_golden_ticket = false;
4173        block.hash = [2; 32];
4174        block.previous_block_hash = [1; 32];
4175        blocks.insert(block.hash, block);
4176
4177        let mut block = Block::new();
4178        block.id = 11;
4179        block.has_golden_ticket = true;
4180        block.hash = [3; 32];
4181        block.previous_block_hash = [2; 32];
4182        blocks.insert(block.hash, block);
4183
4184        let mut block = Block::new();
4185        block.id = 12;
4186        block.has_golden_ticket = false;
4187        block.hash = [4; 32];
4188        block.previous_block_hash = [3; 32];
4189        blocks.insert(block.hash, block);
4190
4191        let mut block = Block::new();
4192        block.id = 13;
4193        block.has_golden_ticket = false;
4194        block.hash = [5; 32];
4195        block.previous_block_hash = [4; 32];
4196        blocks.insert(block.hash, block);
4197
4198        let mut block = Block::new();
4199        block.id = 14;
4200        block.has_golden_ticket = false;
4201        block.hash = [6; 32];
4202        block.previous_block_hash = [5; 32];
4203        blocks.insert(block.hash, block);
4204
4205        let mut block = Block::new();
4206        block.id = 15;
4207        block.has_golden_ticket = true;
4208        block.hash = [7; 32];
4209        block.previous_block_hash = [6; 32];
4210        blocks.insert(block.hash, block);
4211
4212        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4213            blocks.get(&block_hash)
4214        });
4215        assert!(result);
4216    }
4217    #[tokio::test]
4218    #[serial_test::serial]
4219    async fn is_golden_ticket_count_valid_test_3_gt() {
4220        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4221        let mut block = Block::new();
4222        block.id = 98;
4223        block.has_golden_ticket = false;
4224        block.hash = [2; 32];
4225        block.previous_block_hash = [1; 32];
4226        blocks.insert(block.hash, block);
4227
4228        let mut block = Block::new();
4229        block.id = 99;
4230        block.has_golden_ticket = false;
4231        block.hash = [3; 32];
4232        block.previous_block_hash = [2; 32];
4233        blocks.insert(block.hash, block);
4234
4235        let mut block = Block::new();
4236        block.id = 100;
4237        block.has_golden_ticket = false;
4238        block.hash = [4; 32];
4239        block.previous_block_hash = [3; 32];
4240        blocks.insert(block.hash, block);
4241
4242        let mut block = Block::new();
4243        block.id = 101;
4244        block.has_golden_ticket = true;
4245        block.hash = [5; 32];
4246        block.previous_block_hash = [4; 32];
4247        blocks.insert(block.hash, block);
4248
4249        let mut block = Block::new();
4250        block.id = 102;
4251        block.has_golden_ticket = false;
4252        block.hash = [6; 32];
4253        block.previous_block_hash = [5; 32];
4254        blocks.insert(block.hash, block);
4255
4256        let mut block = Block::new();
4257        block.id = 103;
4258        block.has_golden_ticket = false;
4259        block.hash = [7; 32];
4260        block.previous_block_hash = [6; 32];
4261        blocks.insert(block.hash, block);
4262
4263        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4264            blocks.get(&block_hash)
4265        });
4266        assert!(!result);
4267    }
4268    #[ignore]
4269    #[tokio::test]
4270    #[serial_test::serial]
4271    async fn is_golden_ticket_count_valid_test_faraway_gt() {
4272        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4273        let mut block = Block::new();
4274        block.id = 10;
4275        block.has_golden_ticket = true;
4276        block.hash = [2; 32];
4277        block.previous_block_hash = [1; 32];
4278        blocks.insert(block.hash, block);
4279
4280        let mut block = Block::new();
4281        block.id = 11;
4282        block.has_golden_ticket = false;
4283        block.hash = [3; 32];
4284        block.previous_block_hash = [2; 32];
4285        blocks.insert(block.hash, block);
4286
4287        let mut block = Block::new();
4288        block.id = 12;
4289        block.has_golden_ticket = false;
4290        block.hash = [4; 32];
4291        block.previous_block_hash = [3; 32];
4292        blocks.insert(block.hash, block);
4293
4294        let mut block = Block::new();
4295        block.id = 13;
4296        block.has_golden_ticket = false;
4297        block.hash = [5; 32];
4298        block.previous_block_hash = [4; 32];
4299        blocks.insert(block.hash, block);
4300
4301        let mut block = Block::new();
4302        block.id = 14;
4303        block.has_golden_ticket = false;
4304        block.hash = [6; 32];
4305        block.previous_block_hash = [5; 32];
4306        blocks.insert(block.hash, block);
4307
4308        let mut block = Block::new();
4309        block.id = 15;
4310        block.has_golden_ticket = true;
4311        block.hash = [7; 32];
4312        block.previous_block_hash = [6; 32];
4313        blocks.insert(block.hash, block);
4314
4315        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4316            blocks.get(&block_hash)
4317        });
4318        assert!(!result);
4319    }
4320    #[ignore]
4321    #[tokio::test]
4322    #[serial_test::serial]
4323    async fn is_golden_ticket_count_valid_test_faraway_gt_with_one_in_hand() {
4324        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4325        let mut block = Block::new();
4326        block.id = 10;
4327        block.has_golden_ticket = true;
4328        block.hash = [2; 32];
4329        block.previous_block_hash = [1; 32];
4330        blocks.insert(block.hash, block);
4331
4332        let mut block = Block::new();
4333        block.id = 11;
4334        block.has_golden_ticket = false;
4335        block.hash = [3; 32];
4336        block.previous_block_hash = [2; 32];
4337        blocks.insert(block.hash, block);
4338
4339        let mut block = Block::new();
4340        block.id = 12;
4341        block.has_golden_ticket = false;
4342        block.hash = [4; 32];
4343        block.previous_block_hash = [3; 32];
4344        blocks.insert(block.hash, block);
4345
4346        let mut block = Block::new();
4347        block.id = 13;
4348        block.has_golden_ticket = false;
4349        block.hash = [5; 32];
4350        block.previous_block_hash = [4; 32];
4351        blocks.insert(block.hash, block);
4352
4353        let mut block = Block::new();
4354        block.id = 14;
4355        block.has_golden_ticket = false;
4356        block.hash = [6; 32];
4357        block.previous_block_hash = [5; 32];
4358        blocks.insert(block.hash, block);
4359
4360        let mut block = Block::new();
4361        block.id = 15;
4362        block.has_golden_ticket = false;
4363        block.hash = [7; 32];
4364        block.previous_block_hash = [6; 32];
4365        blocks.insert(block.hash, block);
4366
4367        let result = is_golden_ticket_count_valid_([7; 32], true, false, |block_hash| {
4368            blocks.get(&block_hash)
4369        });
4370        assert!(!result);
4371    }
4372    #[tokio::test]
4373    #[serial_test::serial]
4374    async fn is_golden_ticket_count_valid_test_faraway_gt_with_one_in_hand_2() {
4375        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4376        let mut block = Block::new();
4377        block.id = 10;
4378        block.has_golden_ticket = true;
4379        block.hash = [2; 32];
4380        block.previous_block_hash = [1; 32];
4381        blocks.insert(block.hash, block);
4382
4383        let mut block = Block::new();
4384        block.id = 11;
4385        block.has_golden_ticket = true;
4386        block.hash = [3; 32];
4387        block.previous_block_hash = [2; 32];
4388        blocks.insert(block.hash, block);
4389
4390        let mut block = Block::new();
4391        block.id = 12;
4392        block.has_golden_ticket = false;
4393        block.hash = [4; 32];
4394        block.previous_block_hash = [3; 32];
4395        blocks.insert(block.hash, block);
4396
4397        let mut block = Block::new();
4398        block.id = 13;
4399        block.has_golden_ticket = false;
4400        block.hash = [5; 32];
4401        block.previous_block_hash = [4; 32];
4402        blocks.insert(block.hash, block);
4403
4404        let mut block = Block::new();
4405        block.id = 14;
4406        block.has_golden_ticket = false;
4407        block.hash = [6; 32];
4408        block.previous_block_hash = [5; 32];
4409        blocks.insert(block.hash, block);
4410
4411        let mut block = Block::new();
4412        block.id = 15;
4413        block.has_golden_ticket = false;
4414        block.hash = [7; 32];
4415        block.previous_block_hash = [6; 32];
4416        blocks.insert(block.hash, block);
4417
4418        let result = is_golden_ticket_count_valid_([7; 32], true, false, |block_hash| {
4419            blocks.get(&block_hash)
4420        });
4421        assert!(result);
4422    }
4423    #[ignore]
4424    #[tokio::test]
4425    #[serial_test::serial]
4426    async fn is_golden_ticket_count_valid_test_early_blocks() {
4427        // pretty_env_logger::init();
4428        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4429        let mut block = Block::new();
4430        block.id = 1;
4431        block.has_golden_ticket = true;
4432        block.hash = [2; 32];
4433        block.previous_block_hash = [1; 32];
4434        blocks.insert(block.hash, block);
4435
4436        let mut block = Block::new();
4437        block.id = 2;
4438        block.has_golden_ticket = true;
4439        block.hash = [3; 32];
4440        block.previous_block_hash = [2; 32];
4441        blocks.insert(block.hash, block);
4442
4443        let mut block = Block::new();
4444        block.id = 3;
4445        block.has_golden_ticket = false;
4446        block.hash = [4; 32];
4447        block.previous_block_hash = [3; 32];
4448        blocks.insert(block.hash, block);
4449
4450        let mut block = Block::new();
4451        block.id = 4;
4452        block.has_golden_ticket = false;
4453        block.hash = [5; 32];
4454        block.previous_block_hash = [4; 32];
4455        blocks.insert(block.hash, block);
4456
4457        let mut block = Block::new();
4458        block.id = 5;
4459        block.has_golden_ticket = false;
4460        block.hash = [6; 32];
4461        block.previous_block_hash = [5; 32];
4462        blocks.insert(block.hash, block);
4463
4464        let mut block = Block::new();
4465        block.id = 6;
4466        block.has_golden_ticket = false;
4467        block.hash = [7; 32];
4468        block.previous_block_hash = [6; 32];
4469        blocks.insert(block.hash, block);
4470
4471        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4472            blocks.get(&block_hash)
4473        });
4474        assert!(!result);
4475    }
4476    #[tokio::test]
4477    #[serial_test::serial]
4478    async fn is_golden_ticket_count_valid_test_early_blocks_2() {
4479        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4480        let mut block = Block::new();
4481        block.id = 1;
4482        block.has_golden_ticket = false;
4483        block.hash = [2; 32];
4484        block.previous_block_hash = [1; 32];
4485        blocks.insert(block.hash, block);
4486
4487        let mut block = Block::new();
4488        block.id = 2;
4489        block.has_golden_ticket = false;
4490        block.hash = [3; 32];
4491        block.previous_block_hash = [2; 32];
4492        blocks.insert(block.hash, block);
4493
4494        let mut block = Block::new();
4495        block.id = 3;
4496        block.has_golden_ticket = false;
4497        block.hash = [4; 32];
4498        block.previous_block_hash = [3; 32];
4499        blocks.insert(block.hash, block);
4500
4501        let mut block = Block::new();
4502        block.id = 4;
4503        block.has_golden_ticket = false;
4504        block.hash = [5; 32];
4505        block.previous_block_hash = [4; 32];
4506        blocks.insert(block.hash, block);
4507
4508        let result = is_golden_ticket_count_valid_([7; 32], false, false, |block_hash| {
4509            blocks.get(&block_hash)
4510        });
4511        assert!(result);
4512    }
4513    #[ignore]
4514    #[tokio::test]
4515    #[serial_test::serial]
4516    async fn is_golden_ticket_count_valid_test_early_blocks_3() {
4517        let mut blocks: HashMap<SaitoHash, Block> = Default::default();
4518        let mut block = Block::new();
4519        block.id = 1;
4520        block.has_golden_ticket = true;
4521        block.hash = [2; 32];
4522        block.previous_block_hash = [1; 32];
4523        blocks.insert(block.hash, block);
4524
4525        let mut block = Block::new();
4526        block.id = 2;
4527        block.has_golden_ticket = false;
4528        block.hash = [3; 32];
4529        block.previous_block_hash = [2; 32];
4530        blocks.insert(block.hash, block);
4531
4532        let mut block = Block::new();
4533        block.id = 3;
4534        block.has_golden_ticket = false;
4535        block.hash = [4; 32];
4536        block.previous_block_hash = [3; 32];
4537        blocks.insert(block.hash, block);
4538
4539        let mut block = Block::new();
4540        block.id = 4;
4541        block.has_golden_ticket = false;
4542        block.hash = [5; 32];
4543        block.previous_block_hash = [4; 32];
4544        blocks.insert(block.hash, block);
4545
4546        let mut block = Block::new();
4547        block.id = 5;
4548        block.has_golden_ticket = false;
4549        block.hash = [6; 32];
4550        block.previous_block_hash = [5; 32];
4551        blocks.insert(block.hash, block);
4552
4553        let mut block = Block::new();
4554        block.id = 6;
4555        block.has_golden_ticket = false;
4556        block.hash = [7; 32];
4557        block.previous_block_hash = [6; 32];
4558        blocks.insert(block.hash, block);
4559
4560        let result = is_golden_ticket_count_valid_([7; 32], true, false, |block_hash| {
4561            blocks.get(&block_hash)
4562        });
4563        assert!(!result);
4564    }
4565}