Skip to content

refactor(notification): allow multiple observer nodes on one worker #8210

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 5 commits into from
Feb 28, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
149 changes: 99 additions & 50 deletions src/meta/src/manager/notification.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,14 +42,18 @@ pub enum LocalNotification {
}

#[derive(Debug)]
enum Target {
SubscribeType(SubscribeType),
WorkerKey(WorkerKey),
struct Target {
subscribe_type: SubscribeType,
// `None` indicates sending to all subscribers of `subscribe_type`.
worker_key: Option<WorkerKey>,
}

impl From<SubscribeType> for Target {
fn from(value: SubscribeType) -> Self {
Self::SubscribeType(value)
Self {
subscribe_type: value,
worker_key: None,
}
}
}

Expand Down Expand Up @@ -90,15 +94,7 @@ where
version: task.version.unwrap_or_default(),
};

let mut guard = core.lock().await;
match task.target {
Target::SubscribeType(subscribe_type) => {
guard.notify(subscribe_type, response);
}
Target::WorkerKey(worker_key) => {
guard.notify_with_worker_key(worker_key, response);
}
}
core.lock().await.notify(task.target, response);
}
});

Expand Down Expand Up @@ -153,9 +149,17 @@ where
self.notify(target, operation, info, None);
}

pub fn notify_snapshot(&self, worker_key: WorkerKey, meta_snapshot: MetaSnapshot) {
pub fn notify_snapshot(
&self,
worker_key: WorkerKey,
subscribe_type: SubscribeType,
meta_snapshot: MetaSnapshot,
) {
self.notify_without_version(
Target::WorkerKey(worker_key),
Target {
subscribe_type,
worker_key: Some(worker_key),
},
Operation::Snapshot,
Info::Snapshot(meta_snapshot),
)
Expand Down Expand Up @@ -222,12 +226,7 @@ where
tracing::warn!("notification manager exiting.");
return;
}
let senders = match subscribe_type {
SubscribeType::Frontend => &mut core_guard.frontend_senders,
SubscribeType::Hummock => &mut core_guard.hummock_senders,
SubscribeType::Compactor => &mut core_guard.compactor_senders,
SubscribeType::Unspecified => unreachable!(),
};
let senders = core_guard.senders_of(subscribe_type);

senders.insert(worker_key, sender);
}
Expand All @@ -247,13 +246,15 @@ where
}
}

type SenderMap = HashMap<WorkerKey, UnboundedSender<Notification>>;

struct NotificationManagerCore {
/// The notification sender to frontends.
frontend_senders: HashMap<WorkerKey, UnboundedSender<Notification>>,
frontend_senders: SenderMap,
/// The notification sender to nodes that subscribes the hummock.
hummock_senders: HashMap<WorkerKey, UnboundedSender<Notification>>,
hummock_senders: SenderMap,
/// The notification sender to compactor nodes.
compactor_senders: HashMap<WorkerKey, UnboundedSender<Notification>>,
compactor_senders: SenderMap,
/// The notification sender to local subscribers.
local_senders: Vec<UnboundedSender<LocalNotification>>,
exiting: bool,
Expand All @@ -270,46 +271,94 @@ impl NotificationManagerCore {
}
}

fn notify_with_worker_key(&mut self, worker_key: WorkerKey, response: SubscribeResponse) {
for senders in [
&mut self.frontend_senders,
&mut self.hummock_senders,
&mut self.compactor_senders,
] {
fn notify(&mut self, target: Target, response: SubscribeResponse) {
macro_rules! warn_send_failure {
($subscribe_type:expr, $worker_key:expr, $err:expr) => {
tracing::warn!(
"Failed to notify {:?} {:?}: {}",
$subscribe_type,
$worker_key,
$err
);
};
}

let senders = self.senders_of(target.subscribe_type);

if let Some(worker_key) = target.worker_key {
match senders.entry(worker_key.clone()) {
Entry::Occupied(entry) => {
entry.get().send(Ok(response)).unwrap_or_else(|_| {
let _ = entry.get().send(Ok(response)).inspect_err(|err| {
warn_send_failure!(target.subscribe_type, &worker_key, err);
entry.remove_entry();
});
return;
}
Entry::Vacant(_) => continue,
Entry::Vacant(_) => {
tracing::warn!("Failed to find notification sender of {:?}", worker_key)
}
}
} else {
senders.retain(|worker_key, sender| {
sender
.send(Ok(response.clone()))
.inspect_err(|err| {
warn_send_failure!(target.subscribe_type, &worker_key, err);
})
.is_ok()
});
}

tracing::warn!("Failed to find notification sender of {:?}", worker_key);
}

fn notify(&mut self, subscribe_type: SubscribeType, response: SubscribeResponse) {
let senders = match subscribe_type {
fn senders_of(&mut self, subscribe_type: SubscribeType) -> &mut SenderMap {
match subscribe_type {
SubscribeType::Frontend => &mut self.frontend_senders,
SubscribeType::Hummock => &mut self.hummock_senders,
SubscribeType::Compactor => &mut self.compactor_senders,
SubscribeType::Unspecified => unreachable!(),
};
}
}
}

#[cfg(test)]
mod tests {
use risingwave_pb::common::HostAddress;

senders.retain(|worker_key, sender| {
sender
.send(Ok(response.clone()))
.inspect_err(|err| {
tracing::warn!(
"Failed to notify {:?} {:?}: {}",
subscribe_type,
worker_key,
err
)
})
.is_ok()
use super::*;
use crate::storage::MemStore;

#[tokio::test]
async fn test_multiple_subscribers_one_worker() {
let mgr = NotificationManager::new(MemStore::new().into()).await;
let worker_key1 = WorkerKey(HostAddress {
host: "a".to_string(),
port: 1,
});
let worker_key2 = WorkerKey(HostAddress {
host: "a".to_string(),
port: 2,
});
let (tx1, mut rx1) = mpsc::unbounded_channel();
let (tx2, mut rx2) = mpsc::unbounded_channel();
let (tx3, mut rx3) = mpsc::unbounded_channel();
mgr.insert_sender(SubscribeType::Hummock, worker_key1.clone(), tx1)
.await;
mgr.insert_sender(SubscribeType::Frontend, worker_key1.clone(), tx2)
.await;
mgr.insert_sender(SubscribeType::Frontend, worker_key2, tx3)
.await;
mgr.notify_snapshot(
worker_key1.clone(),
SubscribeType::Hummock,
MetaSnapshot::default(),
);
assert!(rx1.recv().await.is_some());
assert!(rx2.try_recv().is_err());
assert!(rx3.try_recv().is_err());

mgr.notify_frontend(Operation::Add, Info::Database(Default::default()))
.await;
assert!(rx1.try_recv().is_err());
assert!(rx2.recv().await.is_some());
assert!(rx3.recv().await.is_some());
}
}
2 changes: 1 addition & 1 deletion src/meta/src/rpc/service/notification_service.rs
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ where

self.env
.notification_manager()
.notify_snapshot(worker_key, meta_snapshot);
.notify_snapshot(worker_key, subscribe_type, meta_snapshot);

Ok(Response::new(UnboundedReceiverStream::new(rx)))
}
Expand Down
2 changes: 1 addition & 1 deletion src/storage/hummock_test/src/mock_notification_client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ impl<S: MetaStore> NotificationClient for MockNotificationClient<S> {
};

self.notification_manager
.notify_snapshot(worker_key, meta_snapshot);
.notify_snapshot(worker_key, subscribe_type, meta_snapshot);

Ok(TestChannel(rx))
}
Expand Down