Line data Source code
1 : //! This module implements pulling WAL from peer safekeepers if compute can't
2 : //! provide it, i.e. safekeeper lags too much.
3 :
4 : use std::fmt;
5 : use std::pin::pin;
6 : use std::time::SystemTime;
7 :
8 : use anyhow::{Context, bail};
9 : use futures::StreamExt;
10 : use postgres_protocol::message::backend::ReplicationMessage;
11 : use reqwest::Certificate;
12 : use safekeeper_api::Term;
13 : use safekeeper_api::membership::INVALID_GENERATION;
14 : use safekeeper_api::models::{PeerInfo, TimelineStatus};
15 : use tokio::select;
16 : use tokio::sync::mpsc::{Receiver, Sender, channel};
17 : use tokio::time::{self, Duration, sleep, timeout};
18 : use tokio_postgres::replication::ReplicationStream;
19 : use tokio_postgres::types::PgLsn;
20 : use tracing::*;
21 : use utils::id::NodeId;
22 : use utils::lsn::Lsn;
23 : use utils::postgres_client::{
24 : ConnectionConfigArgs, PostgresClientProtocol, wal_stream_connection_config,
25 : };
26 :
27 : use crate::SafeKeeperConf;
28 : use crate::receive_wal::{MSG_QUEUE_SIZE, REPLY_QUEUE_SIZE, WalAcceptor};
29 : use crate::safekeeper::{
30 : AcceptorProposerMessage, AppendRequest, AppendRequestHeader, ProposerAcceptorMessage,
31 : ProposerElected, TermHistory, TermLsn, VoteRequest,
32 : };
33 : use crate::timeline::WalResidentTimeline;
34 :
35 : /// Entrypoint for per timeline task which always runs, checking whether
36 : /// recovery for this safekeeper is needed and starting it if so.
37 : #[instrument(name = "recovery", skip_all, fields(ttid = %tli.ttid))]
38 : pub async fn recovery_main(tli: WalResidentTimeline, conf: SafeKeeperConf) {
39 : info!("started");
40 :
41 : let cancel = tli.cancel.clone();
42 : select! {
43 : _ = recovery_main_loop(tli, conf) => { unreachable!() }
44 : _ = cancel.cancelled() => {
45 : info!("stopped");
46 : }
47 : }
48 : }
49 :
50 : /// Should we start fetching WAL from a peer safekeeper, and if yes, from
51 : /// which? Answer is yes, i.e. .donors is not empty if 1) there is something
52 : /// to fetch, and we can do that without running elections; 2) there is no
53 : /// actively streaming compute, as we don't want to compete with it.
54 : ///
55 : /// If donor(s) are choosen, theirs last_log_term is guaranteed to be equal
56 : /// to its last_log_term so we are sure such a leader ever had been elected.
57 : ///
58 : /// All possible donors are returned so that we could keep connection to the
59 : /// current one if it is good even if it slightly lags behind.
60 : ///
61 : /// Note that term conditions above might be not met, but safekeepers are
62 : /// still not aligned on last flush_lsn. Generally in this case until
63 : /// elections are run it is not possible to say which safekeeper should
64 : /// recover from which one -- history which would be committed is different
65 : /// depending on assembled quorum (e.g. classic picture 8 from Raft paper).
66 : /// Thus we don't try to predict it here.
67 5 : async fn recovery_needed(
68 5 : tli: &WalResidentTimeline,
69 5 : heartbeat_timeout: Duration,
70 5 : ) -> RecoveryNeededInfo {
71 5 : let ss = tli.read_shared_state().await;
72 5 : let term = ss.sk.state().acceptor_state.term;
73 5 : let last_log_term = ss.sk.last_log_term();
74 5 : let flush_lsn = ss.sk.flush_lsn();
75 5 : // note that peers contain myself, but that's ok -- we are interested only in peers which are strictly ahead of us.
76 5 : let mut peers = ss.get_peers(heartbeat_timeout);
77 5 : // Sort by <last log term, lsn> pairs.
78 5 : peers.sort_by(|p1, p2| {
79 0 : let tl1 = TermLsn {
80 0 : term: p1.last_log_term,
81 0 : lsn: p1.flush_lsn,
82 0 : };
83 0 : let tl2 = TermLsn {
84 0 : term: p2.last_log_term,
85 0 : lsn: p2.flush_lsn,
86 0 : };
87 0 : tl2.cmp(&tl1) // desc
88 5 : });
89 5 : let num_streaming_computes = tli.get_walreceivers().get_num_streaming();
90 5 : let donors = if num_streaming_computes > 0 {
91 0 : vec![] // If there is a streaming compute, don't try to recover to not intervene.
92 : } else {
93 5 : peers
94 5 : .iter()
95 5 : .filter_map(|candidate| {
96 0 : // Are we interested in this candidate?
97 0 : let candidate_tl = TermLsn {
98 0 : term: candidate.last_log_term,
99 0 : lsn: candidate.flush_lsn,
100 0 : };
101 0 : let my_tl = TermLsn {
102 0 : term: last_log_term,
103 0 : lsn: flush_lsn,
104 0 : };
105 0 : if my_tl < candidate_tl {
106 : // Yes, we are interested. Can we pull from it without
107 : // (re)running elections? It is possible if 1) his term
108 : // is equal to his last_log_term so we could act on
109 : // behalf of leader of this term (we must be sure he was
110 : // ever elected) and 2) our term is not higher, or we'll refuse data.
111 0 : if candidate.term == candidate.last_log_term && candidate.term >= term {
112 0 : Some(Donor::from(candidate))
113 : } else {
114 0 : None
115 : }
116 : } else {
117 0 : None
118 : }
119 5 : })
120 5 : .collect()
121 : };
122 5 : RecoveryNeededInfo {
123 5 : term,
124 5 : last_log_term,
125 5 : flush_lsn,
126 5 : peers,
127 5 : num_streaming_computes,
128 5 : donors,
129 5 : }
130 5 : }
131 : /// Result of Timeline::recovery_needed, contains donor(s) if recovery needed and
132 : /// fields to explain the choice.
133 : #[derive(Debug)]
134 : pub struct RecoveryNeededInfo {
135 : /// my term
136 : pub term: Term,
137 : /// my last_log_term
138 : pub last_log_term: Term,
139 : /// my flush_lsn
140 : pub flush_lsn: Lsn,
141 : /// peers from which we can fetch WAL, for observability.
142 : pub peers: Vec<PeerInfo>,
143 : /// for observability
144 : pub num_streaming_computes: usize,
145 : pub donors: Vec<Donor>,
146 : }
147 :
148 : // Custom to omit not important fields from PeerInfo.
149 : impl fmt::Display for RecoveryNeededInfo {
150 0 : fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
151 0 : write!(f, "{{")?;
152 0 : write!(
153 0 : f,
154 0 : "term: {}, last_log_term: {}, flush_lsn: {}, peers: {{",
155 0 : self.term, self.last_log_term, self.flush_lsn
156 0 : )?;
157 0 : for p in self.peers.iter() {
158 0 : write!(
159 0 : f,
160 0 : "PeerInfo {{ sk_id: {}, term: {}, last_log_term: {}, flush_lsn: {} }}, ",
161 0 : p.sk_id, p.term, p.last_log_term, p.flush_lsn
162 0 : )?;
163 : }
164 0 : write!(
165 0 : f,
166 0 : "}} num_streaming_computes: {}, donors: {:?}",
167 0 : self.num_streaming_computes, self.donors
168 0 : )
169 0 : }
170 : }
171 :
172 : #[derive(Clone, Debug)]
173 : pub struct Donor {
174 : pub sk_id: NodeId,
175 : /// equals to last_log_term
176 : pub term: Term,
177 : pub flush_lsn: Lsn,
178 : pub pg_connstr: String,
179 : pub http_connstr: String,
180 : pub https_connstr: Option<String>,
181 : }
182 :
183 : impl From<&PeerInfo> for Donor {
184 0 : fn from(p: &PeerInfo) -> Self {
185 0 : Donor {
186 0 : sk_id: p.sk_id,
187 0 : term: p.term,
188 0 : flush_lsn: p.flush_lsn,
189 0 : pg_connstr: p.pg_connstr.clone(),
190 0 : http_connstr: p.http_connstr.clone(),
191 0 : https_connstr: p.https_connstr.clone(),
192 0 : }
193 0 : }
194 : }
195 :
196 : const CHECK_INTERVAL_MS: u64 = 2000;
197 :
198 : /// Check regularly whether we need to start recovery.
199 5 : async fn recovery_main_loop(tli: WalResidentTimeline, conf: SafeKeeperConf) {
200 5 : let check_duration = Duration::from_millis(CHECK_INTERVAL_MS);
201 : loop {
202 5 : let recovery_needed_info = recovery_needed(&tli, conf.heartbeat_timeout).await;
203 5 : match recovery_needed_info.donors.first() {
204 0 : Some(donor) => {
205 0 : info!(
206 0 : "starting recovery from donor {}: {}",
207 : donor.sk_id, recovery_needed_info
208 : );
209 0 : let res = tli.wal_residence_guard().await;
210 0 : if let Err(e) = res {
211 0 : warn!("failed to obtain guard: {}", e);
212 0 : continue;
213 0 : }
214 0 : match recover(res.unwrap(), donor, &conf).await {
215 : // Note: 'write_wal rewrites WAL written before' error is
216 : // expected here and might happen if compute and recovery
217 : // concurrently write the same data. Eventually compute
218 : // should win.
219 0 : Err(e) => warn!("recovery failed: {:#}", e),
220 0 : Ok(msg) => info!("recovery finished: {}", msg),
221 : }
222 : }
223 : None => {
224 5 : trace!(
225 0 : "recovery not needed or not possible: {}",
226 : recovery_needed_info
227 : );
228 : }
229 : }
230 5 : sleep(check_duration).await;
231 : }
232 : }
233 :
234 : /// Recover from the specified donor. Returns message explaining normal finish
235 : /// reason or error.
236 0 : async fn recover(
237 0 : tli: WalResidentTimeline,
238 0 : donor: &Donor,
239 0 : conf: &SafeKeeperConf,
240 0 : ) -> anyhow::Result<String> {
241 0 : // Learn donor term switch history to figure out starting point.
242 0 :
243 0 : let mut client = reqwest::Client::builder();
244 0 : for cert in &conf.ssl_ca_certs {
245 0 : client = client.add_root_certificate(Certificate::from_der(cert.contents())?);
246 : }
247 0 : let client = client
248 0 : .build()
249 0 : .context("Failed to build http client for recover")?;
250 :
251 0 : let url = if conf.use_https_safekeeper_api {
252 0 : if let Some(https_connstr) = donor.https_connstr.as_ref() {
253 0 : format!("https://{https_connstr}")
254 : } else {
255 0 : anyhow::bail!(
256 0 : "cannot recover from donor {}: \
257 0 : https is enabled, but https_connstr is not specified",
258 0 : donor.sk_id
259 0 : );
260 : }
261 : } else {
262 0 : format!("http://{}", donor.http_connstr)
263 : };
264 :
265 0 : let timeline_info: TimelineStatus = client
266 0 : .get(format!(
267 0 : "{}/v1/tenant/{}/timeline/{}",
268 0 : url, tli.ttid.tenant_id, tli.ttid.timeline_id
269 0 : ))
270 0 : .send()
271 0 : .await?
272 0 : .json()
273 0 : .await?;
274 0 : if timeline_info.acceptor_state.term != donor.term {
275 0 : bail!(
276 0 : "donor term changed from {} to {}",
277 0 : donor.term,
278 0 : timeline_info.acceptor_state.term
279 0 : );
280 0 : }
281 0 : // convert from API TermSwitchApiEntry into TermLsn.
282 0 : let donor_th = TermHistory(
283 0 : timeline_info
284 0 : .acceptor_state
285 0 : .term_history
286 0 : .iter()
287 0 : .map(|tl| Into::<TermLsn>::into(*tl))
288 0 : .collect(),
289 0 : );
290 0 :
291 0 : // Now understand our term history.
292 0 : let vote_request = ProposerAcceptorMessage::VoteRequest(VoteRequest {
293 0 : generation: INVALID_GENERATION,
294 0 : term: donor.term,
295 0 : });
296 0 : let vote_response = match tli
297 0 : .process_msg(&vote_request)
298 0 : .await
299 0 : .context("VoteRequest handling")?
300 : {
301 0 : Some(AcceptorProposerMessage::VoteResponse(vr)) => vr,
302 : _ => {
303 0 : bail!("unexpected VoteRequest response"); // unreachable
304 : }
305 : };
306 0 : if vote_response.term != donor.term {
307 0 : bail!(
308 0 : "our term changed from {} to {}",
309 0 : donor.term,
310 0 : vote_response.term
311 0 : );
312 0 : }
313 :
314 0 : let last_common_point = match TermHistory::find_highest_common_point(
315 0 : &donor_th,
316 0 : &vote_response.term_history,
317 0 : vote_response.flush_lsn,
318 0 : ) {
319 0 : None => bail!(
320 0 : "couldn't find common point in histories, donor {:?}, sk {:?}",
321 0 : donor_th,
322 0 : vote_response.term_history,
323 0 : ),
324 0 : Some(lcp) => lcp,
325 0 : };
326 0 : info!("found last common point at {:?}", last_common_point);
327 :
328 : // truncate WAL locally
329 0 : let pe = ProposerAcceptorMessage::Elected(ProposerElected {
330 0 : generation: INVALID_GENERATION,
331 0 : term: donor.term,
332 0 : start_streaming_at: last_common_point.lsn,
333 0 : term_history: donor_th,
334 0 : });
335 0 : // Successful ProposerElected handling always returns None. If term changed,
336 0 : // we'll find out that during the streaming. Note: it is expected to get
337 0 : // 'refusing to overwrite correct WAL' here if walproposer reconnected
338 0 : // concurrently, restart helps here.
339 0 : tli.process_msg(&pe)
340 0 : .await
341 0 : .context("ProposerElected handling")?;
342 :
343 0 : recovery_stream(tli, donor, last_common_point.lsn, conf).await
344 0 : }
345 :
346 : // Pull WAL from donor, assuming handshake is already done.
347 0 : async fn recovery_stream(
348 0 : tli: WalResidentTimeline,
349 0 : donor: &Donor,
350 0 : start_streaming_at: Lsn,
351 0 : conf: &SafeKeeperConf,
352 0 : ) -> anyhow::Result<String> {
353 0 : // TODO: pass auth token
354 0 : let connection_conf_args = ConnectionConfigArgs {
355 0 : protocol: PostgresClientProtocol::Vanilla,
356 0 : ttid: tli.ttid,
357 0 : shard_number: None,
358 0 : shard_count: None,
359 0 : shard_stripe_size: None,
360 0 : listen_pg_addr_str: &donor.pg_connstr,
361 0 : auth_token: None,
362 0 : availability_zone: None,
363 0 : };
364 0 : let cfg = wal_stream_connection_config(connection_conf_args)?;
365 0 : let mut cfg = cfg.to_tokio_postgres_config();
366 0 : // It will make safekeeper give out not committed WAL (up to flush_lsn).
367 0 : cfg.application_name(&format!("safekeeper_{}", conf.my_id));
368 0 : cfg.replication_mode(tokio_postgres::config::ReplicationMode::Physical);
369 0 :
370 0 : let connect_timeout = Duration::from_millis(10000);
371 0 : let (client, connection) = match time::timeout(
372 0 : connect_timeout,
373 0 : cfg.connect(tokio_postgres::NoTls),
374 0 : )
375 0 : .await
376 : {
377 0 : Ok(client_and_conn) => client_and_conn?,
378 0 : Err(_elapsed) => {
379 0 : bail!(
380 0 : "timed out while waiting {connect_timeout:?} for connection to peer safekeeper to open"
381 0 : );
382 : }
383 : };
384 0 : trace!("connected to {:?}", donor);
385 :
386 : // The connection object performs the actual communication with the
387 : // server, spawn it off to run on its own.
388 0 : let ttid = tli.ttid;
389 0 : tokio::spawn(async move {
390 0 : if let Err(e) = connection
391 0 : .instrument(info_span!("recovery task connection poll", ttid = %ttid))
392 0 : .await
393 : {
394 : // This logging isn't very useful as error is anyway forwarded to client.
395 0 : trace!(
396 0 : "tokio_postgres connection object finished with error: {}",
397 : e
398 : );
399 0 : }
400 0 : });
401 0 :
402 0 : let query = format!(
403 0 : "START_REPLICATION PHYSICAL {} (term='{}')",
404 0 : start_streaming_at, donor.term
405 0 : );
406 :
407 0 : let copy_stream = client.copy_both_simple(&query).await?;
408 0 : let physical_stream = ReplicationStream::new(copy_stream);
409 0 :
410 0 : // As in normal walreceiver, do networking and writing to disk in parallel.
411 0 : let (msg_tx, msg_rx) = channel(MSG_QUEUE_SIZE);
412 0 : let (reply_tx, reply_rx) = channel(REPLY_QUEUE_SIZE);
413 0 : let wa = WalAcceptor::spawn(tli.wal_residence_guard().await?, msg_rx, reply_tx, None);
414 :
415 0 : let res = tokio::select! {
416 0 : r = network_io(physical_stream, msg_tx, donor.clone(), tli, conf.clone()) => r,
417 0 : r = read_replies(reply_rx, donor.term) => r.map(|()| None),
418 : };
419 :
420 : // Join the spawned WalAcceptor. At this point chans to/from it passed to
421 : // network routines are dropped, so it will exit as soon as it touches them.
422 0 : match wa.await {
423 : Ok(Ok(())) => {
424 : // WalAcceptor finished normally, termination reason is different
425 0 : match res {
426 0 : Ok(Some(success_desc)) => Ok(success_desc),
427 0 : Ok(None) => bail!("unexpected recovery end without error/success"), // can't happen
428 0 : Err(e) => Err(e), // network error or term change
429 : }
430 : }
431 0 : Ok(Err(e)) => Err(e), // error while processing message
432 0 : Err(e) => bail!("WalAcceptor panicked: {}", e),
433 : }
434 0 : }
435 :
436 : // Perform network part of streaming: read data and push it to msg_tx, send KA
437 : // to make sender hear from us. If there is nothing coming for a while, check
438 : // for termination.
439 : // Returns
440 : // - Ok(None) if channel to WalAcceptor closed -- its task should return error.
441 : // - Ok(Some(String)) if recovery successfully completed.
442 : // - Err if error happened while reading/writing to socket.
443 0 : async fn network_io(
444 0 : physical_stream: ReplicationStream,
445 0 : msg_tx: Sender<ProposerAcceptorMessage>,
446 0 : donor: Donor,
447 0 : tli: WalResidentTimeline,
448 0 : conf: SafeKeeperConf,
449 0 : ) -> anyhow::Result<Option<String>> {
450 0 : let mut physical_stream = pin!(physical_stream);
451 0 : let mut last_received_lsn = Lsn::INVALID;
452 0 : // tear down connection if no data arrives withing this period
453 0 : let no_data_timeout = Duration::from_millis(30000);
454 :
455 : loop {
456 0 : let msg = match timeout(no_data_timeout, physical_stream.next()).await {
457 0 : Ok(next) => match next {
458 0 : None => bail!("unexpected end of replication stream"),
459 0 : Some(msg) => msg.context("get replication message")?,
460 : },
461 0 : Err(_) => bail!("no message received within {:?}", no_data_timeout),
462 : };
463 :
464 0 : match msg {
465 0 : ReplicationMessage::XLogData(xlog_data) => {
466 0 : let ar_hdr = AppendRequestHeader {
467 0 : generation: INVALID_GENERATION,
468 0 : term: donor.term,
469 0 : begin_lsn: Lsn(xlog_data.wal_start()),
470 0 : end_lsn: Lsn(xlog_data.wal_start()) + xlog_data.data().len() as u64,
471 0 : commit_lsn: Lsn::INVALID, // do not attempt to advance, peer communication anyway does it
472 0 : truncate_lsn: Lsn::INVALID, // do not attempt to advance
473 0 : };
474 0 : let ar = AppendRequest {
475 0 : h: ar_hdr,
476 0 : wal_data: xlog_data.into_data(),
477 0 : };
478 0 : trace!(
479 0 : "processing AppendRequest {}-{}, len {}",
480 0 : ar.h.begin_lsn,
481 0 : ar.h.end_lsn,
482 0 : ar.wal_data.len()
483 : );
484 0 : last_received_lsn = ar.h.end_lsn;
485 0 : if msg_tx
486 0 : .send(ProposerAcceptorMessage::AppendRequest(ar))
487 0 : .await
488 0 : .is_err()
489 : {
490 0 : return Ok(None); // chan closed, WalAcceptor terminated
491 0 : }
492 : }
493 : ReplicationMessage::PrimaryKeepAlive(_) => {
494 : // keepalive means nothing is being streamed for a while. Check whether we need to stop.
495 0 : let recovery_needed_info = recovery_needed(&tli, conf.heartbeat_timeout).await;
496 : // do current donors still contain one we currently connected to?
497 0 : if !recovery_needed_info
498 0 : .donors
499 0 : .iter()
500 0 : .any(|d| d.sk_id == donor.sk_id)
501 : {
502 : // Most likely it means we are caughtup.
503 : // note: just exiting makes tokio_postgres send CopyFail to the far end.
504 0 : return Ok(Some(format!(
505 0 : "terminating at {} as connected safekeeper {} with term {} is not a donor anymore: {}",
506 0 : last_received_lsn, donor.sk_id, donor.term, recovery_needed_info
507 0 : )));
508 0 : }
509 : }
510 0 : _ => {}
511 : }
512 : // Send reply to each message to keep connection alive. Ideally we
513 : // should do that once in a while instead, but this again requires
514 : // stream split or similar workaround, and recovery is anyway not that
515 : // performance critical.
516 : //
517 : // We do not know here real write/flush LSNs (need to take mutex again
518 : // or check replies which are read in different future), but neither
519 : // sender much cares about them, so just send last received.
520 0 : physical_stream
521 0 : .as_mut()
522 0 : .standby_status_update(
523 0 : PgLsn::from(last_received_lsn.0),
524 0 : PgLsn::from(last_received_lsn.0),
525 0 : PgLsn::from(last_received_lsn.0),
526 0 : SystemTime::now(),
527 0 : 0,
528 0 : )
529 0 : .await?;
530 : }
531 0 : }
532 :
533 : // Read replies from WalAcceptor. We are not interested much in sending them to
534 : // donor safekeeper, so don't route them anywhere. However, we should check if
535 : // term changes and exit if it does.
536 : // Returns Ok(()) if channel closed, Err in case of term change.
537 0 : async fn read_replies(
538 0 : mut reply_rx: Receiver<AcceptorProposerMessage>,
539 0 : donor_term: Term,
540 0 : ) -> anyhow::Result<()> {
541 : loop {
542 0 : match reply_rx.recv().await {
543 0 : Some(msg) => {
544 0 : if let AcceptorProposerMessage::AppendResponse(ar) = msg {
545 0 : if ar.term != donor_term {
546 0 : bail!("donor term changed from {} to {}", donor_term, ar.term);
547 0 : }
548 0 : }
549 : }
550 0 : None => return Ok(()), // chan closed, WalAcceptor terminated
551 : }
552 : }
553 0 : }
|