Allow per validator fee recipient via flag or file in validator client (similar to graffiti / graffiti-file) (#2924)
## Issue Addressed #2883 ## Proposed Changes * Added `suggested-fee-recipient` & `suggested-fee-recipient-file` flags to validator client (similar to graffiti / graffiti-file implementation). * Added proposer preparation service to VC, which sends the fee-recipient of all known validators to the BN via [/eth/v1/validator/prepare_beacon_proposer](https://github.com/ethereum/beacon-APIs/pull/178) api once per slot * Added [/eth/v1/validator/prepare_beacon_proposer](https://github.com/ethereum/beacon-APIs/pull/178) api endpoint and preparation data caching * Added cleanup routine to remove cached proposer preparations when not updated for 2 epochs ## Additional Info Changed the Implementation following the discussion in #2883. Co-authored-by: pk910 <philipp@pk910.de> Co-authored-by: Paul Hauner <paul@paulhauner.com> Co-authored-by: Philipp K <philipp@pk910.de>
This commit is contained in:
parent
d172c0b9fc
commit
5388183884
@ -273,8 +273,14 @@ pub fn cli_run(matches: &ArgMatches, validator_dir: PathBuf) -> Result<(), Strin
|
|||||||
eprintln!("Successfully imported keystore.");
|
eprintln!("Successfully imported keystore.");
|
||||||
num_imported_keystores += 1;
|
num_imported_keystores += 1;
|
||||||
|
|
||||||
let validator_def =
|
let graffiti = None;
|
||||||
ValidatorDefinition::new_keystore_with_password(&dest_keystore, password_opt, None)
|
let suggested_fee_recipient = None;
|
||||||
|
let validator_def = ValidatorDefinition::new_keystore_with_password(
|
||||||
|
&dest_keystore,
|
||||||
|
password_opt,
|
||||||
|
graffiti,
|
||||||
|
suggested_fee_recipient,
|
||||||
|
)
|
||||||
.map_err(|e| format!("Unable to create new validator definition: {:?}", e))?;
|
.map_err(|e| format!("Unable to create new validator definition: {:?}", e))?;
|
||||||
|
|
||||||
defs.push(validator_def);
|
defs.push(validator_def);
|
||||||
|
@ -3101,7 +3101,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
}
|
}
|
||||||
BeaconState::Merge(_) => {
|
BeaconState::Merge(_) => {
|
||||||
let sync_aggregate = get_sync_aggregate()?;
|
let sync_aggregate = get_sync_aggregate()?;
|
||||||
let execution_payload = get_execution_payload(self, &state)?;
|
let execution_payload = get_execution_payload(self, &state, proposer_index)?;
|
||||||
BeaconBlock::Merge(BeaconBlockMerge {
|
BeaconBlock::Merge(BeaconBlockMerge {
|
||||||
slot,
|
slot,
|
||||||
proposer_index,
|
proposer_index,
|
||||||
|
@ -204,14 +204,16 @@ pub fn validate_execution_payload_for_gossip<T: BeaconChainTypes>(
|
|||||||
pub fn get_execution_payload<T: BeaconChainTypes>(
|
pub fn get_execution_payload<T: BeaconChainTypes>(
|
||||||
chain: &BeaconChain<T>,
|
chain: &BeaconChain<T>,
|
||||||
state: &BeaconState<T::EthSpec>,
|
state: &BeaconState<T::EthSpec>,
|
||||||
|
proposer_index: u64,
|
||||||
) -> Result<ExecutionPayload<T::EthSpec>, BlockProductionError> {
|
) -> Result<ExecutionPayload<T::EthSpec>, BlockProductionError> {
|
||||||
Ok(prepare_execution_payload_blocking(chain, state)?.unwrap_or_default())
|
Ok(prepare_execution_payload_blocking(chain, state, proposer_index)?.unwrap_or_default())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Wraps the async `prepare_execution_payload` function as a blocking task.
|
/// Wraps the async `prepare_execution_payload` function as a blocking task.
|
||||||
pub fn prepare_execution_payload_blocking<T: BeaconChainTypes>(
|
pub fn prepare_execution_payload_blocking<T: BeaconChainTypes>(
|
||||||
chain: &BeaconChain<T>,
|
chain: &BeaconChain<T>,
|
||||||
state: &BeaconState<T::EthSpec>,
|
state: &BeaconState<T::EthSpec>,
|
||||||
|
proposer_index: u64,
|
||||||
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
|
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
|
||||||
let execution_layer = chain
|
let execution_layer = chain
|
||||||
.execution_layer
|
.execution_layer
|
||||||
@ -219,7 +221,9 @@ pub fn prepare_execution_payload_blocking<T: BeaconChainTypes>(
|
|||||||
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
|
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
|
||||||
|
|
||||||
execution_layer
|
execution_layer
|
||||||
.block_on_generic(|_| async { prepare_execution_payload(chain, state).await })
|
.block_on_generic(|_| async {
|
||||||
|
prepare_execution_payload(chain, state, proposer_index).await
|
||||||
|
})
|
||||||
.map_err(BlockProductionError::BlockingFailed)?
|
.map_err(BlockProductionError::BlockingFailed)?
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -240,6 +244,7 @@ pub fn prepare_execution_payload_blocking<T: BeaconChainTypes>(
|
|||||||
pub async fn prepare_execution_payload<T: BeaconChainTypes>(
|
pub async fn prepare_execution_payload<T: BeaconChainTypes>(
|
||||||
chain: &BeaconChain<T>,
|
chain: &BeaconChain<T>,
|
||||||
state: &BeaconState<T::EthSpec>,
|
state: &BeaconState<T::EthSpec>,
|
||||||
|
proposer_index: u64,
|
||||||
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
|
) -> Result<Option<ExecutionPayload<T::EthSpec>>, BlockProductionError> {
|
||||||
let spec = &chain.spec;
|
let spec = &chain.spec;
|
||||||
let execution_layer = chain
|
let execution_layer = chain
|
||||||
@ -300,6 +305,7 @@ pub async fn prepare_execution_payload<T: BeaconChainTypes>(
|
|||||||
timestamp,
|
timestamp,
|
||||||
random,
|
random,
|
||||||
finalized_block_hash.unwrap_or_else(Hash256::zero),
|
finalized_block_hash.unwrap_or_else(Hash256::zero),
|
||||||
|
proposer_index,
|
||||||
)
|
)
|
||||||
.await
|
.await
|
||||||
.map_err(BlockProductionError::GetPayloadFailed)?;
|
.map_err(BlockProductionError::GetPayloadFailed)?;
|
||||||
|
@ -700,6 +700,11 @@ where
|
|||||||
|
|
||||||
// Spawn a routine that tracks the status of the execution engines.
|
// Spawn a routine that tracks the status of the execution engines.
|
||||||
execution_layer.spawn_watchdog_routine(beacon_chain.slot_clock.clone());
|
execution_layer.spawn_watchdog_routine(beacon_chain.slot_clock.clone());
|
||||||
|
|
||||||
|
// Spawn a routine that removes expired proposer preparations.
|
||||||
|
execution_layer.spawn_clean_proposer_preparation_routine::<TSlotClock, TEthSpec>(
|
||||||
|
beacon_chain.slot_clock.clone(),
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -10,6 +10,7 @@ use lru::LruCache;
|
|||||||
use sensitive_url::SensitiveUrl;
|
use sensitive_url::SensitiveUrl;
|
||||||
use slog::{crit, debug, error, info, Logger};
|
use slog::{crit, debug, error, info, Logger};
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
|
use std::collections::HashMap;
|
||||||
use std::future::Future;
|
use std::future::Future;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
@ -18,7 +19,7 @@ use tokio::{
|
|||||||
sync::{Mutex, MutexGuard},
|
sync::{Mutex, MutexGuard},
|
||||||
time::{sleep, sleep_until, Instant},
|
time::{sleep, sleep_until, Instant},
|
||||||
};
|
};
|
||||||
use types::ChainSpec;
|
use types::{ChainSpec, Epoch, ProposerPreparationData};
|
||||||
|
|
||||||
pub use engine_api::{http::HttpJsonRpc, ExecutePayloadResponseStatus};
|
pub use engine_api::{http::HttpJsonRpc, ExecutePayloadResponseStatus};
|
||||||
|
|
||||||
@ -30,6 +31,16 @@ pub mod test_utils;
|
|||||||
/// in an LRU cache to avoid redundant lookups. This is the size of that cache.
|
/// in an LRU cache to avoid redundant lookups. This is the size of that cache.
|
||||||
const EXECUTION_BLOCKS_LRU_CACHE_SIZE: usize = 128;
|
const EXECUTION_BLOCKS_LRU_CACHE_SIZE: usize = 128;
|
||||||
|
|
||||||
|
/// A fee recipient address for use during block production. Only used as a very last resort if
|
||||||
|
/// there is no address provided by the user.
|
||||||
|
///
|
||||||
|
/// ## Note
|
||||||
|
///
|
||||||
|
/// This is *not* the zero-address, since Geth has been known to return errors for a coinbase of
|
||||||
|
/// 0x00..00.
|
||||||
|
const DEFAULT_SUGGESTED_FEE_RECIPIENT: [u8; 20] =
|
||||||
|
[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1];
|
||||||
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub enum Error {
|
pub enum Error {
|
||||||
NoEngines,
|
NoEngines,
|
||||||
@ -46,9 +57,16 @@ impl From<ApiError> for Error {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[derive(Clone)]
|
||||||
|
pub struct ProposerPreparationDataEntry {
|
||||||
|
update_epoch: Epoch,
|
||||||
|
preparation_data: ProposerPreparationData,
|
||||||
|
}
|
||||||
|
|
||||||
struct Inner {
|
struct Inner {
|
||||||
engines: Engines<HttpJsonRpc>,
|
engines: Engines<HttpJsonRpc>,
|
||||||
suggested_fee_recipient: Option<Address>,
|
suggested_fee_recipient: Option<Address>,
|
||||||
|
proposer_preparation_data: Mutex<HashMap<u64, ProposerPreparationDataEntry>>,
|
||||||
execution_blocks: Mutex<LruCache<Hash256, ExecutionBlock>>,
|
execution_blocks: Mutex<LruCache<Hash256, ExecutionBlock>>,
|
||||||
executor: TaskExecutor,
|
executor: TaskExecutor,
|
||||||
log: Logger,
|
log: Logger,
|
||||||
@ -96,6 +114,7 @@ impl ExecutionLayer {
|
|||||||
log: log.clone(),
|
log: log.clone(),
|
||||||
},
|
},
|
||||||
suggested_fee_recipient,
|
suggested_fee_recipient,
|
||||||
|
proposer_preparation_data: Mutex::new(HashMap::new()),
|
||||||
execution_blocks: Mutex::new(LruCache::new(EXECUTION_BLOCKS_LRU_CACHE_SIZE)),
|
execution_blocks: Mutex::new(LruCache::new(EXECUTION_BLOCKS_LRU_CACHE_SIZE)),
|
||||||
executor,
|
executor,
|
||||||
log,
|
log,
|
||||||
@ -116,17 +135,18 @@ impl ExecutionLayer {
|
|||||||
&self.inner.executor
|
&self.inner.executor
|
||||||
}
|
}
|
||||||
|
|
||||||
fn suggested_fee_recipient(&self) -> Result<Address, Error> {
|
|
||||||
self.inner
|
|
||||||
.suggested_fee_recipient
|
|
||||||
.ok_or(Error::FeeRecipientUnspecified)
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
|
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
|
||||||
async fn execution_blocks(&self) -> MutexGuard<'_, LruCache<Hash256, ExecutionBlock>> {
|
async fn execution_blocks(&self) -> MutexGuard<'_, LruCache<Hash256, ExecutionBlock>> {
|
||||||
self.inner.execution_blocks.lock().await
|
self.inner.execution_blocks.lock().await
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Note: this function returns a mutex guard, be careful to avoid deadlocks.
|
||||||
|
async fn proposer_preparation_data(
|
||||||
|
&self,
|
||||||
|
) -> MutexGuard<'_, HashMap<u64, ProposerPreparationDataEntry>> {
|
||||||
|
self.inner.proposer_preparation_data.lock().await
|
||||||
|
}
|
||||||
|
|
||||||
fn log(&self) -> &Logger {
|
fn log(&self) -> &Logger {
|
||||||
&self.inner.log
|
&self.inner.log
|
||||||
}
|
}
|
||||||
@ -234,11 +254,124 @@ impl ExecutionLayer {
|
|||||||
self.engines().upcheck_not_synced(Logging::Disabled).await;
|
self.engines().upcheck_not_synced(Logging::Disabled).await;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Spawns a routine which cleans the cached proposer preparations periodically.
|
||||||
|
pub fn spawn_clean_proposer_preparation_routine<S: SlotClock + 'static, T: EthSpec>(
|
||||||
|
&self,
|
||||||
|
slot_clock: S,
|
||||||
|
) {
|
||||||
|
let preparation_cleaner = |el: ExecutionLayer| async move {
|
||||||
|
// Start the loop to periodically clean proposer preparation cache.
|
||||||
|
loop {
|
||||||
|
if let Some(duration_to_next_epoch) =
|
||||||
|
slot_clock.duration_to_next_epoch(T::slots_per_epoch())
|
||||||
|
{
|
||||||
|
// Wait for next epoch
|
||||||
|
sleep(duration_to_next_epoch).await;
|
||||||
|
|
||||||
|
match slot_clock
|
||||||
|
.now()
|
||||||
|
.map(|slot| slot.epoch(T::slots_per_epoch()))
|
||||||
|
{
|
||||||
|
Some(current_epoch) => el
|
||||||
|
.clean_proposer_preparation(current_epoch)
|
||||||
|
.await
|
||||||
|
.map_err(|e| {
|
||||||
|
error!(
|
||||||
|
el.log(),
|
||||||
|
"Failed to clean proposer preparation cache";
|
||||||
|
"error" => format!("{:?}", e)
|
||||||
|
)
|
||||||
|
})
|
||||||
|
.unwrap_or(()),
|
||||||
|
None => error!(el.log(), "Failed to get current epoch from slot clock"),
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
error!(el.log(), "Failed to read slot clock");
|
||||||
|
// If we can't read the slot clock, just wait another slot and retry.
|
||||||
|
sleep(slot_clock.slot_duration()).await;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
self.spawn(preparation_cleaner, "exec_preparation_cleanup");
|
||||||
|
}
|
||||||
|
|
||||||
/// Returns `true` if there is at least one synced and reachable engine.
|
/// Returns `true` if there is at least one synced and reachable engine.
|
||||||
pub async fn is_synced(&self) -> bool {
|
pub async fn is_synced(&self) -> bool {
|
||||||
self.engines().any_synced().await
|
self.engines().any_synced().await
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Updates the proposer preparation data provided by validators
|
||||||
|
pub fn update_proposer_preparation_blocking(
|
||||||
|
&self,
|
||||||
|
update_epoch: Epoch,
|
||||||
|
preparation_data: &[ProposerPreparationData],
|
||||||
|
) -> Result<(), Error> {
|
||||||
|
self.block_on_generic(|_| async move {
|
||||||
|
self.update_proposer_preparation(update_epoch, preparation_data)
|
||||||
|
.await
|
||||||
|
})?
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Updates the proposer preparation data provided by validators
|
||||||
|
async fn update_proposer_preparation(
|
||||||
|
&self,
|
||||||
|
update_epoch: Epoch,
|
||||||
|
preparation_data: &[ProposerPreparationData],
|
||||||
|
) -> Result<(), Error> {
|
||||||
|
let mut proposer_preparation_data = self.proposer_preparation_data().await;
|
||||||
|
for preparation_entry in preparation_data {
|
||||||
|
proposer_preparation_data.insert(
|
||||||
|
preparation_entry.validator_index,
|
||||||
|
ProposerPreparationDataEntry {
|
||||||
|
update_epoch,
|
||||||
|
preparation_data: preparation_entry.clone(),
|
||||||
|
},
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Removes expired entries from cached proposer preparations
|
||||||
|
async fn clean_proposer_preparation(&self, current_epoch: Epoch) -> Result<(), Error> {
|
||||||
|
let mut proposer_preparation_data = self.proposer_preparation_data().await;
|
||||||
|
|
||||||
|
// Keep all entries that have been updated in the last 2 epochs
|
||||||
|
let retain_epoch = current_epoch.saturating_sub(Epoch::new(2));
|
||||||
|
proposer_preparation_data.retain(|_validator_index, preparation_entry| {
|
||||||
|
preparation_entry.update_epoch >= retain_epoch
|
||||||
|
});
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the fee-recipient address that should be used to build a block
|
||||||
|
async fn get_suggested_fee_recipient(&self, proposer_index: u64) -> Address {
|
||||||
|
if let Some(preparation_data_entry) =
|
||||||
|
self.proposer_preparation_data().await.get(&proposer_index)
|
||||||
|
{
|
||||||
|
// The values provided via the API have first priority.
|
||||||
|
preparation_data_entry.preparation_data.fee_recipient
|
||||||
|
} else if let Some(address) = self.inner.suggested_fee_recipient {
|
||||||
|
// If there has been no fee recipient provided via the API, but the BN has been provided
|
||||||
|
// with a global default address, use that.
|
||||||
|
address
|
||||||
|
} else {
|
||||||
|
// If there is no user-provided fee recipient, use a junk value and complain loudly.
|
||||||
|
crit!(
|
||||||
|
self.log(),
|
||||||
|
"Fee recipient unknown";
|
||||||
|
"msg" => "the suggested_fee_recipient was unknown during block production. \
|
||||||
|
a junk address was used, rewards were lost! \
|
||||||
|
check the --suggested-fee-recipient flag and VC configuration.",
|
||||||
|
"proposer_index" => ?proposer_index
|
||||||
|
);
|
||||||
|
|
||||||
|
Address::from_slice(&DEFAULT_SUGGESTED_FEE_RECIPIENT)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Maps to the `engine_getPayload` JSON-RPC call.
|
/// Maps to the `engine_getPayload` JSON-RPC call.
|
||||||
///
|
///
|
||||||
/// However, it will attempt to call `self.prepare_payload` if it cannot find an existing
|
/// However, it will attempt to call `self.prepare_payload` if it cannot find an existing
|
||||||
@ -254,8 +387,10 @@ impl ExecutionLayer {
|
|||||||
timestamp: u64,
|
timestamp: u64,
|
||||||
random: Hash256,
|
random: Hash256,
|
||||||
finalized_block_hash: Hash256,
|
finalized_block_hash: Hash256,
|
||||||
|
proposer_index: u64,
|
||||||
) -> Result<ExecutionPayload<T>, Error> {
|
) -> Result<ExecutionPayload<T>, Error> {
|
||||||
let suggested_fee_recipient = self.suggested_fee_recipient()?;
|
let suggested_fee_recipient = self.get_suggested_fee_recipient(proposer_index).await;
|
||||||
|
|
||||||
debug!(
|
debug!(
|
||||||
self.log(),
|
self.log(),
|
||||||
"Issuing engine_getPayload";
|
"Issuing engine_getPayload";
|
||||||
|
@ -127,9 +127,16 @@ impl<T: EthSpec> MockExecutionLayer<T> {
|
|||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
|
let validator_index = 0;
|
||||||
let payload = self
|
let payload = self
|
||||||
.el
|
.el
|
||||||
.get_payload::<T>(parent_hash, timestamp, random, finalized_block_hash)
|
.get_payload::<T>(
|
||||||
|
parent_hash,
|
||||||
|
timestamp,
|
||||||
|
random,
|
||||||
|
finalized_block_hash,
|
||||||
|
validator_index,
|
||||||
|
)
|
||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
let block_hash = payload.block_hash;
|
let block_hash = payload.block_hash;
|
||||||
|
@ -45,9 +45,9 @@ use tokio::sync::mpsc::UnboundedSender;
|
|||||||
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
|
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
|
||||||
use types::{
|
use types::{
|
||||||
Attestation, AttesterSlashing, BeaconStateError, CommitteeCache, ConfigAndPreset, Epoch,
|
Attestation, AttesterSlashing, BeaconStateError, CommitteeCache, ConfigAndPreset, Epoch,
|
||||||
EthSpec, ForkName, ProposerSlashing, RelativeEpoch, SignedAggregateAndProof, SignedBeaconBlock,
|
EthSpec, ForkName, ProposerPreparationData, ProposerSlashing, RelativeEpoch,
|
||||||
SignedContributionAndProof, SignedVoluntaryExit, Slot, SyncCommitteeMessage,
|
SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
|
||||||
SyncContributionData,
|
Slot, SyncCommitteeMessage, SyncContributionData,
|
||||||
};
|
};
|
||||||
use version::{
|
use version::{
|
||||||
add_consensus_version_header, fork_versioned_response, inconsistent_fork_rejection,
|
add_consensus_version_header, fork_versioned_response, inconsistent_fork_rejection,
|
||||||
@ -2186,6 +2186,53 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
},
|
},
|
||||||
);
|
);
|
||||||
|
|
||||||
|
// POST validator/prepare_beacon_proposer
|
||||||
|
let post_validator_prepare_beacon_proposer = eth1_v1
|
||||||
|
.and(warp::path("validator"))
|
||||||
|
.and(warp::path("prepare_beacon_proposer"))
|
||||||
|
.and(warp::path::end())
|
||||||
|
.and(not_while_syncing_filter.clone())
|
||||||
|
.and(chain_filter.clone())
|
||||||
|
.and(warp::addr::remote())
|
||||||
|
.and(log_filter.clone())
|
||||||
|
.and(warp::body::json())
|
||||||
|
.and_then(
|
||||||
|
|chain: Arc<BeaconChain<T>>,
|
||||||
|
client_addr: Option<SocketAddr>,
|
||||||
|
log: Logger,
|
||||||
|
preparation_data: Vec<ProposerPreparationData>| {
|
||||||
|
blocking_json_task(move || {
|
||||||
|
let execution_layer = chain
|
||||||
|
.execution_layer
|
||||||
|
.as_ref()
|
||||||
|
.ok_or(BeaconChainError::ExecutionLayerMissing)
|
||||||
|
.map_err(warp_utils::reject::beacon_chain_error)?;
|
||||||
|
let current_epoch = chain
|
||||||
|
.epoch()
|
||||||
|
.map_err(warp_utils::reject::beacon_chain_error)?;
|
||||||
|
|
||||||
|
debug!(
|
||||||
|
log,
|
||||||
|
"Received proposer preparation data";
|
||||||
|
"count" => preparation_data.len(),
|
||||||
|
"client" => client_addr
|
||||||
|
.map(|a| a.to_string())
|
||||||
|
.unwrap_or_else(|| "unknown".to_string()),
|
||||||
|
);
|
||||||
|
|
||||||
|
execution_layer
|
||||||
|
.update_proposer_preparation_blocking(current_epoch, &preparation_data)
|
||||||
|
.map_err(|_e| {
|
||||||
|
warp_utils::reject::custom_bad_request(
|
||||||
|
"error processing proposer preparations".to_string(),
|
||||||
|
)
|
||||||
|
})?;
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
})
|
||||||
|
},
|
||||||
|
);
|
||||||
|
|
||||||
// POST validator/sync_committee_subscriptions
|
// POST validator/sync_committee_subscriptions
|
||||||
let post_validator_sync_committee_subscriptions = eth1_v1
|
let post_validator_sync_committee_subscriptions = eth1_v1
|
||||||
.and(warp::path("validator"))
|
.and(warp::path("validator"))
|
||||||
@ -2710,6 +2757,7 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.or(post_validator_contribution_and_proofs.boxed())
|
.or(post_validator_contribution_and_proofs.boxed())
|
||||||
.or(post_validator_beacon_committee_subscriptions.boxed())
|
.or(post_validator_beacon_committee_subscriptions.boxed())
|
||||||
.or(post_validator_sync_committee_subscriptions.boxed())
|
.or(post_validator_sync_committee_subscriptions.boxed())
|
||||||
|
.or(post_validator_prepare_beacon_proposer.boxed())
|
||||||
.or(post_lighthouse_liveness.boxed())
|
.or(post_lighthouse_liveness.boxed())
|
||||||
.or(post_lighthouse_database_reconstruct.boxed())
|
.or(post_lighthouse_database_reconstruct.boxed())
|
||||||
.or(post_lighthouse_database_historical_blocks.boxed()),
|
.or(post_lighthouse_database_historical_blocks.boxed()),
|
||||||
|
@ -409,9 +409,9 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
|
|||||||
.takes_value(true)
|
.takes_value(true)
|
||||||
)
|
)
|
||||||
.arg(
|
.arg(
|
||||||
Arg::with_name("fee-recipient")
|
Arg::with_name("suggested-fee-recipient")
|
||||||
.long("fee-recipient")
|
.long("suggested-fee-recipient")
|
||||||
.value_name("FEE-RECIPIENT")
|
.value_name("SUGGESTED-FEE-RECIPIENT")
|
||||||
.help("Once the merge has happened, this address will receive transaction fees \
|
.help("Once the merge has happened, this address will receive transaction fees \
|
||||||
collected from any blocks produced by this node. Defaults to a junk \
|
collected from any blocks produced by this node. Defaults to a junk \
|
||||||
address whilst the merge is in development stages. THE DEFAULT VALUE \
|
address whilst the merge is in development stages. THE DEFAULT VALUE \
|
||||||
|
@ -14,12 +14,7 @@ use std::net::{IpAddr, Ipv4Addr, ToSocketAddrs};
|
|||||||
use std::net::{TcpListener, UdpSocket};
|
use std::net::{TcpListener, UdpSocket};
|
||||||
use std::path::{Path, PathBuf};
|
use std::path::{Path, PathBuf};
|
||||||
use std::str::FromStr;
|
use std::str::FromStr;
|
||||||
use types::{Address, Checkpoint, Epoch, EthSpec, Hash256, PublicKeyBytes, GRAFFITI_BYTES_LEN};
|
use types::{Checkpoint, Epoch, EthSpec, Hash256, PublicKeyBytes, GRAFFITI_BYTES_LEN};
|
||||||
|
|
||||||
// TODO(merge): remove this default value. It's just there to make life easy during
|
|
||||||
// early testnets.
|
|
||||||
const DEFAULT_SUGGESTED_FEE_RECIPIENT: [u8; 20] =
|
|
||||||
[0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1];
|
|
||||||
|
|
||||||
/// Gets the fully-initialized global client.
|
/// Gets the fully-initialized global client.
|
||||||
///
|
///
|
||||||
@ -253,12 +248,8 @@ pub fn get_config<E: EthSpec>(
|
|||||||
client_config.execution_endpoints = Some(client_config.eth1.endpoints.clone());
|
client_config.execution_endpoints = Some(client_config.eth1.endpoints.clone());
|
||||||
}
|
}
|
||||||
|
|
||||||
client_config.suggested_fee_recipient = Some(
|
client_config.suggested_fee_recipient =
|
||||||
clap_utils::parse_optional(cli_args, "fee-recipient")?
|
clap_utils::parse_optional(cli_args, "suggested-fee-recipient")?;
|
||||||
// TODO(merge): remove this default value. It's just there to make life easy during
|
|
||||||
// early testnets.
|
|
||||||
.unwrap_or_else(|| Address::from(DEFAULT_SUGGESTED_FEE_RECIPIENT)),
|
|
||||||
);
|
|
||||||
|
|
||||||
if let Some(freezer_dir) = cli_args.value_of("freezer-dir") {
|
if let Some(freezer_dir) = cli_args.value_of("freezer-dir") {
|
||||||
client_config.freezer_db_path = Some(PathBuf::from(freezer_dir));
|
client_config.freezer_db_path = Some(PathBuf::from(freezer_dir));
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
* [Voluntary Exits](./voluntary-exit.md)
|
* [Voluntary Exits](./voluntary-exit.md)
|
||||||
* [Validator Monitoring](./validator-monitoring.md)
|
* [Validator Monitoring](./validator-monitoring.md)
|
||||||
* [Doppelganger Protection](./validator-doppelganger.md)
|
* [Doppelganger Protection](./validator-doppelganger.md)
|
||||||
|
* [Suggested Fee Recipient](./suggested-fee-recipient.md)
|
||||||
* [APIs](./api.md)
|
* [APIs](./api.md)
|
||||||
* [Beacon Node API](./api-bn.md)
|
* [Beacon Node API](./api-bn.md)
|
||||||
* [/lighthouse](./api-lighthouse.md)
|
* [/lighthouse](./api-lighthouse.md)
|
||||||
|
@ -315,7 +315,8 @@ Typical Responses | 200
|
|||||||
"enable": true,
|
"enable": true,
|
||||||
"description": "validator_one",
|
"description": "validator_one",
|
||||||
"deposit_gwei": "32000000000",
|
"deposit_gwei": "32000000000",
|
||||||
"graffiti": "Mr F was here"
|
"graffiti": "Mr F was here",
|
||||||
|
"suggested_fee_recipient": "0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"enable": false,
|
"enable": false,
|
||||||
@ -492,6 +493,7 @@ Typical Responses | 200, 400
|
|||||||
"enable": true,
|
"enable": true,
|
||||||
"description": "validator_one",
|
"description": "validator_one",
|
||||||
"graffiti": "Mr F was here",
|
"graffiti": "Mr F was here",
|
||||||
|
"suggested_fee_recipient": "0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d",
|
||||||
"voting_public_key": "0xa062f95fee747144d5e511940624bc6546509eeaeae9383257a9c43e7ddc58c17c2bab4ae62053122184c381b90db380",
|
"voting_public_key": "0xa062f95fee747144d5e511940624bc6546509eeaeae9383257a9c43e7ddc58c17c2bab4ae62053122184c381b90db380",
|
||||||
"url": "http://path-to-web3signer.com",
|
"url": "http://path-to-web3signer.com",
|
||||||
"root_certificate_path": "/path/on/vc/filesystem/to/certificate.pem",
|
"root_certificate_path": "/path/on/vc/filesystem/to/certificate.pem",
|
||||||
@ -503,6 +505,7 @@ Typical Responses | 200, 400
|
|||||||
The following fields may be omitted or nullified to obtain default values:
|
The following fields may be omitted or nullified to obtain default values:
|
||||||
|
|
||||||
- `graffiti`
|
- `graffiti`
|
||||||
|
- `suggested_fee_recipient`
|
||||||
- `root_certificate_path`
|
- `root_certificate_path`
|
||||||
- `request_timeout_ms`
|
- `request_timeout_ms`
|
||||||
|
|
||||||
|
91
book/src/suggested-fee-recipient.md
Normal file
91
book/src/suggested-fee-recipient.md
Normal file
@ -0,0 +1,91 @@
|
|||||||
|
# Suggested Fee Recipient
|
||||||
|
|
||||||
|
*Note: these documents are not relevant until the Bellatrix (Merge) upgrade has occurred.*
|
||||||
|
|
||||||
|
## Fee recipient trust assumptions
|
||||||
|
|
||||||
|
During post-merge block production, the Beacon Node (BN) will provide a `suggested_fee_recipient` to
|
||||||
|
the execution node. This is a 20-byte Ethereum address which the EL might choose to set as the
|
||||||
|
coinbase and the recipient of other fees or rewards.
|
||||||
|
|
||||||
|
There is no guarantee that an execution node will use the `suggested_fee_recipient` to collect fees,
|
||||||
|
it may use any address it chooses. It is assumed that an honest execution node *will* use the
|
||||||
|
`suggested_fee_recipient`, but users should note this trust assumption.
|
||||||
|
|
||||||
|
The `suggested_fee_recipient` can be provided to the VC, who will transmit it to the BN. The also BN
|
||||||
|
has a choice regarding the fee recipient it passes to the execution node, creating another
|
||||||
|
noteworthy trust assumption.
|
||||||
|
|
||||||
|
To be sure *you* control your fee recipient value, run your own BN and execution node (don't use
|
||||||
|
third-party services).
|
||||||
|
|
||||||
|
The Lighthouse VC provides three methods for setting the `suggested_fee_recipient` (also known
|
||||||
|
simply as the "fee recipient") to be passed to the execution layer during block production. The
|
||||||
|
Lighthouse BN also provides a method for defining this value, should the VC not transmit a value.
|
||||||
|
|
||||||
|
Assuming trustworthy nodes, the priority for the four methods is:
|
||||||
|
|
||||||
|
1. `validator_definitions.yml`
|
||||||
|
1. `--suggested-fee-recipient-file`
|
||||||
|
1. `--suggested-fee-recipient` provided to the VC.
|
||||||
|
1. `--suggested-fee-recipient` provided to the BN.
|
||||||
|
|
||||||
|
Users may configure the fee recipient via `validator_definitions.yml` or via the
|
||||||
|
`--suggested-fee-recipient-file` flag. The value in `validator_definitions.yml` will always take
|
||||||
|
precedence.
|
||||||
|
|
||||||
|
### 1. Setting the fee recipient in the `validator_definitions.yml`
|
||||||
|
|
||||||
|
Users can set the fee recipient in `validator_definitions.yml` with the `suggested_fee_recipient`
|
||||||
|
key. This option is recommended for most users, where each validator has a fixed fee recipient.
|
||||||
|
|
||||||
|
Below is an example of the validator_definitions.yml with `suggested_fee_recipient` values:
|
||||||
|
|
||||||
|
```
|
||||||
|
---
|
||||||
|
- enabled: true
|
||||||
|
voting_public_key: "0x87a580d31d7bc69069b55f5a01995a610dd391a26dc9e36e81057a17211983a79266800ab8531f21f1083d7d84085007"
|
||||||
|
type: local_keystore
|
||||||
|
voting_keystore_path: /home/paul/.lighthouse/validators/0x87a580d31d7bc69069b55f5a01995a610dd391a26dc9e36e81057a17211983a79266800ab8531f21f1083d7d84085007/voting-keystore.json
|
||||||
|
voting_keystore_password_path: /home/paul/.lighthouse/secrets/0x87a580d31d7bc69069b55f5a01995a610dd391a26dc9e36e81057a17211983a79266800ab8531f21f1083d7d84085007
|
||||||
|
suggested_fee_recipient: "0x6cc8dcbca744a6e4ffedb98e1d0df903b10abd21"
|
||||||
|
- enabled: false
|
||||||
|
voting_public_key: "0xa5566f9ec3c6e1fdf362634ebec9ef7aceb0e460e5079714808388e5d48f4ae1e12897fed1bea951c17fa389d511e477"
|
||||||
|
type: local_keystore voting_keystore_path: /home/paul/.lighthouse/validators/0xa5566f9ec3c6e1fdf362634ebec9ef7aceb0e460e5079714808388e5d48f4ae1e12897fed1bea951c17fa389d511e477/voting-keystore.json
|
||||||
|
voting_keystore_password: myStrongpa55word123&$
|
||||||
|
suggested_fee_recipient: "0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d"
|
||||||
|
```
|
||||||
|
|
||||||
|
### 2. Using the "--suggested-fee-recipient-file" flag on the validator client
|
||||||
|
|
||||||
|
Users can specify a file with the `--suggested-fee-recipient-file` flag. This option is useful for dynamically
|
||||||
|
changing fee recipients. This file is reloaded each time a validator is chosen to propose a block.
|
||||||
|
|
||||||
|
Usage:
|
||||||
|
`lighthouse vc --suggested-fee-recipient-file fee_recipient.txt`
|
||||||
|
|
||||||
|
The file should contain key value pairs corresponding to validator public keys and their associated
|
||||||
|
fee recipient. The file can optionally contain a `default` key for the default case.
|
||||||
|
|
||||||
|
The following example sets the default and the values for the validators with pubkeys `0x87a5` and
|
||||||
|
`0xa556`:
|
||||||
|
|
||||||
|
```
|
||||||
|
default: 0x6cc8dcbca744a6e4ffedb98e1d0df903b10abd21
|
||||||
|
0x87a580d31d7bc69069b55f5a01995a610dd391a26dc9e36e81057a17211983a79266800ab8531f21f1083d7d84085007: 0x6cc8dcbca744a6e4ffedb98e1d0df903b10abd21
|
||||||
|
0xa5566f9ec3c6e1fdf362634ebec9ef7aceb0e460e5079714808388e5d48f4ae1e12897fed1bea951c17fa389d511e477: 0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d
|
||||||
|
```
|
||||||
|
|
||||||
|
Lighthouse will first search for the fee recipient corresponding to the public key of the proposing
|
||||||
|
validator, if there are no matches for the public key, then it uses the address corresponding to the
|
||||||
|
default key (if present).
|
||||||
|
|
||||||
|
### 3. Using the "--suggested-fee-recipient" flag on the validator client
|
||||||
|
|
||||||
|
The `--suggested-fee-recipient` can be provided to the VC to act as a default value for all
|
||||||
|
validators where a `suggested_fee_recipient` is not loaded from another method.
|
||||||
|
|
||||||
|
### 4. Using the "--suggested-fee-recipient" flag on the beacon node
|
||||||
|
|
||||||
|
The `--suggested-fee-recipient` can be provided to the BN to act as a default value when the
|
||||||
|
validator client does not transmit a `suggested_fee_recipient` to the BN.
|
@ -13,7 +13,7 @@ use std::collections::HashSet;
|
|||||||
use std::fs::{self, OpenOptions};
|
use std::fs::{self, OpenOptions};
|
||||||
use std::io;
|
use std::io;
|
||||||
use std::path::{Path, PathBuf};
|
use std::path::{Path, PathBuf};
|
||||||
use types::{graffiti::GraffitiString, PublicKey};
|
use types::{graffiti::GraffitiString, Address, PublicKey};
|
||||||
use validator_dir::VOTING_KEYSTORE_FILE;
|
use validator_dir::VOTING_KEYSTORE_FILE;
|
||||||
|
|
||||||
/// The file name for the serialized `ValidatorDefinitions` struct.
|
/// The file name for the serialized `ValidatorDefinitions` struct.
|
||||||
@ -93,6 +93,9 @@ pub struct ValidatorDefinition {
|
|||||||
#[serde(skip_serializing_if = "Option::is_none")]
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
pub graffiti: Option<GraffitiString>,
|
pub graffiti: Option<GraffitiString>,
|
||||||
#[serde(default)]
|
#[serde(default)]
|
||||||
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
|
pub suggested_fee_recipient: Option<Address>,
|
||||||
|
#[serde(default)]
|
||||||
pub description: String,
|
pub description: String,
|
||||||
#[serde(flatten)]
|
#[serde(flatten)]
|
||||||
pub signing_definition: SigningDefinition,
|
pub signing_definition: SigningDefinition,
|
||||||
@ -109,6 +112,7 @@ impl ValidatorDefinition {
|
|||||||
voting_keystore_path: P,
|
voting_keystore_path: P,
|
||||||
voting_keystore_password: Option<ZeroizeString>,
|
voting_keystore_password: Option<ZeroizeString>,
|
||||||
graffiti: Option<GraffitiString>,
|
graffiti: Option<GraffitiString>,
|
||||||
|
suggested_fee_recipient: Option<Address>,
|
||||||
) -> Result<Self, Error> {
|
) -> Result<Self, Error> {
|
||||||
let voting_keystore_path = voting_keystore_path.as_ref().into();
|
let voting_keystore_path = voting_keystore_path.as_ref().into();
|
||||||
let keystore =
|
let keystore =
|
||||||
@ -120,6 +124,7 @@ impl ValidatorDefinition {
|
|||||||
voting_public_key,
|
voting_public_key,
|
||||||
description: keystore.description().unwrap_or("").to_string(),
|
description: keystore.description().unwrap_or("").to_string(),
|
||||||
graffiti,
|
graffiti,
|
||||||
|
suggested_fee_recipient,
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
voting_keystore_password_path: None,
|
voting_keystore_password_path: None,
|
||||||
@ -265,6 +270,7 @@ impl ValidatorDefinitions {
|
|||||||
voting_public_key,
|
voting_public_key,
|
||||||
description: keystore.description().unwrap_or("").to_string(),
|
description: keystore.description().unwrap_or("").to_string(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
voting_keystore_password_path,
|
voting_keystore_password_path,
|
||||||
@ -466,4 +472,45 @@ mod tests {
|
|||||||
Some(GraffitiString::from_str("mrfwashere").unwrap())
|
Some(GraffitiString::from_str("mrfwashere").unwrap())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn suggested_fee_recipient_checks() {
|
||||||
|
let no_suggested_fee_recipient = r#"---
|
||||||
|
description: ""
|
||||||
|
enabled: true
|
||||||
|
type: local_keystore
|
||||||
|
voting_keystore_path: ""
|
||||||
|
voting_public_key: "0xaf3c7ddab7e293834710fca2d39d068f884455ede270e0d0293dc818e4f2f0f975355067e8437955cb29aec674e5c9e7"
|
||||||
|
"#;
|
||||||
|
let def: ValidatorDefinition = serde_yaml::from_str(no_suggested_fee_recipient).unwrap();
|
||||||
|
assert!(def.suggested_fee_recipient.is_none());
|
||||||
|
|
||||||
|
let invalid_suggested_fee_recipient = r#"---
|
||||||
|
description: ""
|
||||||
|
enabled: true
|
||||||
|
type: local_keystore
|
||||||
|
suggested_fee_recipient: "foopy"
|
||||||
|
voting_keystore_path: ""
|
||||||
|
voting_public_key: "0xaf3c7ddab7e293834710fca2d39d068f884455ede270e0d0293dc818e4f2f0f975355067e8437955cb29aec674e5c9e7"
|
||||||
|
"#;
|
||||||
|
|
||||||
|
let def: Result<ValidatorDefinition, _> =
|
||||||
|
serde_yaml::from_str(invalid_suggested_fee_recipient);
|
||||||
|
assert!(def.is_err());
|
||||||
|
|
||||||
|
let valid_suggested_fee_recipient = r#"---
|
||||||
|
description: ""
|
||||||
|
enabled: true
|
||||||
|
type: local_keystore
|
||||||
|
suggested_fee_recipient: "0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d"
|
||||||
|
voting_keystore_path: ""
|
||||||
|
voting_public_key: "0xaf3c7ddab7e293834710fca2d39d068f884455ede270e0d0293dc818e4f2f0f975355067e8437955cb29aec674e5c9e7"
|
||||||
|
"#;
|
||||||
|
|
||||||
|
let def: ValidatorDefinition = serde_yaml::from_str(valid_suggested_fee_recipient).unwrap();
|
||||||
|
assert_eq!(
|
||||||
|
def.suggested_fee_recipient,
|
||||||
|
Some(Address::from_str("0xa2e334e71511686bcfe38bb3ee1ad8f6babcc03d").unwrap())
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -892,6 +892,23 @@ impl BeaconNodeHttpClient {
|
|||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// `POST validator/prepare_beacon_proposer`
|
||||||
|
pub async fn post_validator_prepare_beacon_proposer(
|
||||||
|
&self,
|
||||||
|
preparation_data: &[ProposerPreparationData],
|
||||||
|
) -> Result<(), Error> {
|
||||||
|
let mut path = self.eth_path(V1)?;
|
||||||
|
|
||||||
|
path.path_segments_mut()
|
||||||
|
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
|
||||||
|
.push("validator")
|
||||||
|
.push("prepare_beacon_proposer");
|
||||||
|
|
||||||
|
self.post(path, &preparation_data).await?;
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
/// `GET config/fork_schedule`
|
/// `GET config/fork_schedule`
|
||||||
pub async fn get_config_fork_schedule(&self) -> Result<GenericResponse<Vec<Fork>>, Error> {
|
pub async fn get_config_fork_schedule(&self) -> Result<GenericResponse<Vec<Fork>>, Error> {
|
||||||
let mut path = self.eth_path(V1)?;
|
let mut path = self.eth_path(V1)?;
|
||||||
|
@ -23,6 +23,9 @@ pub struct ValidatorRequest {
|
|||||||
#[serde(default)]
|
#[serde(default)]
|
||||||
#[serde(skip_serializing_if = "Option::is_none")]
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
pub graffiti: Option<GraffitiString>,
|
pub graffiti: Option<GraffitiString>,
|
||||||
|
#[serde(default)]
|
||||||
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
|
pub suggested_fee_recipient: Option<Address>,
|
||||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
pub deposit_gwei: u64,
|
pub deposit_gwei: u64,
|
||||||
}
|
}
|
||||||
@ -43,6 +46,9 @@ pub struct CreatedValidator {
|
|||||||
#[serde(default)]
|
#[serde(default)]
|
||||||
#[serde(skip_serializing_if = "Option::is_none")]
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
pub graffiti: Option<GraffitiString>,
|
pub graffiti: Option<GraffitiString>,
|
||||||
|
#[serde(default)]
|
||||||
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
|
pub suggested_fee_recipient: Option<Address>,
|
||||||
pub eth1_deposit_tx_data: String,
|
pub eth1_deposit_tx_data: String,
|
||||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
pub deposit_gwei: u64,
|
pub deposit_gwei: u64,
|
||||||
@ -65,6 +71,7 @@ pub struct KeystoreValidatorsPostRequest {
|
|||||||
pub enable: bool,
|
pub enable: bool,
|
||||||
pub keystore: Keystore,
|
pub keystore: Keystore,
|
||||||
pub graffiti: Option<GraffitiString>,
|
pub graffiti: Option<GraffitiString>,
|
||||||
|
pub suggested_fee_recipient: Option<Address>,
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
|
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
|
||||||
@ -74,6 +81,9 @@ pub struct Web3SignerValidatorRequest {
|
|||||||
#[serde(default)]
|
#[serde(default)]
|
||||||
#[serde(skip_serializing_if = "Option::is_none")]
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
pub graffiti: Option<GraffitiString>,
|
pub graffiti: Option<GraffitiString>,
|
||||||
|
#[serde(default)]
|
||||||
|
#[serde(skip_serializing_if = "Option::is_none")]
|
||||||
|
pub suggested_fee_recipient: Option<Address>,
|
||||||
pub voting_public_key: PublicKey,
|
pub voting_public_key: PublicKey,
|
||||||
pub url: String,
|
pub url: String,
|
||||||
#[serde(default)]
|
#[serde(default)]
|
||||||
|
@ -47,6 +47,7 @@ pub mod graffiti;
|
|||||||
pub mod historical_batch;
|
pub mod historical_batch;
|
||||||
pub mod indexed_attestation;
|
pub mod indexed_attestation;
|
||||||
pub mod pending_attestation;
|
pub mod pending_attestation;
|
||||||
|
pub mod proposer_preparation_data;
|
||||||
pub mod proposer_slashing;
|
pub mod proposer_slashing;
|
||||||
pub mod relative_epoch;
|
pub mod relative_epoch;
|
||||||
pub mod selection_proof;
|
pub mod selection_proof;
|
||||||
@ -126,6 +127,7 @@ pub use crate::participation_flags::ParticipationFlags;
|
|||||||
pub use crate::participation_list::ParticipationList;
|
pub use crate::participation_list::ParticipationList;
|
||||||
pub use crate::pending_attestation::PendingAttestation;
|
pub use crate::pending_attestation::PendingAttestation;
|
||||||
pub use crate::preset::{AltairPreset, BasePreset, BellatrixPreset};
|
pub use crate::preset::{AltairPreset, BasePreset, BellatrixPreset};
|
||||||
|
pub use crate::proposer_preparation_data::ProposerPreparationData;
|
||||||
pub use crate::proposer_slashing::ProposerSlashing;
|
pub use crate::proposer_slashing::ProposerSlashing;
|
||||||
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
|
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
|
||||||
pub use crate::selection_proof::SelectionProof;
|
pub use crate::selection_proof::SelectionProof;
|
||||||
|
12
consensus/types/src/proposer_preparation_data.rs
Normal file
12
consensus/types/src/proposer_preparation_data.rs
Normal file
@ -0,0 +1,12 @@
|
|||||||
|
use crate::*;
|
||||||
|
use serde::{Deserialize, Serialize};
|
||||||
|
|
||||||
|
/// A proposer preparation, created when a validator prepares the beacon node for potential proposers
|
||||||
|
/// by supplying information required when proposing blocks for the given validators.
|
||||||
|
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone)]
|
||||||
|
pub struct ProposerPreparationData {
|
||||||
|
/// The validators index.
|
||||||
|
pub validator_index: u64,
|
||||||
|
/// The fee-recipient address.
|
||||||
|
pub fee_recipient: Address,
|
||||||
|
}
|
@ -493,6 +493,7 @@ fn validator_import_launchpad() {
|
|||||||
enabled: false,
|
enabled: false,
|
||||||
description: "".into(),
|
description: "".into(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
voting_public_key: keystore.public_key().unwrap(),
|
voting_public_key: keystore.public_key().unwrap(),
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
@ -612,6 +613,7 @@ fn validator_import_launchpad_no_password_then_add_password() {
|
|||||||
enabled: true,
|
enabled: true,
|
||||||
description: "".into(),
|
description: "".into(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
voting_public_key: keystore.public_key().unwrap(),
|
voting_public_key: keystore.public_key().unwrap(),
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
@ -635,6 +637,7 @@ fn validator_import_launchpad_no_password_then_add_password() {
|
|||||||
enabled: true,
|
enabled: true,
|
||||||
description: "".into(),
|
description: "".into(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
voting_public_key: keystore.public_key().unwrap(),
|
voting_public_key: keystore.public_key().unwrap(),
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path: dst_keystore_dir.join(KEYSTORE_NAME),
|
voting_keystore_path: dst_keystore_dir.join(KEYSTORE_NAME),
|
||||||
@ -734,6 +737,7 @@ fn validator_import_launchpad_password_file() {
|
|||||||
description: "".into(),
|
description: "".into(),
|
||||||
voting_public_key: keystore.public_key().unwrap(),
|
voting_public_key: keystore.public_key().unwrap(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
voting_keystore_password_path: None,
|
voting_keystore_password_path: None,
|
||||||
|
@ -212,7 +212,7 @@ fn merge_fee_recipient_flag() {
|
|||||||
CommandLineTest::new()
|
CommandLineTest::new()
|
||||||
.flag("merge", None)
|
.flag("merge", None)
|
||||||
.flag(
|
.flag(
|
||||||
"fee-recipient",
|
"suggested-fee-recipient",
|
||||||
Some("0x00000000219ab540356cbb839cbe05303d7705fa"),
|
Some("0x00000000219ab540356cbb839cbe05303d7705fa"),
|
||||||
)
|
)
|
||||||
.run_with_zero_port()
|
.run_with_zero_port()
|
||||||
|
@ -7,8 +7,10 @@ use std::io::Write;
|
|||||||
use std::net::Ipv4Addr;
|
use std::net::Ipv4Addr;
|
||||||
use std::path::PathBuf;
|
use std::path::PathBuf;
|
||||||
use std::process::Command;
|
use std::process::Command;
|
||||||
|
use std::str::FromStr;
|
||||||
use std::string::ToString;
|
use std::string::ToString;
|
||||||
use tempfile::TempDir;
|
use tempfile::TempDir;
|
||||||
|
use types::Address;
|
||||||
|
|
||||||
/// Returns the `lighthouse validator_client` command.
|
/// Returns the `lighthouse validator_client` command.
|
||||||
fn base_cmd() -> Command {
|
fn base_cmd() -> Command {
|
||||||
@ -218,6 +220,83 @@ fn graffiti_file_with_pk_flag() {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Tests for suggested-fee-recipient flags.
|
||||||
|
#[test]
|
||||||
|
fn fee_recipient_flag() {
|
||||||
|
CommandLineTest::new()
|
||||||
|
.flag(
|
||||||
|
"suggested-fee-recipient",
|
||||||
|
Some("0x00000000219ab540356cbb839cbe05303d7705fa"),
|
||||||
|
)
|
||||||
|
.run()
|
||||||
|
.with_config(|config| {
|
||||||
|
assert_eq!(
|
||||||
|
config.fee_recipient,
|
||||||
|
Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap())
|
||||||
|
)
|
||||||
|
});
|
||||||
|
}
|
||||||
|
#[test]
|
||||||
|
fn fee_recipient_file_flag() {
|
||||||
|
let dir = TempDir::new().expect("Unable to create temporary directory");
|
||||||
|
let mut file =
|
||||||
|
File::create(dir.path().join("fee_recipient.txt")).expect("Unable to create file");
|
||||||
|
let new_key = Keypair::random();
|
||||||
|
let pubkeybytes = PublicKeyBytes::from(new_key.pk);
|
||||||
|
let contents = "default:0x00000000219ab540356cbb839cbe05303d7705fa";
|
||||||
|
file.write_all(contents.as_bytes())
|
||||||
|
.expect("Unable to write to file");
|
||||||
|
CommandLineTest::new()
|
||||||
|
.flag(
|
||||||
|
"suggested-fee-recipient-file",
|
||||||
|
dir.path().join("fee_recipient.txt").as_os_str().to_str(),
|
||||||
|
)
|
||||||
|
.run()
|
||||||
|
.with_config(|config| {
|
||||||
|
// Public key not present so load default.
|
||||||
|
assert_eq!(
|
||||||
|
config
|
||||||
|
.fee_recipient_file
|
||||||
|
.clone()
|
||||||
|
.unwrap()
|
||||||
|
.load_fee_recipient(&pubkeybytes)
|
||||||
|
.unwrap(),
|
||||||
|
Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap())
|
||||||
|
)
|
||||||
|
});
|
||||||
|
}
|
||||||
|
#[test]
|
||||||
|
fn fee_recipient_file_with_pk_flag() {
|
||||||
|
let dir = TempDir::new().expect("Unable to create temporary directory");
|
||||||
|
let mut file =
|
||||||
|
File::create(dir.path().join("fee_recipient.txt")).expect("Unable to create file");
|
||||||
|
let new_key = Keypair::random();
|
||||||
|
let pubkeybytes = PublicKeyBytes::from(new_key.pk);
|
||||||
|
let contents = format!(
|
||||||
|
"{}:0x00000000219ab540356cbb839cbe05303d7705fa",
|
||||||
|
pubkeybytes.to_string()
|
||||||
|
);
|
||||||
|
file.write_all(contents.as_bytes())
|
||||||
|
.expect("Unable to write to file");
|
||||||
|
CommandLineTest::new()
|
||||||
|
.flag(
|
||||||
|
"suggested-fee-recipient-file",
|
||||||
|
dir.path().join("fee_recipient.txt").as_os_str().to_str(),
|
||||||
|
)
|
||||||
|
.run()
|
||||||
|
.with_config(|config| {
|
||||||
|
assert_eq!(
|
||||||
|
config
|
||||||
|
.fee_recipient_file
|
||||||
|
.clone()
|
||||||
|
.unwrap()
|
||||||
|
.load_fee_recipient(&pubkeybytes)
|
||||||
|
.unwrap(),
|
||||||
|
Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap())
|
||||||
|
)
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
// Tests for HTTP flags.
|
// Tests for HTTP flags.
|
||||||
#[test]
|
#[test]
|
||||||
fn http_flag() {
|
fn http_flag() {
|
||||||
|
@ -336,6 +336,7 @@ mod tests {
|
|||||||
enabled: true,
|
enabled: true,
|
||||||
voting_public_key: validator_pubkey.clone(),
|
voting_public_key: validator_pubkey.clone(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
description: String::default(),
|
description: String::default(),
|
||||||
signing_definition: SigningDefinition::LocalKeystore {
|
signing_definition: SigningDefinition::LocalKeystore {
|
||||||
voting_keystore_path: signer_rig.keystore_path.clone(),
|
voting_keystore_path: signer_rig.keystore_path.clone(),
|
||||||
@ -351,6 +352,7 @@ mod tests {
|
|||||||
enabled: true,
|
enabled: true,
|
||||||
voting_public_key: validator_pubkey.clone(),
|
voting_public_key: validator_pubkey.clone(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
description: String::default(),
|
description: String::default(),
|
||||||
signing_definition: SigningDefinition::Web3Signer {
|
signing_definition: SigningDefinition::Web3Signer {
|
||||||
url: signer_rig.url.to_string(),
|
url: signer_rig.url.to_string(),
|
||||||
|
@ -127,6 +127,22 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
|
|||||||
.takes_value(true)
|
.takes_value(true)
|
||||||
.conflicts_with("graffiti")
|
.conflicts_with("graffiti")
|
||||||
)
|
)
|
||||||
|
.arg(
|
||||||
|
Arg::with_name("suggested-fee-recipient")
|
||||||
|
.long("suggested-fee-recipient")
|
||||||
|
.help("The fallback address provided to the BN if nothing suitable is found \
|
||||||
|
in the validator definitions or fee recipient file.")
|
||||||
|
.value_name("FEE-RECIPIENT")
|
||||||
|
.takes_value(true)
|
||||||
|
)
|
||||||
|
.arg(
|
||||||
|
Arg::with_name("suggested-fee-recipient-file")
|
||||||
|
.long("suggested-fee-recipient-file")
|
||||||
|
.help("The fallback address provided to the BN if nothing suitable is found \
|
||||||
|
in the validator definitions.")
|
||||||
|
.value_name("FEE-RECIPIENT-FILE")
|
||||||
|
.takes_value(true)
|
||||||
|
)
|
||||||
/* REST API related arguments */
|
/* REST API related arguments */
|
||||||
.arg(
|
.arg(
|
||||||
Arg::with_name("http")
|
Arg::with_name("http")
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
use crate::fee_recipient_file::FeeRecipientFile;
|
||||||
use crate::graffiti_file::GraffitiFile;
|
use crate::graffiti_file::GraffitiFile;
|
||||||
use crate::{http_api, http_metrics};
|
use crate::{http_api, http_metrics};
|
||||||
use clap::ArgMatches;
|
use clap::ArgMatches;
|
||||||
@ -13,7 +14,7 @@ use slog::{info, warn, Logger};
|
|||||||
use std::fs;
|
use std::fs;
|
||||||
use std::net::Ipv4Addr;
|
use std::net::Ipv4Addr;
|
||||||
use std::path::PathBuf;
|
use std::path::PathBuf;
|
||||||
use types::GRAFFITI_BYTES_LEN;
|
use types::{Address, GRAFFITI_BYTES_LEN};
|
||||||
|
|
||||||
pub const DEFAULT_BEACON_NODE: &str = "http://localhost:5052/";
|
pub const DEFAULT_BEACON_NODE: &str = "http://localhost:5052/";
|
||||||
|
|
||||||
@ -41,6 +42,10 @@ pub struct Config {
|
|||||||
pub graffiti: Option<Graffiti>,
|
pub graffiti: Option<Graffiti>,
|
||||||
/// Graffiti file to load per validator graffitis.
|
/// Graffiti file to load per validator graffitis.
|
||||||
pub graffiti_file: Option<GraffitiFile>,
|
pub graffiti_file: Option<GraffitiFile>,
|
||||||
|
/// Fallback fallback address.
|
||||||
|
pub fee_recipient: Option<Address>,
|
||||||
|
/// Fee recipient file to load per validator suggested-fee-recipients.
|
||||||
|
pub fee_recipient_file: Option<FeeRecipientFile>,
|
||||||
/// Configuration for the HTTP REST API.
|
/// Configuration for the HTTP REST API.
|
||||||
pub http_api: http_api::Config,
|
pub http_api: http_api::Config,
|
||||||
/// Configuration for the HTTP REST API.
|
/// Configuration for the HTTP REST API.
|
||||||
@ -79,6 +84,8 @@ impl Default for Config {
|
|||||||
use_long_timeouts: false,
|
use_long_timeouts: false,
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
graffiti_file: None,
|
graffiti_file: None,
|
||||||
|
fee_recipient: None,
|
||||||
|
fee_recipient_file: None,
|
||||||
http_api: <_>::default(),
|
http_api: <_>::default(),
|
||||||
http_metrics: <_>::default(),
|
http_metrics: <_>::default(),
|
||||||
monitoring_api: None,
|
monitoring_api: None,
|
||||||
@ -197,6 +204,25 @@ impl Config {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if let Some(fee_recipient_file_path) = cli_args.value_of("suggested-fee-recipient-file") {
|
||||||
|
let mut fee_recipient_file = FeeRecipientFile::new(fee_recipient_file_path.into());
|
||||||
|
fee_recipient_file
|
||||||
|
.read_fee_recipient_file()
|
||||||
|
.map_err(|e| format!("Error reading suggested-fee-recipient file: {:?}", e))?;
|
||||||
|
config.fee_recipient_file = Some(fee_recipient_file);
|
||||||
|
info!(
|
||||||
|
log,
|
||||||
|
"Successfully loaded suggested-fee-recipient file";
|
||||||
|
"path" => fee_recipient_file_path
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if let Some(input_fee_recipient) =
|
||||||
|
parse_optional::<Address>(cli_args, "suggested-fee-recipient")?
|
||||||
|
{
|
||||||
|
config.fee_recipient = Some(input_fee_recipient);
|
||||||
|
}
|
||||||
|
|
||||||
if let Some(tls_certs) = parse_optional::<String>(cli_args, "beacon-nodes-tls-certs")? {
|
if let Some(tls_certs) = parse_optional::<String>(cli_args, "beacon-nodes-tls-certs")? {
|
||||||
config.beacon_nodes_tls_certs = Some(tls_certs.split(',').map(PathBuf::from).collect());
|
config.beacon_nodes_tls_certs = Some(tls_certs.split(',').map(PathBuf::from).collect());
|
||||||
}
|
}
|
||||||
|
184
validator_client/src/fee_recipient_file.rs
Normal file
184
validator_client/src/fee_recipient_file.rs
Normal file
@ -0,0 +1,184 @@
|
|||||||
|
use serde_derive::{Deserialize, Serialize};
|
||||||
|
use std::collections::HashMap;
|
||||||
|
use std::fs::File;
|
||||||
|
use std::io::{prelude::*, BufReader};
|
||||||
|
use std::path::PathBuf;
|
||||||
|
use std::str::FromStr;
|
||||||
|
|
||||||
|
use bls::PublicKeyBytes;
|
||||||
|
use types::Address;
|
||||||
|
|
||||||
|
#[derive(Debug)]
|
||||||
|
#[allow(clippy::enum_variant_names)]
|
||||||
|
pub enum Error {
|
||||||
|
InvalidFile(std::io::Error),
|
||||||
|
InvalidLine(String),
|
||||||
|
InvalidPublicKey(String),
|
||||||
|
InvalidFeeRecipient(String),
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Struct to load validator fee-recipients from file.
|
||||||
|
/// The fee-recipient file is expected to have the following structure
|
||||||
|
///
|
||||||
|
/// default: 0x00000000219ab540356cbb839cbe05303d7705fa
|
||||||
|
/// public_key1: fee-recipient1
|
||||||
|
/// public_key2: fee-recipient2
|
||||||
|
/// ...
|
||||||
|
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||||
|
pub struct FeeRecipientFile {
|
||||||
|
fee_recipient_path: PathBuf,
|
||||||
|
fee_recipients: HashMap<PublicKeyBytes, Address>,
|
||||||
|
default: Option<Address>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl FeeRecipientFile {
|
||||||
|
pub fn new(fee_recipient_path: PathBuf) -> Self {
|
||||||
|
Self {
|
||||||
|
fee_recipient_path,
|
||||||
|
fee_recipients: HashMap::new(),
|
||||||
|
default: None,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns the fee-recipient corresponding to the given public key if present, else returns the
|
||||||
|
/// default fee-recipient.
|
||||||
|
///
|
||||||
|
/// Returns an error if loading from the fee-recipient file fails.
|
||||||
|
pub fn get_fee_recipient(&self, public_key: &PublicKeyBytes) -> Result<Option<Address>, Error> {
|
||||||
|
Ok(self
|
||||||
|
.fee_recipients
|
||||||
|
.get(public_key)
|
||||||
|
.copied()
|
||||||
|
.or(self.default))
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Loads the fee-recipient file and populates the default fee-recipient and `fee_recipients` hashmap.
|
||||||
|
/// Returns the fee-recipient corresponding to the given public key if present, else returns the
|
||||||
|
/// default fee-recipient.
|
||||||
|
///
|
||||||
|
/// Returns an error if loading from the fee-recipient file fails.
|
||||||
|
pub fn load_fee_recipient(
|
||||||
|
&mut self,
|
||||||
|
public_key: &PublicKeyBytes,
|
||||||
|
) -> Result<Option<Address>, Error> {
|
||||||
|
self.read_fee_recipient_file()?;
|
||||||
|
Ok(self
|
||||||
|
.fee_recipients
|
||||||
|
.get(public_key)
|
||||||
|
.copied()
|
||||||
|
.or(self.default))
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Reads from a fee-recipient file with the specified format and populates the default value
|
||||||
|
/// and the hashmap.
|
||||||
|
///
|
||||||
|
/// Returns an error if the file does not exist, or if the format is invalid.
|
||||||
|
pub fn read_fee_recipient_file(&mut self) -> Result<(), Error> {
|
||||||
|
let file = File::open(self.fee_recipient_path.as_path()).map_err(Error::InvalidFile)?;
|
||||||
|
let reader = BufReader::new(file);
|
||||||
|
|
||||||
|
let lines = reader.lines();
|
||||||
|
|
||||||
|
self.default = None;
|
||||||
|
self.fee_recipients.clear();
|
||||||
|
|
||||||
|
for line in lines {
|
||||||
|
let line = line.map_err(|e| Error::InvalidLine(e.to_string()))?;
|
||||||
|
let (pk_opt, fee_recipient) = read_line(&line)?;
|
||||||
|
match pk_opt {
|
||||||
|
Some(pk) => {
|
||||||
|
self.fee_recipients.insert(pk, fee_recipient);
|
||||||
|
}
|
||||||
|
None => self.default = Some(fee_recipient),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Parses a line from the fee-recipient file.
|
||||||
|
///
|
||||||
|
/// `Ok((None, fee_recipient))` represents the fee-recipient for the default key.
|
||||||
|
/// `Ok((Some(pk), fee_recipient))` represents fee-recipient for the public key `pk`.
|
||||||
|
/// Returns an error if the line is in the wrong format or does not contain a valid public key or fee-recipient.
|
||||||
|
fn read_line(line: &str) -> Result<(Option<PublicKeyBytes>, Address), Error> {
|
||||||
|
if let Some(i) = line.find(':') {
|
||||||
|
let (key, value) = line.split_at(i);
|
||||||
|
// Note: `value.len() >=1` so `value[1..]` is safe
|
||||||
|
let fee_recipient = Address::from_str(value[1..].trim())
|
||||||
|
.map_err(|e| Error::InvalidFeeRecipient(e.to_string()))?;
|
||||||
|
if key == "default" {
|
||||||
|
Ok((None, fee_recipient))
|
||||||
|
} else {
|
||||||
|
let pk = PublicKeyBytes::from_str(key).map_err(Error::InvalidPublicKey)?;
|
||||||
|
Ok((Some(pk), fee_recipient))
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
Err(Error::InvalidLine(format!("Missing delimiter: {}", line)))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[cfg(test)]
|
||||||
|
mod tests {
|
||||||
|
use super::*;
|
||||||
|
use bls::Keypair;
|
||||||
|
use std::io::LineWriter;
|
||||||
|
use tempfile::TempDir;
|
||||||
|
|
||||||
|
const DEFAULT_FEE_RECIPIENT: &str = "0x00000000219ab540356cbb839cbe05303d7705fa";
|
||||||
|
const CUSTOM_FEE_RECIPIENT1: &str = "0x4242424242424242424242424242424242424242";
|
||||||
|
const CUSTOM_FEE_RECIPIENT2: &str = "0x0000000000000000000000000000000000000001";
|
||||||
|
const PK1: &str = "0x800012708dc03f611751aad7a43a082142832b5c1aceed07ff9b543cf836381861352aa923c70eeb02018b638aa306aa";
|
||||||
|
const PK2: &str = "0x80001866ce324de7d80ec73be15e2d064dcf121adf1b34a0d679f2b9ecbab40ce021e03bb877e1a2fe72eaaf475e6e21";
|
||||||
|
|
||||||
|
// Create a fee-recipient file in the required format and return a path to the file.
|
||||||
|
fn create_fee_recipient_file() -> PathBuf {
|
||||||
|
let temp = TempDir::new().unwrap();
|
||||||
|
let pk1 = PublicKeyBytes::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
|
||||||
|
let pk2 = PublicKeyBytes::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
|
||||||
|
|
||||||
|
let file_name = temp.into_path().join("fee_recipient.txt");
|
||||||
|
|
||||||
|
let file = File::create(&file_name).unwrap();
|
||||||
|
let mut fee_recipient_file = LineWriter::new(file);
|
||||||
|
fee_recipient_file
|
||||||
|
.write_all(format!("default: {}\n", DEFAULT_FEE_RECIPIENT).as_bytes())
|
||||||
|
.unwrap();
|
||||||
|
fee_recipient_file
|
||||||
|
.write_all(format!("{}: {}\n", pk1.as_hex_string(), CUSTOM_FEE_RECIPIENT1).as_bytes())
|
||||||
|
.unwrap();
|
||||||
|
fee_recipient_file
|
||||||
|
.write_all(format!("{}: {}\n", pk2.as_hex_string(), CUSTOM_FEE_RECIPIENT2).as_bytes())
|
||||||
|
.unwrap();
|
||||||
|
fee_recipient_file.flush().unwrap();
|
||||||
|
file_name
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn test_load_fee_recipient() {
|
||||||
|
let fee_recipient_file_path = create_fee_recipient_file();
|
||||||
|
let mut gf = FeeRecipientFile::new(fee_recipient_file_path);
|
||||||
|
|
||||||
|
let pk1 = PublicKeyBytes::deserialize(&hex::decode(&PK1[2..]).unwrap()).unwrap();
|
||||||
|
let pk2 = PublicKeyBytes::deserialize(&hex::decode(&PK2[2..]).unwrap()).unwrap();
|
||||||
|
|
||||||
|
// Read once
|
||||||
|
gf.read_fee_recipient_file().unwrap();
|
||||||
|
|
||||||
|
assert_eq!(
|
||||||
|
gf.load_fee_recipient(&pk1).unwrap().unwrap(),
|
||||||
|
Address::from_str(CUSTOM_FEE_RECIPIENT1).unwrap()
|
||||||
|
);
|
||||||
|
assert_eq!(
|
||||||
|
gf.load_fee_recipient(&pk2).unwrap().unwrap(),
|
||||||
|
Address::from_str(CUSTOM_FEE_RECIPIENT2).unwrap()
|
||||||
|
);
|
||||||
|
|
||||||
|
// Random pk should return the default fee-recipient
|
||||||
|
let random_pk = Keypair::random().pk.compress();
|
||||||
|
assert_eq!(
|
||||||
|
gf.load_fee_recipient(&random_pk).unwrap().unwrap(),
|
||||||
|
Address::from_str(DEFAULT_FEE_RECIPIENT).unwrap()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
@ -139,6 +139,7 @@ pub async fn create_validators_mnemonic<P: AsRef<Path>, T: 'static + SlotClock,
|
|||||||
voting_password_string,
|
voting_password_string,
|
||||||
request.enable,
|
request.enable,
|
||||||
request.graffiti.clone(),
|
request.graffiti.clone(),
|
||||||
|
request.suggested_fee_recipient,
|
||||||
)
|
)
|
||||||
.await
|
.await
|
||||||
.map_err(|e| {
|
.map_err(|e| {
|
||||||
@ -152,6 +153,7 @@ pub async fn create_validators_mnemonic<P: AsRef<Path>, T: 'static + SlotClock,
|
|||||||
enabled: request.enable,
|
enabled: request.enable,
|
||||||
description: request.description.clone(),
|
description: request.description.clone(),
|
||||||
graffiti: request.graffiti.clone(),
|
graffiti: request.graffiti.clone(),
|
||||||
|
suggested_fee_recipient: request.suggested_fee_recipient,
|
||||||
voting_pubkey,
|
voting_pubkey,
|
||||||
eth1_deposit_tx_data: eth2_serde_utils::hex::encode(ð1_deposit_data.rlp),
|
eth1_deposit_tx_data: eth2_serde_utils::hex::encode(ð1_deposit_data.rlp),
|
||||||
deposit_gwei: request.deposit_gwei,
|
deposit_gwei: request.deposit_gwei,
|
||||||
@ -170,6 +172,7 @@ pub async fn create_validators_web3signer<T: 'static + SlotClock, E: EthSpec>(
|
|||||||
enabled: request.enable,
|
enabled: request.enable,
|
||||||
voting_public_key: request.voting_public_key.clone(),
|
voting_public_key: request.voting_public_key.clone(),
|
||||||
graffiti: request.graffiti.clone(),
|
graffiti: request.graffiti.clone(),
|
||||||
|
suggested_fee_recipient: request.suggested_fee_recipient,
|
||||||
description: request.description.clone(),
|
description: request.description.clone(),
|
||||||
signing_definition: SigningDefinition::Web3Signer {
|
signing_definition: SigningDefinition::Web3Signer {
|
||||||
url: request.url.clone(),
|
url: request.url.clone(),
|
||||||
|
@ -201,6 +201,7 @@ fn import_single_keystore<T: SlotClock + 'static, E: EthSpec>(
|
|||||||
password,
|
password,
|
||||||
true,
|
true,
|
||||||
None,
|
None,
|
||||||
|
None,
|
||||||
))
|
))
|
||||||
.map_err(|e| format!("failed to initialize validator: {:?}", e))?;
|
.map_err(|e| format!("failed to initialize validator: {:?}", e))?;
|
||||||
|
|
||||||
|
@ -409,6 +409,7 @@ pub fn serve<T: 'static + SlotClock + Clone, E: EthSpec>(
|
|||||||
drop(validator_dir);
|
drop(validator_dir);
|
||||||
let voting_password = body.password.clone();
|
let voting_password = body.password.clone();
|
||||||
let graffiti = body.graffiti.clone();
|
let graffiti = body.graffiti.clone();
|
||||||
|
let suggested_fee_recipient = body.suggested_fee_recipient;
|
||||||
|
|
||||||
let validator_def = {
|
let validator_def = {
|
||||||
if let Some(runtime) = runtime.upgrade() {
|
if let Some(runtime) = runtime.upgrade() {
|
||||||
@ -418,6 +419,7 @@ pub fn serve<T: 'static + SlotClock + Clone, E: EthSpec>(
|
|||||||
voting_password,
|
voting_password,
|
||||||
body.enable,
|
body.enable,
|
||||||
graffiti,
|
graffiti,
|
||||||
|
suggested_fee_recipient,
|
||||||
))
|
))
|
||||||
.map_err(|e| {
|
.map_err(|e| {
|
||||||
warp_utils::reject::custom_server_error(format!(
|
warp_utils::reject::custom_server_error(format!(
|
||||||
|
@ -267,6 +267,7 @@ impl ApiTester {
|
|||||||
enable: !s.disabled.contains(&i),
|
enable: !s.disabled.contains(&i),
|
||||||
description: format!("boi #{}", i),
|
description: format!("boi #{}", i),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
deposit_gwei: E::default_spec().max_effective_balance,
|
deposit_gwei: E::default_spec().max_effective_balance,
|
||||||
})
|
})
|
||||||
.collect::<Vec<_>>();
|
.collect::<Vec<_>>();
|
||||||
@ -397,6 +398,7 @@ impl ApiTester {
|
|||||||
.into(),
|
.into(),
|
||||||
keystore,
|
keystore,
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
};
|
};
|
||||||
|
|
||||||
self.client
|
self.client
|
||||||
@ -414,6 +416,7 @@ impl ApiTester {
|
|||||||
.into(),
|
.into(),
|
||||||
keystore,
|
keystore,
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
};
|
};
|
||||||
|
|
||||||
let response = self
|
let response = self
|
||||||
@ -449,6 +452,7 @@ impl ApiTester {
|
|||||||
enable: s.enabled,
|
enable: s.enabled,
|
||||||
description: format!("{}", i),
|
description: format!("{}", i),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
voting_public_key: kp.pk,
|
voting_public_key: kp.pk,
|
||||||
url: format!("http://signer_{}.com/", i),
|
url: format!("http://signer_{}.com/", i),
|
||||||
root_certificate_path: None,
|
root_certificate_path: None,
|
||||||
@ -574,6 +578,7 @@ fn routes_with_invalid_auth() {
|
|||||||
enable: <_>::default(),
|
enable: <_>::default(),
|
||||||
description: <_>::default(),
|
description: <_>::default(),
|
||||||
graffiti: <_>::default(),
|
graffiti: <_>::default(),
|
||||||
|
suggested_fee_recipient: <_>::default(),
|
||||||
deposit_gwei: <_>::default(),
|
deposit_gwei: <_>::default(),
|
||||||
}])
|
}])
|
||||||
.await
|
.await
|
||||||
@ -602,6 +607,7 @@ fn routes_with_invalid_auth() {
|
|||||||
enable: <_>::default(),
|
enable: <_>::default(),
|
||||||
keystore,
|
keystore,
|
||||||
graffiti: <_>::default(),
|
graffiti: <_>::default(),
|
||||||
|
suggested_fee_recipient: <_>::default(),
|
||||||
})
|
})
|
||||||
.await
|
.await
|
||||||
})
|
})
|
||||||
|
@ -37,6 +37,7 @@ fn web3signer_validator_with_pubkey(pubkey: PublicKey) -> Web3SignerValidatorReq
|
|||||||
enable: true,
|
enable: true,
|
||||||
description: "".into(),
|
description: "".into(),
|
||||||
graffiti: None,
|
graffiti: None,
|
||||||
|
suggested_fee_recipient: None,
|
||||||
voting_public_key: pubkey,
|
voting_public_key: pubkey,
|
||||||
url: web3_signer_url(),
|
url: web3_signer_url(),
|
||||||
root_certificate_path: None,
|
root_certificate_path: None,
|
||||||
|
@ -27,7 +27,7 @@ use std::io::{self, Read};
|
|||||||
use std::path::{Path, PathBuf};
|
use std::path::{Path, PathBuf};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use types::{Graffiti, Keypair, PublicKey, PublicKeyBytes};
|
use types::{Address, Graffiti, Keypair, PublicKey, PublicKeyBytes};
|
||||||
use url::{ParseError, Url};
|
use url::{ParseError, Url};
|
||||||
use validator_dir::Builder as ValidatorDirBuilder;
|
use validator_dir::Builder as ValidatorDirBuilder;
|
||||||
|
|
||||||
@ -104,6 +104,7 @@ impl From<LockfileError> for Error {
|
|||||||
pub struct InitializedValidator {
|
pub struct InitializedValidator {
|
||||||
signing_method: Arc<SigningMethod>,
|
signing_method: Arc<SigningMethod>,
|
||||||
graffiti: Option<Graffiti>,
|
graffiti: Option<Graffiti>,
|
||||||
|
suggested_fee_recipient: Option<Address>,
|
||||||
/// The validators index in `state.validators`, to be updated by an external service.
|
/// The validators index in `state.validators`, to be updated by an external service.
|
||||||
index: Option<u64>,
|
index: Option<u64>,
|
||||||
}
|
}
|
||||||
@ -269,6 +270,7 @@ impl InitializedValidator {
|
|||||||
Ok(Self {
|
Ok(Self {
|
||||||
signing_method: Arc::new(signing_method),
|
signing_method: Arc::new(signing_method),
|
||||||
graffiti: def.graffiti.map(Into::into),
|
graffiti: def.graffiti.map(Into::into),
|
||||||
|
suggested_fee_recipient: def.suggested_fee_recipient,
|
||||||
index: None,
|
index: None,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
@ -538,6 +540,14 @@ impl InitializedValidators {
|
|||||||
self.validators.get(public_key).and_then(|v| v.graffiti)
|
self.validators.get(public_key).and_then(|v| v.graffiti)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns the `suggested_fee_recipient` for a given public key specified in the
|
||||||
|
/// `ValidatorDefinitions`.
|
||||||
|
pub fn suggested_fee_recipient(&self, public_key: &PublicKeyBytes) -> Option<Address> {
|
||||||
|
self.validators
|
||||||
|
.get(public_key)
|
||||||
|
.and_then(|v| v.suggested_fee_recipient)
|
||||||
|
}
|
||||||
|
|
||||||
/// Sets the `InitializedValidator` and `ValidatorDefinition` `enabled` values.
|
/// Sets the `InitializedValidator` and `ValidatorDefinition` `enabled` values.
|
||||||
///
|
///
|
||||||
/// ## Notes
|
/// ## Notes
|
||||||
|
@ -5,10 +5,12 @@ mod check_synced;
|
|||||||
mod cli;
|
mod cli;
|
||||||
mod config;
|
mod config;
|
||||||
mod duties_service;
|
mod duties_service;
|
||||||
|
mod fee_recipient_file;
|
||||||
mod graffiti_file;
|
mod graffiti_file;
|
||||||
mod http_metrics;
|
mod http_metrics;
|
||||||
mod key_cache;
|
mod key_cache;
|
||||||
mod notifier;
|
mod notifier;
|
||||||
|
mod preparation_service;
|
||||||
mod signing_method;
|
mod signing_method;
|
||||||
mod sync_committee_service;
|
mod sync_committee_service;
|
||||||
|
|
||||||
@ -38,6 +40,7 @@ use eth2::{reqwest::ClientBuilder, BeaconNodeHttpClient, StatusCode, Timeouts};
|
|||||||
use http_api::ApiSecret;
|
use http_api::ApiSecret;
|
||||||
use notifier::spawn_notifier;
|
use notifier::spawn_notifier;
|
||||||
use parking_lot::RwLock;
|
use parking_lot::RwLock;
|
||||||
|
use preparation_service::{PreparationService, PreparationServiceBuilder};
|
||||||
use reqwest::Certificate;
|
use reqwest::Certificate;
|
||||||
use slog::{error, info, warn, Logger};
|
use slog::{error, info, warn, Logger};
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
@ -82,6 +85,7 @@ pub struct ProductionValidatorClient<T: EthSpec> {
|
|||||||
attestation_service: AttestationService<SystemTimeSlotClock, T>,
|
attestation_service: AttestationService<SystemTimeSlotClock, T>,
|
||||||
sync_committee_service: SyncCommitteeService<SystemTimeSlotClock, T>,
|
sync_committee_service: SyncCommitteeService<SystemTimeSlotClock, T>,
|
||||||
doppelganger_service: Option<Arc<DoppelgangerService>>,
|
doppelganger_service: Option<Arc<DoppelgangerService>>,
|
||||||
|
preparation_service: PreparationService<SystemTimeSlotClock, T>,
|
||||||
validator_store: Arc<ValidatorStore<SystemTimeSlotClock, T>>,
|
validator_store: Arc<ValidatorStore<SystemTimeSlotClock, T>>,
|
||||||
http_api_listen_addr: Option<SocketAddr>,
|
http_api_listen_addr: Option<SocketAddr>,
|
||||||
config: Config,
|
config: Config,
|
||||||
@ -406,6 +410,15 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
|
|||||||
.runtime_context(context.service_context("attestation".into()))
|
.runtime_context(context.service_context("attestation".into()))
|
||||||
.build()?;
|
.build()?;
|
||||||
|
|
||||||
|
let preparation_service = PreparationServiceBuilder::new()
|
||||||
|
.slot_clock(slot_clock.clone())
|
||||||
|
.validator_store(validator_store.clone())
|
||||||
|
.beacon_nodes(beacon_nodes.clone())
|
||||||
|
.runtime_context(context.service_context("preparation".into()))
|
||||||
|
.fee_recipient(config.fee_recipient)
|
||||||
|
.fee_recipient_file(config.fee_recipient_file.clone())
|
||||||
|
.build()?;
|
||||||
|
|
||||||
let sync_committee_service = SyncCommitteeService::new(
|
let sync_committee_service = SyncCommitteeService::new(
|
||||||
duties_service.clone(),
|
duties_service.clone(),
|
||||||
validator_store.clone(),
|
validator_store.clone(),
|
||||||
@ -427,6 +440,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
|
|||||||
attestation_service,
|
attestation_service,
|
||||||
sync_committee_service,
|
sync_committee_service,
|
||||||
doppelganger_service,
|
doppelganger_service,
|
||||||
|
preparation_service,
|
||||||
validator_store,
|
validator_store,
|
||||||
config,
|
config,
|
||||||
http_api_listen_addr: None,
|
http_api_listen_addr: None,
|
||||||
@ -458,6 +472,11 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
|
|||||||
.start_update_service(&self.context.eth2_config.spec)
|
.start_update_service(&self.context.eth2_config.spec)
|
||||||
.map_err(|e| format!("Unable to start sync committee service: {}", e))?;
|
.map_err(|e| format!("Unable to start sync committee service: {}", e))?;
|
||||||
|
|
||||||
|
self.preparation_service
|
||||||
|
.clone()
|
||||||
|
.start_update_service(&self.context.eth2_config.spec)
|
||||||
|
.map_err(|e| format!("Unable to start preparation service: {}", e))?;
|
||||||
|
|
||||||
if let Some(doppelganger_service) = self.doppelganger_service.clone() {
|
if let Some(doppelganger_service) = self.doppelganger_service.clone() {
|
||||||
DoppelgangerService::start_update_service(
|
DoppelgangerService::start_update_service(
|
||||||
doppelganger_service,
|
doppelganger_service,
|
||||||
|
278
validator_client/src/preparation_service.rs
Normal file
278
validator_client/src/preparation_service.rs
Normal file
@ -0,0 +1,278 @@
|
|||||||
|
use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced};
|
||||||
|
use crate::{
|
||||||
|
fee_recipient_file::FeeRecipientFile,
|
||||||
|
validator_store::{DoppelgangerStatus, ValidatorStore},
|
||||||
|
};
|
||||||
|
use environment::RuntimeContext;
|
||||||
|
use slog::{debug, error, info};
|
||||||
|
use slot_clock::SlotClock;
|
||||||
|
use std::ops::Deref;
|
||||||
|
use std::sync::Arc;
|
||||||
|
use tokio::time::{sleep, Duration};
|
||||||
|
use types::{Address, ChainSpec, EthSpec, ProposerPreparationData};
|
||||||
|
|
||||||
|
/// Builds an `PreparationService`.
|
||||||
|
pub struct PreparationServiceBuilder<T: SlotClock + 'static, E: EthSpec> {
|
||||||
|
validator_store: Option<Arc<ValidatorStore<T, E>>>,
|
||||||
|
slot_clock: Option<T>,
|
||||||
|
beacon_nodes: Option<Arc<BeaconNodeFallback<T, E>>>,
|
||||||
|
context: Option<RuntimeContext<E>>,
|
||||||
|
fee_recipient: Option<Address>,
|
||||||
|
fee_recipient_file: Option<FeeRecipientFile>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: SlotClock + 'static, E: EthSpec> PreparationServiceBuilder<T, E> {
|
||||||
|
pub fn new() -> Self {
|
||||||
|
Self {
|
||||||
|
validator_store: None,
|
||||||
|
slot_clock: None,
|
||||||
|
beacon_nodes: None,
|
||||||
|
context: None,
|
||||||
|
fee_recipient: None,
|
||||||
|
fee_recipient_file: None,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn validator_store(mut self, store: Arc<ValidatorStore<T, E>>) -> Self {
|
||||||
|
self.validator_store = Some(store);
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn slot_clock(mut self, slot_clock: T) -> Self {
|
||||||
|
self.slot_clock = Some(slot_clock);
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn beacon_nodes(mut self, beacon_nodes: Arc<BeaconNodeFallback<T, E>>) -> Self {
|
||||||
|
self.beacon_nodes = Some(beacon_nodes);
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn runtime_context(mut self, context: RuntimeContext<E>) -> Self {
|
||||||
|
self.context = Some(context);
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn fee_recipient(mut self, fee_recipient: Option<Address>) -> Self {
|
||||||
|
self.fee_recipient = fee_recipient;
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn fee_recipient_file(mut self, fee_recipient_file: Option<FeeRecipientFile>) -> Self {
|
||||||
|
self.fee_recipient_file = fee_recipient_file;
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn build(self) -> Result<PreparationService<T, E>, String> {
|
||||||
|
Ok(PreparationService {
|
||||||
|
inner: Arc::new(Inner {
|
||||||
|
validator_store: self
|
||||||
|
.validator_store
|
||||||
|
.ok_or("Cannot build PreparationService without validator_store")?,
|
||||||
|
slot_clock: self
|
||||||
|
.slot_clock
|
||||||
|
.ok_or("Cannot build PreparationService without slot_clock")?,
|
||||||
|
beacon_nodes: self
|
||||||
|
.beacon_nodes
|
||||||
|
.ok_or("Cannot build PreparationService without beacon_nodes")?,
|
||||||
|
context: self
|
||||||
|
.context
|
||||||
|
.ok_or("Cannot build PreparationService without runtime_context")?,
|
||||||
|
fee_recipient: self.fee_recipient,
|
||||||
|
fee_recipient_file: self.fee_recipient_file,
|
||||||
|
}),
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Helper to minimise `Arc` usage.
|
||||||
|
pub struct Inner<T, E: EthSpec> {
|
||||||
|
validator_store: Arc<ValidatorStore<T, E>>,
|
||||||
|
slot_clock: T,
|
||||||
|
beacon_nodes: Arc<BeaconNodeFallback<T, E>>,
|
||||||
|
context: RuntimeContext<E>,
|
||||||
|
fee_recipient: Option<Address>,
|
||||||
|
fee_recipient_file: Option<FeeRecipientFile>,
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Attempts to produce proposer preparations for all known validators at the beginning of each epoch.
|
||||||
|
pub struct PreparationService<T, E: EthSpec> {
|
||||||
|
inner: Arc<Inner<T, E>>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T, E: EthSpec> Clone for PreparationService<T, E> {
|
||||||
|
fn clone(&self) -> Self {
|
||||||
|
Self {
|
||||||
|
inner: self.inner.clone(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T, E: EthSpec> Deref for PreparationService<T, E> {
|
||||||
|
type Target = Inner<T, E>;
|
||||||
|
|
||||||
|
fn deref(&self) -> &Self::Target {
|
||||||
|
self.inner.deref()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: SlotClock + 'static, E: EthSpec> PreparationService<T, E> {
|
||||||
|
/// Starts the service which periodically produces proposer preparations.
|
||||||
|
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
|
||||||
|
let log = self.context.log().clone();
|
||||||
|
|
||||||
|
let slot_duration = Duration::from_secs(spec.seconds_per_slot);
|
||||||
|
let duration_to_next_epoch = self
|
||||||
|
.slot_clock
|
||||||
|
.duration_to_next_epoch(E::slots_per_epoch())
|
||||||
|
.ok_or("Unable to determine duration to next epoch")?;
|
||||||
|
|
||||||
|
info!(
|
||||||
|
log,
|
||||||
|
"Proposer preparation service started";
|
||||||
|
"next_update_millis" => duration_to_next_epoch.as_millis()
|
||||||
|
);
|
||||||
|
|
||||||
|
let executor = self.context.executor.clone();
|
||||||
|
let spec = spec.clone();
|
||||||
|
|
||||||
|
let interval_fut = async move {
|
||||||
|
loop {
|
||||||
|
// Poll the endpoint immediately to ensure fee recipients are received.
|
||||||
|
self.prepare_proposers_and_publish(&spec)
|
||||||
|
.await
|
||||||
|
.map_err(|e| {
|
||||||
|
error!(
|
||||||
|
log,
|
||||||
|
"Error during proposer preparation";
|
||||||
|
"error" => format!("{:?}", e),
|
||||||
|
)
|
||||||
|
})
|
||||||
|
.unwrap_or(());
|
||||||
|
|
||||||
|
if let Some(duration_to_next_slot) = self.slot_clock.duration_to_next_slot() {
|
||||||
|
sleep(duration_to_next_slot).await;
|
||||||
|
} else {
|
||||||
|
error!(log, "Failed to read slot clock");
|
||||||
|
// If we can't read the slot clock, just wait another slot.
|
||||||
|
sleep(slot_duration).await;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
executor.spawn(interval_fut, "preparation_service");
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Prepare proposer preparations and send to beacon node
|
||||||
|
async fn prepare_proposers_and_publish(&self, spec: &ChainSpec) -> Result<(), String> {
|
||||||
|
let preparation_data = self.collect_preparation_data(spec);
|
||||||
|
if !preparation_data.is_empty() {
|
||||||
|
self.publish_preparation_data(preparation_data).await?;
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
fn collect_preparation_data(&self, spec: &ChainSpec) -> Vec<ProposerPreparationData> {
|
||||||
|
let log = self.context.log();
|
||||||
|
|
||||||
|
let fee_recipient_file = self
|
||||||
|
.fee_recipient_file
|
||||||
|
.clone()
|
||||||
|
.map(|mut fee_recipient_file| {
|
||||||
|
fee_recipient_file
|
||||||
|
.read_fee_recipient_file()
|
||||||
|
.map_err(|e| {
|
||||||
|
error!(
|
||||||
|
log,
|
||||||
|
"{}", format!("Error loading fee-recipient file: {:?}", e);
|
||||||
|
);
|
||||||
|
})
|
||||||
|
.unwrap_or(());
|
||||||
|
fee_recipient_file
|
||||||
|
});
|
||||||
|
|
||||||
|
let all_pubkeys: Vec<_> = self
|
||||||
|
.validator_store
|
||||||
|
.voting_pubkeys(DoppelgangerStatus::ignored);
|
||||||
|
|
||||||
|
all_pubkeys
|
||||||
|
.into_iter()
|
||||||
|
.filter_map(|pubkey| {
|
||||||
|
let validator_index = self.validator_store.validator_index(&pubkey);
|
||||||
|
if let Some(validator_index) = validator_index {
|
||||||
|
let fee_recipient = if let Some(from_validator_defs) =
|
||||||
|
self.validator_store.suggested_fee_recipient(&pubkey)
|
||||||
|
{
|
||||||
|
// If there is a `suggested_fee_recipient` in the validator definitions yaml
|
||||||
|
// file, use that value.
|
||||||
|
Some(from_validator_defs)
|
||||||
|
} else {
|
||||||
|
// If there's nothing in the validator defs file, check the fee recipient
|
||||||
|
// file.
|
||||||
|
fee_recipient_file
|
||||||
|
.as_ref()
|
||||||
|
.and_then(|f| match f.get_fee_recipient(&pubkey) {
|
||||||
|
Ok(f) => f,
|
||||||
|
Err(_e) => None,
|
||||||
|
})
|
||||||
|
// If there's nothing in the file, try the process-level default value.
|
||||||
|
.or(self.fee_recipient)
|
||||||
|
};
|
||||||
|
|
||||||
|
if let Some(fee_recipient) = fee_recipient {
|
||||||
|
Some(ProposerPreparationData {
|
||||||
|
validator_index,
|
||||||
|
fee_recipient,
|
||||||
|
})
|
||||||
|
} else {
|
||||||
|
if spec.bellatrix_fork_epoch.is_some() {
|
||||||
|
error!(
|
||||||
|
log,
|
||||||
|
"Validator is missing fee recipient";
|
||||||
|
"msg" => "update validator_definitions.yml",
|
||||||
|
"pubkey" => ?pubkey
|
||||||
|
);
|
||||||
|
}
|
||||||
|
None
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.collect()
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn publish_preparation_data(
|
||||||
|
&self,
|
||||||
|
preparation_data: Vec<ProposerPreparationData>,
|
||||||
|
) -> Result<(), String> {
|
||||||
|
let log = self.context.log();
|
||||||
|
|
||||||
|
// Post the proposer preparations to the BN.
|
||||||
|
let preparation_data_len = preparation_data.len();
|
||||||
|
let preparation_entries = preparation_data.as_slice();
|
||||||
|
match self
|
||||||
|
.beacon_nodes
|
||||||
|
.first_success(RequireSynced::Yes, |beacon_node| async move {
|
||||||
|
beacon_node
|
||||||
|
.post_validator_prepare_beacon_proposer(preparation_entries)
|
||||||
|
.await
|
||||||
|
})
|
||||||
|
.await
|
||||||
|
{
|
||||||
|
Ok(()) => debug!(
|
||||||
|
log,
|
||||||
|
"Published proposer preparation";
|
||||||
|
"count" => preparation_data_len,
|
||||||
|
),
|
||||||
|
Err(e) => error!(
|
||||||
|
log,
|
||||||
|
"Unable to publish proposer preparation";
|
||||||
|
"error" => %e,
|
||||||
|
),
|
||||||
|
}
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
}
|
@ -17,7 +17,7 @@ use std::path::Path;
|
|||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use types::{
|
use types::{
|
||||||
attestation::Error as AttestationError, graffiti::GraffitiString, AggregateAndProof,
|
attestation::Error as AttestationError, graffiti::GraffitiString, Address, AggregateAndProof,
|
||||||
Attestation, BeaconBlock, ChainSpec, ContributionAndProof, Domain, Epoch, EthSpec, Fork,
|
Attestation, BeaconBlock, ChainSpec, ContributionAndProof, Domain, Epoch, EthSpec, Fork,
|
||||||
Graffiti, Hash256, Keypair, PublicKeyBytes, SelectionProof, Signature, SignedAggregateAndProof,
|
Graffiti, Hash256, Keypair, PublicKeyBytes, SelectionProof, Signature, SignedAggregateAndProof,
|
||||||
SignedBeaconBlock, SignedContributionAndProof, Slot, SyncAggregatorSelectionData,
|
SignedBeaconBlock, SignedContributionAndProof, Slot, SyncAggregatorSelectionData,
|
||||||
@ -148,11 +148,13 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
|
|||||||
password: ZeroizeString,
|
password: ZeroizeString,
|
||||||
enable: bool,
|
enable: bool,
|
||||||
graffiti: Option<GraffitiString>,
|
graffiti: Option<GraffitiString>,
|
||||||
|
suggested_fee_recipient: Option<Address>,
|
||||||
) -> Result<ValidatorDefinition, String> {
|
) -> Result<ValidatorDefinition, String> {
|
||||||
let mut validator_def = ValidatorDefinition::new_keystore_with_password(
|
let mut validator_def = ValidatorDefinition::new_keystore_with_password(
|
||||||
voting_keystore_path,
|
voting_keystore_path,
|
||||||
Some(password),
|
Some(password),
|
||||||
graffiti.map(Into::into),
|
graffiti.map(Into::into),
|
||||||
|
suggested_fee_recipient,
|
||||||
)
|
)
|
||||||
.map_err(|e| format!("failed to create validator definitions: {:?}", e))?;
|
.map_err(|e| format!("failed to create validator definitions: {:?}", e))?;
|
||||||
|
|
||||||
@ -351,6 +353,12 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
|
|||||||
self.validators.read().graffiti(validator_pubkey)
|
self.validators.read().graffiti(validator_pubkey)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub fn suggested_fee_recipient(&self, validator_pubkey: &PublicKeyBytes) -> Option<Address> {
|
||||||
|
self.validators
|
||||||
|
.read()
|
||||||
|
.suggested_fee_recipient(validator_pubkey)
|
||||||
|
}
|
||||||
|
|
||||||
pub async fn sign_block(
|
pub async fn sign_block(
|
||||||
&self,
|
&self,
|
||||||
validator_pubkey: PublicKeyBytes,
|
validator_pubkey: PublicKeyBytes,
|
||||||
|
Loading…
Reference in New Issue
Block a user