Skip to content

Commit 9cf2d2b

Browse files
committed
[Bifrost] Seal task for replicated loglet
Initial implementation for the seal task for the replicated loglet. The seal task is not responsible for repairing the tail, just executes seal on an f-majority of nodes of the nodeset.
1 parent f208019 commit 9cf2d2b

File tree

6 files changed

+263
-5
lines changed

6 files changed

+263
-5
lines changed

crates/bifrost/src/providers/replicated_loglet/error.rs

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,13 +14,16 @@ use restate_core::ShutdownError;
1414
use restate_types::errors::MaybeRetryableError;
1515
use restate_types::logs::metadata::SegmentIndex;
1616
use restate_types::logs::LogId;
17+
use restate_types::replicated_loglet::ReplicatedLogletId;
1718

1819
use crate::loglet::OperationError;
1920

2021
#[derive(Debug, thiserror::Error)]
2122
pub(crate) enum ReplicatedLogletError {
2223
#[error("cannot parse loglet configuration for log_id={0} at segment_index={1}: {2}")]
2324
LogletParamsParsingError(LogId, SegmentIndex, serde_json::Error),
25+
#[error("could not seal loglet_id={0}, insufficient nodes available for seal")]
26+
SealFailed(ReplicatedLogletId),
2427
#[error(transparent)]
2528
Shutdown(#[from] ShutdownError),
2629
}
@@ -29,6 +32,7 @@ impl MaybeRetryableError for ReplicatedLogletError {
2932
fn retryable(&self) -> bool {
3033
match self {
3134
Self::LogletParamsParsingError(..) => false,
35+
Self::SealFailed(..) => true,
3236
Self::Shutdown(_) => false,
3337
}
3438
}

crates/bifrost/src/providers/replicated_loglet/loglet.rs

Lines changed: 57 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,10 +15,10 @@ use std::sync::Arc;
1515

1616
use async_trait::async_trait;
1717
use futures::stream::BoxStream;
18-
use tracing::debug;
18+
use tracing::{debug, info};
1919

2020
use restate_core::network::{Networking, TransportConnect};
21-
use restate_core::ShutdownError;
21+
use restate_core::{task_center, ShutdownError};
2222
use restate_types::logs::metadata::SegmentIndex;
2323
use restate_types::logs::{KeyFilter, LogId, LogletOffset, Record, SequenceNumber, TailState};
2424
use restate_types::replicated_loglet::ReplicatedLogletParams;
@@ -27,6 +27,7 @@ use crate::loglet::util::TailOffsetWatch;
2727
use crate::loglet::{Loglet, LogletCommit, OperationError, SendableLogletReadStream};
2828
use crate::providers::replicated_loglet::replication::spread_selector::SelectorStrategy;
2929
use crate::providers::replicated_loglet::sequencer::Sequencer;
30+
use crate::providers::replicated_loglet::tasks::SealTask;
3031

3132
use super::log_server_manager::RemoteLogServerManager;
3233
use super::record_cache::RecordCache;
@@ -168,7 +169,17 @@ impl<T: TransportConnect> Loglet for ReplicatedLoglet<T> {
168169
}
169170

170171
async fn seal(&self) -> Result<(), OperationError> {
171-
todo!()
172+
// todo(asoli): If we are the sequencer node, let the sequencer know.
173+
let _ = SealTask::new(
174+
task_center(),
175+
self.my_params.clone(),
176+
self.logservers_rpc.seal.clone(),
177+
self.known_global_tail.clone(),
178+
)
179+
.run(self.networking.clone())
180+
.await?;
181+
info!(loglet_id=%self.my_params.loglet_id, "Loglet has been sealed successfully");
182+
Ok(())
172183
}
173184
}
174185

@@ -190,7 +201,7 @@ mod tests {
190201
use restate_types::replicated_loglet::{NodeSet, ReplicatedLogletId, ReplicationProperty};
191202
use restate_types::{GenerationalNodeId, PlainNodeId};
192203

193-
use crate::loglet::Loglet;
204+
use crate::loglet::{AppendError, Loglet};
194205

195206
struct TestEnv {
196207
pub loglet: Arc<dyn Loglet>,
@@ -295,4 +306,46 @@ mod tests {
295306
})
296307
.await
297308
}
309+
310+
// ** Single-node replicated-loglet seal **
311+
#[test(tokio::test(start_paused = true))]
312+
async fn test_seal_local_sequencer_single_node() -> Result<()> {
313+
let loglet_id = ReplicatedLogletId::new(122);
314+
let params = ReplicatedLogletParams {
315+
loglet_id,
316+
sequencer: GenerationalNodeId::new(1, 1),
317+
replication: ReplicationProperty::new(NonZeroU8::new(1).unwrap()),
318+
nodeset: NodeSet::from_single(PlainNodeId::new(1)),
319+
write_set: None,
320+
};
321+
322+
run_in_test_env(params, |env| async move {
323+
let batch: Arc<[Record]> = vec![
324+
("record-1", Keys::Single(1)).into(),
325+
("record-2", Keys::Single(2)).into(),
326+
("record-3", Keys::Single(3)).into(),
327+
]
328+
.into();
329+
let offset = env.loglet.enqueue_batch(batch.clone()).await?.await?;
330+
assert_that!(offset, eq(LogletOffset::new(3)));
331+
let offset = env.loglet.enqueue_batch(batch.clone()).await?.await?;
332+
assert_that!(offset, eq(LogletOffset::new(6)));
333+
let tail = env.loglet.find_tail().await?;
334+
assert_that!(tail, eq(TailState::Open(LogletOffset::new(7))));
335+
336+
env.loglet.seal().await?;
337+
let batch: Arc<[Record]> = vec![
338+
("record-4", Keys::Single(4)).into(),
339+
("record-5", Keys::Single(5)).into(),
340+
]
341+
.into();
342+
let not_appended = env.loglet.enqueue_batch(batch).await?.await;
343+
assert_that!(not_appended, err(pat!(AppendError::Sealed)));
344+
let tail = env.loglet.find_tail().await?;
345+
assert_that!(tail, eq(TailState::Sealed(LogletOffset::new(7))));
346+
347+
Ok(())
348+
})
349+
.await
350+
}
298351
}

