fixed broken logs (#7934)

* fixed broken logs

* bring back old lock order

* removed bloom groups from blockchain

* revert unrelated changes

* simplify blockchain_block_blooms
This commit is contained in:
Marek Kotewicz 2018-02-22 11:22:56 +01:00 committed by Rando
parent e0b4506474
commit f8a2e53f3e
7 changed files with 79 additions and 168 deletions

View File

@ -20,7 +20,6 @@ use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use std::mem;
use itertools::Itertools;
use bloomchain as bc;
use heapsize::HeapSizeOf;
use ethereum_types::{H256, Bloom, U256};
use parking_lot::{Mutex, RwLock};
@ -31,7 +30,6 @@ use transaction::*;
use views::*;
use log_entry::{LogEntry, LocalizedLogEntry};
use receipt::Receipt;
use blooms::{BloomGroup, GroupPosition};
use blockchain::best_block::{BestBlock, BestAncientBlock};
use blockchain::block_info::{BlockInfo, BlockLocation, BranchBecomingCanonChainData};
use blockchain::extras::{BlockReceipts, BlockDetails, TransactionAddress, EPOCH_KEY_PREFIX, EpochTransitions};
@ -47,9 +45,6 @@ use rayon::prelude::*;
use ansi_term::Colour;
use kvdb::{DBTransaction, KeyValueDB};
const LOG_BLOOMS_LEVELS: usize = 3;
const LOG_BLOOMS_ELEMENTS_PER_INDEX: usize = 16;
/// Interface for querying blocks by hash and by number.
pub trait BlockProvider {
/// Returns true if the given block is known
@ -149,7 +144,7 @@ pub trait BlockProvider {
}
/// Returns numbers of blocks containing given bloom.
fn blocks_with_bloom(&self, bloom: &Bloom, from_block: BlockNumber, to_block: BlockNumber) -> Vec<BlockNumber>;
fn blocks_with_blooms(&self, bloom: &[Bloom], from_block: BlockNumber, to_block: BlockNumber) -> Vec<BlockNumber>;
/// Returns logs matching given filter.
fn logs<F>(&self, blocks: Vec<BlockNumber>, matches: F, limit: Option<usize>) -> Vec<LocalizedLogEntry>
@ -163,26 +158,14 @@ enum CacheId {
BlockDetails(H256),
BlockHashes(BlockNumber),
TransactionAddresses(H256),
BlocksBlooms(GroupPosition),
BlockReceipts(H256),
}
impl bc::group::BloomGroupDatabase for BlockChain {
fn blooms_at(&self, position: &bc::group::GroupPosition) -> Option<bc::group::BloomGroup> {
let position = GroupPosition::from(position.clone());
let result = self.db.read_with_cache(db::COL_EXTRA, &self.blocks_blooms, &position).map(Into::into);
self.cache_man.lock().note_used(CacheId::BlocksBlooms(position));
result
}
}
/// Structure providing fast access to blockchain data.
///
/// **Does not do input data verification.**
pub struct BlockChain {
// All locks must be captured in the order declared here.
blooms_config: bc::Config,
best_block: RwLock<BestBlock>,
// Stores best block of the first uninterrupted sequence of blocks. `None` if there are no gaps.
// Only updated with `insert_unordered_block`.
@ -199,7 +182,6 @@ pub struct BlockChain {
block_details: RwLock<HashMap<H256, BlockDetails>>,
block_hashes: RwLock<HashMap<BlockNumber, H256>>,
transaction_addresses: RwLock<HashMap<H256, TransactionAddress>>,
blocks_blooms: RwLock<HashMap<GroupPosition, BloomGroup>>,
block_receipts: RwLock<HashMap<H256, BlockReceipts>>,
db: Arc<KeyValueDB>,
@ -349,13 +331,14 @@ impl BlockProvider for BlockChain {
result
}
/// Returns numbers of blocks containing given bloom.
fn blocks_with_bloom(&self, bloom: &Bloom, from_block: BlockNumber, to_block: BlockNumber) -> Vec<BlockNumber> {
let range = from_block as bc::Number..to_block as bc::Number;
let chain = bc::group::BloomGroupChain::new(self.blooms_config, self);
chain.with_bloom(&range, bloom)
.into_iter()
.map(|b| b as BlockNumber)
fn blocks_with_blooms(&self, blooms: &[Bloom], from_block: BlockNumber, to_block: BlockNumber) -> Vec<BlockNumber> {
// +1, cause it's inclusive range
(from_block..to_block + 1)
.into_par_iter()
.filter_map(|number| self.block_hash(number).map(|hash| (number, hash)))
.map(|(number, hash)| (number, self.block_header_data(&hash).expect("hash exists; qed")))
.filter(|&(_, ref header)| blooms.iter().any(|bloom| header.view().log_bloom().contains_bloom(bloom)))
.map(|(number, _)| number)
.collect()
}
@ -371,19 +354,15 @@ impl BlockProvider for BlockChain {
.filter_map(|number| self.block_hash(*number).map(|hash| (*number, hash)))
.filter_map(|(number, hash)| self.block_receipts(&hash).map(|r| (number, hash, r.receipts)))
.filter_map(|(number, hash, receipts)| self.block_body(&hash).map(|ref b| (number, hash, receipts, b.transaction_hashes())))
.flat_map(|(number, hash, mut receipts, mut hashes)| {
if receipts.len() != hashes.len() {
warn!("Block {} ({}) has different number of receipts ({}) to transactions ({}). Database corrupt?", number, hash, receipts.len(), hashes.len());
assert!(false);
}
let mut log_index = receipts.iter().fold(0, |sum, receipt| sum + receipt.logs.len());
.flat_map(|(number, hash, receipts, hashes)| {
assert_eq!(receipts.len(), hashes.len(), "Block {} ({}) has different number of receipts ({}) to transactions ({})", number, hash, receipts.len(), hashes.len());
let mut log_index: usize = receipts.iter().map(|r| r.logs.len()).sum();
let receipts_len = receipts.len();
hashes.reverse();
receipts.reverse();
receipts.into_iter()
.map(|receipt| receipt.logs)
.zip(hashes)
.rev()
.enumerate()
.flat_map(move |(index, (mut logs, tx_hash))| {
let current_log_index = log_index;
@ -487,10 +466,6 @@ impl BlockChain {
let cache_man = CacheManager::new(config.pref_cache_size, config.max_cache_size, 400);
let mut bc = BlockChain {
blooms_config: bc::Config {
levels: LOG_BLOOMS_LEVELS,
elements_per_index: LOG_BLOOMS_ELEMENTS_PER_INDEX,
},
first_block: None,
best_block: RwLock::new(BestBlock::default()),
best_ancient_block: RwLock::new(None),
@ -499,7 +474,6 @@ impl BlockChain {
block_details: RwLock::new(HashMap::new()),
block_hashes: RwLock::new(HashMap::new()),
transaction_addresses: RwLock::new(HashMap::new()),
blocks_blooms: RwLock::new(HashMap::new()),
block_receipts: RwLock::new(HashMap::new()),
db: db.clone(),
cache_man: Mutex::new(cache_man),
@ -714,6 +688,7 @@ impl BlockChain {
/// This is used by snapshot restoration and when downloading missing blocks for the chain gap.
/// `is_best` forces the best block to be updated to this block.
/// `is_ancient` forces the best block of the first block sequence to be updated to this block.
/// `parent_td` is a parent total diffuculty
/// Supply a dummy parent total difficulty when the parent block may not be in the chain.
/// Returns true if the block is disconnected.
pub fn insert_unordered_block(&self, batch: &mut DBTransaction, bytes: &[u8], receipts: Vec<Receipt>, parent_td: Option<U256>, is_best: bool, is_ancient: bool) -> bool {
@ -750,7 +725,6 @@ impl BlockChain {
block_hashes: self.prepare_block_hashes_update(bytes, &info),
block_details: self.prepare_block_details_update(bytes, &info),
block_receipts: self.prepare_block_receipts_update(receipts, &info),
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info,
timestamp: header.timestamp(),
@ -799,7 +773,6 @@ impl BlockChain {
block_hashes: self.prepare_block_hashes_update(bytes, &info),
block_details: update,
block_receipts: self.prepare_block_receipts_update(receipts, &info),
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info,
timestamp: header.timestamp(),
@ -947,7 +920,6 @@ impl BlockChain {
block_hashes: self.prepare_block_hashes_update(bytes, &info),
block_details: self.prepare_block_details_update(bytes, &info),
block_receipts: self.prepare_block_receipts_update(receipts, &info),
blocks_blooms: self.prepare_block_blooms_update(bytes, &info),
transactions_addresses: self.prepare_transaction_addresses_update(bytes, &info),
info: info.clone(),
timestamp: header.timestamp(),
@ -1005,16 +977,10 @@ impl BlockChain {
batch.extend_with_cache(db::COL_EXTRA, &mut *write_receipts, update.block_receipts, CacheUpdatePolicy::Remove);
}
{
let mut write_blocks_blooms = self.blocks_blooms.write();
batch.extend_with_cache(db::COL_EXTRA, &mut *write_blocks_blooms, update.blocks_blooms, CacheUpdatePolicy::Remove);
}
// These cached values must be updated last with all four locks taken to avoid
// cache decoherence
{
let mut best_block = self.pending_best_block.write();
// update best block
match update.info.location {
BlockLocation::Branch => (),
_ => if is_best {
@ -1026,8 +992,9 @@ impl BlockChain {
timestamp: update.timestamp,
block: update.block.to_vec(),
});
},
}
}
let mut write_hashes = self.pending_block_hashes.write();
let mut write_details = self.pending_block_details.write();
let mut write_txs = self.pending_transaction_addresses.write();
@ -1244,59 +1211,6 @@ impl BlockChain {
}
}
/// This functions returns modified blocks blooms.
///
/// To accelerate blooms lookups, blomms are stored in multiple
/// layers (BLOOM_LEVELS, currently 3).
/// ChainFilter is responsible for building and rebuilding these layers.
/// It returns them in HashMap, where values are Blooms and
/// keys are BloomIndexes. BloomIndex represents bloom location on one
/// of these layers.
///
/// To reduce number of queries to databse, block blooms are stored
/// in BlocksBlooms structure which contains info about several
/// (BLOOM_INDEX_SIZE, currently 16) consecutive blocks blooms.
///
/// Later, BloomIndexer is used to map bloom location on filter layer (BloomIndex)
/// to bloom location in database (BlocksBloomLocation).
///
fn prepare_block_blooms_update(&self, block_bytes: &[u8], info: &BlockInfo) -> HashMap<GroupPosition, BloomGroup> {
let block = BlockView::new(block_bytes);
let header = block.header_view();
let log_blooms = match info.location {
BlockLocation::Branch => HashMap::new(),
BlockLocation::CanonChain => {
let log_bloom = header.log_bloom();
if log_bloom.is_zero() {
HashMap::new()
} else {
let chain = bc::group::BloomGroupChain::new(self.blooms_config, self);
chain.insert(info.number as bc::Number, log_bloom)
}
},
BlockLocation::BranchBecomingCanonChain(ref data) => {
let ancestor_number = self.block_number(&data.ancestor).unwrap();
let start_number = ancestor_number + 1;
let range = start_number as bc::Number..self.best_block_number() as bc::Number;
let mut blooms: Vec<Bloom> = data.enacted.iter()
.map(|hash| self.block_header_data(hash).unwrap())
.map(|h| h.log_bloom())
.collect();
blooms.push(header.log_bloom());
let chain = bc::group::BloomGroupChain::new(self.blooms_config, self);
chain.replace(&range, blooms)
}
};
log_blooms.into_iter()
.map(|p| (From::from(p.0), From::from(p.1)))
.collect()
}
/// Get best block hash.
pub fn best_block_hash(&self) -> H256 {
self.best_block.read().hash
@ -1330,7 +1244,6 @@ impl BlockChain {
blocks: self.block_headers.read().heap_size_of_children() + self.block_bodies.read().heap_size_of_children(),
block_details: self.block_details.read().heap_size_of_children(),
transaction_addresses: self.transaction_addresses.read().heap_size_of_children(),
blocks_blooms: self.blocks_blooms.read().heap_size_of_children(),
block_receipts: self.block_receipts.read().heap_size_of_children(),
}
}
@ -1344,7 +1257,6 @@ impl BlockChain {
let mut block_details = self.block_details.write();
let mut block_hashes = self.block_hashes.write();
let mut transaction_addresses = self.transaction_addresses.write();
let mut blocks_blooms = self.blocks_blooms.write();
let mut block_receipts = self.block_receipts.write();
let mut cache_man = self.cache_man.lock();
@ -1356,7 +1268,6 @@ impl BlockChain {
CacheId::BlockDetails(ref h) => { block_details.remove(h); }
CacheId::BlockHashes(ref h) => { block_hashes.remove(h); }
CacheId::TransactionAddresses(ref h) => { transaction_addresses.remove(h); }
CacheId::BlocksBlooms(ref h) => { blocks_blooms.remove(h); }
CacheId::BlockReceipts(ref h) => { block_receipts.remove(h); }
}
}
@ -1366,7 +1277,6 @@ impl BlockChain {
block_details.shrink_to_fit();
block_hashes.shrink_to_fit();
transaction_addresses.shrink_to_fit();
blocks_blooms.shrink_to_fit();
block_receipts.shrink_to_fit();
block_headers.heap_size_of_children() +
@ -1374,7 +1284,6 @@ impl BlockChain {
block_details.heap_size_of_children() +
block_hashes.heap_size_of_children() +
transaction_addresses.heap_size_of_children() +
blocks_blooms.heap_size_of_children() +
block_receipts.heap_size_of_children()
});
}
@ -2051,51 +1960,90 @@ mod tests {
let db = new_db();
let bc = new_chain(&genesis.last().encoded(), db.clone());
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
assert!(blocks_b1.is_empty());
assert!(blocks_b2.is_empty());
insert_block(&db, &bc, &b1.last().encoded(), vec![]);
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
assert_eq!(blocks_b1, vec![1]);
assert!(blocks_b2.is_empty());
insert_block(&db, &bc, &b2.last().encoded(), vec![]);
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
assert_eq!(blocks_b1, vec![1]);
assert_eq!(blocks_b2, vec![2]);
// hasn't been forked yet
insert_block(&db, &bc, &b1a.last().encoded(), vec![]);
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_ba = bc.blocks_with_bloom(&bloom_ba, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
let blocks_ba = bc.blocks_with_blooms(&[bloom_ba], 0, 5);
assert_eq!(blocks_b1, vec![1]);
assert_eq!(blocks_b2, vec![2]);
assert!(blocks_ba.is_empty());
// fork has happend
insert_block(&db, &bc, &b2a.last().encoded(), vec![]);
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_ba = bc.blocks_with_bloom(&bloom_ba, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
let blocks_ba = bc.blocks_with_blooms(&[bloom_ba], 0, 5);
assert!(blocks_b1.is_empty());
assert!(blocks_b2.is_empty());
assert_eq!(blocks_ba, vec![1, 2]);
// fork back
insert_block(&db, &bc, &b3.last().encoded(), vec![]);
let blocks_b1 = bc.blocks_with_bloom(&bloom_b1, 0, 5);
let blocks_b2 = bc.blocks_with_bloom(&bloom_b2, 0, 5);
let blocks_ba = bc.blocks_with_bloom(&bloom_ba, 0, 5);
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 5);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 5);
let blocks_ba = bc.blocks_with_blooms(&[bloom_ba], 0, 5);
assert_eq!(blocks_b1, vec![1]);
assert_eq!(blocks_b2, vec![2]);
assert_eq!(blocks_ba, vec![3]);
}
#[test]
fn test_insert_unordered() {
let bloom_b1: Bloom = "00000020000000000000000000000000000000000000000002000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008000400000000000000000000002000".into();
let bloom_b2: Bloom = "00000000000000000000000000000000000000000000020000001000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000008000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000".into();
let bloom_b3: Bloom = "00000000000000000000000000000000000000000000020000000800000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000008000000000000000000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000".into();
let genesis = BlockBuilder::genesis();
let b1 = genesis.add_block_with_bloom(bloom_b1);
let b2 = b1.add_block_with_bloom(bloom_b2);
let b3 = b2.add_block_with_bloom(bloom_b3);
let b1_total_difficulty = genesis.last().difficulty() + b1.last().difficulty();
let db = new_db();
let bc = new_chain(&genesis.last().encoded(), db.clone());
let mut batch = db.transaction();
bc.insert_unordered_block(&mut batch, &b2.last().encoded(), vec![], Some(b1_total_difficulty), false, false);
bc.commit();
bc.insert_unordered_block(&mut batch, &b3.last().encoded(), vec![], None, true, false);
bc.commit();
bc.insert_unordered_block(&mut batch, &b1.last().encoded(), vec![], None, false, false);
bc.commit();
db.write(batch).unwrap();
assert_eq!(bc.best_block_hash(), b3.last().hash());
assert_eq!(bc.block_hash(1).unwrap(), b1.last().hash());
assert_eq!(bc.block_hash(2).unwrap(), b2.last().hash());
assert_eq!(bc.block_hash(3).unwrap(), b3.last().hash());
let blocks_b1 = bc.blocks_with_blooms(&[bloom_b1], 0, 3);
let blocks_b2 = bc.blocks_with_blooms(&[bloom_b2], 0, 3);
let blocks_b3 = bc.blocks_with_blooms(&[bloom_b3], 0, 3);
assert_eq!(blocks_b1, vec![1]);
assert_eq!(blocks_b2, vec![2]);
assert_eq!(blocks_b3, vec![3]);
}
#[test]
fn test_best_block_update() {
let genesis = BlockBuilder::genesis();

View File

@ -23,8 +23,6 @@ pub struct CacheSize {
pub block_details: usize,
/// Transaction addresses cache size.
pub transaction_addresses: usize,
/// Blooms cache size.
pub blocks_blooms: usize,
/// Block receipts size.
pub block_receipts: usize,
}
@ -32,6 +30,6 @@ pub struct CacheSize {
impl CacheSize {
/// Total amount used by the cache.
pub fn total(&self) -> usize {
self.blocks + self.block_details + self.transaction_addresses + self.blocks_blooms + self.block_receipts
self.blocks + self.block_details + self.transaction_addresses + self.block_receipts
}
}

View File

@ -18,7 +18,6 @@
use std::ops;
use std::io::Write;
use blooms::{GroupPosition, BloomGroup};
use db::Key;
use engines::epoch::{Transition as EpochTransition};
use header::BlockNumber;
@ -37,8 +36,6 @@ pub enum ExtrasIndex {
BlockHash = 1,
/// Transaction address index
TransactionAddress = 2,
/// Block blooms index
BlocksBlooms = 3,
/// Block receipts index
BlockReceipts = 4,
/// Epoch transition data index.
@ -86,31 +83,6 @@ impl Key<BlockDetails> for H256 {
}
}
pub struct LogGroupKey([u8; 6]);
impl ops::Deref for LogGroupKey {
type Target = [u8];
fn deref(&self) -> &Self::Target {
&self.0
}
}
impl Key<BloomGroup> for GroupPosition {
type Target = LogGroupKey;
fn key(&self) -> Self::Target {
let mut result = [0u8; 6];
result[0] = ExtrasIndex::BlocksBlooms as u8;
result[1] = self.level;
result[2] = (self.index >> 24) as u8;
result[3] = (self.index >> 16) as u8;
result[4] = (self.index >> 8) as u8;
result[5] = self.index as u8;
LogGroupKey(result)
}
}
impl Key<TransactionAddress> for H256 {
type Target = H264;

View File

@ -53,6 +53,11 @@ impl Block {
pub fn encoded(&self) -> Bytes {
encode(self).into_vec()
}
#[inline]
pub fn difficulty(&self) -> U256 {
*self.header.difficulty()
}
}
#[derive(Debug)]

View File

@ -3,7 +3,6 @@ use ethereum_types::H256;
use header::BlockNumber;
use blockchain::block_info::BlockInfo;
use blockchain::extras::{BlockDetails, BlockReceipts, TransactionAddress};
use blooms::{BloomGroup, GroupPosition};
/// Block extras update info.
pub struct ExtrasUpdate<'a> {
@ -19,8 +18,6 @@ pub struct ExtrasUpdate<'a> {
pub block_details: HashMap<H256, BlockDetails>,
/// Modified block receipts.
pub block_receipts: HashMap<H256, BlockReceipts>,
/// Modified blocks blooms.
pub blocks_blooms: HashMap<GroupPosition, BloomGroup>,
/// Modified transaction addresses (None signifies removed transactions).
pub transactions_addresses: HashMap<H256, Option<TransactionAddress>>,
}

View File

@ -1677,17 +1677,8 @@ impl BlockChainClient for Client {
};
let chain = self.chain.read();
let blocks = filter.bloom_possibilities().iter()
.map(move |bloom| {
chain.blocks_with_bloom(bloom, from, to)
})
.flat_map(|m| m)
// remove duplicate elements
.collect::<HashSet<u64>>()
.into_iter()
.collect::<Vec<u64>>();
self.chain.read().logs(blocks, |entry| filter.matches(entry), filter.limit)
let blocks = chain.blocks_with_blooms(&filter.bloom_possibilities(), from, to);
chain.logs(blocks, |entry| filter.matches(entry), filter.limit)
}
fn filter_traces(&self, filter: TraceFilter) -> Option<Vec<LocalizedTrace>> {

View File

@ -454,7 +454,7 @@ mod tests {
unimplemented!()
}
fn blocks_with_bloom(&self, _bloom: &Bloom, _from_block: BlockNumber, _to_block: BlockNumber) -> Vec<BlockNumber> {
fn blocks_with_blooms(&self, _blooms: &[Bloom], _from_block: BlockNumber, _to_block: BlockNumber) -> Vec<BlockNumber> {
unimplemented!()
}