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