crates/bifrost/src/providers/replicated_loglet/mod.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ pub mod replication;
2020
mod rpc_routers;
2121
#[allow(dead_code)]
2222
pub mod sequencer;
23+
mod tasks;
2324
#[cfg(any(test, feature = "test-util"))]
2425
pub mod test_util;
2526

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,13 @@
1+
// Copyright (c) 2024 - Restate Software, Inc., Restate GmbH.
2+
// All rights reserved.
3+
//
4+
// Use of this software is governed by the Business Source License
5+
// included in the LICENSE file.
6+
//
7+
// As of the Change Date specified in that file, in accordance with
8+
// the Business Source License, use of this software will be governed
9+
// by the Apache License, Version 2.0.
10+
11+
mod seal;
12+
13+
pub use seal::*;
Lines changed: 176 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,176 @@
1+
// Copyright (c) 2024 - Restate Software, Inc., Restate GmbH.
2+
// All rights reserved.
3+
//
4+
// Use of this software is governed by the Business Source License
5+
// included in the LICENSE file.
6+
//
7+
// As of the Change Date specified in that file, in accordance with
8+
// the Business Source License, use of this software will be governed
9+
// by the Apache License, Version 2.0.
10+
11+
use tokio::sync::mpsc;
12+
use tracing::trace;
13+
14+
use restate_core::network::rpc_router::{RpcError, RpcRouter};
15+
use restate_core::network::{Incoming, Networking, TransportConnect};
16+
use restate_core::{TaskCenter, TaskKind};
17+
use restate_types::config::Configuration;
18+
use restate_types::logs::{LogletOffset, SequenceNumber};
19+
use restate_types::net::log_server::{Seal, Sealed, Status};
20+
use restate_types::replicated_loglet::{
21+
EffectiveNodeSet, ReplicatedLogletId, ReplicatedLogletParams,
22+
};
23+
use restate_types::retries::RetryPolicy;
24+
use restate_types::{GenerationalNodeId, PlainNodeId};
25+
26+
use crate::loglet::util::TailOffsetWatch;
27+
use crate::providers::replicated_loglet::error::ReplicatedLogletError;
28+
use crate::providers::replicated_loglet::replication::NodeSetChecker;
29+
30+
/// Sends a seal request to as many log-servers in the nodeset
31+
///
32+
/// We broadcast the seal to all nodes that we can, but only wait for f-majority
33+
/// responses before acknowleding the seal.
34+
///
35+
/// The seal operation is idempotent. It's safe to seal a loglet if it's already partially or fully
36+
/// sealed. Note that the seal task ignores the "seal" state in the input known_global_tail watch,
37+
/// but it will set it to `true` after the seal.
38+
pub struct SealTask {
39+
task_center: TaskCenter,
40+
my_params: ReplicatedLogletParams,
41+
seal_router: RpcRouter<Seal>,
42+
known_global_tail: TailOffsetWatch,
43+
}
44+
45+
impl SealTask {
46+
pub fn new(
47+
task_center: TaskCenter,
48+
my_params: ReplicatedLogletParams,
49+
seal_router: RpcRouter<Seal>,
50+
known_global_tail: TailOffsetWatch,
51+
) -> Self {
52+
Self {
53+
task_center,
54+
my_params,
55+
seal_router,
56+
known_global_tail,
57+
}
58+
}
59+
60+
pub async fn run<T: TransportConnect>(
61+
self,
62+
networking: Networking<T>,
63+
) -> Result<LogletOffset, ReplicatedLogletError> {
64+
// Use the entire nodeset except for StorageState::Disabled.
65+
let effective_nodeset = EffectiveNodeSet::new(
66+
&self.my_params.nodeset,
67+
&networking.metadata().nodes_config_ref(),
68+
);
69+
70+
let (tx, mut rx) = mpsc::unbounded_channel();
71+
72+
let mut nodeset_checker = NodeSetChecker::<'_, bool>::new(
73+
&effective_nodeset,
74+
&networking.metadata().nodes_config_ref(),
75+
&self.my_params.replication,
76+
);
77+
78+
let retry_policy = Configuration::pinned()
79+
.bifrost
80+
.replicated_loglet
81+
.log_server_retry_policy
82+
.clone();
83+
84+
for node in effective_nodeset.iter() {
85+
let task = SealSingleNode {
86+
node_id: *node,
87+
loglet_id: self.my_params.loglet_id,
88+
sequencer: self.my_params.sequencer,
89+
seal_router: self.seal_router.clone(),
90+
networking: networking.clone(),
91+
known_global_tail: self.known_global_tail.clone(),
92+
};
93+
self.task_center.spawn_child(
94+
TaskKind::Disposable,
95+
"send-seal-request",
96+
None,
97+
task.run(tx.clone(), retry_policy.clone()),
98+
)?;
99+
}
100+
drop(tx);
101+
102+
// Max observed local-tail from sealed nodes
103+
let mut max_tail = LogletOffset::INVALID;
104+
while let Some((node_id, local_tail)) = rx.recv().await {
105+
max_tail = std::cmp::max(max_tail, local_tail);
106+
nodeset_checker.set_attribute(node_id, true);
107+
108+
// Do we have f-majority responses?
109+
if nodeset_checker.check_fmajority(|sealed| *sealed).passed() {
110+
self.known_global_tail.notify_seal();
111+
// note that the rest of seal requests will continue in the background
112+
return Ok(max_tail);
113+
}
114+
}
115+
116+
// no more tasks left. We this means that we failed to seal
117+
Err(ReplicatedLogletError::SealFailed(self.my_params.loglet_id))
118+
}
119+
}
120+
121+
struct SealSingleNode<T> {
122+
node_id: PlainNodeId,
123+
loglet_id: ReplicatedLogletId,
124+
sequencer: GenerationalNodeId,
125+
seal_router: RpcRouter<Seal>,
126+
networking: Networking<T>,
127+
known_global_tail: TailOffsetWatch,
128+
}
129+
130+
impl<T: TransportConnect> SealSingleNode<T> {
131+
/// Returns local-tail. Note that this will _only_ return if seal was successful, otherwise,
132+
/// it'll continue to retry.
133+
pub async fn run(
134+
self,
135+
tx: mpsc::UnboundedSender<(PlainNodeId, LogletOffset)>,
136+
retry_policy: RetryPolicy,
137+
) -> anyhow::Result<()> {
138+
let mut retry_iter = retry_policy.into_iter();
139+
loop {
140+
match self.do_seal().await {
141+
Ok(res) if res.body().sealed || res.body().status == Status::Ok => {
142+
let _ = tx.send((self.node_id, res.body().local_tail));
143+
return Ok(());
144+
}
145+
// not sealed, or seal has failed
146+
Ok(res) => {
147+
// Sent, but sealing not successful
148+
trace!(loglet_id = %self.loglet_id, "Seal failed on node {} with status {:?}", self.node_id, res.body().status);
149+
}
150+
Err(_) => {
151+
trace!(loglet_id = %self.loglet_id, "Failed to send seal message to node {}", self.node_id);
152+
}
153+
}
154+
if let Some(pause) = retry_iter.next() {
155+
tokio::time::sleep(pause).await;
156+
} else {
157+
return Err(anyhow::anyhow!(format!(
158+
"Exhausted retries while attempting to seal the loglet {} on node {}",
159+
self.loglet_id, self.node_id
160+
)));
161+
}
162+
}
163+
}
164+
165+
async fn do_seal(&self) -> Result<Incoming<Sealed>, RpcError<Seal>> {
166+
let request = Seal {
167+
loglet_id: self.loglet_id,
168+
sequencer: self.sequencer.clone(),
169+
known_global_tail: self.known_global_tail.latest_offset(),
170+
};
171+
trace!(loglet_id = %self.loglet_id, "Sending seal message to node {}", self.node_id);
172+
self.seal_router
173+
.call(&self.networking, self.node_id, request)
174+
.await
175+
}
176+
}

crates/types/src/config/bifrost.rs

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -232,6 +232,11 @@ pub struct ReplicatedLogletOptions {
232232
///
233233
/// Timeout waiting on log server response
234234
pub log_server_timeout: Duration,
235+
236+
/// log_server RPC retry policy
237+
///
238+
/// Retry policy for log server RPCs
239+
pub log_server_retry_policy: RetryPolicy,
235240
}
236241

237242
impl Default for ReplicatedLogletOptions {
@@ -241,11 +246,17 @@ impl Default for ReplicatedLogletOptions {
241246

242247
sequencer_backoff_strategy: RetryPolicy::exponential(
243248
Duration::from_millis(100),
244-
0.1,
249+
2.0,
245250
None,
246251
Some(Duration::from_millis(2000)),
247252
),
248253
log_server_timeout: Duration::from_millis(500),
254+
log_server_retry_policy: RetryPolicy::exponential(
255+
Duration::from_millis(250),
256+
2.0,
257+
Some(10),
258+
Some(Duration::from_millis(2000)),
259+
),
249260
}
250261
}
251262
}

0 commit comments

Comments
 (0)