Line data Source code
1 : //! Safekeeper communication endpoint to WAL proposer (compute node).
2 : //! Gets messages from the network, passes them down to consensus module and
3 : //! sends replies back.
4 :
5 : use crate::handler::SafekeeperPostgresHandler;
6 : use crate::metrics::{
7 : WAL_RECEIVERS, WAL_RECEIVER_QUEUE_DEPTH, WAL_RECEIVER_QUEUE_DEPTH_TOTAL,
8 : WAL_RECEIVER_QUEUE_SIZE_TOTAL,
9 : };
10 : use crate::safekeeper::AcceptorProposerMessage;
11 : use crate::safekeeper::ProposerAcceptorMessage;
12 : use crate::timeline::WalResidentTimeline;
13 : use crate::GlobalTimelines;
14 : use anyhow::{anyhow, Context};
15 : use bytes::BytesMut;
16 : use parking_lot::MappedMutexGuard;
17 : use parking_lot::Mutex;
18 : use parking_lot::MutexGuard;
19 : use postgres_backend::CopyStreamHandlerEnd;
20 : use postgres_backend::PostgresBackend;
21 : use postgres_backend::PostgresBackendReader;
22 : use postgres_backend::QueryError;
23 : use pq_proto::BeMessage;
24 : use safekeeper_api::models::{ConnectionId, WalReceiverState, WalReceiverStatus};
25 : use safekeeper_api::ServerInfo;
26 : use std::future;
27 : use std::net::SocketAddr;
28 : use std::sync::Arc;
29 : use tokio::io::AsyncRead;
30 : use tokio::io::AsyncWrite;
31 : use tokio::sync::mpsc::error::SendTimeoutError;
32 : use tokio::sync::mpsc::{channel, Receiver, Sender};
33 : use tokio::task;
34 : use tokio::task::JoinHandle;
35 : use tokio::time::{Duration, Instant, MissedTickBehavior};
36 : use tracing::*;
37 : use utils::id::TenantTimelineId;
38 : use utils::lsn::Lsn;
39 : use utils::pageserver_feedback::PageserverFeedback;
40 :
41 : const DEFAULT_FEEDBACK_CAPACITY: usize = 8;
42 :
43 : /// Registry of WalReceivers (compute connections). Timeline holds it (wrapped
44 : /// in Arc).
45 : pub struct WalReceivers {
46 : mutex: Mutex<WalReceiversShared>,
47 : pageserver_feedback_tx: tokio::sync::broadcast::Sender<PageserverFeedback>,
48 :
49 : num_computes_tx: tokio::sync::watch::Sender<usize>,
50 : num_computes_rx: tokio::sync::watch::Receiver<usize>,
51 : }
52 :
53 : /// Id under which walreceiver is registered in shmem.
54 : type WalReceiverId = usize;
55 :
56 : impl WalReceivers {
57 0 : pub fn new() -> Arc<WalReceivers> {
58 0 : let (pageserver_feedback_tx, _) =
59 0 : tokio::sync::broadcast::channel(DEFAULT_FEEDBACK_CAPACITY);
60 0 :
61 0 : let (num_computes_tx, num_computes_rx) = tokio::sync::watch::channel(0usize);
62 0 :
63 0 : Arc::new(WalReceivers {
64 0 : mutex: Mutex::new(WalReceiversShared { slots: Vec::new() }),
65 0 : pageserver_feedback_tx,
66 0 : num_computes_tx,
67 0 : num_computes_rx,
68 0 : })
69 0 : }
70 :
71 : /// Register new walreceiver. Returned guard provides access to the slot and
72 : /// automatically deregisters in Drop.
73 0 : pub fn register(self: &Arc<WalReceivers>, conn_id: Option<ConnectionId>) -> WalReceiverGuard {
74 0 : let mut shared = self.mutex.lock();
75 0 : let slots = &mut shared.slots;
76 0 : let walreceiver = WalReceiverState {
77 0 : conn_id,
78 0 : status: WalReceiverStatus::Voting,
79 0 : };
80 : // find empty slot or create new one
81 0 : let pos = if let Some(pos) = slots.iter().position(|s| s.is_none()) {
82 0 : slots[pos] = Some(walreceiver);
83 0 : pos
84 : } else {
85 0 : let pos = slots.len();
86 0 : slots.push(Some(walreceiver));
87 0 : pos
88 : };
89 :
90 0 : self.update_num(&shared);
91 0 : WAL_RECEIVERS.inc();
92 0 :
93 0 : WalReceiverGuard {
94 0 : id: pos,
95 0 : walreceivers: self.clone(),
96 0 : }
97 0 : }
98 :
99 : /// Get reference to locked slot contents. Slot must exist (registered
100 : /// earlier).
101 0 : fn get_slot<'a>(
102 0 : self: &'a Arc<WalReceivers>,
103 0 : id: WalReceiverId,
104 0 : ) -> MappedMutexGuard<'a, WalReceiverState> {
105 0 : MutexGuard::map(self.mutex.lock(), |locked| {
106 0 : locked.slots[id]
107 0 : .as_mut()
108 0 : .expect("walreceiver doesn't exist")
109 0 : })
110 0 : }
111 :
112 : /// Get number of walreceivers (compute connections).
113 0 : pub fn get_num(self: &Arc<WalReceivers>) -> usize {
114 0 : self.mutex.lock().get_num()
115 0 : }
116 :
117 : /// Get channel for number of walreceivers.
118 0 : pub fn get_num_rx(self: &Arc<WalReceivers>) -> tokio::sync::watch::Receiver<usize> {
119 0 : self.num_computes_rx.clone()
120 0 : }
121 :
122 : /// Should get called after every update of slots.
123 0 : fn update_num(self: &Arc<WalReceivers>, shared: &MutexGuard<WalReceiversShared>) {
124 0 : let num = shared.get_num();
125 0 : self.num_computes_tx.send_replace(num);
126 0 : }
127 :
128 : /// Get state of all walreceivers.
129 0 : pub fn get_all(self: &Arc<WalReceivers>) -> Vec<WalReceiverState> {
130 0 : self.mutex.lock().slots.iter().flatten().cloned().collect()
131 0 : }
132 :
133 : /// Get number of streaming walreceivers (normally 0 or 1) from compute.
134 0 : pub fn get_num_streaming(self: &Arc<WalReceivers>) -> usize {
135 0 : self.mutex
136 0 : .lock()
137 0 : .slots
138 0 : .iter()
139 0 : .flatten()
140 0 : // conn_id.is_none skips recovery which also registers here
141 0 : .filter(|s| s.conn_id.is_some() && matches!(s.status, WalReceiverStatus::Streaming))
142 0 : .count()
143 0 : }
144 :
145 : /// Unregister walreceiver.
146 0 : fn unregister(self: &Arc<WalReceivers>, id: WalReceiverId) {
147 0 : let mut shared = self.mutex.lock();
148 0 : shared.slots[id] = None;
149 0 : self.update_num(&shared);
150 0 : WAL_RECEIVERS.dec();
151 0 : }
152 :
153 : /// Broadcast pageserver feedback to connected walproposers.
154 0 : pub fn broadcast_pageserver_feedback(&self, feedback: PageserverFeedback) {
155 0 : // Err means there is no subscribers, it is fine.
156 0 : let _ = self.pageserver_feedback_tx.send(feedback);
157 0 : }
158 : }
159 :
160 : /// Only a few connections are expected (normally one), so store in Vec.
161 : struct WalReceiversShared {
162 : slots: Vec<Option<WalReceiverState>>,
163 : }
164 :
165 : impl WalReceiversShared {
166 : /// Get number of walreceivers (compute connections).
167 0 : fn get_num(&self) -> usize {
168 0 : self.slots.iter().flatten().count()
169 0 : }
170 : }
171 :
172 : /// Scope guard to access slot in WalReceivers registry and unregister from
173 : /// it in Drop.
174 : pub struct WalReceiverGuard {
175 : id: WalReceiverId,
176 : walreceivers: Arc<WalReceivers>,
177 : }
178 :
179 : impl WalReceiverGuard {
180 : /// Get reference to locked shared state contents.
181 0 : fn get(&self) -> MappedMutexGuard<WalReceiverState> {
182 0 : self.walreceivers.get_slot(self.id)
183 0 : }
184 : }
185 :
186 : impl Drop for WalReceiverGuard {
187 0 : fn drop(&mut self) {
188 0 : self.walreceivers.unregister(self.id);
189 0 : }
190 : }
191 :
192 : pub const MSG_QUEUE_SIZE: usize = 256;
193 : pub const REPLY_QUEUE_SIZE: usize = 16;
194 :
195 : impl SafekeeperPostgresHandler {
196 : /// Wrapper around handle_start_wal_push_guts handling result. Error is
197 : /// handled here while we're still in walreceiver ttid span; with API
198 : /// extension, this can probably be moved into postgres_backend.
199 0 : pub async fn handle_start_wal_push<IO: AsyncRead + AsyncWrite + Unpin>(
200 0 : &mut self,
201 0 : pgb: &mut PostgresBackend<IO>,
202 0 : ) -> Result<(), QueryError> {
203 0 : let mut tli: Option<WalResidentTimeline> = None;
204 0 : if let Err(end) = self.handle_start_wal_push_guts(pgb, &mut tli).await {
205 : // Log the result and probably send it to the client, closing the stream.
206 0 : let handle_end_fut = pgb.handle_copy_stream_end(end);
207 : // If we managed to create the timeline, augment logging with current LSNs etc.
208 0 : if let Some(tli) = tli {
209 0 : let info = tli.get_safekeeper_info(&self.conf).await;
210 0 : handle_end_fut
211 0 : .instrument(info_span!("", term=%info.term, last_log_term=%info.last_log_term, flush_lsn=%Lsn(info.flush_lsn), commit_lsn=%Lsn(info.commit_lsn)))
212 0 : .await;
213 : } else {
214 0 : handle_end_fut.await;
215 : }
216 0 : }
217 0 : Ok(())
218 0 : }
219 :
220 0 : pub async fn handle_start_wal_push_guts<IO: AsyncRead + AsyncWrite + Unpin>(
221 0 : &mut self,
222 0 : pgb: &mut PostgresBackend<IO>,
223 0 : tli: &mut Option<WalResidentTimeline>,
224 0 : ) -> Result<(), CopyStreamHandlerEnd> {
225 0 : // The `tli` parameter is only used for passing _out_ a timeline, one should
226 0 : // not have been passed in.
227 0 : assert!(tli.is_none());
228 :
229 : // Notify the libpq client that it's allowed to send `CopyData` messages
230 0 : pgb.write_message(&BeMessage::CopyBothResponse).await?;
231 :
232 : // Experiments [1] confirm that doing network IO in one (this) thread and
233 : // processing with disc IO in another significantly improves
234 : // performance; we spawn off WalAcceptor thread for message processing
235 : // to this end.
236 : //
237 : // [1] https://github.com/neondatabase/neon/pull/1318
238 0 : let (msg_tx, msg_rx) = channel(MSG_QUEUE_SIZE);
239 0 : let (reply_tx, reply_rx) = channel(REPLY_QUEUE_SIZE);
240 0 : let mut acceptor_handle: Option<JoinHandle<anyhow::Result<()>>> = None;
241 :
242 : // Concurrently receive and send data; replies are not synchronized with
243 : // sends, so this avoids deadlocks.
244 0 : let mut pgb_reader = pgb.split().context("START_WAL_PUSH split")?;
245 0 : let peer_addr = *pgb.get_peer_addr();
246 0 :
247 0 : let mut network_reader = NetworkReader {
248 0 : ttid: self.ttid,
249 0 : conn_id: self.conn_id,
250 0 : pgb_reader: &mut pgb_reader,
251 0 : peer_addr,
252 0 : acceptor_handle: &mut acceptor_handle,
253 0 : global_timelines: self.global_timelines.clone(),
254 0 : };
255 :
256 : // Read first message and create timeline if needed.
257 0 : let res = network_reader.read_first_message().await;
258 :
259 0 : let network_res = if let Ok((timeline, next_msg)) = res {
260 0 : let pageserver_feedback_rx: tokio::sync::broadcast::Receiver<PageserverFeedback> =
261 0 : timeline
262 0 : .get_walreceivers()
263 0 : .pageserver_feedback_tx
264 0 : .subscribe();
265 0 : *tli = Some(timeline.wal_residence_guard().await?);
266 :
267 0 : let timeline_cancel = timeline.cancel.clone();
268 0 : tokio::select! {
269 : // todo: add read|write .context to these errors
270 0 : r = network_reader.run(msg_tx, msg_rx, reply_tx, timeline, next_msg) => r,
271 0 : r = network_write(pgb, reply_rx, pageserver_feedback_rx) => r,
272 0 : _ = timeline_cancel.cancelled() => {
273 0 : return Err(CopyStreamHandlerEnd::Cancelled);
274 : }
275 : }
276 : } else {
277 0 : res.map(|_| ())
278 : };
279 :
280 : // Join pg backend back.
281 0 : pgb.unsplit(pgb_reader)?;
282 :
283 : // Join the spawned WalAcceptor. At this point chans to/from it passed
284 : // to network routines are dropped, so it will exit as soon as it
285 : // touches them.
286 0 : match acceptor_handle {
287 : None => {
288 : // failed even before spawning; read_network should have error
289 0 : Err(network_res.expect_err("no error with WalAcceptor not spawn"))
290 : }
291 0 : Some(handle) => {
292 0 : let wal_acceptor_res = handle.await;
293 :
294 : // If there was any network error, return it.
295 0 : network_res?;
296 :
297 : // Otherwise, WalAcceptor thread must have errored.
298 0 : match wal_acceptor_res {
299 0 : Ok(Ok(_)) => Ok(()), // Clean shutdown
300 0 : Ok(Err(e)) => Err(CopyStreamHandlerEnd::Other(e.context("WAL acceptor"))),
301 0 : Err(_) => Err(CopyStreamHandlerEnd::Other(anyhow!(
302 0 : "WalAcceptor task panicked",
303 0 : ))),
304 : }
305 : }
306 : }
307 0 : }
308 : }
309 :
310 : struct NetworkReader<'a, IO> {
311 : ttid: TenantTimelineId,
312 : conn_id: ConnectionId,
313 : pgb_reader: &'a mut PostgresBackendReader<IO>,
314 : peer_addr: SocketAddr,
315 : // WalAcceptor is spawned when we learn server info from walproposer and
316 : // create timeline; handle is put here.
317 : acceptor_handle: &'a mut Option<JoinHandle<anyhow::Result<()>>>,
318 : global_timelines: Arc<GlobalTimelines>,
319 : }
320 :
321 : impl<IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'_, IO> {
322 0 : async fn read_first_message(
323 0 : &mut self,
324 0 : ) -> Result<(WalResidentTimeline, ProposerAcceptorMessage), CopyStreamHandlerEnd> {
325 : // Receive information about server to create timeline, if not yet.
326 0 : let next_msg = read_message(self.pgb_reader).await?;
327 0 : let tli = match next_msg {
328 0 : ProposerAcceptorMessage::Greeting(ref greeting) => {
329 0 : info!(
330 0 : "start handshake with walproposer {} sysid {} timeline {}",
331 : self.peer_addr, greeting.system_id, greeting.tli,
332 : );
333 0 : let server_info = ServerInfo {
334 0 : pg_version: greeting.pg_version,
335 0 : system_id: greeting.system_id,
336 0 : wal_seg_size: greeting.wal_seg_size,
337 0 : };
338 0 : let tli = self
339 0 : .global_timelines
340 0 : .create(self.ttid, server_info, Lsn::INVALID, Lsn::INVALID)
341 0 : .await
342 0 : .context("create timeline")?;
343 0 : tli.wal_residence_guard().await?
344 : }
345 : _ => {
346 0 : return Err(CopyStreamHandlerEnd::Other(anyhow::anyhow!(
347 0 : "unexpected message {next_msg:?} instead of greeting"
348 0 : )))
349 : }
350 : };
351 0 : Ok((tli, next_msg))
352 0 : }
353 :
354 : /// This function is cancellation-safe (only does network I/O and channel read/writes).
355 0 : async fn run(
356 0 : self,
357 0 : msg_tx: Sender<ProposerAcceptorMessage>,
358 0 : msg_rx: Receiver<ProposerAcceptorMessage>,
359 0 : reply_tx: Sender<AcceptorProposerMessage>,
360 0 : tli: WalResidentTimeline,
361 0 : next_msg: ProposerAcceptorMessage,
362 0 : ) -> Result<(), CopyStreamHandlerEnd> {
363 0 : *self.acceptor_handle = Some(WalAcceptor::spawn(
364 0 : tli,
365 0 : msg_rx,
366 0 : reply_tx,
367 0 : Some(self.conn_id),
368 0 : ));
369 0 :
370 0 : // Forward all messages to WalAcceptor
371 0 : read_network_loop(self.pgb_reader, msg_tx, next_msg).await
372 0 : }
373 : }
374 :
375 : /// Read next message from walproposer.
376 : /// TODO: Return Ok(None) on graceful termination.
377 0 : async fn read_message<IO: AsyncRead + AsyncWrite + Unpin>(
378 0 : pgb_reader: &mut PostgresBackendReader<IO>,
379 0 : ) -> Result<ProposerAcceptorMessage, CopyStreamHandlerEnd> {
380 0 : let copy_data = pgb_reader.read_copy_message().await?;
381 0 : let msg = ProposerAcceptorMessage::parse(copy_data)?;
382 0 : Ok(msg)
383 0 : }
384 :
385 0 : async fn read_network_loop<IO: AsyncRead + AsyncWrite + Unpin>(
386 0 : pgb_reader: &mut PostgresBackendReader<IO>,
387 0 : msg_tx: Sender<ProposerAcceptorMessage>,
388 0 : mut next_msg: ProposerAcceptorMessage,
389 0 : ) -> Result<(), CopyStreamHandlerEnd> {
390 : /// Threshold for logging slow WalAcceptor sends.
391 : const SLOW_THRESHOLD: Duration = Duration::from_secs(5);
392 :
393 : loop {
394 0 : let started = Instant::now();
395 0 : let size = next_msg.size();
396 0 :
397 0 : match msg_tx.send_timeout(next_msg, SLOW_THRESHOLD).await {
398 0 : Ok(()) => {}
399 : // Slow send, log a message and keep trying. Log context has timeline ID.
400 0 : Err(SendTimeoutError::Timeout(next_msg)) => {
401 0 : warn!(
402 0 : "slow WalAcceptor send blocked for {:.3}s",
403 0 : Instant::now().duration_since(started).as_secs_f64()
404 : );
405 0 : if msg_tx.send(next_msg).await.is_err() {
406 0 : return Ok(()); // WalAcceptor terminated
407 0 : }
408 0 : warn!(
409 0 : "slow WalAcceptor send completed after {:.3}s",
410 0 : Instant::now().duration_since(started).as_secs_f64()
411 : )
412 : }
413 : // WalAcceptor terminated.
414 0 : Err(SendTimeoutError::Closed(_)) => return Ok(()),
415 : }
416 :
417 : // Update metrics. Will be decremented in WalAcceptor.
418 0 : WAL_RECEIVER_QUEUE_DEPTH_TOTAL.inc();
419 0 : WAL_RECEIVER_QUEUE_SIZE_TOTAL.add(size as i64);
420 :
421 0 : next_msg = read_message(pgb_reader).await?;
422 : }
423 0 : }
424 :
425 : /// Read replies from WalAcceptor and pass them back to socket. Returns Ok(())
426 : /// if reply_rx closed; it must mean WalAcceptor terminated, joining it should
427 : /// tell the error.
428 : ///
429 : /// This function is cancellation-safe (only does network I/O and channel read/writes).
430 0 : async fn network_write<IO: AsyncRead + AsyncWrite + Unpin>(
431 0 : pgb_writer: &mut PostgresBackend<IO>,
432 0 : mut reply_rx: Receiver<AcceptorProposerMessage>,
433 0 : mut pageserver_feedback_rx: tokio::sync::broadcast::Receiver<PageserverFeedback>,
434 0 : ) -> Result<(), CopyStreamHandlerEnd> {
435 0 : let mut buf = BytesMut::with_capacity(128);
436 0 :
437 0 : // storing append_response to inject PageserverFeedback into it
438 0 : let mut last_append_response = None;
439 :
440 : loop {
441 : // trying to read either AcceptorProposerMessage or PageserverFeedback
442 0 : let msg = tokio::select! {
443 0 : reply = reply_rx.recv() => {
444 0 : if let Some(msg) = reply {
445 0 : if let AcceptorProposerMessage::AppendResponse(append_response) = &msg {
446 0 : last_append_response = Some(append_response.clone());
447 0 : }
448 0 : Some(msg)
449 : } else {
450 0 : return Ok(()); // chan closed, WalAcceptor terminated
451 : }
452 : }
453 :
454 0 : feedback = pageserver_feedback_rx.recv() =>
455 0 : match (feedback, &last_append_response) {
456 0 : (Ok(feedback), Some(append_response)) => {
457 0 : // clone AppendResponse and inject PageserverFeedback into it
458 0 : let mut append_response = append_response.clone();
459 0 : append_response.pageserver_feedback = Some(feedback);
460 0 : Some(AcceptorProposerMessage::AppendResponse(append_response))
461 : }
462 0 : _ => None,
463 : },
464 : };
465 :
466 0 : let Some(msg) = msg else {
467 0 : continue;
468 : };
469 :
470 0 : buf.clear();
471 0 : msg.serialize(&mut buf)?;
472 0 : pgb_writer.write_message(&BeMessage::CopyData(&buf)).await?;
473 : }
474 0 : }
475 :
476 : /// The WAL flush interval. This ensures we periodically flush the WAL and send AppendResponses to
477 : /// walproposer, even when it's writing a steady stream of messages.
478 : const FLUSH_INTERVAL: Duration = Duration::from_secs(1);
479 :
480 : /// The metrics computation interval.
481 : ///
482 : /// The Prometheus poll interval is 60 seconds at the time of writing. We sample the queue depth
483 : /// every 5 seconds, for 12 samples per poll. This will give a count of up to 12x active timelines.
484 : const METRICS_INTERVAL: Duration = Duration::from_secs(5);
485 :
486 : /// Encapsulates a task which takes messages from msg_rx, processes and pushes
487 : /// replies to reply_tx.
488 : ///
489 : /// Reading from socket and writing to disk in parallel is beneficial for
490 : /// performance, this struct provides the writing to disk part.
491 : pub struct WalAcceptor {
492 : tli: WalResidentTimeline,
493 : msg_rx: Receiver<ProposerAcceptorMessage>,
494 : reply_tx: Sender<AcceptorProposerMessage>,
495 : conn_id: Option<ConnectionId>,
496 : }
497 :
498 : impl WalAcceptor {
499 : /// Spawn task with WalAcceptor running, return handle to it. Task returns
500 : /// Ok(()) if either of channels has closed, and Err if any error during
501 : /// message processing is encountered.
502 : ///
503 : /// conn_id None means WalAcceptor is used by recovery initiated at this safekeeper.
504 0 : pub fn spawn(
505 0 : tli: WalResidentTimeline,
506 0 : msg_rx: Receiver<ProposerAcceptorMessage>,
507 0 : reply_tx: Sender<AcceptorProposerMessage>,
508 0 : conn_id: Option<ConnectionId>,
509 0 : ) -> JoinHandle<anyhow::Result<()>> {
510 0 : task::spawn(async move {
511 0 : let mut wa = WalAcceptor {
512 0 : tli,
513 0 : msg_rx,
514 0 : reply_tx,
515 0 : conn_id,
516 0 : };
517 0 :
518 0 : let span_ttid = wa.tli.ttid; // satisfy borrow checker
519 0 : wa.run()
520 0 : .instrument(
521 0 : info_span!("WAL acceptor", cid = %conn_id.unwrap_or(0), ttid = %span_ttid),
522 : )
523 0 : .await
524 0 : })
525 0 : }
526 :
527 : /// The main loop. Returns Ok(()) if either msg_rx or reply_tx got closed;
528 : /// it must mean that network thread terminated.
529 : ///
530 : /// This function is *not* cancellation safe, it does local disk I/O: it should always
531 : /// be allowed to run to completion. It respects Timeline::cancel and shuts down cleanly
532 : /// when that gets triggered.
533 0 : async fn run(&mut self) -> anyhow::Result<()> {
534 0 : let walreceiver_guard = self.tli.get_walreceivers().register(self.conn_id);
535 0 :
536 0 : // Periodically flush the WAL and compute metrics.
537 0 : let mut flush_ticker = tokio::time::interval(FLUSH_INTERVAL);
538 0 : flush_ticker.set_missed_tick_behavior(MissedTickBehavior::Delay);
539 0 : flush_ticker.tick().await; // skip the initial, immediate tick
540 :
541 0 : let mut metrics_ticker = tokio::time::interval(METRICS_INTERVAL);
542 0 : metrics_ticker.set_missed_tick_behavior(MissedTickBehavior::Skip);
543 0 :
544 0 : // Tracks whether we have unflushed appends.
545 0 : let mut dirty = false;
546 :
547 0 : while !self.tli.is_cancelled() {
548 0 : let reply = tokio::select! {
549 : // Process inbound message.
550 0 : msg = self.msg_rx.recv() => {
551 : // If disconnected, break to flush WAL and return.
552 0 : let Some(mut msg) = msg else {
553 0 : break;
554 : };
555 :
556 : // Update gauge metrics.
557 0 : WAL_RECEIVER_QUEUE_DEPTH_TOTAL.dec();
558 0 : WAL_RECEIVER_QUEUE_SIZE_TOTAL.sub(msg.size() as i64);
559 0 :
560 0 : // Update walreceiver state in shmem for reporting.
561 0 : if let ProposerAcceptorMessage::Elected(_) = &msg {
562 0 : walreceiver_guard.get().status = WalReceiverStatus::Streaming;
563 0 : }
564 :
565 : // Don't flush the WAL on every append, only periodically via flush_ticker.
566 : // This batches multiple appends per fsync. If the channel is empty after
567 : // sending the reply, we'll schedule an immediate flush.
568 : //
569 : // Note that a flush can still happen on segment bounds, which will result
570 : // in an AppendResponse.
571 0 : if let ProposerAcceptorMessage::AppendRequest(append_request) = msg {
572 0 : msg = ProposerAcceptorMessage::NoFlushAppendRequest(append_request);
573 0 : dirty = true;
574 0 : }
575 :
576 0 : self.tli.process_msg(&msg).await?
577 : }
578 :
579 : // While receiving AppendRequests, flush the WAL periodically and respond with an
580 : // AppendResponse to let walproposer know we're still alive.
581 0 : _ = flush_ticker.tick(), if dirty => {
582 0 : dirty = false;
583 0 : self.tli
584 0 : .process_msg(&ProposerAcceptorMessage::FlushWAL)
585 0 : .await?
586 : }
587 :
588 : // If there are no pending messages, flush the WAL immediately.
589 : //
590 : // TODO: this should be done via flush_ticker.reset_immediately(), but that's always
591 : // delayed by 1ms due to this bug: https://github.com/tokio-rs/tokio/issues/6866.
592 0 : _ = future::ready(()), if dirty && self.msg_rx.is_empty() => {
593 0 : dirty = false;
594 0 : flush_ticker.reset();
595 0 : self.tli
596 0 : .process_msg(&ProposerAcceptorMessage::FlushWAL)
597 0 : .await?
598 : }
599 :
600 : // Update histogram metrics periodically.
601 0 : _ = metrics_ticker.tick() => {
602 0 : WAL_RECEIVER_QUEUE_DEPTH.observe(self.msg_rx.len() as f64);
603 0 : None // no reply
604 : }
605 :
606 0 : _ = self.tli.cancel.cancelled() => {
607 0 : break;
608 : }
609 : };
610 :
611 : // Send reply, if any.
612 0 : if let Some(reply) = reply {
613 0 : if self.reply_tx.send(reply).await.is_err() {
614 0 : break; // disconnected, break to flush WAL and return
615 0 : }
616 0 : }
617 : }
618 :
619 : // Flush WAL on disconnect, see https://github.com/neondatabase/neon/issues/9259.
620 0 : if dirty && !self.tli.cancel.is_cancelled() {
621 0 : self.tli
622 0 : .process_msg(&ProposerAcceptorMessage::FlushWAL)
623 0 : .await?;
624 0 : }
625 :
626 0 : Ok(())
627 0 : }
628 : }
629 :
630 : /// On drop, drain msg_rx and update metrics to avoid leaks.
631 : impl Drop for WalAcceptor {
632 0 : fn drop(&mut self) {
633 0 : self.msg_rx.close(); // prevent further sends
634 0 : while let Ok(msg) = self.msg_rx.try_recv() {
635 0 : WAL_RECEIVER_QUEUE_DEPTH_TOTAL.dec();
636 0 : WAL_RECEIVER_QUEUE_SIZE_TOTAL.sub(msg.size() as i64);
637 0 : }
638 0 : }
639 : }
|