Implement Pruner to delete unwanted data. (#70)

* Implement Pruner.

* Put pruner in a crate.

* Fix clippy.

* Add rpc zgs_getShardConfig.

* Fix.

* Increase wait time.

* Add pruner_test and use max_num_chunks instead of size_limit.

* Add back shard config and fix test.

* fix: serde format

* Remove unneeded PORA_CHUNK_SIZE.

* Fix tests.

---------

Co-authored-by: MiniFrenchBread <103425574+MiniFrenchBread@users.noreply.github.com>
This commit is contained in:
peilun-conflux 2024-05-31 13:11:06 +08:00 committed by GitHub
parent c0f9d5cc31
commit ef82f64393
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
24 changed files with 459 additions and 120 deletions

30
Cargo.lock generated
View File

@ -159,9 +159,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299"
[[package]] [[package]]
name = "anyhow" name = "anyhow"
version = "1.0.81" version = "1.0.86"
source = "registry+https://github.com/rust-lang/crates.io-index" source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "0952808a6c2afd1aa8947271f3a60f1a6763c7b912d210184c5149b5cf147247" checksum = "b3d1d046238990b9cf5bcde22a3fb3584ee5cf65fb2765f454ed428c7a0063da"
dependencies = [ dependencies = [
"backtrace", "backtrace",
] ]
@ -3236,6 +3236,15 @@ dependencies = [
"either", "either",
] ]
[[package]]
name = "itertools"
version = "0.13.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "413ee7dfc52ee1a4949ceeb7dbc8a33f2d6c088194d9f922fb8318faf1f01186"
dependencies = [
"either",
]
[[package]] [[package]]
name = "itoa" name = "itoa"
version = "1.0.11" version = "1.0.11"
@ -5723,6 +5732,19 @@ version = "2.28.0"
source = "registry+https://github.com/rust-lang/crates.io-index" source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "106dd99e98437432fed6519dedecfade6a06a73bb7b2a1e019fdd2bee5778d94" checksum = "106dd99e98437432fed6519dedecfade6a06a73bb7b2a1e019fdd2bee5778d94"
[[package]]
name = "pruner"
version = "0.1.0"
dependencies = [
"anyhow",
"miner",
"rand 0.8.5",
"storage",
"task_executor",
"tokio",
"tracing",
]
[[package]] [[package]]
name = "quick-error" name = "quick-error"
version = "1.2.3" version = "1.2.3"
@ -6853,7 +6875,7 @@ dependencies = [
"ethereum-types 0.14.1", "ethereum-types 0.14.1",
"hex", "hex",
"hex-literal", "hex-literal",
"itertools 0.10.5", "itertools 0.13.0",
"kvdb", "kvdb",
"kvdb-memorydb", "kvdb-memorydb",
"kvdb-rocksdb", "kvdb-rocksdb",
@ -6861,6 +6883,7 @@ dependencies = [
"merkle_tree", "merkle_tree",
"rand 0.8.5", "rand 0.8.5",
"rayon", "rayon",
"serde",
"shared_types", "shared_types",
"static_assertions", "static_assertions",
"tempdir", "tempdir",
@ -8360,6 +8383,7 @@ dependencies = [
"log_entry_sync", "log_entry_sync",
"miner", "miner",
"network", "network",
"pruner",
"router", "router",
"rpc", "rpc",
"serde", "serde",

View File

@ -17,6 +17,7 @@ members = [
"node/log_entry_sync", "node/log_entry_sync",
"node/miner", "node/miner",
"node/network", "node/network",
"node/pruner",
"node/router", "node/router",
"node/rpc", "node/rpc",
"node/shared_types", "node/shared_types",

View File

@ -17,6 +17,7 @@ zgs_spec = { path = "../common/spec" }
log_entry_sync = { path = "./log_entry_sync" } log_entry_sync = { path = "./log_entry_sync" }
miner = { path = "./miner" } miner = { path = "./miner" }
network = { path = "./network" } network = { path = "./network" }
pruner = { path = "./pruner" }
router = { path = "./router" } router = { path = "./router" }
rpc = { path = "./rpc" } rpc = { path = "./rpc" }
shared_types = { path = "./shared_types" } shared_types = { path = "./shared_types" }

View File

@ -6,95 +6,7 @@ use ethers::providers::Middleware;
use ethers::providers::Provider; use ethers::providers::Provider;
use ethers::signers::LocalWallet; use ethers::signers::LocalWallet;
use ethers::signers::Signer; use ethers::signers::Signer;
use zgs_spec::BYTES_PER_LOAD; use storage::config::ShardConfig;
#[derive(Clone, Copy, Debug)]
pub struct ShardConfig {
pub shard_group_load_chunks: usize,
pub shard_id: usize,
pub num_shard: usize,
}
impl Default for ShardConfig {
fn default() -> Self {
Self {
shard_group_load_chunks: 1,
shard_id: 0,
num_shard: 1,
}
}
}
impl ShardConfig {
pub fn new(
shard_group_bytes: Option<usize>,
shard_position: &Option<String>,
) -> Result<Option<Self>, String> {
let (group_bytes, (id, num)) = match (shard_group_bytes, shard_position) {
(None, None) => {
return Ok(None);
}
(Some(bytes), Some(position)) => (bytes, Self::parse_position(position)?),
_ => {
return Err(
"`shard_group_bytes` and `shard_position` should be set simultaneously".into(),
);
}
};
if group_bytes < BYTES_PER_LOAD || group_bytes & (group_bytes - 1) != 0 {
return Err(format!(
"Incorrect shard group bytes: {}, should be power of two and >= {}",
group_bytes, BYTES_PER_LOAD
));
}
let group_chunks = group_bytes / BYTES_PER_LOAD;
if id >= num {
return Err(format!(
"Incorrect shard_id: expected [0, {}), actual {}",
num, id
));
}
if group_chunks < num {
return Err(format!("Incorrect shard_group_number: the shard group contains {} loading chunks, which cannot be divided into {} shards", group_chunks, num));
}
Ok(Some(ShardConfig {
shard_group_load_chunks: group_chunks,
shard_id: id,
num_shard: num,
}))
}
pub fn shard_chunks(&self) -> u64 {
(self.shard_group_load_chunks / self.num_shard) as u64
}
pub fn shard_mask(&self) -> u64 {
let x = self.shard_group_load_chunks as u64 - self.shard_chunks();
!x
}
pub fn parse_position(input: &str) -> Result<(usize, usize), String> {
let parts: Vec<&str> = input.trim().split('/').map(|s| s.trim()).collect();
if parts.len() != 2 {
return Err("Incorrect format, expected like: '0 / 8'".into());
}
let numerator = parts[0]
.parse::<usize>()
.map_err(|e| format!("Cannot parse shard position {:?}", e))?;
let denominator = parts[1]
.parse::<usize>()
.map_err(|e| format!("Cannot parse shard position {:?}", e))?;
Ok((numerator, denominator))
}
}
pub struct MinerConfig { pub struct MinerConfig {
pub(crate) miner_id: Option<H256>, pub(crate) miner_id: Option<H256>,
@ -105,7 +17,7 @@ pub struct MinerConfig {
pub(crate) submission_gas: Option<U256>, pub(crate) submission_gas: Option<U256>,
pub(crate) cpu_percentage: u64, pub(crate) cpu_percentage: u64,
pub(crate) iter_batch: usize, pub(crate) iter_batch: usize,
pub(crate) shard_config: Option<ShardConfig>, pub(crate) shard_config: ShardConfig,
} }
pub type MineServiceMiddleware = SignerMiddleware<Provider<Http>, LocalWallet>; pub type MineServiceMiddleware = SignerMiddleware<Provider<Http>, LocalWallet>;
@ -121,7 +33,7 @@ impl MinerConfig {
submission_gas: Option<U256>, submission_gas: Option<U256>,
cpu_percentage: u64, cpu_percentage: u64,
iter_batch: usize, iter_batch: usize,
shard_config: Option<ShardConfig>, shard_config: ShardConfig,
) -> Option<MinerConfig> { ) -> Option<MinerConfig> {
miner_key.map(|miner_key| MinerConfig { miner_key.map(|miner_key| MinerConfig {
miner_id, miner_id,

View File

@ -15,8 +15,9 @@ mod service;
mod submitter; mod submitter;
mod watcher; mod watcher;
pub use config::{MinerConfig, ShardConfig}; pub use config::MinerConfig;
pub use loader::PoraLoader; pub use loader::PoraLoader;
pub use mine::MineRangeConfig; pub use mine::MineRangeConfig;
pub use miner_id::load_miner_id; pub use miner_id::load_miner_id;
pub use service::{MineService, MinerMessage}; pub use service::{MineService, MinerMessage};
pub use storage::config::ShardConfig;

View File

@ -9,12 +9,12 @@ use tokio::time::{sleep, Duration, Instant};
use zgs_spec::{SECTORS_PER_LOAD, SECTORS_PER_MAX_MINING_RANGE, SECTORS_PER_PRICING}; use zgs_spec::{SECTORS_PER_LOAD, SECTORS_PER_MAX_MINING_RANGE, SECTORS_PER_PRICING};
use crate::recall_range::RecallRange; use crate::recall_range::RecallRange;
use crate::ShardConfig;
use crate::{ use crate::{
pora::{AnswerWithoutProof, Miner}, pora::{AnswerWithoutProof, Miner},
watcher::MineContextMessage, watcher::MineContextMessage,
MinerConfig, MinerMessage, PoraLoader, MinerConfig, MinerMessage, PoraLoader,
}; };
use storage::config::ShardConfig;
use std::sync::Arc; use std::sync::Arc;
@ -36,16 +36,16 @@ struct PoraPuzzle {
context: MineContext, context: MineContext,
target_quality: U256, target_quality: U256,
} }
#[derive(Clone, Copy, Debug, Default)] #[derive(Clone, Debug, Default)]
pub struct MineRangeConfig { pub struct MineRangeConfig {
start_position: Option<u64>, start_position: Option<u64>,
end_position: Option<u64>, end_position: Option<u64>,
shard_config: Option<ShardConfig>, shard_config: ShardConfig,
} }
impl MineRangeConfig { impl MineRangeConfig {
#[inline] #[inline]
fn to_valid_range(self, context: &MineContext) -> Option<RecallRange> { fn to_valid_range(&self, context: &MineContext) -> Option<RecallRange> {
let self_start_position = self.start_position?; let self_start_position = self.start_position?;
let self_end_position = self.end_position?; let self_end_position = self.end_position?;
@ -64,10 +64,8 @@ impl MineRangeConfig {
Some(RecallRange { Some(RecallRange {
start_position, start_position,
mining_length, mining_length,
shard_mask: self.shard_config.map_or(u64::MAX, |c| c.shard_mask()), shard_mask: self.shard_config.miner_shard_mask(),
shard_id: self shard_id: self.shard_config.miner_shard_id(),
.shard_config
.map_or(0, |c| c.shard_id as u64 * c.shard_chunks()),
}) })
} }
@ -145,6 +143,9 @@ impl PoraService {
info!("Change end position to: {:?}", pos); info!("Change end position to: {:?}", pos);
self.mine_range.end_position = pos; self.mine_range.end_position = pos;
} }
Ok(MinerMessage::SetShardConfig(shard_config)) => {
self.mine_range.shard_config = shard_config;
}
Err(broadcast::error::RecvError::Closed)=>{ Err(broadcast::error::RecvError::Closed)=>{
warn!("Unexpected: Mine service config channel closed."); warn!("Unexpected: Mine service config channel closed.");
channel_opened = false; channel_opened = false;

View File

@ -4,6 +4,7 @@ use crate::submitter::Submitter;
use crate::{config::MinerConfig, mine::PoraService, watcher::MineContextWatcher}; use crate::{config::MinerConfig, mine::PoraService, watcher::MineContextWatcher};
use network::NetworkMessage; use network::NetworkMessage;
use std::sync::Arc; use std::sync::Arc;
use storage::config::ShardConfig;
use storage::log_store::Store; use storage::log_store::Store;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use tokio::sync::{broadcast, RwLock}; use tokio::sync::{broadcast, RwLock};
@ -16,6 +17,9 @@ pub enum MinerMessage {
/// Change mining range /// Change mining range
SetStartPosition(Option<u64>), SetStartPosition(Option<u64>),
SetEndPosition(Option<u64>), SetEndPosition(Option<u64>),
/// Change shard config
SetShardConfig(ShardConfig),
} }
pub struct MineService; pub struct MineService;

View File

@ -146,6 +146,15 @@ pub struct AnnounceChunks {
pub timestamp: u32, pub timestamp: u32,
} }
#[derive(Debug, Clone, PartialEq, Eq, Hash, Encode, Decode)]
pub struct AnnounceShardConfig {
pub num_shard: usize,
pub shard_id: usize,
pub peer_id: WrappedPeerId,
pub at: WrappedMultiaddr,
pub timestamp: u32,
}
#[derive(Debug, Clone, PartialEq, Eq, Hash, Encode, Decode)] #[derive(Debug, Clone, PartialEq, Eq, Hash, Encode, Decode)]
pub struct SignedMessage<T: Encode + Decode> { pub struct SignedMessage<T: Encode + Decode> {
pub inner: T, pub inner: T,

13
node/pruner/Cargo.toml Normal file
View File

@ -0,0 +1,13 @@
[package]
name = "pruner"
version = "0.1.0"
edition = "2021"
[dependencies]
storage = { path = "../storage" }
miner = { path = "../miner" }
anyhow = "1.0.86"
tokio = "1.37.0"
rand = "0.8.5"
task_executor = { path = "../../common/task_executor" }
tracing = "0.1.40"

132
node/pruner/src/lib.rs Normal file
View File

@ -0,0 +1,132 @@
use anyhow::Result;
use miner::MinerMessage;
use rand::Rng;
use std::path::PathBuf;
use std::sync::Arc;
use std::time::Duration;
use storage::config::{ShardConfig, SHARD_CONFIG_KEY};
use storage::log_store::config::ConfigurableExt;
use storage::log_store::Store;
use task_executor::TaskExecutor;
use tokio::sync::{broadcast, RwLock};
use tracing::debug;
// Start pruning when the db directory size exceeds 0.9 * limit.
const PRUNE_THRESHOLD: f32 = 0.9;
#[derive(Debug)]
pub struct PrunerConfig {
pub shard_config: ShardConfig,
pub db_path: PathBuf,
pub max_num_chunks: usize,
pub check_time: Duration,
pub batch_size: usize,
pub batch_wait_time: Duration,
}
impl PrunerConfig {
fn start_prune_size(&self) -> u64 {
(self.max_num_chunks as f32 * PRUNE_THRESHOLD) as u64
}
}
pub struct Pruner {
config: PrunerConfig,
store: Arc<RwLock<dyn Store>>,
miner_sender: Option<broadcast::Sender<MinerMessage>>,
}
impl Pruner {
pub async fn spawn(
executor: TaskExecutor,
mut config: PrunerConfig,
store: Arc<RwLock<dyn Store>>,
miner_sender: Option<broadcast::Sender<MinerMessage>>,
) -> Result<()> {
if let Some(shard_config) = get_shard_config(&store).await? {
config.shard_config = shard_config;
}
let pruner = Pruner {
config,
store,
miner_sender,
};
pruner.put_shard_config().await?;
executor.spawn(
async move {
pruner.start().await.expect("pruner error");
},
"pruner",
);
Ok(())
}
pub async fn start(mut self) -> Result<()> {
loop {
if let Some(delete_list) = self.maybe_update().await? {
debug!(new_config = ?self.config.shard_config, "new shard config");
self.put_shard_config().await?;
let mut batch = Vec::with_capacity(self.config.batch_size);
let mut iter = delete_list.peekable();
while let Some(index) = iter.next() {
batch.push(index);
if batch.len() == self.config.batch_size || iter.peek().is_none() {
debug!(start = batch.first(), end = batch.last(), "prune batch");
self.store.write().await.remove_chunks_batch(&batch)?;
batch = Vec::with_capacity(self.config.batch_size);
tokio::time::sleep(self.config.batch_wait_time).await;
}
}
}
tokio::time::sleep(self.config.check_time).await;
}
}
async fn maybe_update(&mut self) -> Result<Option<Box<dyn Send + Iterator<Item = u64>>>> {
let current_size = self.store.read().await.flow().get_num_entries()?;
debug!(
current_size = current_size,
config = ?self.config.shard_config,
"maybe_update"
);
if current_size >= self.config.start_prune_size() {
// Update config and generate delete list should be done in a single lock to ensure
// the list is complete.
let config = &mut self.config.shard_config;
let old_shard_id = config.shard_id;
let old_num_shard = config.num_shard;
// Update new config
let rand_bit = {
let mut rng = rand::thread_rng();
rng.gen::<bool>()
};
config.shard_id = old_shard_id + rand_bit as usize * old_num_shard;
config.num_shard *= 2;
// Generate delete list
let flow_len = self.store.read().await.get_context()?.1;
let start_index = old_shard_id + (!rand_bit) as usize * old_num_shard;
return Ok(Some(Box::new(
(start_index as u64..flow_len).step_by(config.num_shard),
)));
}
Ok(None)
}
async fn put_shard_config(&self) -> Result<()> {
if let Some(sender) = &self.miner_sender {
sender.send(MinerMessage::SetShardConfig(self.config.shard_config))?;
}
let mut store = self.store.write().await;
store
.flow_mut()
.update_shard_config(self.config.shard_config);
store.set_config_encoded(&SHARD_CONFIG_KEY, &self.config.shard_config)
}
}
async fn get_shard_config(store: &RwLock<dyn Store>) -> Result<Option<ShardConfig>> {
store.read().await.get_config_decoded(&SHARD_CONFIG_KEY)
}

View File

@ -2,6 +2,7 @@ use crate::types::{FileInfo, Segment, SegmentWithProof, Status};
use jsonrpsee::core::RpcResult; use jsonrpsee::core::RpcResult;
use jsonrpsee::proc_macros::rpc; use jsonrpsee::proc_macros::rpc;
use shared_types::DataRoot; use shared_types::DataRoot;
use storage::config::ShardConfig;
#[rpc(server, client, namespace = "zgs")] #[rpc(server, client, namespace = "zgs")]
pub trait Rpc { pub trait Rpc {
@ -34,4 +35,7 @@ pub trait Rpc {
#[method(name = "getFileInfoByTxSeq")] #[method(name = "getFileInfoByTxSeq")]
async fn get_file_info_by_tx_seq(&self, tx_seq: u64) -> RpcResult<Option<FileInfo>>; async fn get_file_info_by_tx_seq(&self, tx_seq: u64) -> RpcResult<Option<FileInfo>>;
#[method(name = "getShardConfig")]
async fn get_shard_config(&self) -> RpcResult<ShardConfig>;
} }

View File

@ -7,6 +7,8 @@ use jsonrpsee::core::async_trait;
use jsonrpsee::core::RpcResult; use jsonrpsee::core::RpcResult;
use shared_types::{DataRoot, Transaction, CHUNK_SIZE}; use shared_types::{DataRoot, Transaction, CHUNK_SIZE};
use std::fmt::{Debug, Formatter, Result}; use std::fmt::{Debug, Formatter, Result};
use storage::config::{ShardConfig, SHARD_CONFIG_KEY};
use storage::log_store::config::ConfigurableExt;
use storage::try_option; use storage::try_option;
pub struct RpcServerImpl { pub struct RpcServerImpl {
@ -152,6 +154,22 @@ impl RpcServer for RpcServerImpl {
Ok(Some(self.get_file_info_by_tx(tx).await?)) Ok(Some(self.get_file_info_by_tx(tx).await?))
} }
async fn get_shard_config(&self) -> RpcResult<ShardConfig> {
debug!("zgs_getShardConfig");
let shard_config = self
.ctx
.log_store
.get_store()
.read()
.await
.get_config_decoded(&SHARD_CONFIG_KEY)?
.ok_or(error::invalid_params(
"shard_config",
"shard_config is unavailable",
))?;
Ok(shard_config)
}
} }
impl RpcServerImpl { impl RpcServerImpl {

View File

@ -7,6 +7,7 @@ use network::{
self, Keypair, NetworkConfig, NetworkGlobals, NetworkMessage, RequestId, self, Keypair, NetworkConfig, NetworkGlobals, NetworkMessage, RequestId,
Service as LibP2PService, Service as LibP2PService,
}; };
use pruner::{Pruner, PrunerConfig};
use router::RouterService; use router::RouterService;
use rpc::RPCConfig; use rpc::RPCConfig;
use std::sync::Arc; use std::sync::Arc;
@ -49,6 +50,8 @@ struct LogSyncComponents {
send: broadcast::Sender<LogSyncEvent>, send: broadcast::Sender<LogSyncEvent>,
} }
struct PrunerComponents {}
/// Builds a `Client` instance. /// Builds a `Client` instance.
/// ///
/// ## Notes /// ## Notes
@ -65,6 +68,7 @@ pub struct ClientBuilder {
sync: Option<SyncComponents>, sync: Option<SyncComponents>,
miner: Option<MinerComponents>, miner: Option<MinerComponents>,
log_sync: Option<LogSyncComponents>, log_sync: Option<LogSyncComponents>,
pruner: Option<PrunerComponents>,
} }
impl ClientBuilder { impl ClientBuilder {
@ -174,6 +178,19 @@ impl ClientBuilder {
Ok(self) Ok(self)
} }
pub async fn with_pruner(mut self, config: Option<PrunerConfig>) -> Result<Self, String> {
if let Some(config) = config {
let miner_send = self.miner.as_ref().map(|miner| miner.send.clone());
let store = require!("pruner", self, store).clone();
let executor = require!("pruner", self, runtime_context).clone().executor;
Pruner::spawn(executor, config, store, miner_send)
.await
.map_err(|e| e.to_string())?;
self.pruner = Some(PrunerComponents {});
}
Ok(self)
}
/// Starts the networking stack. /// Starts the networking stack.
pub fn with_router(mut self, router_config: router::Config) -> Result<Self, String> { pub fn with_router(mut self, router_config: router::Config) -> Result<Self, String> {
let executor = require!("router", self, runtime_context).clone().executor; let executor = require!("router", self, runtime_context).clone().executor;

View File

@ -3,9 +3,12 @@
use crate::ZgsConfig; use crate::ZgsConfig;
use ethereum_types::{H256, U256}; use ethereum_types::{H256, U256};
use log_entry_sync::{CacheConfig, ContractAddress, LogSyncConfig}; use log_entry_sync::{CacheConfig, ContractAddress, LogSyncConfig};
use miner::{MinerConfig, ShardConfig}; use miner::MinerConfig;
use network::NetworkConfig; use network::NetworkConfig;
use pruner::PrunerConfig;
use rpc::RPCConfig; use rpc::RPCConfig;
use std::time::Duration;
use storage::config::ShardConfig;
use storage::StorageConfig; use storage::StorageConfig;
impl ZgsConfig { impl ZgsConfig {
@ -143,7 +146,7 @@ impl ZgsConfig {
let cpu_percentage = self.miner_cpu_percentage; let cpu_percentage = self.miner_cpu_percentage;
let iter_batch = self.mine_iter_batch_size; let iter_batch = self.mine_iter_batch_size;
let shard_config = ShardConfig::new(self.shard_group_bytes, &self.shard_position)?; let shard_config = ShardConfig::new(&self.shard_position)?;
Ok(MinerConfig::new( Ok(MinerConfig::new(
miner_id, miner_id,
@ -172,4 +175,20 @@ impl ZgsConfig {
router_config.libp2p_nodes = network_config.libp2p_nodes.to_vec(); router_config.libp2p_nodes = network_config.libp2p_nodes.to_vec();
Ok(router_config) Ok(router_config)
} }
pub fn pruner_config(&self) -> Result<Option<PrunerConfig>, String> {
if let Some(max_num_chunks) = self.db_max_num_chunks {
let shard_config = ShardConfig::new(&self.shard_position)?;
Ok(Some(PrunerConfig {
shard_config,
db_path: self.db_dir.clone().into(),
max_num_chunks,
check_time: Duration::from_secs(self.prune_check_time_s),
batch_size: self.prune_batch_size,
batch_wait_time: Duration::from_millis(self.prune_batch_wait_time_ms),
}))
} else {
Ok(None)
}
}
} }

View File

@ -54,6 +54,10 @@ build_config! {
// db // db
(db_dir, (String), "db".to_string()) (db_dir, (String), "db".to_string())
(db_max_num_chunks, (Option<usize>), None)
(prune_check_time_s, (u64), 60)
(prune_batch_size, (usize), 1024)
(prune_batch_wait_time_ms, (u64), 1000)
// misc // misc
(log_config_file, (String), "log_config".to_string()) (log_config_file, (String), "log_config".to_string())
@ -66,7 +70,6 @@ build_config! {
(miner_submission_gas, (Option<u64>), None) (miner_submission_gas, (Option<u64>), None)
(miner_cpu_percentage, (u64), 100) (miner_cpu_percentage, (u64), 100)
(mine_iter_batch_size, (usize), 100) (mine_iter_batch_size, (usize), 100)
(shard_group_bytes, (Option<usize>), None)
(shard_position, (Option<String>), None) (shard_position, (Option<String>), None)
} }

View File

@ -17,6 +17,7 @@ async fn start_node(context: RuntimeContext, config: ZgsConfig) -> Result<Client
let log_sync_config = config.log_sync_config()?; let log_sync_config = config.log_sync_config()?;
let miner_config = config.mine_config()?; let miner_config = config.mine_config()?;
let router_config = config.router_config(&network_config)?; let router_config = config.router_config(&network_config)?;
let pruner_config = config.pruner_config()?;
ClientBuilder::default() ClientBuilder::default()
.with_runtime_context(context) .with_runtime_context(context)
@ -31,6 +32,8 @@ async fn start_node(context: RuntimeContext, config: ZgsConfig) -> Result<Client
.with_miner(miner_config) .with_miner(miner_config)
.await? .await?
.with_router(router_config)? .with_router(router_config)?
.with_pruner(pruner_config)
.await?
.with_rpc(rpc_config, config.chunk_pool_config()) .with_rpc(rpc_config, config.chunk_pool_config())
.await? .await?
.build() .build()

View File

@ -25,7 +25,8 @@ typenum = "1.15.0"
bitmaps = "^3.2" bitmaps = "^3.2"
static_assertions = "1.1" static_assertions = "1.1"
tiny-keccak = "*" tiny-keccak = "*"
itertools = "0.10.5" itertools = "0.13.0"
serde = { version = "1.0.197", features = ["derive"] }
[dev-dependencies] [dev-dependencies]
tempdir = "0.3.7" tempdir = "0.3.7"

View File

@ -1,6 +1,83 @@
use serde::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use std::path::PathBuf; use std::path::PathBuf;
pub const SHARD_CONFIG_KEY: &str = "shard_config";
#[derive(Clone)] #[derive(Clone)]
pub struct Config { pub struct Config {
pub db_dir: PathBuf, pub db_dir: PathBuf,
} }
#[derive(Clone, Copy, Debug, Decode, Encode, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct ShardConfig {
pub shard_id: usize,
pub num_shard: usize,
}
impl Default for ShardConfig {
fn default() -> Self {
Self {
shard_id: 0,
num_shard: 1,
}
}
}
impl ShardConfig {
pub fn new(shard_position: &Option<String>) -> Result<Self, String> {
let (id, num) = if let Some(position) = shard_position {
Self::parse_position(position)?
} else {
(0, 1)
};
if id >= num {
return Err(format!(
"Incorrect shard_id: expected [0, {}), actual {}",
num, id
));
}
if !num.is_power_of_two() {
return Err(format!(
"Incorrect shard group bytes: {}, should be power of two",
num
));
}
Ok(ShardConfig {
shard_id: id,
num_shard: num,
})
}
pub fn miner_shard_mask(&self) -> u64 {
!(self.num_shard - 1) as u64
}
pub fn miner_shard_id(&self) -> u64 {
self.shard_id as u64
}
pub fn in_range(&self, segment_index: u64) -> bool {
segment_index as usize % self.num_shard == self.shard_id
}
pub fn parse_position(input: &str) -> Result<(usize, usize), String> {
let parts: Vec<&str> = input.trim().split('/').map(|s| s.trim()).collect();
if parts.len() != 2 {
return Err("Incorrect format, expected like: '0 / 8'".into());
}
let numerator = parts[0]
.parse::<usize>()
.map_err(|e| format!("Cannot parse shard position {:?}", e))?;
let denominator = parts[1]
.parse::<usize>()
.map_err(|e| format!("Cannot parse shard position {:?}", e))?;
Ok((numerator, denominator))
}
}

View File

@ -8,6 +8,8 @@ pub use config::Config as StorageConfig;
pub use log_store::log_manager::LogManager; pub use log_store::log_manager::LogManager;
pub use ethereum_types::H256; pub use ethereum_types::H256;
use kvdb_memorydb::InMemory;
use kvdb_rocksdb::Database;
pub trait ZgsKeyValueDB: KeyValueDB { pub trait ZgsKeyValueDB: KeyValueDB {
fn put(&self, col: u32, key: &[u8], value: &[u8]) -> std::io::Result<()> { fn put(&self, col: u32, key: &[u8], value: &[u8]) -> std::io::Result<()> {
@ -35,6 +37,18 @@ pub trait ZgsKeyValueDB: KeyValueDB {
tx.delete_prefix(col, key_prefix); tx.delete_prefix(col, key_prefix);
self.write(tx) self.write(tx)
} }
fn num_keys(&self, col: u32) -> std::io::Result<u64>;
} }
impl<T: KeyValueDB> ZgsKeyValueDB for T {} impl ZgsKeyValueDB for Database {
fn num_keys(&self, col: u32) -> std::io::Result<u64> {
self.num_keys(col)
}
}
impl ZgsKeyValueDB for InMemory {
fn num_keys(&self, _col: u32) -> std::io::Result<u64> {
todo!("not used")
}
}

View File

@ -1,8 +1,9 @@
use super::load_chunk::EntryBatch; use super::load_chunk::EntryBatch;
use super::{MineLoadChunk, SealAnswer, SealTask}; use super::{MineLoadChunk, SealAnswer, SealTask};
use crate::config::ShardConfig;
use crate::error::Error; use crate::error::Error;
use crate::log_store::log_manager::{ use crate::log_store::log_manager::{
bytes_to_entries, COL_ENTRY_BATCH, COL_ENTRY_BATCH_ROOT, COL_FLOW_MPT_NODES, bytes_to_entries, COL_ENTRY_BATCH, COL_ENTRY_BATCH_ROOT, COL_FLOW_MPT_NODES, PORA_CHUNK_SIZE,
}; };
use crate::log_store::{FlowRead, FlowSeal, FlowWrite}; use crate::log_store::{FlowRead, FlowSeal, FlowWrite};
use crate::{try_option, ZgsKeyValueDB}; use crate::{try_option, ZgsKeyValueDB};
@ -76,17 +77,23 @@ impl FlowStore {
pub fn put_mpt_node_list(&self, node_list: Vec<(usize, usize, DataRoot)>) -> Result<()> { pub fn put_mpt_node_list(&self, node_list: Vec<(usize, usize, DataRoot)>) -> Result<()> {
self.db.put_mpt_node_list(node_list) self.db.put_mpt_node_list(node_list)
} }
pub fn delete_batch_list(&self, batch_list: &[u64]) -> Result<()> {
self.db.delete_batch_list(batch_list)
}
} }
#[derive(Clone, Debug)] #[derive(Clone, Debug)]
pub struct FlowConfig { pub struct FlowConfig {
pub batch_size: usize, pub batch_size: usize,
pub shard_config: ShardConfig,
} }
impl Default for FlowConfig { impl Default for FlowConfig {
fn default() -> Self { fn default() -> Self {
Self { Self {
batch_size: SECTORS_PER_LOAD, batch_size: SECTORS_PER_LOAD,
shard_config: Default::default(),
} }
} }
} }
@ -189,6 +196,15 @@ impl FlowRead for FlowStore {
} }
Ok(Some(mine_chunk)) Ok(Some(mine_chunk))
} }
fn get_num_entries(&self) -> Result<u64> {
// This is an over-estimation as it assumes each batch is full.
self.db
.kvdb
.num_keys(COL_ENTRY_BATCH)
.map(|num_batches| num_batches * PORA_CHUNK_SIZE as u64)
.map_err(Into::into)
}
} }
impl FlowWrite for FlowStore { impl FlowWrite for FlowStore {
@ -211,6 +227,10 @@ impl FlowWrite for FlowStore {
.expect("in range"); .expect("in range");
let chunk_index = chunk.start_index / self.config.batch_size as u64; let chunk_index = chunk.start_index / self.config.batch_size as u64;
if !self.config.shard_config.in_range(chunk_index) {
// The data are in a shard range that we are not storing.
continue;
}
// TODO: Try to avoid loading from db if possible. // TODO: Try to avoid loading from db if possible.
let mut batch = self let mut batch = self
@ -245,6 +265,10 @@ impl FlowWrite for FlowStore {
}); });
Ok(()) Ok(())
} }
fn update_shard_config(&mut self, shard_config: ShardConfig) {
self.config.shard_config = shard_config;
}
} }
impl FlowSeal for FlowStore { impl FlowSeal for FlowStore {
@ -293,7 +317,7 @@ impl FlowSeal for FlowStore {
for (load_index, answers_in_chunk) in &answers for (load_index, answers_in_chunk) in &answers
.into_iter() .into_iter()
.filter(is_consistent) .filter(is_consistent)
.group_by(|answer| answer.seal_index / SEALS_PER_LOAD as u64) .chunk_by(|answer| answer.seal_index / SEALS_PER_LOAD as u64)
{ {
let mut batch_chunk = self let mut batch_chunk = self
.db .db
@ -526,6 +550,14 @@ impl FlowDBStore {
} }
Ok(node_list) Ok(node_list)
} }
fn delete_batch_list(&self, batch_list: &[u64]) -> Result<()> {
let mut tx = self.kvdb.transaction();
for i in batch_list {
tx.delete(COL_ENTRY_BATCH, &i.to_be_bytes());
}
Ok(self.kvdb.write(tx)?)
}
} }
#[derive(DeriveEncode, DeriveDecode, Clone, Debug)] #[derive(DeriveEncode, DeriveDecode, Clone, Debug)]

View File

@ -135,8 +135,8 @@ impl LogStoreChunkWrite for LogManager {
Ok(true) Ok(true)
} }
fn remove_all_chunks(&self, _tx_seq: u64) -> crate::error::Result<()> { fn remove_chunks_batch(&self, batch_list: &[u64]) -> crate::error::Result<()> {
todo!() self.flow_store.delete_batch_list(batch_list)
} }
} }

View File

@ -1,3 +1,4 @@
use crate::config::ShardConfig;
use append_merkle::MerkleTreeInitialData; use append_merkle::MerkleTreeInitialData;
use ethereum_types::H256; use ethereum_types::H256;
use shared_types::{ use shared_types::{
@ -144,8 +145,9 @@ pub trait LogStoreChunkWrite {
maybe_file_proof: Option<FlowProof>, maybe_file_proof: Option<FlowProof>,
) -> Result<bool>; ) -> Result<bool>;
/// Delete all chunks of a tx. /// Delete a list of chunk batches from the db.
fn remove_all_chunks(&self, tx_seq: u64) -> Result<()>; /// `batch_list` is a `Vec` of entry batch index.
fn remove_chunks_batch(&self, batch_list: &[u64]) -> Result<()>;
} }
pub trait LogChunkStore: LogStoreChunkRead + LogStoreChunkWrite + Send + Sync + 'static {} pub trait LogChunkStore: LogStoreChunkRead + LogStoreChunkWrite + Send + Sync + 'static {}
@ -193,6 +195,9 @@ pub trait FlowRead {
fn get_chunk_root_list(&self) -> Result<MerkleTreeInitialData<DataRoot>>; fn get_chunk_root_list(&self) -> Result<MerkleTreeInitialData<DataRoot>>;
fn load_sealed_data(&self, chunk_index: u64) -> Result<Option<MineLoadChunk>>; fn load_sealed_data(&self, chunk_index: u64) -> Result<Option<MineLoadChunk>>;
// An estimation of the number of entries in the flow db.
fn get_num_entries(&self) -> Result<u64>;
} }
pub trait FlowWrite { pub trait FlowWrite {
@ -204,6 +209,9 @@ pub trait FlowWrite {
/// Remove all the entries after `start_index`. /// Remove all the entries after `start_index`.
/// This is used to remove deprecated data in case of chain reorg. /// This is used to remove deprecated data in case of chain reorg.
fn truncate(&mut self, start_index: u64) -> Result<()>; fn truncate(&mut self, start_index: u64) -> Result<()>;
/// Update the shard config.
fn update_shard_config(&mut self, shard_config: ShardConfig);
} }
pub struct SealTask { pub struct SealTask {

View File

@ -10,13 +10,14 @@ import math
PRICE_PER_SECTOR = math.ceil(10 * (10 ** 18) / (2 ** 30) * 256 / 12) PRICE_PER_SECTOR = math.ceil(10 * (10 ** 18) / (2 ** 30) * 256 / 12)
class MineTest(TestFramework): class MineTest(TestFramework):
def setup_params(self): def setup_params(self):
self.num_blockchain_nodes = 1 self.num_blockchain_nodes = 1
self.num_nodes = 1 self.num_nodes = 1
self.zgs_node_configs[0] = { self.zgs_node_configs[0] = {
"db_max_num_chunks": 2**30,
"miner_key": GENESIS_PRIV_KEY, "miner_key": GENESIS_PRIV_KEY,
"shard_group_bytes": 4 * 1024 * 1024,
"shard_position": "3 / 8", "shard_position": "3 / 8",
} }
self.enable_market = True self.enable_market = True
@ -24,7 +25,6 @@ class MineTest(TestFramework):
self.launch_wait_seconds = 15 self.launch_wait_seconds = 15
self.log.info("Contract Info: Est. block time %.2f, Mine period %d", self.block_time, self.mine_period) self.log.info("Contract Info: Est. block time %.2f, Mine period %d", self.block_time, self.mine_period)
def submit_data(self, item, size, no_submit = False): def submit_data(self, item, size, no_submit = False):
submissions_before = self.contract.num_submissions() submissions_before = self.contract.num_submissions()
client = self.nodes[0] client = self.nodes[0]
@ -37,7 +37,7 @@ class MineTest(TestFramework):
if not no_submit: if not no_submit:
wait_until(lambda: client.zgs_get_file_info(data_root) is not None) wait_until(lambda: client.zgs_get_file_info(data_root) is not None)
segment = submit_data(client, chunk_data) segment = submit_data(client, chunk_data)
wait_until(lambda: client.zgs_get_file_info(data_root)["finalized"]) # wait_until(lambda: client.zgs_get_file_info(data_root)["finalized"])
def run_test(self): def run_test(self):
blockchain = self.blockchain_nodes[0] blockchain = self.blockchain_nodes[0]
@ -67,7 +67,7 @@ class MineTest(TestFramework):
self.contract.update_context() self.contract.update_context()
self.log.info("Wait for mine answer") self.log.info("Wait for mine answer")
wait_until(lambda: self.mine_contract.last_mined_epoch() == start_epoch + 1) wait_until(lambda: self.mine_contract.last_mined_epoch() == start_epoch + 1, timeout=120)
rewards = self.reward_contract.reward_distributes() rewards = self.reward_contract.reward_distributes()
assert_equal(len(self.reward_contract.reward_distributes()), start_epoch + 1) assert_equal(len(self.reward_contract.reward_distributes()), start_epoch + 1)
@ -82,7 +82,6 @@ class MineTest(TestFramework):
assert_equal(current_epoch, start_epoch + 1); assert_equal(current_epoch, start_epoch + 1);
self.log.info("Sumission Done, epoch is %d, current block number %d", self.contract.epoch(), int(blockchain.eth_blockNumber(), 16)) self.log.info("Sumission Done, epoch is %d, current block number %d", self.contract.epoch(), int(blockchain.eth_blockNumber(), 16))
self.log.info("Wait for mine context release") self.log.info("Wait for mine context release")
wait_until(lambda: self.contract.epoch() >= start_epoch + 2, timeout=180) wait_until(lambda: self.contract.epoch() >= start_epoch + 2, timeout=180)
self.contract.update_context() self.contract.update_context()
@ -97,6 +96,5 @@ class MineTest(TestFramework):
assert_greater_than(secondReward, 100 * firstReward / (start_epoch + 1)) assert_greater_than(secondReward, 100 * firstReward / (start_epoch + 1))
if __name__ == "__main__": if __name__ == "__main__":
MineTest(blockchain_node_type=BlockChainNodeType.BSC).main() MineTest(blockchain_node_type=BlockChainNodeType.BSC).main()

47
tests/pruner_test.py Executable file
View File

@ -0,0 +1,47 @@
#!/usr/bin/env python3
import time
from test_framework.test_framework import TestFramework
from utility.submission import create_submission, submit_data
from utility.utils import wait_until, assert_equal
class PrunerTest(TestFramework):
def setup_params(self):
self.num_blockchain_nodes = 1
self.num_nodes = 1
self.zgs_node_configs[0] = {
"db_max_num_chunks": 16 * 1024,
"prune_check_time_s": 1,
"prune_batch_wait_time_ms": 10,
}
def run_test(self):
client = self.nodes[0]
chunk_data = b"\x02" * 16 * 256 * 1024
submissions, data_root = create_submission(chunk_data)
self.contract.submit(submissions)
wait_until(lambda: self.contract.num_submissions() == 1)
wait_until(lambda: client.zgs_get_file_info(data_root) is not None)
segment = submit_data(client, chunk_data)
self.log.info("segment: %s", len(segment))
# Wait for 1 sec for the shard config to be updated
time.sleep(1)
shard_config = client.rpc.zgs_getShardConfig()
shard_id = int(shard_config["shardId"])
num_shard = int(shard_config["numShard"])
for i in range(len(segment)):
seg = client.zgs_download_segment(data_root, i * 1024, (i + 1) * 1024)
if i % num_shard == shard_id:
# base64 encoding size
assert_equal(len(seg), 349528)
else:
assert_equal(seg, None)
if __name__ == "__main__":
PrunerTest().main()