Line data Source code
1 : //! WAL receiver logic that ensures the pageserver gets connectected to safekeeper,
2 : //! that contains the latest WAL to stream and this connection does not go stale.
3 : //!
4 : //! To achieve that, a storage broker is used: safekepers propagate their timelines' state in it,
5 : //! the manager subscribes for changes and accumulates those to query the one with the biggest Lsn for connection.
6 : //! Current connection state is tracked too, to ensure it's not getting stale.
7 : //!
8 : //! After every connection or storage broker update fetched, the state gets updated correspondingly and rechecked for the new conneciton leader,
9 : //! then a (re)connection happens, if necessary.
10 : //! Only WAL streaming task expects to be finished, other loops (storage broker, connection management) never exit unless cancelled explicitly via the dedicated channel.
11 :
12 : use std::{collections::HashMap, num::NonZeroU64, ops::ControlFlow, sync::Arc, time::Duration};
13 :
14 : use super::{TaskStateUpdate, WalReceiverConf};
15 : use crate::context::{DownloadBehavior, RequestContext};
16 : use crate::metrics::{
17 : WALRECEIVER_ACTIVE_MANAGERS, WALRECEIVER_BROKER_UPDATES, WALRECEIVER_CANDIDATES_ADDED,
18 : WALRECEIVER_CANDIDATES_REMOVED, WALRECEIVER_SWITCHES,
19 : };
20 : use crate::task_mgr::TaskKind;
21 : use crate::tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline};
22 : use anyhow::Context;
23 : use chrono::{NaiveDateTime, Utc};
24 : use pageserver_api::models::TimelineState;
25 :
26 : use storage_broker::proto::TenantTimelineId as ProtoTenantTimelineId;
27 : use storage_broker::proto::{
28 : FilterTenantTimelineId, MessageType, SafekeeperDiscoveryRequest, SafekeeperDiscoveryResponse,
29 : SubscribeByFilterRequest, TypeSubscription, TypedMessage,
30 : };
31 : use storage_broker::{BrokerClientChannel, Code, Streaming};
32 : use tokio_util::sync::CancellationToken;
33 : use tracing::*;
34 :
35 : use postgres_connection::PgConnectionConfig;
36 : use utils::backoff::{
37 : exponential_backoff, DEFAULT_BASE_BACKOFF_SECONDS, DEFAULT_MAX_BACKOFF_SECONDS,
38 : };
39 : use utils::postgres_client::{
40 : wal_stream_connection_config, ConnectionConfigArgs, PostgresClientProtocol,
41 : };
42 : use utils::{
43 : id::{NodeId, TenantTimelineId},
44 : lsn::Lsn,
45 : };
46 :
47 : use super::{
48 : walreceiver_connection::WalConnectionStatus, walreceiver_connection::WalReceiverError,
49 : TaskEvent, TaskHandle,
50 : };
51 :
52 : pub(crate) struct Cancelled;
53 :
54 : /// Attempts to subscribe for timeline updates, pushed by safekeepers into the broker.
55 : /// Based on the updates, desides whether to start, keep or stop a WAL receiver task.
56 : /// If storage broker subscription is cancelled, exits.
57 : ///
58 : /// # Cancel-Safety
59 : ///
60 : /// Not cancellation-safe. Use `cancel` token to request cancellation.
61 0 : pub(super) async fn connection_manager_loop_step(
62 0 : broker_client: &mut BrokerClientChannel,
63 0 : connection_manager_state: &mut ConnectionManagerState,
64 0 : ctx: &RequestContext,
65 0 : cancel: &CancellationToken,
66 0 : manager_status: &std::sync::RwLock<Option<ConnectionManagerStatus>>,
67 0 : ) -> Result<(), Cancelled> {
68 0 : match tokio::select! {
69 0 : _ = cancel.cancelled() => { return Err(Cancelled); },
70 0 : st = connection_manager_state.timeline.wait_to_become_active(ctx) => { st }
71 : } {
72 0 : Ok(()) => {}
73 0 : Err(new_state) => {
74 0 : debug!(
75 : ?new_state,
76 0 : "state changed, stopping wal connection manager loop"
77 : );
78 0 : return Err(Cancelled);
79 : }
80 : }
81 :
82 0 : WALRECEIVER_ACTIVE_MANAGERS.inc();
83 0 : scopeguard::defer! {
84 0 : WALRECEIVER_ACTIVE_MANAGERS.dec();
85 0 : }
86 0 :
87 0 : let id = TenantTimelineId {
88 0 : tenant_id: connection_manager_state.timeline.tenant_shard_id.tenant_id,
89 0 : timeline_id: connection_manager_state.timeline.timeline_id,
90 0 : };
91 0 :
92 0 : let mut timeline_state_updates = connection_manager_state
93 0 : .timeline
94 0 : .subscribe_for_state_updates();
95 0 :
96 0 : let mut wait_lsn_status = connection_manager_state
97 0 : .timeline
98 0 : .subscribe_for_wait_lsn_updates();
99 0 :
100 0 : // TODO: create a separate config option for discovery request interval
101 0 : let discovery_request_interval = connection_manager_state.conf.lagging_wal_timeout;
102 0 : let mut last_discovery_ts: Option<std::time::Instant> = None;
103 :
104 : // Subscribe to the broker updates. Stream shares underlying TCP connection
105 : // with other streams on this client (other connection managers). When
106 : // object goes out of scope, stream finishes in drop() automatically.
107 0 : let mut broker_subscription = subscribe_for_timeline_updates(broker_client, id, cancel).await?;
108 0 : debug!("Subscribed for broker timeline updates");
109 :
110 : loop {
111 0 : let time_until_next_retry = connection_manager_state.time_until_next_retry();
112 0 : let any_activity = connection_manager_state.wal_connection.is_some()
113 0 : || !connection_manager_state.wal_stream_candidates.is_empty();
114 :
115 : // These things are happening concurrently:
116 : //
117 : // - cancellation request
118 : // - keep receiving WAL on the current connection
119 : // - if the shared state says we need to change connection, disconnect and return
120 : // - this runs in a separate task and we receive updates via a watch channel
121 : // - change connection if the rules decide so, or if the current connection dies
122 : // - receive updates from broker
123 : // - this might change the current desired connection
124 : // - timeline state changes to something that does not allow walreceiver to run concurrently
125 : // - if there's no connection and no candidates, try to send a discovery request
126 :
127 : // NB: make sure each of the select expressions are cancellation-safe
128 : // (no need for arms to be cancellation-safe).
129 0 : tokio::select! {
130 0 : _ = cancel.cancelled() => { return Err(Cancelled); }
131 0 : Some(wal_connection_update) = async {
132 0 : match connection_manager_state.wal_connection.as_mut() {
133 0 : Some(wal_connection) => Some(wal_connection.connection_task.next_task_event().await),
134 0 : None => None,
135 : }
136 0 : } => {
137 0 : let wal_connection = connection_manager_state.wal_connection.as_mut()
138 0 : .expect("Should have a connection, as checked by the corresponding select! guard");
139 0 : match wal_connection_update {
140 0 : TaskEvent::Update(TaskStateUpdate::Started) => {},
141 0 : TaskEvent::Update(TaskStateUpdate::Progress(new_status)) => {
142 0 : if new_status.has_processed_wal {
143 0 : // We have advanced last_record_lsn by processing the WAL received
144 0 : // from this safekeeper. This is good enough to clean unsuccessful
145 0 : // retries history and allow reconnecting to this safekeeper without
146 0 : // sleeping for a long time.
147 0 : connection_manager_state.wal_connection_retries.remove(&wal_connection.sk_id);
148 0 : }
149 0 : wal_connection.status = new_status;
150 : }
151 0 : TaskEvent::End(walreceiver_task_result) => {
152 0 : match walreceiver_task_result {
153 0 : Ok(()) => debug!("WAL receiving task finished"),
154 0 : Err(e) => error!("wal receiver task finished with an error: {e:?}"),
155 : }
156 0 : connection_manager_state.drop_old_connection(false).await;
157 : },
158 : }
159 : },
160 :
161 : // Got a new update from the broker
162 0 : broker_update = broker_subscription.message() /* TODO: review cancellation-safety */ => {
163 0 : match broker_update {
164 0 : Ok(Some(broker_update)) => connection_manager_state.register_timeline_update(broker_update),
165 0 : Err(status) => {
166 0 : match status.code() {
167 0 : Code::Unknown if status.message().contains("stream closed because of a broken pipe") || status.message().contains("connection reset") => {
168 0 : // tonic's error handling doesn't provide a clear code for disconnections: we get
169 0 : // "h2 protocol error: error reading a body from connection: stream closed because of a broken pipe"
170 0 : info!("broker disconnected: {status}");
171 : },
172 : _ => {
173 0 : warn!("broker subscription failed: {status}");
174 : }
175 : }
176 0 : return Ok(());
177 : }
178 : Ok(None) => {
179 0 : error!("broker subscription stream ended"); // can't happen
180 0 : return Ok(());
181 : }
182 : }
183 : },
184 :
185 0 : new_event = async {
186 : // Reminder: this match arm needs to be cancellation-safe.
187 : loop {
188 0 : if connection_manager_state.timeline.current_state() == TimelineState::Loading {
189 0 : warn!("wal connection manager should only be launched after timeline has become active");
190 0 : }
191 0 : match timeline_state_updates.changed().await {
192 : Ok(()) => {
193 0 : let new_state = connection_manager_state.timeline.current_state();
194 0 : match new_state {
195 : // we're already active as walreceiver, no need to reactivate
196 0 : TimelineState::Active => continue,
197 : TimelineState::Broken { .. } | TimelineState::Stopping => {
198 0 : debug!("timeline entered terminal state {new_state:?}, stopping wal connection manager loop");
199 0 : return ControlFlow::Break(());
200 : }
201 : TimelineState::Loading => {
202 0 : warn!("timeline transitioned back to Loading state, that should not happen");
203 0 : return ControlFlow::Continue(());
204 : }
205 : }
206 : }
207 0 : Err(_sender_dropped_error) => return ControlFlow::Break(()),
208 : }
209 : }
210 0 : } => match new_event {
211 : ControlFlow::Continue(()) => {
212 0 : return Ok(());
213 : }
214 : ControlFlow::Break(()) => {
215 0 : debug!("Timeline is no longer active, stopping wal connection manager loop");
216 0 : return Err(Cancelled);
217 : }
218 : },
219 :
220 0 : Some(()) = async {
221 0 : match time_until_next_retry {
222 0 : Some(sleep_time) => {
223 0 : tokio::time::sleep(sleep_time).await;
224 0 : Some(())
225 : },
226 : None => {
227 0 : debug!("No candidates to retry, waiting indefinitely for the broker events");
228 0 : None
229 : }
230 : }
231 0 : } => debug!("Waking up for the next retry after waiting for {time_until_next_retry:?}"),
232 :
233 0 : Some(()) = async {
234 0 : // Reminder: this match arm needs to be cancellation-safe.
235 0 : // Calculating time needed to wait until sending the next discovery request.
236 0 : // Current implementation is conservative and sends discovery requests only when there are no candidates.
237 0 :
238 0 : if any_activity {
239 : // No need to send discovery requests if there is an active connection or candidates.
240 0 : return None;
241 0 : }
242 :
243 : // Waiting for an active wait_lsn request.
244 0 : while wait_lsn_status.borrow().is_none() {
245 0 : if wait_lsn_status.changed().await.is_err() {
246 : // wait_lsn_status channel was closed, exiting
247 0 : warn!("wait_lsn_status channel was closed in connection_manager_loop_step");
248 0 : return None;
249 0 : }
250 : }
251 :
252 : // All preconditions met, preparing to send a discovery request.
253 0 : let now = std::time::Instant::now();
254 0 : let next_discovery_ts = last_discovery_ts
255 0 : .map(|ts| ts + discovery_request_interval)
256 0 : .unwrap_or_else(|| now);
257 0 :
258 0 : if next_discovery_ts > now {
259 : // Prevent sending discovery requests too frequently.
260 0 : tokio::time::sleep(next_discovery_ts - now).await;
261 0 : }
262 :
263 0 : let tenant_timeline_id = Some(ProtoTenantTimelineId {
264 0 : tenant_id: id.tenant_id.as_ref().to_owned(),
265 0 : timeline_id: id.timeline_id.as_ref().to_owned(),
266 0 : });
267 0 : let request = SafekeeperDiscoveryRequest { tenant_timeline_id };
268 0 : let msg = TypedMessage {
269 0 : r#type: MessageType::SafekeeperDiscoveryRequest as i32,
270 0 : safekeeper_timeline_info: None,
271 0 : safekeeper_discovery_request: Some(request),
272 0 : safekeeper_discovery_response: None,
273 0 : };
274 0 :
275 0 : last_discovery_ts = Some(std::time::Instant::now());
276 0 : debug!("No active connection and no candidates, sending discovery request to the broker");
277 :
278 : // Cancellation safety: we want to send a message to the broker, but publish_one()
279 : // function can get cancelled by the other select! arm. This is absolutely fine, because
280 : // we just want to receive broker updates and discovery is not important if we already
281 : // receive updates.
282 : //
283 : // It is possible that `last_discovery_ts` will be updated, but the message will not be sent.
284 : // This is totally fine because of the reason above.
285 :
286 : // This is a fire-and-forget request, we don't care about the response
287 0 : let _ = broker_client.publish_one(msg).await;
288 0 : debug!("Discovery request sent to the broker");
289 0 : None
290 0 : } => {}
291 : }
292 :
293 0 : if let Some(new_candidate) = connection_manager_state.next_connection_candidate() {
294 0 : info!("Switching to new connection candidate: {new_candidate:?}");
295 0 : connection_manager_state
296 0 : .change_connection(new_candidate, ctx)
297 0 : .await
298 0 : }
299 0 : *manager_status.write().unwrap() = Some(connection_manager_state.manager_status());
300 : }
301 0 : }
302 :
303 : /// Endlessly try to subscribe for broker updates for a given timeline.
304 0 : async fn subscribe_for_timeline_updates(
305 0 : broker_client: &mut BrokerClientChannel,
306 0 : id: TenantTimelineId,
307 0 : cancel: &CancellationToken,
308 0 : ) -> Result<Streaming<TypedMessage>, Cancelled> {
309 0 : let mut attempt = 0;
310 : loop {
311 0 : exponential_backoff(
312 0 : attempt,
313 0 : DEFAULT_BASE_BACKOFF_SECONDS,
314 0 : DEFAULT_MAX_BACKOFF_SECONDS,
315 0 : cancel,
316 0 : )
317 0 : .await;
318 0 : attempt += 1;
319 0 :
320 0 : // subscribe to the specific timeline
321 0 : let request = SubscribeByFilterRequest {
322 0 : types: vec![
323 0 : TypeSubscription {
324 0 : r#type: MessageType::SafekeeperTimelineInfo as i32,
325 0 : },
326 0 : TypeSubscription {
327 0 : r#type: MessageType::SafekeeperDiscoveryResponse as i32,
328 0 : },
329 0 : ],
330 0 : tenant_timeline_id: Some(FilterTenantTimelineId {
331 0 : enabled: true,
332 0 : tenant_timeline_id: Some(ProtoTenantTimelineId {
333 0 : tenant_id: id.tenant_id.as_ref().to_owned(),
334 0 : timeline_id: id.timeline_id.as_ref().to_owned(),
335 0 : }),
336 0 : }),
337 0 : };
338 0 :
339 0 : match {
340 0 : tokio::select! {
341 0 : r = broker_client.subscribe_by_filter(request) => { r }
342 0 : _ = cancel.cancelled() => { return Err(Cancelled); }
343 : }
344 : } {
345 0 : Ok(resp) => {
346 0 : return Ok(resp.into_inner());
347 : }
348 0 : Err(e) => {
349 0 : // Safekeeper nodes can stop pushing timeline updates to the broker, when no new writes happen and
350 0 : // entire WAL is streamed. Keep this noticeable with logging, but do not warn/error.
351 0 : info!("Attempt #{attempt}, failed to subscribe for timeline {id} updates in broker: {e:#}");
352 0 : continue;
353 : }
354 : }
355 : }
356 0 : }
357 :
358 : const WALCONNECTION_RETRY_MIN_BACKOFF_SECONDS: f64 = 0.1;
359 : const WALCONNECTION_RETRY_MAX_BACKOFF_SECONDS: f64 = 15.0;
360 : const WALCONNECTION_RETRY_BACKOFF_MULTIPLIER: f64 = 1.5;
361 :
362 : /// All data that's needed to run endless broker loop and keep the WAL streaming connection alive, if possible.
363 : pub(super) struct ConnectionManagerState {
364 : id: TenantTimelineId,
365 : /// Use pageserver data about the timeline to filter out some of the safekeepers.
366 : timeline: Arc<Timeline>,
367 : /// Child token of [`super::WalReceiver::cancel`], inherited to all tasks we spawn.
368 : cancel: CancellationToken,
369 : conf: WalReceiverConf,
370 : /// Current connection to safekeeper for WAL streaming.
371 : wal_connection: Option<WalConnection>,
372 : /// Info about retries and unsuccessful attempts to connect to safekeepers.
373 : wal_connection_retries: HashMap<NodeId, RetryInfo>,
374 : /// Data about all timelines, available for connection, fetched from storage broker, grouped by their corresponding safekeeper node id.
375 : wal_stream_candidates: HashMap<NodeId, BrokerSkTimeline>,
376 : }
377 :
378 : /// An information about connection manager's current connection and connection candidates.
379 : #[derive(Debug, Clone)]
380 : pub struct ConnectionManagerStatus {
381 : existing_connection: Option<WalConnectionStatus>,
382 : wal_stream_candidates: HashMap<NodeId, BrokerSkTimeline>,
383 : }
384 :
385 : impl ConnectionManagerStatus {
386 : /// Generates a string, describing current connection status in a form, suitable for logging.
387 0 : pub fn to_human_readable_string(&self) -> String {
388 0 : let mut resulting_string = String::new();
389 0 : match &self.existing_connection {
390 0 : Some(connection) => {
391 0 : if connection.has_processed_wal {
392 0 : resulting_string.push_str(&format!(
393 0 : " (update {}): streaming WAL from node {}, ",
394 0 : connection.latest_wal_update.format("%Y-%m-%d %H:%M:%S"),
395 0 : connection.node,
396 0 : ));
397 0 :
398 0 : match (connection.streaming_lsn, connection.commit_lsn) {
399 0 : (None, None) => resulting_string.push_str("no streaming data"),
400 0 : (None, Some(commit_lsn)) => {
401 0 : resulting_string.push_str(&format!("commit Lsn: {commit_lsn}"))
402 : }
403 0 : (Some(streaming_lsn), None) => {
404 0 : resulting_string.push_str(&format!("streaming Lsn: {streaming_lsn}"))
405 : }
406 0 : (Some(streaming_lsn), Some(commit_lsn)) => resulting_string.push_str(
407 0 : &format!("commit|streaming Lsn: {commit_lsn}|{streaming_lsn}"),
408 0 : ),
409 : }
410 0 : } else if connection.is_connected {
411 0 : resulting_string.push_str(&format!(
412 0 : " (update {}): connecting to node {}",
413 0 : connection
414 0 : .latest_connection_update
415 0 : .format("%Y-%m-%d %H:%M:%S"),
416 0 : connection.node,
417 0 : ));
418 0 : } else {
419 0 : resulting_string.push_str(&format!(
420 0 : " (update {}): initializing node {} connection",
421 0 : connection
422 0 : .latest_connection_update
423 0 : .format("%Y-%m-%d %H:%M:%S"),
424 0 : connection.node,
425 0 : ));
426 0 : }
427 : }
428 0 : None => resulting_string.push_str(": disconnected"),
429 : }
430 :
431 0 : resulting_string.push_str(", safekeeper candidates (id|update_time|commit_lsn): [");
432 0 : let mut candidates = self.wal_stream_candidates.iter().peekable();
433 0 : while let Some((node_id, candidate_info)) = candidates.next() {
434 0 : resulting_string.push_str(&format!(
435 0 : "({}|{}|{})",
436 0 : node_id,
437 0 : candidate_info.latest_update.format("%H:%M:%S"),
438 0 : Lsn(candidate_info.timeline.commit_lsn)
439 0 : ));
440 0 : if candidates.peek().is_some() {
441 0 : resulting_string.push_str(", ");
442 0 : }
443 : }
444 0 : resulting_string.push(']');
445 0 :
446 0 : resulting_string
447 0 : }
448 : }
449 :
450 : /// Current connection data.
451 : #[derive(Debug)]
452 : struct WalConnection {
453 : /// Time when the connection was initiated.
454 : started_at: NaiveDateTime,
455 : /// Current safekeeper pageserver is connected to for WAL streaming.
456 : sk_id: NodeId,
457 : /// Availability zone of the safekeeper.
458 : availability_zone: Option<String>,
459 : /// Status of the connection.
460 : status: WalConnectionStatus,
461 : /// WAL streaming task handle.
462 : connection_task: TaskHandle<WalConnectionStatus>,
463 : /// Have we discovered that other safekeeper has more recent WAL than we do?
464 : discovered_new_wal: Option<NewCommittedWAL>,
465 : }
466 :
467 : /// Notion of a new committed WAL, which exists on other safekeeper.
468 : #[derive(Debug, Clone, Copy)]
469 : struct NewCommittedWAL {
470 : /// LSN of the new committed WAL.
471 : lsn: Lsn,
472 : /// When we discovered that the new committed WAL exists on other safekeeper.
473 : discovered_at: NaiveDateTime,
474 : }
475 :
476 : #[derive(Debug, Clone, Copy)]
477 : struct RetryInfo {
478 : next_retry_at: Option<NaiveDateTime>,
479 : retry_duration_seconds: f64,
480 : }
481 :
482 : /// Data about the timeline to connect to, received from the broker.
483 : #[derive(Debug, Clone)]
484 : struct BrokerSkTimeline {
485 : timeline: SafekeeperDiscoveryResponse,
486 : /// Time at which the data was fetched from the broker last time, to track the stale data.
487 : latest_update: NaiveDateTime,
488 : }
489 :
490 : impl ConnectionManagerState {
491 0 : pub(super) fn new(
492 0 : timeline: Arc<Timeline>,
493 0 : conf: WalReceiverConf,
494 0 : cancel: CancellationToken,
495 0 : ) -> Self {
496 0 : let id = TenantTimelineId {
497 0 : tenant_id: timeline.tenant_shard_id.tenant_id,
498 0 : timeline_id: timeline.timeline_id,
499 0 : };
500 0 : Self {
501 0 : id,
502 0 : timeline,
503 0 : cancel,
504 0 : conf,
505 0 : wal_connection: None,
506 0 : wal_stream_candidates: HashMap::new(),
507 0 : wal_connection_retries: HashMap::new(),
508 0 : }
509 0 : }
510 :
511 10 : fn spawn<Fut>(
512 10 : &self,
513 10 : task: impl FnOnce(
514 10 : tokio::sync::watch::Sender<TaskStateUpdate<WalConnectionStatus>>,
515 10 : CancellationToken,
516 10 : ) -> Fut
517 10 : + Send
518 10 : + 'static,
519 10 : ) -> TaskHandle<WalConnectionStatus>
520 10 : where
521 10 : Fut: std::future::Future<Output = anyhow::Result<()>> + Send,
522 10 : {
523 10 : // TODO: get rid of TaskHandle
524 10 : super::TaskHandle::spawn(&self.cancel, task)
525 10 : }
526 :
527 : /// Shuts down the current connection (if any) and immediately starts another one with the given connection string.
528 0 : async fn change_connection(&mut self, new_sk: NewWalConnectionCandidate, ctx: &RequestContext) {
529 0 : WALRECEIVER_SWITCHES
530 0 : .with_label_values(&[new_sk.reason.name()])
531 0 : .inc();
532 0 :
533 0 : self.drop_old_connection(true).await;
534 :
535 0 : let node_id = new_sk.safekeeper_id;
536 0 : let connect_timeout = self.conf.wal_connect_timeout;
537 0 : let ingest_batch_size = self.conf.ingest_batch_size;
538 0 : let timeline = Arc::clone(&self.timeline);
539 0 : let ctx = ctx.detached_child(
540 0 : TaskKind::WalReceiverConnectionHandler,
541 0 : DownloadBehavior::Download,
542 0 : );
543 :
544 0 : let span = info_span!("connection", %node_id);
545 0 : let connection_handle = self.spawn(move |events_sender, cancellation| {
546 0 : async move {
547 0 : debug_assert_current_span_has_tenant_and_timeline_id();
548 :
549 0 : let res = super::walreceiver_connection::handle_walreceiver_connection(
550 0 : timeline,
551 0 : new_sk.wal_source_connconf,
552 0 : events_sender,
553 0 : cancellation.clone(),
554 0 : connect_timeout,
555 0 : ctx,
556 0 : node_id,
557 0 : ingest_batch_size,
558 0 : )
559 0 : .await;
560 :
561 0 : match res {
562 0 : Ok(()) => Ok(()),
563 0 : Err(e) => {
564 0 : match e {
565 0 : WalReceiverError::SuccessfulCompletion(msg) => {
566 0 : info!("walreceiver connection handling ended with success: {msg}");
567 0 : Ok(())
568 : }
569 0 : WalReceiverError::ExpectedSafekeeperError(e) => {
570 0 : info!("walreceiver connection handling ended: {e}");
571 0 : Ok(())
572 : }
573 : WalReceiverError::ClosedGate => {
574 0 : info!(
575 0 : "walreceiver connection handling ended because of closed gate"
576 : );
577 0 : Ok(())
578 : }
579 0 : WalReceiverError::Other(e) => {
580 0 : // give out an error to have task_mgr give it a really verbose logging
581 0 : if cancellation.is_cancelled() {
582 : // Ideally we would learn about this via some path other than Other, but
583 : // that requires refactoring all the intermediate layers of ingest code
584 : // that only emit anyhow::Error
585 0 : Ok(())
586 : } else {
587 0 : Err(e).context("walreceiver connection handling failure")
588 : }
589 : }
590 : }
591 : }
592 : }
593 0 : }
594 0 : .instrument(span)
595 0 : });
596 0 :
597 0 : let now = Utc::now().naive_utc();
598 0 : self.wal_connection = Some(WalConnection {
599 0 : started_at: now,
600 0 : sk_id: new_sk.safekeeper_id,
601 0 : availability_zone: new_sk.availability_zone,
602 0 : status: WalConnectionStatus {
603 0 : is_connected: false,
604 0 : has_processed_wal: false,
605 0 : latest_connection_update: now,
606 0 : latest_wal_update: now,
607 0 : streaming_lsn: None,
608 0 : commit_lsn: None,
609 0 : node: node_id,
610 0 : },
611 0 : connection_task: connection_handle,
612 0 : discovered_new_wal: None,
613 0 : });
614 0 : }
615 :
616 : /// Drops the current connection (if any) and updates retry timeout for the next
617 : /// connection attempt to the same safekeeper.
618 : ///
619 : /// # Cancel-Safety
620 : ///
621 : /// Not cancellation-safe.
622 0 : async fn drop_old_connection(&mut self, needs_shutdown: bool) {
623 0 : let wal_connection = match self.wal_connection.take() {
624 0 : Some(wal_connection) => wal_connection,
625 0 : None => return,
626 : };
627 :
628 0 : if needs_shutdown {
629 0 : wal_connection
630 0 : .connection_task
631 0 : .shutdown()
632 : // This here is why this function isn't cancellation-safe.
633 : // If we got cancelled here, then self.wal_connection is already None and we lose track of the task.
634 : // Even if our caller diligently calls Self::shutdown(), it will find a self.wal_connection=None
635 : // and thus be ineffective.
636 0 : .await;
637 0 : }
638 :
639 0 : let retry = self
640 0 : .wal_connection_retries
641 0 : .entry(wal_connection.sk_id)
642 0 : .or_insert(RetryInfo {
643 0 : next_retry_at: None,
644 0 : retry_duration_seconds: WALCONNECTION_RETRY_MIN_BACKOFF_SECONDS,
645 0 : });
646 0 :
647 0 : let now = Utc::now().naive_utc();
648 0 :
649 0 : // Schedule the next retry attempt. We want to have exponential backoff for connection attempts,
650 0 : // and we add backoff to the time when we started the connection attempt. If the connection
651 0 : // was active for a long time, then next_retry_at will be in the past.
652 0 : retry.next_retry_at =
653 0 : wal_connection
654 0 : .started_at
655 0 : .checked_add_signed(chrono::Duration::milliseconds(
656 0 : (retry.retry_duration_seconds * 1000.0) as i64,
657 0 : ));
658 :
659 0 : if let Some(next) = &retry.next_retry_at {
660 0 : if next > &now {
661 0 : info!(
662 0 : "Next connection retry to {:?} is at {}",
663 : wal_connection.sk_id, next
664 : );
665 0 : }
666 0 : }
667 :
668 0 : let next_retry_duration =
669 0 : retry.retry_duration_seconds * WALCONNECTION_RETRY_BACKOFF_MULTIPLIER;
670 0 : // Clamp the next retry duration to the maximum allowed.
671 0 : let next_retry_duration = next_retry_duration.min(WALCONNECTION_RETRY_MAX_BACKOFF_SECONDS);
672 0 : // Clamp the next retry duration to the minimum allowed.
673 0 : let next_retry_duration = next_retry_duration.max(WALCONNECTION_RETRY_MIN_BACKOFF_SECONDS);
674 0 :
675 0 : retry.retry_duration_seconds = next_retry_duration;
676 0 : }
677 :
678 : /// Returns time needed to wait to have a new candidate for WAL streaming.
679 0 : fn time_until_next_retry(&self) -> Option<Duration> {
680 0 : let now = Utc::now().naive_utc();
681 :
682 0 : let next_retry_at = self
683 0 : .wal_connection_retries
684 0 : .values()
685 0 : .filter_map(|retry| retry.next_retry_at)
686 0 : .filter(|next_retry_at| next_retry_at > &now)
687 0 : .min()?;
688 :
689 0 : (next_retry_at - now).to_std().ok()
690 0 : }
691 :
692 : /// Adds another broker timeline into the state, if its more recent than the one already added there for the same key.
693 0 : fn register_timeline_update(&mut self, typed_msg: TypedMessage) {
694 0 : let mut is_discovery = false;
695 0 : let timeline_update = match typed_msg.r#type() {
696 : MessageType::SafekeeperTimelineInfo => {
697 0 : let info = match typed_msg.safekeeper_timeline_info {
698 0 : Some(info) => info,
699 : None => {
700 0 : warn!("bad proto message from broker: no safekeeper_timeline_info");
701 0 : return;
702 : }
703 : };
704 0 : SafekeeperDiscoveryResponse {
705 0 : safekeeper_id: info.safekeeper_id,
706 0 : tenant_timeline_id: info.tenant_timeline_id,
707 0 : commit_lsn: info.commit_lsn,
708 0 : safekeeper_connstr: info.safekeeper_connstr,
709 0 : availability_zone: info.availability_zone,
710 0 : standby_horizon: info.standby_horizon,
711 0 : }
712 : }
713 : MessageType::SafekeeperDiscoveryResponse => {
714 0 : is_discovery = true;
715 0 : match typed_msg.safekeeper_discovery_response {
716 0 : Some(response) => response,
717 : None => {
718 0 : warn!("bad proto message from broker: no safekeeper_discovery_response");
719 0 : return;
720 : }
721 : }
722 : }
723 : _ => {
724 : // unexpected message
725 0 : return;
726 : }
727 : };
728 :
729 0 : WALRECEIVER_BROKER_UPDATES.inc();
730 0 :
731 0 : trace!(
732 0 : "safekeeper info update: standby_horizon(cutoff)={}",
733 : timeline_update.standby_horizon
734 : );
735 0 : if timeline_update.standby_horizon != 0 {
736 0 : // ignore reports from safekeepers not connected to replicas
737 0 : self.timeline
738 0 : .standby_horizon
739 0 : .store(Lsn(timeline_update.standby_horizon));
740 0 : self.timeline
741 0 : .metrics
742 0 : .standby_horizon_gauge
743 0 : .set(timeline_update.standby_horizon as i64);
744 0 : }
745 :
746 0 : let new_safekeeper_id = NodeId(timeline_update.safekeeper_id);
747 0 : let old_entry = self.wal_stream_candidates.insert(
748 0 : new_safekeeper_id,
749 0 : BrokerSkTimeline {
750 0 : timeline: timeline_update,
751 0 : latest_update: Utc::now().naive_utc(),
752 0 : },
753 0 : );
754 0 :
755 0 : if old_entry.is_none() {
756 0 : info!(
757 : ?is_discovery,
758 : %new_safekeeper_id,
759 0 : "New SK node was added",
760 : );
761 0 : WALRECEIVER_CANDIDATES_ADDED.inc();
762 0 : }
763 0 : }
764 :
765 : /// Cleans up stale broker records and checks the rest for the new connection candidate.
766 : /// Returns a new candidate, if the current state is absent or somewhat lagging, `None` otherwise.
767 : /// The current rules for approving new candidates:
768 : /// * pick a candidate different from the connected safekeeper with biggest `commit_lsn` and lowest failed connection attemps
769 : /// * if there's no such entry, no new candidate found, abort
770 : /// * otherwise check if the candidate is much better than the current one
771 : ///
772 : /// To understand exact rules for determining if the candidate is better than the current one, refer to this function's implementation.
773 : /// General rules are following:
774 : /// * if connected safekeeper is not present, pick the candidate
775 : /// * if we haven't received any updates for some time, pick the candidate
776 : /// * if the candidate commit_lsn is much higher than the current one, pick the candidate
777 : /// * if the candidate commit_lsn is same, but candidate is located in the same AZ as the pageserver, pick the candidate
778 : /// * if connected safekeeper stopped sending us new WAL which is available on other safekeeper, pick the candidate
779 : ///
780 : /// This way we ensure to keep up with the most up-to-date safekeeper and don't try to jump from one safekeeper to another too frequently.
781 : /// Both thresholds are configured per tenant.
782 18 : fn next_connection_candidate(&mut self) -> Option<NewWalConnectionCandidate> {
783 18 : self.cleanup_old_candidates();
784 18 :
785 18 : match &self.wal_connection {
786 10 : Some(existing_wal_connection) => {
787 10 : let connected_sk_node = existing_wal_connection.sk_id;
788 :
789 10 : let (new_sk_id, new_safekeeper_broker_data, new_wal_source_connconf) =
790 10 : self.select_connection_candidate(Some(connected_sk_node))?;
791 10 : let new_availability_zone = new_safekeeper_broker_data.availability_zone.clone();
792 10 :
793 10 : let now = Utc::now().naive_utc();
794 10 : if let Ok(latest_interaciton) =
795 10 : (now - existing_wal_connection.status.latest_connection_update).to_std()
796 : {
797 : // Drop connection if we haven't received keepalive message for a while.
798 10 : if latest_interaciton > self.conf.wal_connect_timeout {
799 2 : return Some(NewWalConnectionCandidate {
800 2 : safekeeper_id: new_sk_id,
801 2 : wal_source_connconf: new_wal_source_connconf,
802 2 : availability_zone: new_availability_zone,
803 2 : reason: ReconnectReason::NoKeepAlives {
804 2 : last_keep_alive: Some(
805 2 : existing_wal_connection.status.latest_connection_update,
806 2 : ),
807 2 : check_time: now,
808 2 : threshold: self.conf.wal_connect_timeout,
809 2 : },
810 2 : });
811 8 : }
812 0 : }
813 :
814 8 : if !existing_wal_connection.status.is_connected {
815 : // We haven't connected yet and we shouldn't switch until connection timeout (condition above).
816 0 : return None;
817 8 : }
818 :
819 8 : if let Some(current_commit_lsn) = existing_wal_connection.status.commit_lsn {
820 8 : let new_commit_lsn = Lsn(new_safekeeper_broker_data.commit_lsn);
821 8 : // Check if the new candidate has much more WAL than the current one.
822 8 : match new_commit_lsn.0.checked_sub(current_commit_lsn.0) {
823 8 : Some(new_sk_lsn_advantage) => {
824 8 : if new_sk_lsn_advantage >= self.conf.max_lsn_wal_lag.get() {
825 2 : return Some(NewWalConnectionCandidate {
826 2 : safekeeper_id: new_sk_id,
827 2 : wal_source_connconf: new_wal_source_connconf,
828 2 : availability_zone: new_availability_zone,
829 2 : reason: ReconnectReason::LaggingWal {
830 2 : current_commit_lsn,
831 2 : new_commit_lsn,
832 2 : threshold: self.conf.max_lsn_wal_lag,
833 2 : },
834 2 : });
835 6 : }
836 6 : // If we have a candidate with the same commit_lsn as the current one, which is in the same AZ as pageserver,
837 6 : // and the current one is not, switch to the new one.
838 6 : if self.conf.availability_zone.is_some()
839 2 : && existing_wal_connection.availability_zone
840 2 : != self.conf.availability_zone
841 2 : && self.conf.availability_zone == new_availability_zone
842 : {
843 2 : return Some(NewWalConnectionCandidate {
844 2 : safekeeper_id: new_sk_id,
845 2 : availability_zone: new_availability_zone,
846 2 : wal_source_connconf: new_wal_source_connconf,
847 2 : reason: ReconnectReason::SwitchAvailabilityZone,
848 2 : });
849 4 : }
850 : }
851 0 : None => debug!(
852 0 : "Best SK candidate has its commit_lsn behind connected SK's commit_lsn"
853 : ),
854 : }
855 0 : }
856 :
857 4 : let current_lsn = match existing_wal_connection.status.streaming_lsn {
858 4 : Some(lsn) => lsn,
859 0 : None => self.timeline.get_last_record_lsn(),
860 : };
861 4 : let current_commit_lsn = existing_wal_connection
862 4 : .status
863 4 : .commit_lsn
864 4 : .unwrap_or(current_lsn);
865 4 : let candidate_commit_lsn = Lsn(new_safekeeper_broker_data.commit_lsn);
866 4 :
867 4 : // Keep discovered_new_wal only if connected safekeeper has not caught up yet.
868 4 : let mut discovered_new_wal = existing_wal_connection
869 4 : .discovered_new_wal
870 4 : .filter(|new_wal| new_wal.lsn > current_commit_lsn);
871 4 :
872 4 : if discovered_new_wal.is_none() {
873 : // Check if the new candidate has more WAL than the current one.
874 : // If the new candidate has more WAL than the current one, we consider switching to the new candidate.
875 2 : discovered_new_wal = if candidate_commit_lsn > current_commit_lsn {
876 2 : trace!(
877 0 : "New candidate has commit_lsn {}, higher than current_commit_lsn {}",
878 : candidate_commit_lsn,
879 : current_commit_lsn
880 : );
881 2 : Some(NewCommittedWAL {
882 2 : lsn: candidate_commit_lsn,
883 2 : discovered_at: Utc::now().naive_utc(),
884 2 : })
885 : } else {
886 0 : None
887 : };
888 2 : }
889 :
890 4 : let waiting_for_new_lsn_since = if current_lsn < current_commit_lsn {
891 : // Connected safekeeper has more WAL, but we haven't received updates for some time.
892 0 : trace!(
893 0 : "Connected safekeeper has more WAL, but we haven't received updates for {:?}. current_lsn: {}, current_commit_lsn: {}",
894 0 : (now - existing_wal_connection.status.latest_wal_update).to_std(),
895 : current_lsn,
896 : current_commit_lsn
897 : );
898 0 : Some(existing_wal_connection.status.latest_wal_update)
899 : } else {
900 4 : discovered_new_wal.as_ref().map(|new_wal| {
901 4 : // We know that new WAL is available on other safekeeper, but connected safekeeper don't have it.
902 4 : new_wal
903 4 : .discovered_at
904 4 : .max(existing_wal_connection.status.latest_wal_update)
905 4 : })
906 : };
907 :
908 : // If we haven't received any WAL updates for a while and candidate has more WAL, switch to it.
909 4 : if let Some(waiting_for_new_lsn_since) = waiting_for_new_lsn_since {
910 4 : if let Ok(waiting_for_new_wal) = (now - waiting_for_new_lsn_since).to_std() {
911 2 : if candidate_commit_lsn > current_commit_lsn
912 2 : && waiting_for_new_wal > self.conf.lagging_wal_timeout
913 : {
914 2 : return Some(NewWalConnectionCandidate {
915 2 : safekeeper_id: new_sk_id,
916 2 : wal_source_connconf: new_wal_source_connconf,
917 2 : availability_zone: new_availability_zone,
918 2 : reason: ReconnectReason::NoWalTimeout {
919 2 : current_lsn,
920 2 : current_commit_lsn,
921 2 : candidate_commit_lsn,
922 2 : last_wal_interaction: Some(
923 2 : existing_wal_connection.status.latest_wal_update,
924 2 : ),
925 2 : check_time: now,
926 2 : threshold: self.conf.lagging_wal_timeout,
927 2 : },
928 2 : });
929 0 : }
930 2 : }
931 0 : }
932 :
933 2 : self.wal_connection.as_mut().unwrap().discovered_new_wal = discovered_new_wal;
934 : }
935 : None => {
936 6 : let (new_sk_id, new_safekeeper_broker_data, new_wal_source_connconf) =
937 8 : self.select_connection_candidate(None)?;
938 6 : return Some(NewWalConnectionCandidate {
939 6 : safekeeper_id: new_sk_id,
940 6 : availability_zone: new_safekeeper_broker_data.availability_zone.clone(),
941 6 : wal_source_connconf: new_wal_source_connconf,
942 6 : reason: ReconnectReason::NoExistingConnection,
943 6 : });
944 : }
945 : }
946 :
947 2 : None
948 18 : }
949 :
950 : /// Selects the best possible candidate, based on the data collected from the broker updates about the safekeepers.
951 : /// Optionally, omits the given node, to support gracefully switching from a healthy safekeeper to another.
952 : ///
953 : /// The candidate that is chosen:
954 : /// * has no pending retry cooldown
955 : /// * has greatest commit_lsn among the ones that are left
956 18 : fn select_connection_candidate(
957 18 : &self,
958 18 : node_to_omit: Option<NodeId>,
959 18 : ) -> Option<(NodeId, &SafekeeperDiscoveryResponse, PgConnectionConfig)> {
960 18 : self.applicable_connection_candidates()
961 26 : .filter(|&(sk_id, _, _)| Some(sk_id) != node_to_omit)
962 20 : .max_by_key(|(_, info, _)| info.commit_lsn)
963 18 : }
964 :
965 : /// Returns a list of safekeepers that have valid info and ready for connection.
966 : /// Some safekeepers are filtered by the retry cooldown.
967 18 : fn applicable_connection_candidates(
968 18 : &self,
969 18 : ) -> impl Iterator<Item = (NodeId, &SafekeeperDiscoveryResponse, PgConnectionConfig)> {
970 18 : let now = Utc::now().naive_utc();
971 18 :
972 18 : self.wal_stream_candidates
973 18 : .iter()
974 36 : .filter(|(_, info)| Lsn(info.timeline.commit_lsn) != Lsn::INVALID)
975 32 : .filter(move |(sk_id, _)| {
976 32 : let next_retry_at = self
977 32 : .wal_connection_retries
978 32 : .get(sk_id)
979 32 : .and_then(|retry_info| {
980 2 : retry_info.next_retry_at
981 32 : });
982 32 :
983 32 : next_retry_at.is_none() || next_retry_at.unwrap() <= now
984 32 : }).filter_map(|(sk_id, broker_info)| {
985 30 : let info = &broker_info.timeline;
986 30 : if info.safekeeper_connstr.is_empty() {
987 4 : return None; // no connection string, ignore sk
988 26 : }
989 :
990 26 : let (shard_number, shard_count, shard_stripe_size) = match self.conf.protocol {
991 : PostgresClientProtocol::Vanilla => {
992 26 : (None, None, None)
993 : },
994 : PostgresClientProtocol::Interpreted => {
995 0 : let shard_identity = self.timeline.get_shard_identity();
996 0 : (
997 0 : Some(shard_identity.number.0),
998 0 : Some(shard_identity.count.0),
999 0 : Some(shard_identity.stripe_size.0),
1000 0 : )
1001 : }
1002 : };
1003 :
1004 26 : let connection_conf_args = ConnectionConfigArgs {
1005 26 : protocol: self.conf.protocol,
1006 26 : ttid: self.id,
1007 26 : shard_number,
1008 26 : shard_count,
1009 26 : shard_stripe_size,
1010 26 : listen_pg_addr_str: info.safekeeper_connstr.as_ref(),
1011 26 : auth_token: self.conf.auth_token.as_ref().map(|t| t.as_str()),
1012 26 : availability_zone: self.conf.availability_zone.as_deref()
1013 26 : };
1014 26 :
1015 26 : match wal_stream_connection_config(connection_conf_args) {
1016 26 : Ok(connstr) => Some((*sk_id, info, connstr)),
1017 0 : Err(e) => {
1018 0 : error!("Failed to create wal receiver connection string from broker data of safekeeper node {}: {e:#}", sk_id);
1019 0 : None
1020 : }
1021 : }
1022 30 : })
1023 18 : }
1024 :
1025 : /// Remove candidates which haven't sent broker updates for a while.
1026 18 : fn cleanup_old_candidates(&mut self) {
1027 18 : let mut node_ids_to_remove = Vec::with_capacity(self.wal_stream_candidates.len());
1028 18 : let lagging_wal_timeout = self.conf.lagging_wal_timeout;
1029 18 :
1030 38 : self.wal_stream_candidates.retain(|node_id, broker_info| {
1031 38 : if let Ok(time_since_latest_broker_update) =
1032 38 : (Utc::now().naive_utc() - broker_info.latest_update).to_std()
1033 : {
1034 38 : let should_retain = time_since_latest_broker_update < lagging_wal_timeout;
1035 38 : if !should_retain {
1036 2 : node_ids_to_remove.push(*node_id);
1037 36 : }
1038 38 : should_retain
1039 : } else {
1040 0 : true
1041 : }
1042 38 : });
1043 18 :
1044 18 : if !node_ids_to_remove.is_empty() {
1045 4 : for node_id in node_ids_to_remove {
1046 2 : info!("Safekeeper node {node_id} did not send events for over {lagging_wal_timeout:?}, not retrying the connections");
1047 2 : self.wal_connection_retries.remove(&node_id);
1048 2 : WALRECEIVER_CANDIDATES_REMOVED.inc();
1049 : }
1050 16 : }
1051 18 : }
1052 :
1053 : /// # Cancel-Safety
1054 : ///
1055 : /// Not cancellation-safe.
1056 0 : pub(super) async fn shutdown(mut self) {
1057 0 : if let Some(wal_connection) = self.wal_connection.take() {
1058 0 : wal_connection.connection_task.shutdown().await;
1059 0 : }
1060 0 : }
1061 :
1062 0 : fn manager_status(&self) -> ConnectionManagerStatus {
1063 0 : ConnectionManagerStatus {
1064 0 : existing_connection: self.wal_connection.as_ref().map(|conn| conn.status),
1065 0 : wal_stream_candidates: self.wal_stream_candidates.clone(),
1066 0 : }
1067 0 : }
1068 : }
1069 :
1070 : #[derive(Debug)]
1071 : struct NewWalConnectionCandidate {
1072 : safekeeper_id: NodeId,
1073 : wal_source_connconf: PgConnectionConfig,
1074 : availability_zone: Option<String>,
1075 : reason: ReconnectReason,
1076 : }
1077 :
1078 : /// Stores the reason why WAL connection was switched, for furter debugging purposes.
1079 : #[derive(Debug, PartialEq, Eq)]
1080 : enum ReconnectReason {
1081 : NoExistingConnection,
1082 : LaggingWal {
1083 : current_commit_lsn: Lsn,
1084 : new_commit_lsn: Lsn,
1085 : threshold: NonZeroU64,
1086 : },
1087 : SwitchAvailabilityZone,
1088 : NoWalTimeout {
1089 : current_lsn: Lsn,
1090 : current_commit_lsn: Lsn,
1091 : candidate_commit_lsn: Lsn,
1092 : last_wal_interaction: Option<NaiveDateTime>,
1093 : check_time: NaiveDateTime,
1094 : threshold: Duration,
1095 : },
1096 : NoKeepAlives {
1097 : last_keep_alive: Option<NaiveDateTime>,
1098 : check_time: NaiveDateTime,
1099 : threshold: Duration,
1100 : },
1101 : }
1102 :
1103 : impl ReconnectReason {
1104 0 : fn name(&self) -> &str {
1105 0 : match self {
1106 0 : ReconnectReason::NoExistingConnection => "NoExistingConnection",
1107 0 : ReconnectReason::LaggingWal { .. } => "LaggingWal",
1108 0 : ReconnectReason::SwitchAvailabilityZone => "SwitchAvailabilityZone",
1109 0 : ReconnectReason::NoWalTimeout { .. } => "NoWalTimeout",
1110 0 : ReconnectReason::NoKeepAlives { .. } => "NoKeepAlives",
1111 : }
1112 0 : }
1113 : }
1114 :
1115 : #[cfg(test)]
1116 : mod tests {
1117 : use super::*;
1118 : use crate::tenant::harness::{TenantHarness, TIMELINE_ID};
1119 : use pageserver_api::config::defaults::DEFAULT_WAL_RECEIVER_PROTOCOL;
1120 : use url::Host;
1121 :
1122 38 : fn dummy_broker_sk_timeline(
1123 38 : commit_lsn: u64,
1124 38 : safekeeper_connstr: &str,
1125 38 : latest_update: NaiveDateTime,
1126 38 : ) -> BrokerSkTimeline {
1127 38 : BrokerSkTimeline {
1128 38 : timeline: SafekeeperDiscoveryResponse {
1129 38 : safekeeper_id: 0,
1130 38 : tenant_timeline_id: None,
1131 38 : commit_lsn,
1132 38 : safekeeper_connstr: safekeeper_connstr.to_owned(),
1133 38 : availability_zone: None,
1134 38 : standby_horizon: 0,
1135 38 : },
1136 38 : latest_update,
1137 38 : }
1138 38 : }
1139 :
1140 : #[tokio::test]
1141 2 : async fn no_connection_no_candidate() -> anyhow::Result<()> {
1142 2 : let harness = TenantHarness::create("no_connection_no_candidate").await?;
1143 26 : let mut state = dummy_state(&harness).await;
1144 2 : let now = Utc::now().naive_utc();
1145 2 :
1146 2 : let lagging_wal_timeout = chrono::Duration::from_std(state.conf.lagging_wal_timeout)?;
1147 2 : let delay_over_threshold = now - lagging_wal_timeout - lagging_wal_timeout;
1148 2 :
1149 2 : state.wal_connection = None;
1150 2 : state.wal_stream_candidates = HashMap::from([
1151 2 : (NodeId(0), dummy_broker_sk_timeline(1, "", now)),
1152 2 : (NodeId(1), dummy_broker_sk_timeline(0, "no_commit_lsn", now)),
1153 2 : (NodeId(2), dummy_broker_sk_timeline(0, "no_commit_lsn", now)),
1154 2 : (
1155 2 : NodeId(3),
1156 2 : dummy_broker_sk_timeline(
1157 2 : 1 + state.conf.max_lsn_wal_lag.get(),
1158 2 : "delay_over_threshold",
1159 2 : delay_over_threshold,
1160 2 : ),
1161 2 : ),
1162 2 : ]);
1163 2 :
1164 2 : let no_candidate = state.next_connection_candidate();
1165 2 : assert!(
1166 2 : no_candidate.is_none(),
1167 2 : "Expected no candidate selected out of non full data options, but got {no_candidate:?}"
1168 2 : );
1169 2 :
1170 2 : Ok(())
1171 2 : }
1172 :
1173 : #[tokio::test]
1174 2 : async fn connection_no_candidate() -> anyhow::Result<()> {
1175 2 : let harness = TenantHarness::create("connection_no_candidate").await?;
1176 24 : let mut state = dummy_state(&harness).await;
1177 2 : let now = Utc::now().naive_utc();
1178 2 :
1179 2 : let connected_sk_id = NodeId(0);
1180 2 : let current_lsn = 100_000;
1181 2 :
1182 2 : let connection_status = WalConnectionStatus {
1183 2 : is_connected: true,
1184 2 : has_processed_wal: true,
1185 2 : latest_connection_update: now,
1186 2 : latest_wal_update: now,
1187 2 : commit_lsn: Some(Lsn(current_lsn)),
1188 2 : streaming_lsn: Some(Lsn(current_lsn)),
1189 2 : node: NodeId(1),
1190 2 : };
1191 2 :
1192 2 : state.conf.max_lsn_wal_lag = NonZeroU64::new(100).unwrap();
1193 2 : state.wal_connection = Some(WalConnection {
1194 2 : started_at: now,
1195 2 : sk_id: connected_sk_id,
1196 2 : availability_zone: None,
1197 2 : status: connection_status,
1198 2 : connection_task: state.spawn(move |sender, _| async move {
1199 2 : sender
1200 2 : .send(TaskStateUpdate::Progress(connection_status))
1201 2 : .ok();
1202 2 : Ok(())
1203 2 : }),
1204 2 : discovered_new_wal: None,
1205 2 : });
1206 2 : state.wal_stream_candidates = HashMap::from([
1207 2 : (
1208 2 : connected_sk_id,
1209 2 : dummy_broker_sk_timeline(
1210 2 : current_lsn + state.conf.max_lsn_wal_lag.get() * 2,
1211 2 : DUMMY_SAFEKEEPER_HOST,
1212 2 : now,
1213 2 : ),
1214 2 : ),
1215 2 : (
1216 2 : NodeId(1),
1217 2 : dummy_broker_sk_timeline(current_lsn, "not_advanced_lsn", now),
1218 2 : ),
1219 2 : (
1220 2 : NodeId(2),
1221 2 : dummy_broker_sk_timeline(
1222 2 : current_lsn + state.conf.max_lsn_wal_lag.get() / 2,
1223 2 : "not_enough_advanced_lsn",
1224 2 : now,
1225 2 : ),
1226 2 : ),
1227 2 : ]);
1228 2 :
1229 2 : let no_candidate = state.next_connection_candidate();
1230 2 : assert!(
1231 2 : no_candidate.is_none(),
1232 2 : "Expected no candidate selected out of valid options since candidate Lsn data is ignored and others' was not advanced enough, but got {no_candidate:?}"
1233 2 : );
1234 2 :
1235 2 : Ok(())
1236 2 : }
1237 :
1238 : #[tokio::test]
1239 2 : async fn no_connection_candidate() -> anyhow::Result<()> {
1240 2 : let harness = TenantHarness::create("no_connection_candidate").await?;
1241 26 : let mut state = dummy_state(&harness).await;
1242 2 : let now = Utc::now().naive_utc();
1243 2 :
1244 2 : state.wal_connection = None;
1245 2 : state.wal_stream_candidates = HashMap::from([(
1246 2 : NodeId(0),
1247 2 : dummy_broker_sk_timeline(
1248 2 : 1 + state.conf.max_lsn_wal_lag.get(),
1249 2 : DUMMY_SAFEKEEPER_HOST,
1250 2 : now,
1251 2 : ),
1252 2 : )]);
1253 2 :
1254 2 : let only_candidate = state
1255 2 : .next_connection_candidate()
1256 2 : .expect("Expected one candidate selected out of the only data option, but got none");
1257 2 : assert_eq!(only_candidate.safekeeper_id, NodeId(0));
1258 2 : assert_eq!(
1259 2 : only_candidate.reason,
1260 2 : ReconnectReason::NoExistingConnection,
1261 2 : "Should select new safekeeper due to missing connection, even if there's also a lag in the wal over the threshold"
1262 2 : );
1263 2 : assert_eq!(
1264 2 : only_candidate.wal_source_connconf.host(),
1265 2 : &Host::Domain(DUMMY_SAFEKEEPER_HOST.to_owned())
1266 2 : );
1267 2 :
1268 2 : let selected_lsn = 100_000;
1269 2 : state.wal_stream_candidates = HashMap::from([
1270 2 : (
1271 2 : NodeId(0),
1272 2 : dummy_broker_sk_timeline(selected_lsn - 100, "smaller_commit_lsn", now),
1273 2 : ),
1274 2 : (
1275 2 : NodeId(1),
1276 2 : dummy_broker_sk_timeline(selected_lsn, DUMMY_SAFEKEEPER_HOST, now),
1277 2 : ),
1278 2 : (
1279 2 : NodeId(2),
1280 2 : dummy_broker_sk_timeline(selected_lsn + 100, "", now),
1281 2 : ),
1282 2 : ]);
1283 2 : let biggest_wal_candidate = state.next_connection_candidate().expect(
1284 2 : "Expected one candidate selected out of multiple valid data options, but got none",
1285 2 : );
1286 2 :
1287 2 : assert_eq!(biggest_wal_candidate.safekeeper_id, NodeId(1));
1288 2 : assert_eq!(
1289 2 : biggest_wal_candidate.reason,
1290 2 : ReconnectReason::NoExistingConnection,
1291 2 : "Should select new safekeeper due to missing connection, even if there's also a lag in the wal over the threshold"
1292 2 : );
1293 2 : assert_eq!(
1294 2 : biggest_wal_candidate.wal_source_connconf.host(),
1295 2 : &Host::Domain(DUMMY_SAFEKEEPER_HOST.to_owned())
1296 2 : );
1297 2 :
1298 2 : Ok(())
1299 2 : }
1300 :
1301 : #[tokio::test]
1302 2 : async fn candidate_with_many_connection_failures() -> anyhow::Result<()> {
1303 2 : let harness = TenantHarness::create("candidate_with_many_connection_failures").await?;
1304 26 : let mut state = dummy_state(&harness).await;
1305 2 : let now = Utc::now().naive_utc();
1306 2 :
1307 2 : let current_lsn = Lsn(100_000).align();
1308 2 : let bigger_lsn = Lsn(current_lsn.0 + 100).align();
1309 2 :
1310 2 : state.wal_connection = None;
1311 2 : state.wal_stream_candidates = HashMap::from([
1312 2 : (
1313 2 : NodeId(0),
1314 2 : dummy_broker_sk_timeline(bigger_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1315 2 : ),
1316 2 : (
1317 2 : NodeId(1),
1318 2 : dummy_broker_sk_timeline(current_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1319 2 : ),
1320 2 : ]);
1321 2 : state.wal_connection_retries = HashMap::from([(
1322 2 : NodeId(0),
1323 2 : RetryInfo {
1324 2 : next_retry_at: now.checked_add_signed(chrono::Duration::hours(1)),
1325 2 : retry_duration_seconds: WALCONNECTION_RETRY_MAX_BACKOFF_SECONDS,
1326 2 : },
1327 2 : )]);
1328 2 :
1329 2 : let candidate_with_less_errors = state
1330 2 : .next_connection_candidate()
1331 2 : .expect("Expected one candidate selected, but got none");
1332 2 : assert_eq!(
1333 2 : candidate_with_less_errors.safekeeper_id,
1334 2 : NodeId(1),
1335 2 : "Should select the node with no pending retry cooldown"
1336 2 : );
1337 2 :
1338 2 : Ok(())
1339 2 : }
1340 :
1341 : #[tokio::test]
1342 2 : async fn lsn_wal_over_threshold_current_candidate() -> anyhow::Result<()> {
1343 2 : let harness = TenantHarness::create("lsn_wal_over_threshcurrent_candidate").await?;
1344 26 : let mut state = dummy_state(&harness).await;
1345 2 : let current_lsn = Lsn(100_000).align();
1346 2 : let now = Utc::now().naive_utc();
1347 2 :
1348 2 : let connected_sk_id = NodeId(0);
1349 2 : let new_lsn = Lsn(current_lsn.0 + state.conf.max_lsn_wal_lag.get() + 1);
1350 2 :
1351 2 : let connection_status = WalConnectionStatus {
1352 2 : is_connected: true,
1353 2 : has_processed_wal: true,
1354 2 : latest_connection_update: now,
1355 2 : latest_wal_update: now,
1356 2 : commit_lsn: Some(current_lsn),
1357 2 : streaming_lsn: Some(current_lsn),
1358 2 : node: connected_sk_id,
1359 2 : };
1360 2 :
1361 2 : state.wal_connection = Some(WalConnection {
1362 2 : started_at: now,
1363 2 : sk_id: connected_sk_id,
1364 2 : availability_zone: None,
1365 2 : status: connection_status,
1366 2 : connection_task: state.spawn(move |sender, _| async move {
1367 2 : sender
1368 2 : .send(TaskStateUpdate::Progress(connection_status))
1369 2 : .ok();
1370 2 : Ok(())
1371 2 : }),
1372 2 : discovered_new_wal: None,
1373 2 : });
1374 2 : state.wal_stream_candidates = HashMap::from([
1375 2 : (
1376 2 : connected_sk_id,
1377 2 : dummy_broker_sk_timeline(current_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1378 2 : ),
1379 2 : (
1380 2 : NodeId(1),
1381 2 : dummy_broker_sk_timeline(new_lsn.0, "advanced_by_lsn_safekeeper", now),
1382 2 : ),
1383 2 : ]);
1384 2 :
1385 2 : let over_threshcurrent_candidate = state.next_connection_candidate().expect(
1386 2 : "Expected one candidate selected out of multiple valid data options, but got none",
1387 2 : );
1388 2 :
1389 2 : assert_eq!(over_threshcurrent_candidate.safekeeper_id, NodeId(1));
1390 2 : assert_eq!(
1391 2 : over_threshcurrent_candidate.reason,
1392 2 : ReconnectReason::LaggingWal {
1393 2 : current_commit_lsn: current_lsn,
1394 2 : new_commit_lsn: new_lsn,
1395 2 : threshold: state.conf.max_lsn_wal_lag
1396 2 : },
1397 2 : "Should select bigger WAL safekeeper if it starts to lag enough"
1398 2 : );
1399 2 : assert_eq!(
1400 2 : over_threshcurrent_candidate.wal_source_connconf.host(),
1401 2 : &Host::Domain("advanced_by_lsn_safekeeper".to_owned())
1402 2 : );
1403 2 :
1404 2 : Ok(())
1405 2 : }
1406 :
1407 : #[tokio::test]
1408 2 : async fn timeout_connection_threshold_current_candidate() -> anyhow::Result<()> {
1409 2 : let harness =
1410 2 : TenantHarness::create("timeout_connection_threshold_current_candidate").await?;
1411 26 : let mut state = dummy_state(&harness).await;
1412 2 : let current_lsn = Lsn(100_000).align();
1413 2 : let now = Utc::now().naive_utc();
1414 2 :
1415 2 : let wal_connect_timeout = chrono::Duration::from_std(state.conf.wal_connect_timeout)?;
1416 2 : let time_over_threshold =
1417 2 : Utc::now().naive_utc() - wal_connect_timeout - wal_connect_timeout;
1418 2 :
1419 2 : let connection_status = WalConnectionStatus {
1420 2 : is_connected: true,
1421 2 : has_processed_wal: true,
1422 2 : latest_connection_update: time_over_threshold,
1423 2 : latest_wal_update: time_over_threshold,
1424 2 : commit_lsn: Some(current_lsn),
1425 2 : streaming_lsn: Some(current_lsn),
1426 2 : node: NodeId(1),
1427 2 : };
1428 2 :
1429 2 : state.wal_connection = Some(WalConnection {
1430 2 : started_at: now,
1431 2 : sk_id: NodeId(1),
1432 2 : availability_zone: None,
1433 2 : status: connection_status,
1434 2 : connection_task: state.spawn(move |sender, _| async move {
1435 2 : sender
1436 2 : .send(TaskStateUpdate::Progress(connection_status))
1437 2 : .ok();
1438 2 : Ok(())
1439 2 : }),
1440 2 : discovered_new_wal: None,
1441 2 : });
1442 2 : state.wal_stream_candidates = HashMap::from([(
1443 2 : NodeId(0),
1444 2 : dummy_broker_sk_timeline(current_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1445 2 : )]);
1446 2 :
1447 2 : let over_threshcurrent_candidate = state.next_connection_candidate().expect(
1448 2 : "Expected one candidate selected out of multiple valid data options, but got none",
1449 2 : );
1450 2 :
1451 2 : assert_eq!(over_threshcurrent_candidate.safekeeper_id, NodeId(0));
1452 2 : match over_threshcurrent_candidate.reason {
1453 2 : ReconnectReason::NoKeepAlives {
1454 2 : last_keep_alive,
1455 2 : threshold,
1456 2 : ..
1457 2 : } => {
1458 2 : assert_eq!(last_keep_alive, Some(time_over_threshold));
1459 2 : assert_eq!(threshold, state.conf.lagging_wal_timeout);
1460 2 : }
1461 2 : unexpected => panic!("Unexpected reason: {unexpected:?}"),
1462 2 : }
1463 2 : assert_eq!(
1464 2 : over_threshcurrent_candidate.wal_source_connconf.host(),
1465 2 : &Host::Domain(DUMMY_SAFEKEEPER_HOST.to_owned())
1466 2 : );
1467 2 :
1468 2 : Ok(())
1469 2 : }
1470 :
1471 : #[tokio::test]
1472 2 : async fn timeout_wal_over_threshold_current_candidate() -> anyhow::Result<()> {
1473 2 : let harness = TenantHarness::create("timeout_wal_over_threshold_current_candidate").await?;
1474 26 : let mut state = dummy_state(&harness).await;
1475 2 : let current_lsn = Lsn(100_000).align();
1476 2 : let new_lsn = Lsn(100_100).align();
1477 2 : let now = Utc::now().naive_utc();
1478 2 :
1479 2 : let lagging_wal_timeout = chrono::Duration::from_std(state.conf.lagging_wal_timeout)?;
1480 2 : let time_over_threshold =
1481 2 : Utc::now().naive_utc() - lagging_wal_timeout - lagging_wal_timeout;
1482 2 :
1483 2 : let connection_status = WalConnectionStatus {
1484 2 : is_connected: true,
1485 2 : has_processed_wal: true,
1486 2 : latest_connection_update: now,
1487 2 : latest_wal_update: time_over_threshold,
1488 2 : commit_lsn: Some(current_lsn),
1489 2 : streaming_lsn: Some(current_lsn),
1490 2 : node: NodeId(1),
1491 2 : };
1492 2 :
1493 2 : state.wal_connection = Some(WalConnection {
1494 2 : started_at: now,
1495 2 : sk_id: NodeId(1),
1496 2 : availability_zone: None,
1497 2 : status: connection_status,
1498 2 : connection_task: state.spawn(move |_, _| async move { Ok(()) }),
1499 2 : discovered_new_wal: Some(NewCommittedWAL {
1500 2 : discovered_at: time_over_threshold,
1501 2 : lsn: new_lsn,
1502 2 : }),
1503 2 : });
1504 2 : state.wal_stream_candidates = HashMap::from([(
1505 2 : NodeId(0),
1506 2 : dummy_broker_sk_timeline(new_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1507 2 : )]);
1508 2 :
1509 2 : let over_threshcurrent_candidate = state.next_connection_candidate().expect(
1510 2 : "Expected one candidate selected out of multiple valid data options, but got none",
1511 2 : );
1512 2 :
1513 2 : assert_eq!(over_threshcurrent_candidate.safekeeper_id, NodeId(0));
1514 2 : match over_threshcurrent_candidate.reason {
1515 2 : ReconnectReason::NoWalTimeout {
1516 2 : current_lsn,
1517 2 : current_commit_lsn,
1518 2 : candidate_commit_lsn,
1519 2 : last_wal_interaction,
1520 2 : threshold,
1521 2 : ..
1522 2 : } => {
1523 2 : assert_eq!(current_lsn, current_lsn);
1524 2 : assert_eq!(current_commit_lsn, current_lsn);
1525 2 : assert_eq!(candidate_commit_lsn, new_lsn);
1526 2 : assert_eq!(last_wal_interaction, Some(time_over_threshold));
1527 2 : assert_eq!(threshold, state.conf.lagging_wal_timeout);
1528 2 : }
1529 2 : unexpected => panic!("Unexpected reason: {unexpected:?}"),
1530 2 : }
1531 2 : assert_eq!(
1532 2 : over_threshcurrent_candidate.wal_source_connconf.host(),
1533 2 : &Host::Domain(DUMMY_SAFEKEEPER_HOST.to_owned())
1534 2 : );
1535 2 :
1536 2 : Ok(())
1537 2 : }
1538 :
1539 : const DUMMY_SAFEKEEPER_HOST: &str = "safekeeper_connstr";
1540 :
1541 16 : async fn dummy_state(harness: &TenantHarness) -> ConnectionManagerState {
1542 158 : let (tenant, ctx) = harness.load().await;
1543 16 : let timeline = tenant
1544 16 : .create_test_timeline(TIMELINE_ID, Lsn(0x8), crate::DEFAULT_PG_VERSION, &ctx)
1545 48 : .await
1546 16 : .expect("Failed to create an empty timeline for dummy wal connection manager");
1547 16 :
1548 16 : ConnectionManagerState {
1549 16 : id: TenantTimelineId {
1550 16 : tenant_id: harness.tenant_shard_id.tenant_id,
1551 16 : timeline_id: TIMELINE_ID,
1552 16 : },
1553 16 : timeline,
1554 16 : cancel: CancellationToken::new(),
1555 16 : conf: WalReceiverConf {
1556 16 : protocol: DEFAULT_WAL_RECEIVER_PROTOCOL,
1557 16 : wal_connect_timeout: Duration::from_secs(1),
1558 16 : lagging_wal_timeout: Duration::from_secs(1),
1559 16 : max_lsn_wal_lag: NonZeroU64::new(1024 * 1024).unwrap(),
1560 16 : auth_token: None,
1561 16 : availability_zone: None,
1562 16 : ingest_batch_size: 1,
1563 16 : },
1564 16 : wal_connection: None,
1565 16 : wal_stream_candidates: HashMap::new(),
1566 16 : wal_connection_retries: HashMap::new(),
1567 16 : }
1568 16 : }
1569 :
1570 : #[tokio::test]
1571 2 : async fn switch_to_same_availability_zone() -> anyhow::Result<()> {
1572 2 : // Pageserver and one of safekeepers will be in the same availability zone
1573 2 : // and pageserver should prefer to connect to it.
1574 2 : let test_az = Some("test_az".to_owned());
1575 2 :
1576 2 : let harness = TenantHarness::create("switch_to_same_availability_zone").await?;
1577 26 : let mut state = dummy_state(&harness).await;
1578 2 : state.conf.availability_zone.clone_from(&test_az);
1579 2 : let current_lsn = Lsn(100_000).align();
1580 2 : let now = Utc::now().naive_utc();
1581 2 :
1582 2 : let connected_sk_id = NodeId(0);
1583 2 :
1584 2 : let connection_status = WalConnectionStatus {
1585 2 : is_connected: true,
1586 2 : has_processed_wal: true,
1587 2 : latest_connection_update: now,
1588 2 : latest_wal_update: now,
1589 2 : commit_lsn: Some(current_lsn),
1590 2 : streaming_lsn: Some(current_lsn),
1591 2 : node: connected_sk_id,
1592 2 : };
1593 2 :
1594 2 : state.wal_connection = Some(WalConnection {
1595 2 : started_at: now,
1596 2 : sk_id: connected_sk_id,
1597 2 : availability_zone: None,
1598 2 : status: connection_status,
1599 2 : connection_task: state.spawn(move |sender, _| async move {
1600 2 : sender
1601 2 : .send(TaskStateUpdate::Progress(connection_status))
1602 2 : .ok();
1603 2 : Ok(())
1604 2 : }),
1605 2 : discovered_new_wal: None,
1606 2 : });
1607 2 :
1608 2 : // We have another safekeeper with the same commit_lsn, and it have the same availability zone as
1609 2 : // the current pageserver.
1610 2 : let mut same_az_sk = dummy_broker_sk_timeline(current_lsn.0, "same_az", now);
1611 2 : same_az_sk.timeline.availability_zone.clone_from(&test_az);
1612 2 :
1613 2 : state.wal_stream_candidates = HashMap::from([
1614 2 : (
1615 2 : connected_sk_id,
1616 2 : dummy_broker_sk_timeline(current_lsn.0, DUMMY_SAFEKEEPER_HOST, now),
1617 2 : ),
1618 2 : (NodeId(1), same_az_sk),
1619 2 : ]);
1620 2 :
1621 2 : // We expect that pageserver will switch to the safekeeper in the same availability zone,
1622 2 : // even if it has the same commit_lsn.
1623 2 : let next_candidate = state.next_connection_candidate().expect(
1624 2 : "Expected one candidate selected out of multiple valid data options, but got none",
1625 2 : );
1626 2 :
1627 2 : assert_eq!(next_candidate.safekeeper_id, NodeId(1));
1628 2 : assert_eq!(
1629 2 : next_candidate.reason,
1630 2 : ReconnectReason::SwitchAvailabilityZone,
1631 2 : "Should switch to the safekeeper in the same availability zone, if it has the same commit_lsn"
1632 2 : );
1633 2 : assert_eq!(
1634 2 : next_candidate.wal_source_connconf.host(),
1635 2 : &Host::Domain("same_az".to_owned())
1636 2 : );
1637 2 :
1638 2 : Ok(())
1639 2 : }
1640 : }
|