Skip to content

Commit 54a2c5e

Browse files
author
Hyunsik Jeong
committed
Add reseal timer to client
1 parent e2c8808 commit 54a2c5e

File tree

6 files changed

+167
-58
lines changed

6 files changed

+167
-58
lines changed

codechain/run_node.rs

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -86,14 +86,22 @@ fn discovery_start(service: &NetworkService, cfg: &config::Network) -> Result<()
8686
Ok(())
8787
}
8888

89-
fn client_start(cfg: &config::Operating, scheme: &Scheme, miner: Arc<Miner>) -> Result<ClientService, String> {
89+
fn client_start(
90+
timer_loop: &TimerLoop,
91+
cfg: &config::Operating,
92+
scheme: &Scheme,
93+
miner: Arc<Miner>,
94+
) -> Result<ClientService, String> {
9095
cinfo!(CLIENT, "Starting client");
9196
let db_path = cfg.db_path.as_ref().map(|s| s.as_str()).unwrap();
9297
let client_path = Path::new(db_path);
9398
let client_config = Default::default();
9499
let service = ClientService::start(&client_config, &scheme, &client_path, miner)
95100
.map_err(|e| format!("Client service error: {}", e))?;
96101

102+
let reseal_timer = timer_loop.new_timer("Client reseal timer", service.client());
103+
service.client().register_reseal_timer(reseal_timer);
104+
97105
Ok(service)
98106
}
99107

@@ -229,7 +237,7 @@ pub fn run_node(matches: &ArgMatches) -> Result<(), String> {
229237
unlock_accounts(&*ap, &pf)?;
230238

231239
let miner = new_miner(&config, &scheme, ap.clone())?;
232-
let client = client_start(&config.operating, &scheme, miner.clone())?;
240+
let client = client_start(&timer_loop, &config.operating, &scheme, miner.clone())?;
233241
let sync = BlockSyncExtension::new(client.client());
234242

235243
scheme.engine.register_chain_notify(client.client().as_ref());

core/src/client/client.rs

Lines changed: 63 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ use cnetwork::NodeId;
2525
use cstate::{
2626
ActionHandler, AssetScheme, AssetSchemeAddress, OwnedAsset, OwnedAssetAddress, StateDB, TopLevelState, TopStateView,
2727
};
28+
use ctimer::{TimeoutHandler, TimerApi, TimerToken};
2829
use ctypes::invoice::Invoice;
2930
use ctypes::transaction::Transaction;
3031
use ctypes::{BlockNumber, ShardId};
@@ -41,7 +42,8 @@ use super::{
4142
AccountData, AssetClient, Balance, BlockChain as BlockChainTrait, BlockChainClient, BlockChainInfo, BlockInfo,
4243
BlockProducer, ChainInfo, ChainNotify, ClientConfig, DatabaseClient, EngineClient, EngineInfo,
4344
Error as ClientError, ExecuteClient, ImportBlock, ImportResult, ImportSealedBlock, MiningBlockChainClient,
44-
ParcelInfo, PrepareOpenBlock, RegularKey, RegularKeyOwner, ReopenBlock, Seq, Shard, StateOrBlock, TransactionInfo,
45+
ParcelInfo, PrepareOpenBlock, RegularKey, RegularKeyOwner, ReopenBlock, ResealTimer, Seq, Shard, StateOrBlock,
46+
TransactionInfo,
4547
};
4648
use crate::block::{ClosedBlock, IsBlock, OpenBlock, SealedBlock};
4749
use crate::blockchain::{
@@ -79,6 +81,9 @@ pub struct Client {
7981
genesis_accounts: Vec<Address>,
8082

8183
importer: Importer,
84+
85+
/// Timer for reseal_min_period/reseal_max_period on miner client
86+
reseal_timer: RwLock<Option<TimerApi>>,
8287
}
8388

8489
impl Client {
@@ -121,13 +126,18 @@ impl Client {
121126
queue_parcels: AtomicUsize::new(0),
122127
genesis_accounts,
123128
importer,
129+
reseal_timer: RwLock::new(None),
124130
});
125131

126132
// ensure buffered changes are flushed.
127133
client.db.read().flush().map_err(ClientError::Database)?;
128134
Ok(client)
129135
}
130136

137+
pub fn register_reseal_timer(&self, timer: TimerApi) {
138+
self.register_timer(timer);
139+
}
140+
131141
/// Returns engine reference.
132142
pub fn engine(&self) -> &CodeChainEngine {
133143
&*self.engine
@@ -260,6 +270,58 @@ impl Client {
260270
}
261271
}
262272

273+
const RESEAL_MAX_TIMER_TOKEN: TimerToken = 0;
274+
const RESEAL_MIN_TIMER_TOKEN: TimerToken = 1;
275+
276+
impl TimeoutHandler for Client {
277+
fn on_timeout(&self, token: TimerToken) {
278+
match token {
279+
RESEAL_MAX_TIMER_TOKEN => {
280+
// Working in PoW only
281+
if self.engine().seals_internally().is_none() && !self.importer.miner.prepare_work_sealing(self) {
282+
self.update_sealing();
283+
}
284+
}
285+
RESEAL_MIN_TIMER_TOKEN => {
286+
if let Some(reseal_timer) = self.reseal_timer.read().as_ref() {
287+
reseal_timer.cancel(RESEAL_MAX_TIMER_TOKEN).expect("Reseal max timer clear succeeds");
288+
};
289+
if !self.ready_parcels().is_empty() {
290+
self.update_sealing();
291+
}
292+
}
293+
_ => unreachable!(),
294+
}
295+
}
296+
}
297+
298+
impl ResealTimer for Client {
299+
fn register_timer(&self, timer: TimerApi) {
300+
timer
301+
.schedule_once(self.importer.miner.get_options().reseal_max_period, RESEAL_MAX_TIMER_TOKEN)
302+
.expect("Reseal max timer set succeeds");
303+
*self.reseal_timer.write() = Some(timer);
304+
}
305+
306+
fn set_max_timer(&self) {
307+
if let Some(reseal_timer) = self.reseal_timer.read().as_ref() {
308+
reseal_timer.cancel(RESEAL_MAX_TIMER_TOKEN).expect("Reseal max timer clear succeeds");
309+
reseal_timer
310+
.schedule_once(self.importer.miner.get_options().reseal_max_period, RESEAL_MAX_TIMER_TOKEN)
311+
.expect("Reseal max timer set succeeds");
312+
};
313+
}
314+
315+
fn set_min_timer(&self) {
316+
if let Some(reseal_timer) = self.reseal_timer.read().as_ref() {
317+
reseal_timer.cancel(RESEAL_MIN_TIMER_TOKEN).expect("Reseal min timer clear succeeds");
318+
reseal_timer
319+
.schedule_once(self.importer.miner.get_options().reseal_min_period, RESEAL_MIN_TIMER_TOKEN)
320+
.expect("Reseal min timer set succeeds");
321+
};
322+
}
323+
}
324+
263325
impl DatabaseClient for Client {
264326
fn database(&self) -> Arc<KeyValueDB> {
265327
Arc::clone(&self.db())

core/src/client/mod.rs

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ use ckey::{Address, PlatformAddress, Public};
3535
use cmerkle::Result as TrieResult;
3636
use cnetwork::NodeId;
3737
use cstate::{ActionHandler, AssetScheme, AssetSchemeAddress, OwnedAsset, TopStateView};
38+
use ctimer::TimerApi;
3839
use ctypes::invoice::Invoice;
3940
use ctypes::transaction::Transaction;
4041
use ctypes::{BlockNumber, ShardId};
@@ -183,6 +184,16 @@ pub trait Shard {
183184
fn shard_root(&self, shard_id: ShardId, state: StateOrBlock) -> Option<H256>;
184185
}
185186

187+
/// Provides a timer API for reseal_min_period/reseal_max_period on miner client
188+
pub trait ResealTimer {
189+
/// Register timer API
190+
fn register_timer(&self, timer: TimerApi);
191+
/// Set reseal min timer as reseal_min_period, for creating blocks with parcels which are pending because of reseal_min_period
192+
fn set_min_timer(&self);
193+
/// Set reseal max timer as reseal_max_period, for creating empty blocks every reseal_max_period
194+
fn set_max_timer(&self);
195+
}
196+
186197
/// Provides methods to access account info
187198
pub trait AccountData: Seq + Balance {}
188199

@@ -200,7 +211,7 @@ pub trait BlockChain: ChainInfo + BlockInfo + ParcelInfo + TransactionInfo {}
200211

201212
/// Blockchain database client. Owns and manages a blockchain and a block queue.
202213
pub trait BlockChainClient:
203-
Sync + Send + AccountData + BlockChain + ImportBlock + RegularKeyOwner + ChainTimeInfo {
214+
Sync + Send + AccountData + BlockChain + ImportBlock + RegularKeyOwner + ChainTimeInfo + ResealTimer {
204215
/// Get block queue information.
205216
fn queue_info(&self) -> BlockQueueInfo;
206217

core/src/client/test_client.rs

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ use ckey::{public_to_address, Address, Generator, NetworkId, PlatformAddress, Ra
3939
use cmerkle::skewed_merkle_root;
4040
use cnetwork::NodeId;
4141
use cstate::{ActionHandler, StateDB};
42+
use ctimer::{TimeoutHandler, TimerApi, TimerToken};
4243
use ctypes::invoice::Invoice;
4344
use ctypes::parcel::{Action, Parcel};
4445
use ctypes::transaction::Transaction;
@@ -55,8 +56,8 @@ use crate::blockchain_info::BlockChainInfo;
5556
use crate::client::ImportResult;
5657
use crate::client::{
5758
AccountData, Balance, BlockChain, BlockChainClient, BlockInfo, BlockProducer, BlockStatus, ChainInfo, ImportBlock,
58-
ImportSealedBlock, MiningBlockChainClient, ParcelInfo, PrepareOpenBlock, RegularKeyOwner, ReopenBlock, Seq,
59-
StateOrBlock, TransactionInfo,
59+
ImportSealedBlock, MiningBlockChainClient, ParcelInfo, PrepareOpenBlock, RegularKeyOwner, ReopenBlock, ResealTimer,
60+
Seq, StateOrBlock, TransactionInfo,
6061
};
6162
use crate::db::{COL_STATE, NUM_COLUMNS};
6263
use crate::encoded;
@@ -529,6 +530,18 @@ impl BlockChainClient for TestBlockChainClient {
529530
}
530531
}
531532

533+
impl TimeoutHandler for TestBlockChainClient {
534+
fn on_timeout(&self, _token: TimerToken) {}
535+
}
536+
537+
impl ResealTimer for TestBlockChainClient {
538+
fn register_timer(&self, _timer: TimerApi) {}
539+
540+
fn set_max_timer(&self) {}
541+
542+
fn set_min_timer(&self) {}
543+
}
544+
532545
impl ChainTimeInfo for TestBlockChainClient {
533546
fn best_block_number(&self) -> u64 {
534547
0

core/src/miner/miner.rs

Lines changed: 60 additions & 51 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ use crate::account_provider::{AccountProvider, SignError};
3737
use crate::block::{Block, ClosedBlock, IsBlock};
3838
use crate::client::{
3939
AccountData, BlockChain, BlockProducer, ImportSealedBlock, MiningBlockChainClient, RegularKey, RegularKeyOwner,
40+
ResealTimer,
4041
};
4142
use crate::consensus::{CodeChainEngine, EngineType};
4243
use crate::error::Error;
@@ -177,8 +178,12 @@ impl Miner {
177178
self.map_pending_block(|b| b.header().clone(), latest_block_number)
178179
}
179180

181+
pub fn get_options(&self) -> &MinerOptions {
182+
&self.options
183+
}
184+
180185
/// Check is reseal is allowed and necessary.
181-
fn requires_reseal(&self, best_block: BlockNumber) -> bool {
186+
fn requires_reseal<C: ResealTimer>(&self, client: &C, best_block: BlockNumber) -> bool {
182187
let has_local_parcels = self.mem_pool.read().has_local_pending_parcels();
183188
let mut sealing_work = self.sealing_work.lock();
184189
if sealing_work.enabled {
@@ -206,6 +211,7 @@ impl Miner {
206211
} else {
207212
// sealing enabled and we don't want to sleep.
208213
*self.next_allowed_reseal.lock() = Instant::now() + self.options.reseal_min_period;
214+
client.set_min_timer();
209215
true
210216
}
211217
} else {
@@ -343,53 +349,6 @@ impl Miner {
343349
})
344350
}
345351

346-
/// Returns true if we had to prepare new pending block.
347-
fn prepare_work_sealing<C: AccountData + BlockChain + BlockProducer + RegularKeyOwner + ChainTimeInfo>(
348-
&self,
349-
client: &C,
350-
) -> bool {
351-
ctrace!(MINER, "prepare_work_sealing: entering");
352-
let prepare_new = {
353-
let mut sealing_work = self.sealing_work.lock();
354-
let have_work = sealing_work.queue.peek_last_ref().is_some();
355-
ctrace!(MINER, "prepare_work_sealing: have_work={}", have_work);
356-
if !have_work {
357-
sealing_work.enabled = true;
358-
true
359-
} else {
360-
false
361-
}
362-
};
363-
if prepare_new {
364-
// --------------------------------------------------------------------------
365-
// | NOTE Code below requires transaction_queue and sealing_work locks. |
366-
// | Make sure to release the locks before calling that method. |
367-
// --------------------------------------------------------------------------
368-
match self.prepare_block(client) {
369-
Ok((block, original_work_hash)) => {
370-
self.prepare_work(block, original_work_hash);
371-
}
372-
Err(err) => {
373-
ctrace!(MINER, "prepare_work_sealing: cannot prepare block: {:?}", err);
374-
}
375-
}
376-
}
377-
let mut sealing_block_last_request = self.sealing_block_last_request.lock();
378-
let best_number = client.chain_info().best_block_number;
379-
if *sealing_block_last_request != best_number {
380-
ctrace!(
381-
MINER,
382-
"prepare_work_sealing: Miner received request (was {}, now {}) - waking up.",
383-
*sealing_block_last_request,
384-
best_number
385-
);
386-
*sealing_block_last_request = best_number;
387-
}
388-
389-
// Return if we restarted
390-
prepare_new
391-
}
392-
393352
/// Prepares work which has to be done to seal.
394353
fn prepare_work(&self, block: ClosedBlock, original_work_hash: Option<H256>) {
395354
let (work, is_new) = {
@@ -704,12 +663,59 @@ impl MinerService for Miner {
704663
self.engine.engine_type()
705664
}
706665

666+
fn prepare_work_sealing<C: AccountData + BlockChain + BlockProducer + RegularKeyOwner + ChainTimeInfo>(
667+
&self,
668+
client: &C,
669+
) -> bool {
670+
ctrace!(MINER, "prepare_work_sealing: entering");
671+
let prepare_new = {
672+
let mut sealing_work = self.sealing_work.lock();
673+
let have_work = sealing_work.queue.peek_last_ref().is_some();
674+
ctrace!(MINER, "prepare_work_sealing: have_work={}", have_work);
675+
if !have_work {
676+
sealing_work.enabled = true;
677+
true
678+
} else {
679+
false
680+
}
681+
};
682+
if prepare_new {
683+
// --------------------------------------------------------------------------
684+
// | NOTE Code below requires transaction_queue and sealing_work locks. |
685+
// | Make sure to release the locks before calling that method. |
686+
// --------------------------------------------------------------------------
687+
match self.prepare_block(client) {
688+
Ok((block, original_work_hash)) => {
689+
self.prepare_work(block, original_work_hash);
690+
}
691+
Err(err) => {
692+
ctrace!(MINER, "prepare_work_sealing: cannot prepare block: {:?}", err);
693+
}
694+
}
695+
}
696+
let mut sealing_block_last_request = self.sealing_block_last_request.lock();
697+
let best_number = client.chain_info().best_block_number;
698+
if *sealing_block_last_request != best_number {
699+
ctrace!(
700+
MINER,
701+
"prepare_work_sealing: Miner received request (was {}, now {}) - waking up.",
702+
*sealing_block_last_request,
703+
best_number
704+
);
705+
*sealing_block_last_request = best_number;
706+
}
707+
708+
// Return if we restarted
709+
prepare_new
710+
}
711+
707712
fn update_sealing<C>(&self, chain: &C)
708713
where
709-
C: AccountData + BlockChain + BlockProducer + ImportSealedBlock + RegularKeyOwner + ChainTimeInfo, {
714+
C: AccountData + BlockChain + BlockProducer + ImportSealedBlock + RegularKeyOwner + ResealTimer + ChainTimeInfo,
715+
{
710716
ctrace!(MINER, "update_sealing: preparing a block");
711717

712-
if self.requires_reseal(chain.chain_info().best_block_number) {
718+
if self.requires_reseal(chain, chain.chain_info().best_block_number) {
713719
let (block, original_work_hash) = match self.prepare_block(chain) {
714720
Ok((block, original_work_hash)) => (block, original_work_hash),
715721
Err(err) => {
@@ -728,7 +734,10 @@ impl MinerService for Miner {
728734
Some(false) => ctrace!(MINER, "update_sealing: engine is not keen to seal internally right now"),
729735
None => {
730736
ctrace!(MINER, "update_sealing: engine does not seal internally, preparing work");
731-
self.prepare_work(block, original_work_hash)
737+
self.prepare_work(block, original_work_hash);
738+
// Set the reseal max timer, for creating empty blocks every reseal_max_period
739+
// Not related to next_mandatory_reseal, which is used in seal_and_import_block_internally
740+
chain.set_max_timer();
732741
}
733742
}
734743
}

core/src/miner/mod.rs

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ use crate::account_provider::{AccountProvider, SignError};
3434
use crate::block::ClosedBlock;
3535
use crate::client::{
3636
AccountData, BlockChain, BlockProducer, ImportSealedBlock, MiningBlockChainClient, RegularKey, RegularKeyOwner,
37+
ResealTimer,
3738
};
3839
use crate::consensus::EngineType;
3940
use crate::error::Error;
@@ -79,10 +80,15 @@ pub trait MinerService: Send + Sync {
7980
/// Get the type of consensus engine.
8081
fn engine_type(&self) -> EngineType;
8182

83+
/// Returns true if we had to prepare new pending block.
84+
fn prepare_work_sealing<C>(&self, &C) -> bool
85+
where
86+
C: AccountData + BlockChain + BlockProducer + RegularKeyOwner + ChainTimeInfo;
87+
8288
/// New chain head event. Restart mining operation.
8389
fn update_sealing<C>(&self, chain: &C)
8490
where
85-
C: AccountData + BlockChain + BlockProducer + ImportSealedBlock + RegularKeyOwner + ChainTimeInfo;
91+
C: AccountData + BlockChain + BlockProducer + ImportSealedBlock + RegularKeyOwner + ResealTimer + ChainTimeInfo;
8692

8793
/// Submit `seal` as a valid solution for the header of `pow_hash`.
8894
/// Will check the seal, but not actually insert the block into the chain.

0 commit comments

Comments
 (0)