Compare commits

...

8 Commits

Author SHA1 Message Date
0g-peterzhb
e7cda8924d
Merge b0e273e141 into 949462084a 2024-09-30 18:43:39 +08:00
MiniFrenchBread
949462084a
fix: finalize check on tx sync (#220)
Some checks are pending
abi-consistent-check / build-and-compare (push) Waiting to run
code-coverage / unittest-cov (push) Waiting to run
rust / check (push) Waiting to run
rust / test (push) Waiting to run
rust / lints (push) Waiting to run
functional-test / test (push) Waiting to run
* fix: finalize

* fix: check

* fix: end_segment_index

* fix: fmt
2024-09-30 18:43:11 +08:00
MiniFrenchBread
07ac814e57
test: use fixed 0g binary (#218)
* test: use latest 0g binary

* feat: use bug fixed 0g binary
2024-09-30 14:56:38 +08:00
peilun-conflux
69b71fc0b2
Fix issue in reverting the last incomplete tx. (#215) 2024-09-27 10:02:47 +08:00
Bo QIU
ae6ecfec96
reduce default sync threads (#214) 2024-09-26 14:55:00 +08:00
Bo QIU
ad80b22a1b
Optimize rpc config (#213) 2024-09-25 16:48:40 +08:00
Bo QIU
84c415e959
opt zgs version (#211) 2024-09-24 16:42:38 +08:00
Bo QIU
9cde84ae15
Refactor network peer db configs (#209) 2024-09-24 11:59:34 +08:00
27 changed files with 302 additions and 198 deletions

1
Cargo.lock generated
View File

@ -5002,6 +5002,7 @@ dependencies = [
"directory", "directory",
"dirs 4.0.0", "dirs 4.0.0",
"discv5", "discv5",
"duration-str",
"error-chain", "error-chain",
"eth2_ssz", "eth2_ssz",
"eth2_ssz_derive", "eth2_ssz_derive",

View File

@ -7,41 +7,42 @@ use target_info::Target;
/// ///
/// ## Example /// ## Example
/// ///
/// `Lighthouse/v1.5.1-67da032+` /// `v0.5.2` or `v0.5.2-1-67da032+`
pub const VERSION: &str = git_version!( pub const VERSION: &str = git_version!(
args = [ args = [
"--always", "--always",
"--dirty=+", "--dirty=+",
"--abbrev=7", "--abbrev=7",
// NOTE: using --match instead of --exclude for compatibility with old Git // NOTE: using --match instead of --exclude for compatibility with old Git
"--match=thiswillnevermatchlol" // "--match=thiswillnevermatchlol"
"--tags",
], ],
prefix = "zgs/v0.0.1-", // prefix = "zgs/v0.0.1-",
fallback = "unknown" fallback = "unknown"
); );
/// Returns `VERSION`, but with platform information appended to the end. /// Returns `VERSION`, but with `zgs` prefix and platform information appended to the end.
/// ///
/// ## Example /// ## Example
/// ///
/// `zgs/v0.0.1-67da032+/x86_64-linux` /// `zgs/v0.5.2/x86_64-linux`
pub fn version_with_platform() -> String { pub fn version_with_platform() -> String {
format!("{}/{}-{}", VERSION, Target::arch(), Target::os()) format!("zgs/{}/{}-{}", VERSION, Target::arch(), Target::os())
} }
#[cfg(test)] // #[cfg(test)]
mod test { // mod test {
use super::*; // use super::*;
use regex::Regex; // use regex::Regex;
#[test] // #[test]
fn version_formatting() { // fn version_formatting() {
let re = // let re =
Regex::new(r"^zgs/v[0-9]+\.[0-9]+\.[0-9]+(-rc.[0-9])?-[[:xdigit:]]{7}\+?$").unwrap(); // Regex::new(r"^v[0-9]+\.[0-9]+\.[0-9]+(-rc.[0-9])?-[[:xdigit:]]{7}\+?$").unwrap();
assert!( // assert!(
re.is_match(VERSION), // re.is_match(VERSION),
"version doesn't match regex: {}", // "version doesn't match regex: {}",
VERSION // VERSION
); // );
} // }
} // }

View File

@ -5,7 +5,7 @@ edition = "2021"
[dependencies] [dependencies]
anyhow = { version = "1.0.58", features = ["backtrace"] } anyhow = { version = "1.0.58", features = ["backtrace"] }
clap = { version = "4.5.17", features = ["cargo"] } clap = { version = "4.5.17", features = ["cargo", "string"] }
ctrlc = "3.2.2" ctrlc = "3.2.2"
error-chain = "0.12.4" error-chain = "0.12.4"
ethereum-types = "0.14" ethereum-types = "0.14"

View File

@ -12,7 +12,7 @@ use std::fmt::Debug;
use std::future::Future; use std::future::Future;
use std::sync::Arc; use std::sync::Arc;
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use storage::log_store::log_manager::sector_to_segment; use storage::log_store::log_manager::PORA_CHUNK_SIZE;
use storage::log_store::{tx_store::BlockHashAndSubmissionIndex, Store}; use storage::log_store::{tx_store::BlockHashAndSubmissionIndex, Store};
use task_executor::{ShutdownReason, TaskExecutor}; use task_executor::{ShutdownReason, TaskExecutor};
use thiserror::Error; use thiserror::Error;
@ -479,11 +479,30 @@ impl LogSyncManager {
return false; return false;
} }
} else { } else {
// check if current node need to save at least one segment
let store = self.store.clone(); let store = self.store.clone();
let shard_config = store.flow().get_shard_config(); let shard_config = store.flow().get_shard_config();
if sector_to_segment(bytes_to_chunks(tx.size as usize) as u64) let start_segment_index = tx.start_entry_index as usize / PORA_CHUNK_SIZE;
< shard_config.shard_id let sector_size = bytes_to_chunks(tx.size as usize);
{ let end_segment_index = start_segment_index
+ ((sector_size + PORA_CHUNK_SIZE - 1) / PORA_CHUNK_SIZE)
- 1;
let mut can_finalize = false;
if end_segment_index < shard_config.shard_id {
can_finalize = true;
} else {
// check if there is a number N between [start_segment_index, end_segment_index] that satisfy:
// N % num_shard = shard_id
let min_n_gte_start =
(start_segment_index + shard_config.num_shard - 1 - shard_config.shard_id)
/ shard_config.num_shard;
let max_n_lte_end =
(end_segment_index - shard_config.shard_id) / shard_config.num_shard;
if min_n_gte_start > max_n_lte_end {
can_finalize = true;
}
}
if can_finalize {
if let Err(e) = store.finalize_tx_with_hash(tx.seq, tx.hash()) { if let Err(e) = store.finalize_tx_with_hash(tx.seq, tx.hash()) {
error!("finalize file that does not need to store: e={:?}", e); error!("finalize file that does not need to store: e={:?}", e);
return false; return false;

View File

@ -40,6 +40,7 @@ unsigned-varint = { version = "=0.7.1", features = ["codec"] }
if-addrs = "0.10.1" if-addrs = "0.10.1"
slog = "2.7.0" slog = "2.7.0"
igd = "0.12.1" igd = "0.12.1"
duration-str = "0.5.1"
[dependencies.libp2p] [dependencies.libp2p]
version = "0.45.1" version = "0.45.1"

View File

@ -1,3 +1,4 @@
use crate::peer_manager::peerdb::PeerDBConfig;
use crate::types::GossipKind; use crate::types::GossipKind;
use crate::{Enr, PeerIdSerialized}; use crate::{Enr, PeerIdSerialized};
use directory::{ use directory::{
@ -126,6 +127,8 @@ pub struct Config {
/// The id of the storage network. /// The id of the storage network.
pub network_id: NetworkIdentity, pub network_id: NetworkIdentity,
pub peer_db: PeerDBConfig,
} }
impl Default for Config { impl Default for Config {
@ -204,6 +207,7 @@ impl Default for Config {
topics: Vec::new(), topics: Vec::new(),
metrics_enabled: false, metrics_enabled: false,
network_id: Default::default(), network_id: Default::default(),
peer_db: Default::default(),
} }
} }
} }

View File

@ -3,13 +3,15 @@ use crate::{
multiaddr::{Multiaddr, Protocol}, multiaddr::{Multiaddr, Protocol},
Enr, Gossipsub, PeerId, Enr, Gossipsub, PeerId,
}; };
use duration_str::deserialize_duration;
use peer_info::{ConnectionDirection, PeerConnectionStatus, PeerInfo}; use peer_info::{ConnectionDirection, PeerConnectionStatus, PeerInfo};
use rand::seq::SliceRandom; use rand::seq::SliceRandom;
use score::{PeerAction, ReportSource, Score, ScoreState}; use score::{PeerAction, ReportSource, Score, ScoreState};
use std::cmp::Ordering; use serde::{Deserialize, Serialize};
use std::collections::{HashMap, HashSet}; use std::collections::{HashMap, HashSet};
use std::net::{IpAddr, SocketAddr}; use std::net::{IpAddr, SocketAddr};
use std::time::Instant; use std::time::Instant;
use std::{cmp::Ordering, time::Duration};
use sync_status::SyncStatus; use sync_status::SyncStatus;
pub mod client; pub mod client;
@ -17,21 +19,41 @@ pub mod peer_info;
pub mod score; pub mod score;
pub mod sync_status; pub mod sync_status;
/// Max number of disconnected nodes to remember.
const MAX_DC_PEERS: usize = 500;
/// The maximum number of banned nodes to remember.
pub const MAX_BANNED_PEERS: usize = 1000;
/// We ban an IP if there are more than `BANNED_PEERS_PER_IP_THRESHOLD` banned peers with this IP. /// We ban an IP if there are more than `BANNED_PEERS_PER_IP_THRESHOLD` banned peers with this IP.
const BANNED_PEERS_PER_IP_THRESHOLD: usize = 5; const BANNED_PEERS_PER_IP_THRESHOLD: usize = 5;
/// Relative factor of peers that are allowed to have a negative gossipsub score without penalizing
/// them in lighthouse. #[derive(Debug, Clone, Copy, Serialize, Deserialize)]
const ALLOWED_NEGATIVE_GOSSIPSUB_FACTOR: f32 = 0.1; #[serde(default)]
/// The time we allow peers to be in the dialing state in our PeerDb before we revert them to a pub struct PeerDBConfig {
/// disconnected state. /// The maximum number of disconnected nodes to remember.
const DIAL_TIMEOUT: u64 = 15; pub max_disconnected_peers: usize,
/// The maximum number of banned nodes to remember.
pub max_banned_peers: usize,
/// We ban an IP if there are more than `BANNED_PEERS_PER_IP_THRESHOLD` banned peers with this IP.
pub banned_peers_per_ip_threshold: usize,
/// Relative factor of peers that are allowed to have a negative gossipsub score without penalizing them in lighthouse.
pub allowed_negative_gossipsub_factor: f32,
/// The time we allow peers to be in the dialing state in our PeerDb before we revert them to a disconnected state.
#[serde(deserialize_with = "deserialize_duration")]
pub dail_timeout: Duration,
}
impl Default for PeerDBConfig {
fn default() -> Self {
Self {
max_disconnected_peers: 500,
max_banned_peers: 1000,
banned_peers_per_ip_threshold: 5,
allowed_negative_gossipsub_factor: 0.1,
dail_timeout: Duration::from_secs(15),
}
}
}
/// Storage of known peers, their reputation and information /// Storage of known peers, their reputation and information
pub struct PeerDB { pub struct PeerDB {
config: PeerDBConfig,
/// The collection of known connected peers, their status and reputation /// The collection of known connected peers, their status and reputation
peers: HashMap<PeerId, PeerInfo>, peers: HashMap<PeerId, PeerInfo>,
/// The number of disconnected nodes in the database. /// The number of disconnected nodes in the database.
@ -41,13 +63,14 @@ pub struct PeerDB {
} }
impl PeerDB { impl PeerDB {
pub fn new(trusted_peers: Vec<PeerId>) -> Self { pub fn new(config: PeerDBConfig, trusted_peers: Vec<PeerId>) -> Self {
// Initialize the peers hashmap with trusted peers // Initialize the peers hashmap with trusted peers
let peers = trusted_peers let peers = trusted_peers
.into_iter() .into_iter()
.map(|peer_id| (peer_id, PeerInfo::trusted_peer_info())) .map(|peer_id| (peer_id, PeerInfo::trusted_peer_info()))
.collect(); .collect();
Self { Self {
config,
disconnected_peers: 0, disconnected_peers: 0,
banned_peers_count: BannedPeersCount::default(), banned_peers_count: BannedPeersCount::default(),
peers, peers,
@ -316,9 +339,7 @@ impl PeerDB {
.iter() .iter()
.filter_map(|(peer_id, info)| { .filter_map(|(peer_id, info)| {
if let PeerConnectionStatus::Dialing { since } = info.connection_status() { if let PeerConnectionStatus::Dialing { since } = info.connection_status() {
if (*since) + std::time::Duration::from_secs(DIAL_TIMEOUT) if (*since) + self.config.dail_timeout < std::time::Instant::now() {
< std::time::Instant::now()
{
return Some(*peer_id); return Some(*peer_id);
} }
} }
@ -422,7 +443,7 @@ impl PeerDB {
peers.sort_unstable_by(|(.., s1), (.., s2)| s2.partial_cmp(s1).unwrap_or(Ordering::Equal)); peers.sort_unstable_by(|(.., s1), (.., s2)| s2.partial_cmp(s1).unwrap_or(Ordering::Equal));
let mut to_ignore_negative_peers = let mut to_ignore_negative_peers =
(target_peers as f32 * ALLOWED_NEGATIVE_GOSSIPSUB_FACTOR).ceil() as usize; (target_peers as f32 * self.config.allowed_negative_gossipsub_factor).ceil() as usize;
for (peer_id, info, score) in peers { for (peer_id, info, score) in peers {
let previous_state = info.score_state(); let previous_state = info.score_state();
@ -946,11 +967,11 @@ impl PeerDB {
let excess_peers = self let excess_peers = self
.banned_peers_count .banned_peers_count
.banned_peers() .banned_peers()
.saturating_sub(MAX_BANNED_PEERS); .saturating_sub(self.config.max_banned_peers);
let mut unbanned_peers = Vec::with_capacity(excess_peers); let mut unbanned_peers = Vec::with_capacity(excess_peers);
// Remove excess banned peers // Remove excess banned peers
while self.banned_peers_count.banned_peers() > MAX_BANNED_PEERS { while self.banned_peers_count.banned_peers() > self.config.max_banned_peers {
if let Some((to_drop, unbanned_ips)) = if let Some((id, info, _)) = self if let Some((to_drop, unbanned_ips)) = if let Some((id, info, _)) = self
.peers .peers
.iter() .iter()
@ -982,7 +1003,7 @@ impl PeerDB {
} }
// Remove excess disconnected peers // Remove excess disconnected peers
while self.disconnected_peers > MAX_DC_PEERS { while self.disconnected_peers > self.config.max_disconnected_peers {
if let Some(to_drop) = self if let Some(to_drop) = self
.peers .peers
.iter() .iter()
@ -1210,7 +1231,7 @@ mod tests {
} }
fn get_db() -> PeerDB { fn get_db() -> PeerDB {
PeerDB::new(vec![]) PeerDB::new(PeerDBConfig::default(), vec![])
} }
#[test] #[test]
@ -1265,7 +1286,7 @@ mod tests {
use std::collections::BTreeMap; use std::collections::BTreeMap;
let mut peer_list = BTreeMap::new(); let mut peer_list = BTreeMap::new();
for id in 0..MAX_DC_PEERS + 1 { for id in 0..pdb.config.max_disconnected_peers + 1 {
let new_peer = PeerId::random(); let new_peer = PeerId::random();
pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None);
peer_list.insert(id, new_peer); peer_list.insert(id, new_peer);
@ -1276,11 +1297,15 @@ mod tests {
pdb.inject_disconnect(p); pdb.inject_disconnect(p);
// Allow the timing to update correctly // Allow the timing to update correctly
} }
assert_eq!(pdb.disconnected_peers, MAX_DC_PEERS); assert_eq!(pdb.disconnected_peers, pdb.config.max_disconnected_peers);
assert_eq!(pdb.disconnected_peers, pdb.disconnected_peers().count()); assert_eq!(pdb.disconnected_peers, pdb.disconnected_peers().count());
// Only the oldest peer should have been removed // Only the oldest peer should have been removed
for (id, peer_id) in peer_list.iter().rev().take(MAX_DC_PEERS) { for (id, peer_id) in peer_list
.iter()
.rev()
.take(pdb.config.max_disconnected_peers)
{
println!("Testing id {}", id); println!("Testing id {}", id);
assert!( assert!(
pdb.peer_info(peer_id).is_some(), pdb.peer_info(peer_id).is_some(),
@ -1301,7 +1326,7 @@ mod tests {
use std::collections::BTreeMap; use std::collections::BTreeMap;
let mut peer_list = BTreeMap::new(); let mut peer_list = BTreeMap::new();
for id in 0..MAX_DC_PEERS + 20 { for id in 0..pdb.config.max_disconnected_peers + 20 {
let new_peer = PeerId::random(); let new_peer = PeerId::random();
pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None);
peer_list.insert(id, new_peer); peer_list.insert(id, new_peer);
@ -1314,7 +1339,7 @@ mod tests {
println!("{}", pdb.disconnected_peers); println!("{}", pdb.disconnected_peers);
peer_list.clear(); peer_list.clear();
for id in 0..MAX_DC_PEERS + 20 { for id in 0..pdb.config.max_disconnected_peers + 20 {
let new_peer = PeerId::random(); let new_peer = PeerId::random();
pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&new_peer, "/ip4/0.0.0.0".parse().unwrap(), None);
peer_list.insert(id, new_peer); peer_list.insert(id, new_peer);
@ -1345,7 +1370,7 @@ mod tests {
fn test_disconnected_are_bounded() { fn test_disconnected_are_bounded() {
let mut pdb = get_db(); let mut pdb = get_db();
for _ in 0..MAX_DC_PEERS + 1 { for _ in 0..pdb.config.max_disconnected_peers + 1 {
let p = PeerId::random(); let p = PeerId::random();
pdb.connect_ingoing(&p, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&p, "/ip4/0.0.0.0".parse().unwrap(), None);
} }
@ -1356,14 +1381,14 @@ mod tests {
} }
assert_eq!(pdb.disconnected_peers, pdb.disconnected_peers().count()); assert_eq!(pdb.disconnected_peers, pdb.disconnected_peers().count());
assert_eq!(pdb.disconnected_peers, MAX_DC_PEERS); assert_eq!(pdb.disconnected_peers, pdb.config.max_disconnected_peers);
} }
#[test] #[test]
fn test_banned_are_bounded() { fn test_banned_are_bounded() {
let mut pdb = get_db(); let mut pdb = get_db();
for _ in 0..MAX_BANNED_PEERS + 1 { for _ in 0..pdb.config.max_banned_peers + 1 {
let p = PeerId::random(); let p = PeerId::random();
pdb.connect_ingoing(&p, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&p, "/ip4/0.0.0.0".parse().unwrap(), None);
} }
@ -1374,7 +1399,10 @@ mod tests {
pdb.inject_disconnect(&p); pdb.inject_disconnect(&p);
} }
assert_eq!(pdb.banned_peers_count.banned_peers(), MAX_BANNED_PEERS); assert_eq!(
pdb.banned_peers_count.banned_peers(),
pdb.config.max_banned_peers
);
} }
#[test] #[test]
@ -1908,7 +1936,7 @@ mod tests {
#[allow(clippy::float_cmp)] #[allow(clippy::float_cmp)]
fn test_trusted_peers_score() { fn test_trusted_peers_score() {
let trusted_peer = PeerId::random(); let trusted_peer = PeerId::random();
let mut pdb: PeerDB = PeerDB::new(vec![trusted_peer]); let mut pdb: PeerDB = PeerDB::new(PeerDBConfig::default(), vec![trusted_peer]);
pdb.connect_ingoing(&trusted_peer, "/ip4/0.0.0.0".parse().unwrap(), None); pdb.connect_ingoing(&trusted_peer, "/ip4/0.0.0.0".parse().unwrap(), None);

View File

@ -84,6 +84,7 @@ impl<AppReqId: ReqId> Service<AppReqId> {
.iter() .iter()
.map(|x| PeerId::from(x.clone())) .map(|x| PeerId::from(x.clone()))
.collect(), .collect(),
config.peer_db,
config.network_id.clone(), config.network_id.clone(),
)); ));

View File

@ -1,5 +1,6 @@
//! A collection of variables that are accessible outside of the network thread itself. //! A collection of variables that are accessible outside of the network thread itself.
use crate::peer_manager::peerdb::PeerDB; use crate::peer_manager::peerdb::PeerDB;
use crate::peer_manager::peerdb::PeerDBConfig;
use crate::Client; use crate::Client;
use crate::EnrExt; use crate::EnrExt;
use crate::{Enr, GossipTopic, Multiaddr, PeerId}; use crate::{Enr, GossipTopic, Multiaddr, PeerId};
@ -34,6 +35,7 @@ impl NetworkGlobals {
tcp_port: u16, tcp_port: u16,
udp_port: u16, udp_port: u16,
trusted_peers: Vec<PeerId>, trusted_peers: Vec<PeerId>,
peer_db_config: PeerDBConfig,
network_id: NetworkIdentity, network_id: NetworkIdentity,
) -> Self { ) -> Self {
NetworkGlobals { NetworkGlobals {
@ -42,7 +44,7 @@ impl NetworkGlobals {
listen_multiaddrs: RwLock::new(Vec::new()), listen_multiaddrs: RwLock::new(Vec::new()),
listen_port_tcp: AtomicU16::new(tcp_port), listen_port_tcp: AtomicU16::new(tcp_port),
listen_port_udp: AtomicU16::new(udp_port), listen_port_udp: AtomicU16::new(udp_port),
peers: RwLock::new(PeerDB::new(trusted_peers)), peers: RwLock::new(PeerDB::new(peer_db_config, trusted_peers)),
gossipsub_subscriptions: RwLock::new(HashSet::new()), gossipsub_subscriptions: RwLock::new(HashSet::new()),
network_id: RwLock::new(network_id), network_id: RwLock::new(network_id),
} }
@ -110,6 +112,13 @@ impl NetworkGlobals {
let enr_key: discv5::enr::CombinedKey = let enr_key: discv5::enr::CombinedKey =
discv5::enr::CombinedKey::from_libp2p(&keypair).unwrap(); discv5::enr::CombinedKey::from_libp2p(&keypair).unwrap();
let enr = discv5::enr::EnrBuilder::new("v4").build(&enr_key).unwrap(); let enr = discv5::enr::EnrBuilder::new("v4").build(&enr_key).unwrap();
NetworkGlobals::new(enr, 9000, 9000, vec![], Default::default()) NetworkGlobals::new(
enr,
9000,
9000,
vec![],
Default::default(),
Default::default(),
)
} }
} }

View File

@ -9,7 +9,7 @@ use common::{
swarm, swarm,
}; };
use network::{ use network::{
peer_manager::{self, config::Config, PeerManagerEvent}, peer_manager::{config::Config, peerdb::PeerDBConfig, PeerManagerEvent},
NetworkGlobals, PeerAction, PeerInfo, PeerManager, ReportSource, NetworkGlobals, PeerAction, PeerInfo, PeerManager, ReportSource,
}; };
@ -101,7 +101,7 @@ async fn banned_peers_consistency() {
}; };
let excess_banned_peers = 15; let excess_banned_peers = 15;
let peers_to_ban = peer_manager::peerdb::MAX_BANNED_PEERS + excess_banned_peers; let peers_to_ban = PeerDBConfig::default().max_banned_peers + excess_banned_peers;
// Build all the dummy peers needed. // Build all the dummy peers needed.
let (mut swarm_pool, peers) = { let (mut swarm_pool, peers) = {

View File

@ -948,8 +948,14 @@ mod tests {
let keypair = Keypair::generate_secp256k1(); let keypair = Keypair::generate_secp256k1();
let enr_key = CombinedKey::from_libp2p(&keypair).unwrap(); let enr_key = CombinedKey::from_libp2p(&keypair).unwrap();
let enr = EnrBuilder::new("v4").build(&enr_key).unwrap(); let enr = EnrBuilder::new("v4").build(&enr_key).unwrap();
let network_globals = let network_globals = NetworkGlobals::new(
NetworkGlobals::new(enr, 30000, 30000, vec![], Default::default()); enr,
30000,
30000,
vec![],
Default::default(),
Default::default(),
);
let listen_addr: Multiaddr = "/ip4/127.0.0.1/tcp/30000".parse().unwrap(); let listen_addr: Multiaddr = "/ip4/127.0.0.1/tcp/30000".parse().unwrap();
network_globals.listen_multiaddrs.write().push(listen_addr); network_globals.listen_multiaddrs.write().push(listen_addr);

View File

@ -1,11 +1,27 @@
use std::net::SocketAddr; use std::{net::SocketAddr, str::FromStr};
#[derive(Clone)] use serde::{Deserialize, Serialize};
#[derive(Clone, Copy, Debug, Serialize, Deserialize)]
#[serde(default)]
pub struct Config { pub struct Config {
pub enabled: bool, pub enabled: bool,
pub listen_address: SocketAddr, pub listen_address: SocketAddr,
pub listen_address_admin: Option<SocketAddr>, pub listen_address_admin: SocketAddr,
pub chunks_per_segment: usize, pub chunks_per_segment: usize,
pub max_request_body_size: u32, pub max_request_body_size: u32,
pub max_cache_file_size: usize, pub max_cache_file_size: usize,
} }
impl Default for Config {
fn default() -> Self {
Self {
enabled: true,
listen_address: SocketAddr::from_str("0.0.0.0:5678").unwrap(),
listen_address_admin: SocketAddr::from_str("127.0.0.1:5679").unwrap(),
chunks_per_segment: 1024,
max_request_body_size: 100 * 1024 * 1024, // 100MB
max_cache_file_size: 10 * 1024 * 1024, // 10MB
}
}
}

View File

@ -20,7 +20,6 @@ use jsonrpsee::http_server::{HttpServerBuilder, HttpServerHandle};
use network::NetworkGlobals; use network::NetworkGlobals;
use network::NetworkMessage; use network::NetworkMessage;
use std::error::Error; use std::error::Error;
use std::net::SocketAddr;
use std::sync::Arc; use std::sync::Arc;
use storage_async::Store; use storage_async::Store;
use sync::{SyncRequest, SyncResponse, SyncSender}; use sync::{SyncRequest, SyncResponse, SyncSender};
@ -69,9 +68,10 @@ impl Context {
pub async fn run_server( pub async fn run_server(
ctx: Context, ctx: Context,
) -> Result<(HttpServerHandle, Option<HttpServerHandle>), Box<dyn Error>> { ) -> Result<(HttpServerHandle, Option<HttpServerHandle>), Box<dyn Error>> {
let handles = match ctx.config.listen_address_admin { let handles = if ctx.config.listen_address.port() != ctx.config.listen_address_admin.port() {
Some(listen_addr_private) => run_server_public_private(ctx, listen_addr_private).await?, run_server_public_private(ctx).await?
None => (run_server_all(ctx).await?, None), } else {
(run_server_all(ctx).await?, None)
}; };
info!("Server started"); info!("Server started");
@ -107,7 +107,6 @@ async fn run_server_all(ctx: Context) -> Result<HttpServerHandle, Box<dyn Error>
/// Run 2 RPC servers (public & private) for different namespace RPCs. /// Run 2 RPC servers (public & private) for different namespace RPCs.
async fn run_server_public_private( async fn run_server_public_private(
ctx: Context, ctx: Context,
listen_addr_private: SocketAddr,
) -> Result<(HttpServerHandle, Option<HttpServerHandle>), Box<dyn Error>> { ) -> Result<(HttpServerHandle, Option<HttpServerHandle>), Box<dyn Error>> {
// public rpc // public rpc
let zgs = (zgs::RpcServerImpl { ctx: ctx.clone() }).into_rpc(); let zgs = (zgs::RpcServerImpl { ctx: ctx.clone() }).into_rpc();
@ -127,7 +126,7 @@ async fn run_server_public_private(
.start(zgs)?; .start(zgs)?;
let handle_private = server_builder(ctx.clone()) let handle_private = server_builder(ctx.clone())
.build(listen_addr_private) .build(ctx.config.listen_address_admin)
.await? .await?
.start(admin)?; .start(admin)?;

View File

@ -10,4 +10,5 @@ pub fn cli_app() -> Command {
) )
.arg(arg!(--"db-max-num-chunks" [NUM] "Sets the max number of chunks to store in db (Default: None)")) .arg(arg!(--"db-max-num-chunks" [NUM] "Sets the max number of chunks to store in db (Default: None)"))
.allow_external_subcommands(true) .allow_external_subcommands(true)
.version(zgs_version::VERSION)
} }

View File

@ -7,7 +7,6 @@ use log_entry_sync::{CacheConfig, ContractAddress, LogSyncConfig};
use miner::MinerConfig; use miner::MinerConfig;
use network::NetworkConfig; use network::NetworkConfig;
use pruner::PrunerConfig; use pruner::PrunerConfig;
use rpc::RPCConfig;
use shared_types::{NetworkIdentity, ProtocolVersion}; use shared_types::{NetworkIdentity, ProtocolVersion};
use std::net::IpAddr; use std::net::IpAddr;
use std::time::Duration; use std::time::Duration;
@ -96,6 +95,8 @@ impl ZgsConfig {
network_config.target_peers = self.network_target_peers; network_config.target_peers = self.network_target_peers;
network_config.private = self.network_private; network_config.private = self.network_private;
network_config.peer_db = self.network_peer_db;
Ok(network_config) Ok(network_config)
} }
@ -105,32 +106,6 @@ impl ZgsConfig {
}) })
} }
pub fn rpc_config(&self) -> Result<RPCConfig, String> {
let listen_address = self
.rpc_listen_address
.parse::<std::net::SocketAddr>()
.map_err(|e| format!("Unable to parse rpc_listen_address: {:?}", e))?;
let listen_address_admin = if self.rpc_listen_address_admin.is_empty() {
None
} else {
Some(
self.rpc_listen_address_admin
.parse::<std::net::SocketAddr>()
.map_err(|e| format!("Unable to parse rpc_listen_address_admin: {:?}", e))?,
)
};
Ok(RPCConfig {
enabled: self.rpc_enabled,
listen_address,
listen_address_admin,
max_request_body_size: self.max_request_body_size,
chunks_per_segment: self.rpc_chunks_per_segment,
max_cache_file_size: self.rpc_max_cache_file_size,
})
}
pub fn log_sync_config(&self) -> Result<LogSyncConfig, String> { pub fn log_sync_config(&self) -> Result<LogSyncConfig, String> {
let contract_address = self let contract_address = self
.log_contract_address .log_contract_address

View File

@ -48,14 +48,6 @@ build_config! {
(remove_finalized_block_interval_minutes, (u64), 30) (remove_finalized_block_interval_minutes, (u64), 30)
(watch_loop_wait_time_ms, (u64), 500) (watch_loop_wait_time_ms, (u64), 500)
// rpc
(rpc_enabled, (bool), true)
(rpc_listen_address, (String), "0.0.0.0:5678".to_string())
(rpc_listen_address_admin, (String), "127.0.0.1:5679".to_string())
(max_request_body_size, (u32), 100*1024*1024) // 100MB
(rpc_chunks_per_segment, (usize), 1024)
(rpc_max_cache_file_size, (usize), 10*1024*1024) //10MB
// chunk pool // chunk pool
(chunk_pool_write_window_size, (usize), 4) (chunk_pool_write_window_size, (usize), 4)
(chunk_pool_max_cached_chunks_all, (usize), 4*1024*1024) // 1G (chunk_pool_max_cached_chunks_all, (usize), 4*1024*1024) // 1G
@ -91,6 +83,9 @@ build_config! {
pub struct ZgsConfig { pub struct ZgsConfig {
pub raw_conf: RawConfiguration, pub raw_conf: RawConfiguration,
/// Network peer db config, configured by [network_peer_db] section by `config` crate.
pub network_peer_db: network::peer_manager::peerdb::PeerDBConfig,
// router config, configured by [router] section by `config` crate. // router config, configured by [router] section by `config` crate.
pub router: router::Config, pub router: router::Config,
@ -100,6 +95,9 @@ pub struct ZgsConfig {
// file location cache config, configured by [file_location_cache] section by `config` crate. // file location cache config, configured by [file_location_cache] section by `config` crate.
pub file_location_cache: file_location_cache::Config, pub file_location_cache: file_location_cache::Config,
// rpc config, configured by [rpc] section by `config` crate.
pub rpc: rpc::RPCConfig,
// metrics config, configured by [metrics] section by `config` crate. // metrics config, configured by [metrics] section by `config` crate.
pub metrics: metrics::MetricsConfiguration, pub metrics: metrics::MetricsConfiguration,
} }

View File

@ -13,7 +13,6 @@ use std::error::Error;
async fn start_node(context: RuntimeContext, config: ZgsConfig) -> Result<Client, String> { async fn start_node(context: RuntimeContext, config: ZgsConfig) -> Result<Client, String> {
let network_config = config.network_config().await?; let network_config = config.network_config().await?;
let storage_config = config.storage_config()?; let storage_config = config.storage_config()?;
let rpc_config = config.rpc_config()?;
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)?;
@ -33,7 +32,7 @@ async fn start_node(context: RuntimeContext, config: ZgsConfig) -> Result<Client
.await? .await?
.with_pruner(pruner_config) .with_pruner(pruner_config)
.await? .await?
.with_rpc(rpc_config, config.chunk_pool_config()?) .with_rpc(config.rpc, config.chunk_pool_config()?)
.await? .await?
.with_router(router_config)? .with_router(router_config)?
.build() .build()

View File

@ -189,15 +189,8 @@ impl EntryBatchData {
} }
pub fn insert_data(&mut self, start_byte: usize, mut data: Vec<u8>) -> Result<Vec<u16>> { pub fn insert_data(&mut self, start_byte: usize, mut data: Vec<u8>) -> Result<Vec<u16>> {
assert!(start_byte % BYTES_PER_SECTOR == 0); assert_eq!(start_byte % BYTES_PER_SECTOR, 0);
assert!(data.len() % BYTES_PER_SECTOR == 0); assert_eq!(data.len() % BYTES_PER_SECTOR, 0);
if data.is_empty() || self.get(start_byte, data.len()) == Some(&data) {
// TODO(zz): This assumes the caller has processed chain reorg (truncate flow) before
// inserting new data, and the data of the same file are always inserted with the
// same pattern.
return Ok(vec![]);
}
// Check if the entry is completed // Check if the entry is completed
let (list, subtree_list) = if let EntryBatchData::Incomplete(x) = self { let (list, subtree_list) = if let EntryBatchData::Incomplete(x) = self {

View File

@ -127,6 +127,14 @@ impl EntryBatch {
/// Return `Error` if the new data overlaps with old data. /// Return `Error` if the new data overlaps with old data.
/// Convert `Incomplete` to `Completed` if the chunk is completed after the insertion. /// Convert `Incomplete` to `Completed` if the chunk is completed after the insertion.
pub fn insert_data(&mut self, offset: usize, data: Vec<u8>) -> Result<Vec<u16>> { pub fn insert_data(&mut self, offset: usize, data: Vec<u8>) -> Result<Vec<u16>> {
if data.is_empty()
|| self
.get_unsealed_data(offset, data.len() / BYTES_PER_SECTOR)
.as_ref()
== Some(&data)
{
return Ok(vec![]);
}
self.data.insert_data(offset * BYTES_PER_SECTOR, data) self.data.insert_data(offset * BYTES_PER_SECTOR, data)
} }

View File

@ -708,6 +708,7 @@ impl LogManager {
}; };
if let Some(tx) = last_tx_to_insert { if let Some(tx) = last_tx_to_insert {
log_manager.revert_to(tx.seq - 1)?;
log_manager.put_tx(tx)?; log_manager.put_tx(tx)?;
let mut merkle = log_manager.merkle.write(); let mut merkle = log_manager.merkle.write();
for (index, h) in extra_leaves { for (index, h) in extra_leaves {

View File

@ -66,7 +66,7 @@ impl Default for Config {
// sync service config // sync service config
heartbeat_interval: Duration::from_secs(5), heartbeat_interval: Duration::from_secs(5),
auto_sync_enabled: false, auto_sync_enabled: false,
max_sync_files: 32, max_sync_files: 8,
sync_file_by_rpc_enabled: true, sync_file_by_rpc_enabled: true,
sync_file_on_announcement_enabled: false, sync_file_on_announcement_enabled: false,
@ -86,7 +86,7 @@ impl Default for Config {
auto_sync_idle_interval: Duration::from_secs(3), auto_sync_idle_interval: Duration::from_secs(3),
auto_sync_error_interval: Duration::from_secs(10), auto_sync_error_interval: Duration::from_secs(10),
max_sequential_workers: 0, max_sequential_workers: 0,
max_random_workers: 30, max_random_workers: 2,
sequential_find_peer_timeout: Duration::from_secs(60), sequential_find_peer_timeout: Duration::from_secs(60),
random_find_peer_timeout: Duration::from_secs(500), random_find_peer_timeout: Duration::from_secs(500),
} }

View File

@ -120,28 +120,6 @@ log_sync_start_block_number = 595059
# Watch_loop (eth_getLogs) trigger interval. # Watch_loop (eth_getLogs) trigger interval.
# watch_loop_wait_time_ms = 500 # watch_loop_wait_time_ms = 500
#######################################################################
### RPC Config Options ###
#######################################################################
# Whether to provide RPC service.
# rpc_enabled = true
# HTTP server address to bind for public RPC.
# rpc_listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# rpc_listen_address_admin = "127.0.0.1:5679"
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Number of chunks for a single segment.
# rpc_chunks_per_segment = 1024
# Maximum file size that allowed to cache in memory (by default, 10MB).
# rpc_max_cache_file_size = 10485760
####################################################################### #######################################################################
### Chunk Pool Config Options ### ### Chunk Pool Config Options ###
####################################################################### #######################################################################
@ -218,6 +196,18 @@ reward_contract_address = "0x0496D0817BD8519e0de4894Dc379D35c35275609"
# #
# prune_batch_wait_time_ms = 1000 # prune_batch_wait_time_ms = 1000
#######################################################################
### Network Peer DB Config Options ###
#######################################################################
# [network_peer_db]
# The maximum number of disconnected nodes to remember.
# max_disconnected_peers = 500
# The maximum number of banned nodes to remember.
# max_banned_peers = 1000
####################################################################### #######################################################################
### Router Config Options ### ### Router Config Options ###
####################################################################### #######################################################################
@ -243,7 +233,7 @@ batcher_announcement_capacity = 100
auto_sync_enabled = true auto_sync_enabled = true
# Maximum number of files in sync from other peers simultaneously. # Maximum number of files in sync from other peers simultaneously.
# max_sync_files = 32 # max_sync_files = 8
# Enable to start a file sync via RPC (e.g. `admin_startSyncFile`). # Enable to start a file sync via RPC (e.g. `admin_startSyncFile`).
# sync_file_by_rpc_enabled = true # sync_file_by_rpc_enabled = true
@ -271,7 +261,7 @@ auto_sync_enabled = true
# max_sequential_workers = 0 # max_sequential_workers = 0
# Maximum threads to sync files randomly. # Maximum threads to sync files randomly.
# max_random_workers = 30 # max_random_workers = 2
# Timeout to terminate a file sync in sequence. # Timeout to terminate a file sync in sequence.
# sequential_find_peer_timeout = "60s" # sequential_find_peer_timeout = "60s"
@ -300,6 +290,30 @@ auto_sync_enabled = true
# If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache. # If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache.
# entry_expiration_time_secs = 86400 # entry_expiration_time_secs = 86400
#######################################################################
### RPC Config Options ###
#######################################################################
# [rpc]
# Whether to provide RPC service.
# enabled = true
# HTTP server address to bind for public RPC.
# listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# listen_address_admin = "127.0.0.1:5679"
# Number of chunks for a single segment.
# chunks_per_segment = 1024
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Maximum file size that allowed to cache in memory (by default, 10MB).
# max_cache_file_size = 10485760
####################################################################### #######################################################################
### Metrics Options ### ### Metrics Options ###
####################################################################### #######################################################################

View File

@ -120,28 +120,6 @@ log_sync_start_block_number = 595059
# Watch_loop (eth_getLogs) trigger interval. # Watch_loop (eth_getLogs) trigger interval.
# watch_loop_wait_time_ms = 500 # watch_loop_wait_time_ms = 500
#######################################################################
### RPC Config Options ###
#######################################################################
# Whether to provide RPC service.
# rpc_enabled = true
# HTTP server address to bind for public RPC.
# rpc_listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# rpc_listen_address_admin = "127.0.0.1:5679"
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Number of chunks for a single segment.
# rpc_chunks_per_segment = 1024
# Maximum file size that allowed to cache in memory (by default, 10MB).
# rpc_max_cache_file_size = 10485760
####################################################################### #######################################################################
### Chunk Pool Config Options ### ### Chunk Pool Config Options ###
####################################################################### #######################################################################
@ -230,6 +208,18 @@ reward_contract_address = "0x51998C4d486F406a788B766d93510980ae1f9360"
# #
# prune_batch_wait_time_ms = 1000 # prune_batch_wait_time_ms = 1000
#######################################################################
### Network Peer DB Config Options ###
#######################################################################
# [network_peer_db]
# The maximum number of disconnected nodes to remember.
# max_disconnected_peers = 500
# The maximum number of banned nodes to remember.
# max_banned_peers = 1000
####################################################################### #######################################################################
### Router Config Options ### ### Router Config Options ###
####################################################################### #######################################################################
@ -255,7 +245,7 @@ batcher_announcement_capacity = 100
auto_sync_enabled = true auto_sync_enabled = true
# Maximum number of files in sync from other peers simultaneously. # Maximum number of files in sync from other peers simultaneously.
# max_sync_files = 32 # max_sync_files = 8
# Enable to start a file sync via RPC (e.g. `admin_startSyncFile`). # Enable to start a file sync via RPC (e.g. `admin_startSyncFile`).
# sync_file_by_rpc_enabled = true # sync_file_by_rpc_enabled = true
@ -283,7 +273,7 @@ auto_sync_enabled = true
# max_sequential_workers = 0 # max_sequential_workers = 0
# Maximum threads to sync files randomly. # Maximum threads to sync files randomly.
# max_random_workers = 30 # max_random_workers = 2
# Timeout to terminate a file sync in sequence. # Timeout to terminate a file sync in sequence.
# sequential_find_peer_timeout = "60s" # sequential_find_peer_timeout = "60s"
@ -312,6 +302,30 @@ auto_sync_enabled = true
# If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache. # If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache.
# entry_expiration_time_secs = 86400 # entry_expiration_time_secs = 86400
#######################################################################
### RPC Config Options ###
#######################################################################
# [rpc]
# Whether to provide RPC service.
# enabled = true
# HTTP server address to bind for public RPC.
# listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# listen_address_admin = "127.0.0.1:5679"
# Number of chunks for a single segment.
# chunks_per_segment = 1024
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Maximum file size that allowed to cache in memory (by default, 10MB).
# max_cache_file_size = 10485760
####################################################################### #######################################################################
### Metrics Options ### ### Metrics Options ###
####################################################################### #######################################################################

View File

@ -120,28 +120,6 @@
# Watch_loop (eth_getLogs) trigger interval. # Watch_loop (eth_getLogs) trigger interval.
# watch_loop_wait_time_ms = 500 # watch_loop_wait_time_ms = 500
#######################################################################
### RPC Config Options ###
#######################################################################
# Whether to provide RPC service.
# rpc_enabled = true
# HTTP server address to bind for public RPC.
# rpc_listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# rpc_listen_address_admin = "127.0.0.1:5679"
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Number of chunks for a single segment.
# rpc_chunks_per_segment = 1024
# Maximum file size that allowed to cache in memory (by default, 10MB).
# rpc_max_cache_file_size = 10485760
####################################################################### #######################################################################
### Chunk Pool Config Options ### ### Chunk Pool Config Options ###
####################################################################### #######################################################################
@ -232,6 +210,18 @@
# #
# prune_batch_wait_time_ms = 1000 # prune_batch_wait_time_ms = 1000
#######################################################################
### Network Peer DB Config Options ###
#######################################################################
# [network_peer_db]
# The maximum number of disconnected nodes to remember.
# max_disconnected_peers = 500
# The maximum number of banned nodes to remember.
# max_banned_peers = 1000
####################################################################### #######################################################################
### Router Config Options ### ### Router Config Options ###
####################################################################### #######################################################################
@ -257,7 +247,7 @@
# auto_sync_enabled = false # auto_sync_enabled = false
# Maximum number of files in sync from other peers simultaneously. # Maximum number of files in sync from other peers simultaneously.
# max_sync_files = 32 # max_sync_files = 8
# Enable to start a file sync via RPC (e.g. `admin_startSyncFile`). # Enable to start a file sync via RPC (e.g. `admin_startSyncFile`).
# sync_file_by_rpc_enabled = true # sync_file_by_rpc_enabled = true
@ -285,7 +275,7 @@
# max_sequential_workers = 0 # max_sequential_workers = 0
# Maximum threads to sync files randomly. # Maximum threads to sync files randomly.
# max_random_workers = 30 # max_random_workers = 2
# Timeout to terminate a file sync in sequence. # Timeout to terminate a file sync in sequence.
# sequential_find_peer_timeout = "60s" # sequential_find_peer_timeout = "60s"
@ -314,6 +304,30 @@
# If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache. # If the timestamp in the storage location information exceeds this duration from the current time, it will be removed from the cache.
# entry_expiration_time_secs = 86400 # entry_expiration_time_secs = 86400
#######################################################################
### RPC Config Options ###
#######################################################################
# [rpc]
# Whether to provide RPC service.
# enabled = true
# HTTP server address to bind for public RPC.
# listen_address = "0.0.0.0:5678"
# HTTP server address to bind for admin and debug RPC.
# listen_address_admin = "127.0.0.1:5679"
# Number of chunks for a single segment.
# chunks_per_segment = 1024
# Maximum data size of RPC request body (by default, 100MB).
# max_request_body_size = 104857600
# Maximum file size that allowed to cache in memory (by default, 10MB).
# max_cache_file_size = 10485760
####################################################################### #######################################################################
### Metrics Options ### ### Metrics Options ###
####################################################################### #######################################################################

View File

@ -28,7 +28,7 @@ for ((i=0; i<$NUM_NODES; i++)) do
TMP_GENESIS=$ROOT_DIR/node$i/config/tmp_genesis.json TMP_GENESIS=$ROOT_DIR/node$i/config/tmp_genesis.json
# Replace stake with neuron # Replace stake with neuron
$SED_I 's/stake/ua0gi/g' "$GENESIS" $SED_I 's/"stake"/"ua0gi"/g' "$GENESIS"
# Replace the default evm denom of aphoton with neuron # Replace the default evm denom of aphoton with neuron
$SED_I 's/aphoton/neuron/g' "$GENESIS" $SED_I 's/aphoton/neuron/g' "$GENESIS"

View File

@ -34,11 +34,15 @@ class ZgsNode(TestNode):
for i in range(index): for i in range(index):
libp2p_nodes.append(f"/ip4/127.0.0.1/tcp/{p2p_port(i)}") libp2p_nodes.append(f"/ip4/127.0.0.1/tcp/{p2p_port(i)}")
rpc_listen_address = f"127.0.0.1:{rpc_port(index)}"
indexed_config = { indexed_config = {
"network_libp2p_port": p2p_port(index), "network_libp2p_port": p2p_port(index),
"network_discovery_port": p2p_port(index), "network_discovery_port": p2p_port(index),
"rpc_listen_address": f"127.0.0.1:{rpc_port(index)}", "rpc": {
"rpc_listen_address_admin": "", "listen_address": rpc_listen_address,
"listen_address_admin": rpc_listen_address,
},
"network_libp2p_nodes": libp2p_nodes, "network_libp2p_nodes": libp2p_nodes,
"log_contract_address": log_contract_address, "log_contract_address": log_contract_address,
"mine_contract_address": mine_contract_address, "mine_contract_address": mine_contract_address,
@ -50,7 +54,7 @@ class ZgsNode(TestNode):
# Overwrite with personalized configs. # Overwrite with personalized configs.
update_config(local_conf, updated_config) update_config(local_conf, updated_config)
data_dir = os.path.join(root_dir, "zgs_node" + str(index)) data_dir = os.path.join(root_dir, "zgs_node" + str(index))
rpc_url = "http://" + local_conf["rpc_listen_address"] rpc_url = "http://" + rpc_listen_address
super().__init__( super().__init__(
NodeType.Zgs, NodeType.Zgs,
index, index,

View File

@ -16,7 +16,6 @@ BSC_BINARY = "geth.exe" if is_windows_platform() else "geth"
ZG_BINARY = "0gchaind.exe" if is_windows_platform() else "0gchaind" ZG_BINARY = "0gchaind.exe" if is_windows_platform() else "0gchaind"
CLIENT_BINARY = "0g-storage-client.exe" if is_windows_platform() else "0g-storage-client" CLIENT_BINARY = "0g-storage-client.exe" if is_windows_platform() else "0g-storage-client"
ZG_GIT_REV = "7bc25a060fab9c17bc9942b6747cd07a668d3042" # v0.1.0
CLI_GIT_REV = "98d74b7e7e6084fc986cb43ce2c66692dac094a6" CLI_GIT_REV = "98d74b7e7e6084fc986cb43ce2c66692dac094a6"
@unique @unique
@ -76,8 +75,7 @@ def build_zg(dir: str) -> BuildBinaryResult:
dir=dir, dir=dir,
binary_name=ZG_BINARY, binary_name=ZG_BINARY,
github_url="https://github.com/0glabs/0g-chain.git", github_url="https://github.com/0glabs/0g-chain.git",
git_rev=ZG_GIT_REV, build_cmd="git fetch origin pull/74/head:pr-74; git checkout pr-74; make install; cp $(go env GOPATH)/bin/0gchaind .",
build_cmd="make install; cp $(go env GOPATH)/bin/0gchaind .",
compiled_relative_path=[], compiled_relative_path=[],
) )