Detect and notify when deserializable shreds are available (#11816)
* Add logic to check for complete data ranges * Add RPC signature notification Co-authored-by: Carl <carl@solana.com>
This commit is contained in:
82
core/src/completed_data_sets_service.rs
Normal file
82
core/src/completed_data_sets_service.rs
Normal file
@ -0,0 +1,82 @@
|
||||
use crate::rpc_subscriptions::RpcSubscriptions;
|
||||
use crossbeam_channel::{Receiver, RecvTimeoutError, Sender};
|
||||
use solana_ledger::blockstore::{Blockstore, CompletedDataSetInfo};
|
||||
use solana_sdk::signature::Signature;
|
||||
use std::{
|
||||
sync::{
|
||||
atomic::{AtomicBool, Ordering},
|
||||
Arc,
|
||||
},
|
||||
thread::{self, Builder, JoinHandle},
|
||||
time::Duration,
|
||||
};
|
||||
|
||||
pub type CompletedDataSetsReceiver = Receiver<Vec<CompletedDataSetInfo>>;
|
||||
pub type CompletedDataSetsSender = Sender<Vec<CompletedDataSetInfo>>;
|
||||
|
||||
pub struct CompletedDataSetsService {
|
||||
thread_hdl: JoinHandle<()>,
|
||||
}
|
||||
|
||||
impl CompletedDataSetsService {
|
||||
pub fn new(
|
||||
completed_sets_receiver: CompletedDataSetsReceiver,
|
||||
blockstore: Arc<Blockstore>,
|
||||
rpc_subscriptions: Arc<RpcSubscriptions>,
|
||||
exit: &Arc<AtomicBool>,
|
||||
) -> Self {
|
||||
let exit = exit.clone();
|
||||
let thread_hdl = Builder::new()
|
||||
.name("completed-data-set-service".to_string())
|
||||
.spawn(move || loop {
|
||||
if exit.load(Ordering::Relaxed) {
|
||||
break;
|
||||
}
|
||||
if let Err(RecvTimeoutError::Disconnected) = Self::recv_completed_data_sets(
|
||||
&completed_sets_receiver,
|
||||
&blockstore,
|
||||
&rpc_subscriptions,
|
||||
) {
|
||||
break;
|
||||
}
|
||||
})
|
||||
.unwrap();
|
||||
Self { thread_hdl }
|
||||
}
|
||||
|
||||
fn recv_completed_data_sets(
|
||||
completed_sets_receiver: &CompletedDataSetsReceiver,
|
||||
blockstore: &Blockstore,
|
||||
rpc_subscriptions: &RpcSubscriptions,
|
||||
) -> Result<(), RecvTimeoutError> {
|
||||
let completed_data_sets = completed_sets_receiver.recv_timeout(Duration::from_secs(1))?;
|
||||
for completed_set_info in std::iter::once(completed_data_sets)
|
||||
.chain(completed_sets_receiver.try_iter())
|
||||
.flatten()
|
||||
{
|
||||
let CompletedDataSetInfo {
|
||||
slot,
|
||||
start_index,
|
||||
end_index,
|
||||
} = completed_set_info;
|
||||
match blockstore.get_entries_in_data_block(slot, start_index, end_index, None) {
|
||||
Ok(entries) => {
|
||||
let transactions = entries
|
||||
.into_iter()
|
||||
.flat_map(|e| e.transactions.into_iter().map(|t| t.signatures[0]))
|
||||
.collect::<Vec<Signature>>();
|
||||
if !transactions.is_empty() {
|
||||
rpc_subscriptions.notify_signatures_received((slot, transactions));
|
||||
}
|
||||
}
|
||||
Err(e) => warn!("completed-data-set-service deserialize error: {:?}", e),
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn join(self) -> thread::Result<()> {
|
||||
self.thread_hdl.join()
|
||||
}
|
||||
}
|
@ -12,6 +12,7 @@ pub mod banking_stage;
|
||||
pub mod broadcast_stage;
|
||||
pub mod cluster_info_vote_listener;
|
||||
pub mod commitment_service;
|
||||
pub mod completed_data_sets_service;
|
||||
mod deprecated;
|
||||
pub mod shred_fetch_stage;
|
||||
#[macro_use]
|
||||
|
@ -20,6 +20,7 @@ pub enum Error {
|
||||
ReadyTimeoutError,
|
||||
RecvTimeoutError(std::sync::mpsc::RecvTimeoutError),
|
||||
CrossbeamSendError,
|
||||
TryCrossbeamSendError,
|
||||
TryRecvError(std::sync::mpsc::TryRecvError),
|
||||
Serialize(std::boxed::Box<bincode::ErrorKind>),
|
||||
TransactionError(transaction::TransactionError),
|
||||
@ -87,6 +88,11 @@ impl<T> std::convert::From<crossbeam_channel::SendError<T>> for Error {
|
||||
Error::CrossbeamSendError
|
||||
}
|
||||
}
|
||||
impl<T> std::convert::From<crossbeam_channel::TrySendError<T>> for Error {
|
||||
fn from(_e: crossbeam_channel::TrySendError<T>) -> Error {
|
||||
Error::TryCrossbeamSendError
|
||||
}
|
||||
}
|
||||
impl<T> std::convert::From<std::sync::mpsc::SendError<T>> for Error {
|
||||
fn from(_e: std::sync::mpsc::SendError<T>) -> Error {
|
||||
Error::SendError
|
||||
|
@ -5,6 +5,7 @@ use crate::{
|
||||
cluster_info_vote_listener::VerifiedVoteReceiver,
|
||||
cluster_slots::ClusterSlots,
|
||||
cluster_slots_service::ClusterSlotsService,
|
||||
completed_data_sets_service::CompletedDataSetsSender,
|
||||
contact_info::ContactInfo,
|
||||
repair_service::DuplicateSlotsResetSender,
|
||||
repair_service::RepairInfo,
|
||||
@ -419,6 +420,7 @@ impl RetransmitStage {
|
||||
duplicate_slots_reset_sender: DuplicateSlotsResetSender,
|
||||
verified_vote_receiver: VerifiedVoteReceiver,
|
||||
repair_validators: Option<HashSet<Pubkey>>,
|
||||
completed_data_sets_sender: CompletedDataSetsSender,
|
||||
) -> Self {
|
||||
let (retransmit_sender, retransmit_receiver) = channel();
|
||||
|
||||
@ -472,6 +474,7 @@ impl RetransmitStage {
|
||||
},
|
||||
cluster_slots,
|
||||
verified_vote_receiver,
|
||||
completed_data_sets_sender,
|
||||
);
|
||||
|
||||
let thread_hdls = t_retransmit;
|
||||
|
@ -6,14 +6,12 @@ use jsonrpc_derive::rpc;
|
||||
use jsonrpc_pubsub::{typed::Subscriber, Session, SubscriptionId};
|
||||
use solana_account_decoder::UiAccount;
|
||||
use solana_client::{
|
||||
rpc_config::{RpcAccountInfoConfig, RpcProgramAccountsConfig},
|
||||
rpc_config::{RpcAccountInfoConfig, RpcProgramAccountsConfig, RpcSignatureSubscribeConfig},
|
||||
rpc_response::{Response as RpcResponse, RpcKeyedAccount, RpcSignatureResult},
|
||||
};
|
||||
#[cfg(test)]
|
||||
use solana_runtime::bank_forks::BankForks;
|
||||
use solana_sdk::{
|
||||
clock::Slot, commitment_config::CommitmentConfig, pubkey::Pubkey, signature::Signature,
|
||||
};
|
||||
use solana_sdk::{clock::Slot, pubkey::Pubkey, signature::Signature};
|
||||
#[cfg(test)]
|
||||
use std::sync::RwLock;
|
||||
use std::{
|
||||
@ -89,7 +87,7 @@ pub trait RpcSolPubSub {
|
||||
meta: Self::Metadata,
|
||||
subscriber: Subscriber<RpcResponse<RpcSignatureResult>>,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
config: Option<RpcSignatureSubscribeConfig>,
|
||||
);
|
||||
|
||||
// Unsubscribe from signature notification subscription.
|
||||
@ -248,7 +246,7 @@ impl RpcSolPubSub for RpcSolPubSubImpl {
|
||||
_meta: Self::Metadata,
|
||||
subscriber: Subscriber<RpcResponse<RpcSignatureResult>>,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
signature_subscribe_config: Option<RpcSignatureSubscribeConfig>,
|
||||
) {
|
||||
info!("signature_subscribe");
|
||||
match param::<Signature>(&signature_str, "signature") {
|
||||
@ -259,8 +257,12 @@ impl RpcSolPubSub for RpcSolPubSubImpl {
|
||||
"signature_subscribe: signature={:?} id={:?}",
|
||||
signature, sub_id
|
||||
);
|
||||
self.subscriptions
|
||||
.add_signature_subscription(signature, commitment, sub_id, subscriber);
|
||||
self.subscriptions.add_signature_subscription(
|
||||
signature,
|
||||
signature_subscribe_config,
|
||||
sub_id,
|
||||
subscriber,
|
||||
);
|
||||
}
|
||||
Err(e) => subscriber.reject(e).unwrap(),
|
||||
}
|
||||
@ -359,6 +361,7 @@ mod tests {
|
||||
use jsonrpc_pubsub::{PubSubHandler, Session};
|
||||
use serial_test_derive::serial;
|
||||
use solana_account_decoder::{parse_account_data::parse_account_data, UiAccountEncoding};
|
||||
use solana_client::rpc_response::ProcessedSignatureResult;
|
||||
use solana_runtime::{
|
||||
bank::Bank,
|
||||
bank_forks::BankForks,
|
||||
@ -369,6 +372,7 @@ mod tests {
|
||||
},
|
||||
};
|
||||
use solana_sdk::{
|
||||
commitment_config::CommitmentConfig,
|
||||
hash::Hash,
|
||||
message::Message,
|
||||
pubkey::Pubkey,
|
||||
@ -442,7 +446,8 @@ mod tests {
|
||||
|
||||
// Test signature confirmation notification
|
||||
let (response, _) = robust_poll_or_panic(receiver);
|
||||
let expected_res = RpcSignatureResult { err: None };
|
||||
let expected_res =
|
||||
RpcSignatureResult::ProcessedSignatureResult(ProcessedSignatureResult { err: None });
|
||||
let expected = json!({
|
||||
"jsonrpc": "2.0",
|
||||
"method": "signatureNotification",
|
||||
@ -454,6 +459,38 @@ mod tests {
|
||||
"subscription": 0,
|
||||
}
|
||||
});
|
||||
|
||||
assert_eq!(serde_json::to_string(&expected).unwrap(), response);
|
||||
|
||||
// Test "received"
|
||||
let session = create_session();
|
||||
let (subscriber, _id_receiver, receiver) = Subscriber::new_test("signatureNotification");
|
||||
rpc.signature_subscribe(
|
||||
session,
|
||||
subscriber,
|
||||
tx.signatures[0].to_string(),
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: None,
|
||||
enable_received_notification: Some(true),
|
||||
}),
|
||||
);
|
||||
let received_slot = 1;
|
||||
rpc.subscriptions
|
||||
.notify_signatures_received((received_slot, vec![tx.signatures[0]]));
|
||||
// Test signature confirmation notification
|
||||
let (response, _) = robust_poll_or_panic(receiver);
|
||||
let expected_res = RpcSignatureResult::ReceivedSignature;
|
||||
let expected = json!({
|
||||
"jsonrpc": "2.0",
|
||||
"method": "signatureNotification",
|
||||
"params": {
|
||||
"result": {
|
||||
"context": { "slot": received_slot },
|
||||
"value": expected_res,
|
||||
},
|
||||
"subscription": 1,
|
||||
}
|
||||
});
|
||||
assert_eq!(serde_json::to_string(&expected).unwrap(), response);
|
||||
}
|
||||
|
||||
|
@ -10,9 +10,11 @@ use jsonrpc_pubsub::{
|
||||
use serde::Serialize;
|
||||
use solana_account_decoder::{parse_token::spl_token_id_v2_0, UiAccount, UiAccountEncoding};
|
||||
use solana_client::{
|
||||
rpc_config::{RpcAccountInfoConfig, RpcProgramAccountsConfig},
|
||||
rpc_config::{RpcAccountInfoConfig, RpcProgramAccountsConfig, RpcSignatureSubscribeConfig},
|
||||
rpc_filter::RpcFilterType,
|
||||
rpc_response::{Response, RpcKeyedAccount, RpcResponseContext, RpcSignatureResult},
|
||||
rpc_response::{
|
||||
ProcessedSignatureResult, Response, RpcKeyedAccount, RpcResponseContext, RpcSignatureResult,
|
||||
},
|
||||
};
|
||||
use solana_runtime::{
|
||||
bank::Bank,
|
||||
@ -67,6 +69,7 @@ enum NotificationEntry {
|
||||
Frozen(Slot),
|
||||
Bank(CommitmentSlots),
|
||||
Gossip(Slot),
|
||||
SignaturesReceived((Slot, Vec<Signature>)),
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for NotificationEntry {
|
||||
@ -79,6 +82,9 @@ impl std::fmt::Debug for NotificationEntry {
|
||||
NotificationEntry::Bank(commitment_slots) => {
|
||||
write!(f, "Bank({{slot: {:?}}})", commitment_slots.slot)
|
||||
}
|
||||
NotificationEntry::SignaturesReceived(slot_signatures) => {
|
||||
write!(f, "SignaturesReceived({:?})", slot_signatures)
|
||||
}
|
||||
NotificationEntry::Gossip(slot) => write!(f, "Gossip({:?})", slot),
|
||||
}
|
||||
}
|
||||
@ -108,7 +114,10 @@ type RpcProgramSubscriptions = RwLock<
|
||||
>,
|
||||
>;
|
||||
type RpcSignatureSubscriptions = RwLock<
|
||||
HashMap<Signature, HashMap<SubscriptionId, SubscriptionData<Response<RpcSignatureResult>, ()>>>,
|
||||
HashMap<
|
||||
Signature,
|
||||
HashMap<SubscriptionId, SubscriptionData<Response<RpcSignatureResult>, bool>>,
|
||||
>,
|
||||
>;
|
||||
type RpcSlotSubscriptions = RwLock<HashMap<SubscriptionId, Sink<SlotInfo>>>;
|
||||
type RpcVoteSubscriptions = RwLock<HashMap<SubscriptionId, Sink<RpcVote>>>;
|
||||
@ -134,13 +143,11 @@ fn add_subscription<K, S, T>(
|
||||
last_notified_slot: RwLock::new(last_notified_slot),
|
||||
config,
|
||||
};
|
||||
if let Some(current_hashmap) = subscriptions.get_mut(&hashmap_key) {
|
||||
current_hashmap.insert(sub_id, subscription_data);
|
||||
return;
|
||||
}
|
||||
let mut hashmap = HashMap::new();
|
||||
hashmap.insert(sub_id, subscription_data);
|
||||
subscriptions.insert(hashmap_key, hashmap);
|
||||
|
||||
subscriptions
|
||||
.entry(hashmap_key)
|
||||
.or_default()
|
||||
.insert(sub_id, subscription_data);
|
||||
}
|
||||
|
||||
fn remove_subscription<K, S, T>(
|
||||
@ -279,15 +286,15 @@ fn filter_signature_result(
|
||||
result: Option<transaction::Result<()>>,
|
||||
_signature: &Signature,
|
||||
last_notified_slot: Slot,
|
||||
_config: Option<()>,
|
||||
_config: Option<bool>,
|
||||
_bank: Option<Arc<Bank>>,
|
||||
) -> (Box<dyn Iterator<Item = RpcSignatureResult>>, Slot) {
|
||||
(
|
||||
Box::new(
|
||||
result
|
||||
.into_iter()
|
||||
.map(|result| RpcSignatureResult { err: result.err() }),
|
||||
),
|
||||
Box::new(result.into_iter().map(|result| {
|
||||
RpcSignatureResult::ProcessedSignatureResult(ProcessedSignatureResult {
|
||||
err: result.err(),
|
||||
})
|
||||
})),
|
||||
last_notified_slot,
|
||||
)
|
||||
}
|
||||
@ -629,13 +636,18 @@ impl RpcSubscriptions {
|
||||
pub fn add_signature_subscription(
|
||||
&self,
|
||||
signature: Signature,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
signature_subscribe_config: Option<RpcSignatureSubscribeConfig>,
|
||||
sub_id: SubscriptionId,
|
||||
subscriber: Subscriber<Response<RpcSignatureResult>>,
|
||||
) {
|
||||
let (commitment, enable_received_notification) = signature_subscribe_config
|
||||
.map(|config| (config.commitment, config.enable_received_notification))
|
||||
.unwrap_or((None, Some(false)));
|
||||
|
||||
let commitment_level = commitment
|
||||
.unwrap_or_else(CommitmentConfig::recent)
|
||||
.commitment;
|
||||
|
||||
let mut subscriptions = if commitment_level == CommitmentLevel::SingleGossip {
|
||||
self.subscriptions
|
||||
.gossip_signature_subscriptions
|
||||
@ -651,7 +663,7 @@ impl RpcSubscriptions {
|
||||
sub_id,
|
||||
subscriber,
|
||||
0, // last_notified_slot is not utilized for signature subscriptions
|
||||
None,
|
||||
enable_received_notification,
|
||||
);
|
||||
}
|
||||
|
||||
@ -696,6 +708,10 @@ impl RpcSubscriptions {
|
||||
self.enqueue_notification(NotificationEntry::Slot(SlotInfo { slot, parent, root }));
|
||||
}
|
||||
|
||||
pub fn notify_signatures_received(&self, slot_signatures: (Slot, Vec<Signature>)) {
|
||||
self.enqueue_notification(NotificationEntry::SignaturesReceived(slot_signatures));
|
||||
}
|
||||
|
||||
pub fn add_vote_subscription(&self, sub_id: SubscriptionId, subscriber: Subscriber<RpcVote>) {
|
||||
let sink = subscriber.assign_id(sub_id.clone()).unwrap();
|
||||
let mut subscriptions = self.subscriptions.vote_subscriptions.write().unwrap();
|
||||
@ -840,6 +856,13 @@ impl RpcSubscriptions {
|
||||
);
|
||||
}
|
||||
}
|
||||
NotificationEntry::SignaturesReceived(slot_signatures) => {
|
||||
RpcSubscriptions::process_signatures_received(
|
||||
&slot_signatures,
|
||||
&subscriptions.signature_subscriptions,
|
||||
¬ifier,
|
||||
)
|
||||
}
|
||||
},
|
||||
Err(RecvTimeoutError::Timeout) => {
|
||||
// not a problem - try reading again
|
||||
@ -939,6 +962,40 @@ impl RpcSubscriptions {
|
||||
}
|
||||
}
|
||||
|
||||
fn process_signatures_received(
|
||||
(received_slot, signatures): &(Slot, Vec<Signature>),
|
||||
signature_subscriptions: &Arc<RpcSignatureSubscriptions>,
|
||||
notifier: &RpcNotifier,
|
||||
) {
|
||||
for signature in signatures {
|
||||
if let Some(hashmap) = signature_subscriptions.read().unwrap().get(signature) {
|
||||
for (
|
||||
_,
|
||||
SubscriptionData {
|
||||
sink,
|
||||
config: is_received_notification_enabled,
|
||||
..
|
||||
},
|
||||
) in hashmap.iter()
|
||||
{
|
||||
if is_received_notification_enabled
|
||||
.expect("All signature subscriptions must have this config field set")
|
||||
{
|
||||
notifier.notify(
|
||||
Response {
|
||||
context: RpcResponseContext {
|
||||
slot: *received_slot,
|
||||
},
|
||||
value: RpcSignatureResult::ReceivedSignature,
|
||||
},
|
||||
&sink,
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn shutdown(&mut self) -> std::thread::Result<()> {
|
||||
if let Some(runtime) = self.notifier_runtime.take() {
|
||||
info!("RPC Notifier runtime - shutting down");
|
||||
@ -1245,31 +1302,55 @@ pub(crate) mod tests {
|
||||
Subscriber::new_test("signatureNotification");
|
||||
let (processed_sub, _id_receiver, processed_recv) =
|
||||
Subscriber::new_test("signatureNotification");
|
||||
let (processed_sub3, _id_receiver, processed_recv3) =
|
||||
Subscriber::new_test("signatureNotification");
|
||||
|
||||
subscriptions.add_signature_subscription(
|
||||
past_bank_tx.signatures[0],
|
||||
Some(CommitmentConfig::recent()),
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: Some(CommitmentConfig::recent()),
|
||||
enable_received_notification: Some(false),
|
||||
}),
|
||||
SubscriptionId::Number(1 as u64),
|
||||
past_bank_sub1,
|
||||
);
|
||||
subscriptions.add_signature_subscription(
|
||||
past_bank_tx.signatures[0],
|
||||
Some(CommitmentConfig::root()),
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: Some(CommitmentConfig::root()),
|
||||
enable_received_notification: Some(false),
|
||||
}),
|
||||
SubscriptionId::Number(2 as u64),
|
||||
past_bank_sub2,
|
||||
);
|
||||
subscriptions.add_signature_subscription(
|
||||
processed_tx.signatures[0],
|
||||
Some(CommitmentConfig::recent()),
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: Some(CommitmentConfig::recent()),
|
||||
enable_received_notification: Some(false),
|
||||
}),
|
||||
SubscriptionId::Number(3 as u64),
|
||||
processed_sub,
|
||||
);
|
||||
subscriptions.add_signature_subscription(
|
||||
unprocessed_tx.signatures[0],
|
||||
Some(CommitmentConfig::recent()),
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: Some(CommitmentConfig::recent()),
|
||||
enable_received_notification: Some(false),
|
||||
}),
|
||||
SubscriptionId::Number(4 as u64),
|
||||
Subscriber::new_test("signatureNotification").0,
|
||||
);
|
||||
// Add a subscription that gets `received` notifications
|
||||
subscriptions.add_signature_subscription(
|
||||
unprocessed_tx.signatures[0],
|
||||
Some(RpcSignatureSubscribeConfig {
|
||||
commitment: Some(CommitmentConfig::recent()),
|
||||
enable_received_notification: Some(true),
|
||||
}),
|
||||
SubscriptionId::Number(5 as u64),
|
||||
processed_sub3,
|
||||
);
|
||||
|
||||
{
|
||||
let sig_subs = subscriptions
|
||||
@ -1282,46 +1363,62 @@ pub(crate) mod tests {
|
||||
assert!(sig_subs.contains_key(&processed_tx.signatures[0]));
|
||||
}
|
||||
let mut commitment_slots = CommitmentSlots::default();
|
||||
commitment_slots.slot = 1;
|
||||
let received_slot = 1;
|
||||
commitment_slots.slot = received_slot;
|
||||
subscriptions
|
||||
.notify_signatures_received((received_slot, vec![unprocessed_tx.signatures[0]]));
|
||||
subscriptions.notify_subscribers(commitment_slots);
|
||||
let expected_res = RpcSignatureResult { err: None };
|
||||
|
||||
let expected_res =
|
||||
RpcSignatureResult::ProcessedSignatureResult(ProcessedSignatureResult { err: None });
|
||||
let received_expected_res = RpcSignatureResult::ReceivedSignature;
|
||||
struct Notification {
|
||||
slot: Slot,
|
||||
id: u64,
|
||||
}
|
||||
|
||||
let expected_notification = |exp: Notification| -> String {
|
||||
let json = json!({
|
||||
"jsonrpc": "2.0",
|
||||
"method": "signatureNotification",
|
||||
"params": {
|
||||
"result": {
|
||||
"context": { "slot": exp.slot },
|
||||
"value": &expected_res,
|
||||
},
|
||||
"subscription": exp.id,
|
||||
}
|
||||
});
|
||||
serde_json::to_string(&json).unwrap()
|
||||
};
|
||||
let expected_notification =
|
||||
|exp: Notification, expected_res: &RpcSignatureResult| -> String {
|
||||
let json = json!({
|
||||
"jsonrpc": "2.0",
|
||||
"method": "signatureNotification",
|
||||
"params": {
|
||||
"result": {
|
||||
"context": { "slot": exp.slot },
|
||||
"value": expected_res,
|
||||
},
|
||||
"subscription": exp.id,
|
||||
}
|
||||
});
|
||||
serde_json::to_string(&json).unwrap()
|
||||
};
|
||||
|
||||
// Expect to receive a notification from bank 1 because this subscription is
|
||||
// looking for 0 confirmations and so checks the current bank
|
||||
let expected = expected_notification(Notification { slot: 1, id: 1 });
|
||||
let expected = expected_notification(Notification { slot: 1, id: 1 }, &expected_res);
|
||||
let (response, _) = robust_poll_or_panic(past_bank_recv1);
|
||||
assert_eq!(expected, response);
|
||||
|
||||
// Expect to receive a notification from bank 0 because this subscription is
|
||||
// looking for 1 confirmation and so checks the past bank
|
||||
let expected = expected_notification(Notification { slot: 0, id: 2 });
|
||||
let expected = expected_notification(Notification { slot: 0, id: 2 }, &expected_res);
|
||||
let (response, _) = robust_poll_or_panic(past_bank_recv2);
|
||||
assert_eq!(expected, response);
|
||||
|
||||
let expected = expected_notification(Notification { slot: 1, id: 3 });
|
||||
let expected = expected_notification(Notification { slot: 1, id: 3 }, &expected_res);
|
||||
let (response, _) = robust_poll_or_panic(processed_recv);
|
||||
assert_eq!(expected, response);
|
||||
|
||||
// Expect a "received" notification
|
||||
let expected = expected_notification(
|
||||
Notification {
|
||||
slot: received_slot,
|
||||
id: 5,
|
||||
},
|
||||
&received_expected_res,
|
||||
);
|
||||
let (response, _) = robust_poll_or_panic(processed_recv3);
|
||||
assert_eq!(expected, response);
|
||||
|
||||
// Subscription should be automatically removed after notification
|
||||
let sig_subs = subscriptions
|
||||
.subscriptions
|
||||
@ -1334,7 +1431,7 @@ pub(crate) mod tests {
|
||||
// Unprocessed signature subscription should not be removed
|
||||
assert_eq!(
|
||||
sig_subs.get(&unprocessed_tx.signatures[0]).unwrap().len(),
|
||||
1
|
||||
2
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,7 @@ use crate::{
|
||||
cluster_info::ClusterInfo,
|
||||
cluster_info_vote_listener::{VerifiedVoteReceiver, VoteTracker},
|
||||
cluster_slots::ClusterSlots,
|
||||
completed_data_sets_service::CompletedDataSetsSender,
|
||||
ledger_cleanup_service::LedgerCleanupService,
|
||||
poh_recorder::PohRecorder,
|
||||
replay_stage::{ReplayStage, ReplayStageConfig},
|
||||
@ -100,6 +101,7 @@ impl Tvu {
|
||||
retransmit_slots_sender: RetransmitSlotsSender,
|
||||
verified_vote_receiver: VerifiedVoteReceiver,
|
||||
replay_vote_sender: ReplayVoteSender,
|
||||
completed_data_sets_sender: CompletedDataSetsSender,
|
||||
tvu_config: TvuConfig,
|
||||
) -> Self {
|
||||
let keypair: Arc<Keypair> = cluster_info.keypair.clone();
|
||||
@ -152,6 +154,7 @@ impl Tvu {
|
||||
duplicate_slots_reset_sender,
|
||||
verified_vote_receiver,
|
||||
tvu_config.repair_validators,
|
||||
completed_data_sets_sender,
|
||||
);
|
||||
|
||||
let (ledger_cleanup_slot_sender, ledger_cleanup_slot_receiver) = channel();
|
||||
@ -249,6 +252,7 @@ pub mod tests {
|
||||
};
|
||||
use serial_test_derive::serial;
|
||||
use solana_ledger::{
|
||||
blockstore::BlockstoreSignals,
|
||||
create_new_tmp_ledger,
|
||||
genesis_utils::{create_genesis_config, GenesisConfigInfo},
|
||||
};
|
||||
@ -275,9 +279,13 @@ pub mod tests {
|
||||
let cref1 = Arc::new(cluster_info1);
|
||||
|
||||
let (blockstore_path, _) = create_new_tmp_ledger!(&genesis_config);
|
||||
let (blockstore, l_receiver, completed_slots_receiver) =
|
||||
Blockstore::open_with_signal(&blockstore_path, None)
|
||||
.expect("Expected to successfully open ledger");
|
||||
let BlockstoreSignals {
|
||||
blockstore,
|
||||
ledger_signal_receiver,
|
||||
completed_slots_receiver,
|
||||
..
|
||||
} = Blockstore::open_with_signal(&blockstore_path, None)
|
||||
.expect("Expected to successfully open ledger");
|
||||
let blockstore = Arc::new(blockstore);
|
||||
let bank = bank_forks.working_bank();
|
||||
let (exit, poh_recorder, poh_service, _entry_receiver) =
|
||||
@ -288,6 +296,7 @@ pub mod tests {
|
||||
let (retransmit_slots_sender, _retransmit_slots_receiver) = unbounded();
|
||||
let (_verified_vote_sender, verified_vote_receiver) = unbounded();
|
||||
let (replay_vote_sender, _replay_vote_receiver) = unbounded();
|
||||
let (completed_data_sets_sender, _completed_data_sets_receiver) = unbounded();
|
||||
let bank_forks = Arc::new(RwLock::new(bank_forks));
|
||||
let tvu = Tvu::new(
|
||||
&vote_keypair.pubkey(),
|
||||
@ -303,7 +312,7 @@ pub mod tests {
|
||||
}
|
||||
},
|
||||
blockstore,
|
||||
l_receiver,
|
||||
ledger_signal_receiver,
|
||||
&Arc::new(RpcSubscriptions::new(
|
||||
&exit,
|
||||
bank_forks.clone(),
|
||||
@ -322,6 +331,7 @@ pub mod tests {
|
||||
retransmit_slots_sender,
|
||||
verified_vote_receiver,
|
||||
replay_vote_sender,
|
||||
completed_data_sets_sender,
|
||||
TvuConfig::default(),
|
||||
);
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
|
@ -4,6 +4,7 @@ use crate::{
|
||||
broadcast_stage::BroadcastStageType,
|
||||
cluster_info::{ClusterInfo, Node},
|
||||
cluster_info_vote_listener::VoteTracker,
|
||||
completed_data_sets_service::CompletedDataSetsService,
|
||||
contact_info::ContactInfo,
|
||||
gossip_service::{discover_cluster, GossipService},
|
||||
poh_recorder::{PohRecorder, GRACE_TICKS_FACTOR, MAX_GRACE_SLOTS},
|
||||
@ -21,12 +22,12 @@ use crate::{
|
||||
transaction_status_service::TransactionStatusService,
|
||||
tvu::{Sockets, Tvu, TvuConfig},
|
||||
};
|
||||
use crossbeam_channel::unbounded;
|
||||
use crossbeam_channel::{bounded, unbounded};
|
||||
use rand::{thread_rng, Rng};
|
||||
use solana_banks_server::rpc_banks_service::RpcBanksService;
|
||||
use solana_ledger::{
|
||||
bank_forks_utils,
|
||||
blockstore::{Blockstore, CompletedSlotsReceiver, PurgeType},
|
||||
blockstore::{Blockstore, BlockstoreSignals, CompletedSlotsReceiver, PurgeType},
|
||||
blockstore_db::BlockstoreRecoveryMode,
|
||||
blockstore_processor::{self, TransactionStatusSender},
|
||||
create_new_tmp_ledger,
|
||||
@ -64,6 +65,8 @@ use std::{
|
||||
time::Duration,
|
||||
};
|
||||
|
||||
pub const MAX_COMPLETED_DATA_SETS_IN_CHANNEL: usize = 100_000;
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct ValidatorConfig {
|
||||
pub dev_halt_at_slot: Option<Slot>,
|
||||
@ -156,6 +159,7 @@ pub struct Validator {
|
||||
rewards_recorder_service: Option<RewardsRecorderService>,
|
||||
gossip_service: GossipService,
|
||||
serve_repair_service: ServeRepairService,
|
||||
completed_data_sets_service: CompletedDataSetsService,
|
||||
snapshot_packager_service: Option<SnapshotPackagerService>,
|
||||
poh_recorder: Arc<Mutex<PohRecorder>>,
|
||||
poh_service: PohService,
|
||||
@ -284,6 +288,15 @@ impl Validator {
|
||||
block_commitment_cache.clone(),
|
||||
));
|
||||
|
||||
let (completed_data_sets_sender, completed_data_sets_receiver) =
|
||||
bounded(MAX_COMPLETED_DATA_SETS_IN_CHANNEL);
|
||||
let completed_data_sets_service = CompletedDataSetsService::new(
|
||||
completed_data_sets_receiver,
|
||||
blockstore.clone(),
|
||||
subscriptions.clone(),
|
||||
&exit,
|
||||
);
|
||||
|
||||
let rpc_override_health_check = Arc::new(AtomicBool::new(false));
|
||||
let rpc_service = config
|
||||
.rpc_ports
|
||||
@ -468,6 +481,7 @@ impl Validator {
|
||||
retransmit_slots_sender,
|
||||
verified_vote_receiver,
|
||||
replay_vote_sender.clone(),
|
||||
completed_data_sets_sender,
|
||||
TvuConfig {
|
||||
max_ledger_shreds: config.max_ledger_shreds,
|
||||
halt_on_trusted_validators_accounts_hash_mismatch: config
|
||||
@ -509,6 +523,7 @@ impl Validator {
|
||||
transaction_status_service,
|
||||
rewards_recorder_service,
|
||||
snapshot_packager_service,
|
||||
completed_data_sets_service,
|
||||
tpu,
|
||||
tvu,
|
||||
poh_service,
|
||||
@ -579,6 +594,7 @@ impl Validator {
|
||||
self.serve_repair_service.join()?;
|
||||
self.tpu.join()?;
|
||||
self.tvu.join()?;
|
||||
self.completed_data_sets_service.join()?;
|
||||
self.ip_echo_server.shutdown_now();
|
||||
|
||||
Ok(())
|
||||
@ -622,9 +638,13 @@ fn new_banks_from_ledger(
|
||||
}
|
||||
}
|
||||
|
||||
let (mut blockstore, ledger_signal_receiver, completed_slots_receiver) =
|
||||
Blockstore::open_with_signal(ledger_path, config.wal_recovery_mode.clone())
|
||||
.expect("Failed to open ledger database");
|
||||
let BlockstoreSignals {
|
||||
mut blockstore,
|
||||
ledger_signal_receiver,
|
||||
completed_slots_receiver,
|
||||
..
|
||||
} = Blockstore::open_with_signal(ledger_path, config.wal_recovery_mode.clone())
|
||||
.expect("Failed to open ledger database");
|
||||
blockstore.set_no_compaction(config.no_rocksdb_compaction);
|
||||
|
||||
let process_options = blockstore_processor::ProcessOptions {
|
||||
|
@ -5,6 +5,7 @@ use crate::{
|
||||
cluster_info::ClusterInfo,
|
||||
cluster_info_vote_listener::VerifiedVoteReceiver,
|
||||
cluster_slots::ClusterSlots,
|
||||
completed_data_sets_service::CompletedDataSetsSender,
|
||||
repair_response,
|
||||
repair_service::{RepairInfo, RepairService},
|
||||
result::{Error, Result},
|
||||
@ -123,6 +124,7 @@ fn run_insert<F>(
|
||||
leader_schedule_cache: &Arc<LeaderScheduleCache>,
|
||||
handle_duplicate: F,
|
||||
metrics: &mut BlockstoreInsertionMetrics,
|
||||
completed_data_sets_sender: &CompletedDataSetsSender,
|
||||
) -> Result<()>
|
||||
where
|
||||
F: Fn(Shred),
|
||||
@ -138,13 +140,13 @@ where
|
||||
let mut i = 0;
|
||||
shreds.retain(|shred| (verify_repair(&shred, &repair_infos[i]), i += 1).0);
|
||||
|
||||
blockstore.insert_shreds_handle_duplicate(
|
||||
completed_data_sets_sender.try_send(blockstore.insert_shreds_handle_duplicate(
|
||||
shreds,
|
||||
Some(leader_schedule_cache),
|
||||
false,
|
||||
&handle_duplicate,
|
||||
metrics,
|
||||
)?;
|
||||
)?)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@ -302,6 +304,7 @@ impl WindowService {
|
||||
shred_filter: F,
|
||||
cluster_slots: Arc<ClusterSlots>,
|
||||
verified_vote_receiver: VerifiedVoteReceiver,
|
||||
completed_data_sets_sender: CompletedDataSetsSender,
|
||||
) -> WindowService
|
||||
where
|
||||
F: 'static
|
||||
@ -333,6 +336,7 @@ impl WindowService {
|
||||
leader_schedule_cache,
|
||||
insert_receiver,
|
||||
duplicate_sender,
|
||||
completed_data_sets_sender,
|
||||
);
|
||||
|
||||
let t_window = Self::start_recv_window_thread(
|
||||
@ -387,6 +391,7 @@ impl WindowService {
|
||||
leader_schedule_cache: &Arc<LeaderScheduleCache>,
|
||||
insert_receiver: CrossbeamReceiver<(Vec<Shred>, Vec<Option<RepairMeta>>)>,
|
||||
duplicate_sender: CrossbeamSender<Shred>,
|
||||
completed_data_sets_sender: CompletedDataSetsSender,
|
||||
) -> JoinHandle<()> {
|
||||
let exit = exit.clone();
|
||||
let blockstore = blockstore.clone();
|
||||
@ -415,6 +420,7 @@ impl WindowService {
|
||||
&leader_schedule_cache,
|
||||
&handle_duplicate,
|
||||
&mut metrics,
|
||||
&completed_data_sets_sender,
|
||||
) {
|
||||
if Self::should_exit_on_error(e, &mut handle_timeout, &handle_error) {
|
||||
break;
|
||||
|
Reference in New Issue
Block a user