Line data Source code
1 : //! The Page Service listens for client connections and serves their GetPage@LSN
2 : //! requests.
3 :
4 : use std::any::Any;
5 : use std::borrow::Cow;
6 : use std::num::NonZeroUsize;
7 : use std::os::fd::AsRawFd;
8 : use std::pin::Pin;
9 : use std::str::FromStr;
10 : use std::sync::Arc;
11 : use std::task::{Context, Poll};
12 : use std::time::{Duration, Instant, SystemTime};
13 : use std::{io, str};
14 :
15 : use anyhow::{Context as _, bail};
16 : use bytes::{Buf as _, BufMut as _, BytesMut};
17 : use chrono::Utc;
18 : use futures::future::BoxFuture;
19 : use futures::{FutureExt, Stream};
20 : use itertools::Itertools;
21 : use jsonwebtoken::TokenData;
22 : use once_cell::sync::OnceCell;
23 : use pageserver_api::config::{
24 : GetVectoredConcurrentIo, PageServicePipeliningConfig, PageServicePipeliningConfigPipelined,
25 : PageServiceProtocolPipelinedBatchingStrategy, PageServiceProtocolPipelinedExecutionStrategy,
26 : };
27 : use pageserver_api::key::rel_block_to_key;
28 : use pageserver_api::models::{PageTraceEvent, TenantState};
29 : use pageserver_api::pagestream_api::{
30 : self, PagestreamBeMessage, PagestreamDbSizeRequest, PagestreamDbSizeResponse,
31 : PagestreamErrorResponse, PagestreamExistsRequest, PagestreamExistsResponse,
32 : PagestreamFeMessage, PagestreamGetPageRequest, PagestreamGetSlruSegmentRequest,
33 : PagestreamGetSlruSegmentResponse, PagestreamNblocksRequest, PagestreamNblocksResponse,
34 : PagestreamProtocolVersion, PagestreamRequest,
35 : };
36 : use pageserver_api::reltag::SlruKind;
37 : use pageserver_api::shard::TenantShardId;
38 : use pageserver_page_api as page_api;
39 : use pageserver_page_api::proto;
40 : use postgres_backend::{
41 : AuthType, PostgresBackend, PostgresBackendReader, QueryError, is_expected_io_error,
42 : };
43 : use postgres_ffi::BLCKSZ;
44 : use postgres_ffi_types::constants::DEFAULTTABLESPACE_OID;
45 : use pq_proto::framed::ConnectionError;
46 : use pq_proto::{BeMessage, FeMessage, FeStartupPacket, RowDescriptor};
47 : use smallvec::{SmallVec, smallvec};
48 : use strum_macros::IntoStaticStr;
49 : use tokio::io::{AsyncRead, AsyncReadExt as _, AsyncWrite, AsyncWriteExt as _, BufWriter};
50 : use tokio::task::JoinHandle;
51 : use tokio_util::sync::CancellationToken;
52 : use tonic::service::Interceptor as _;
53 : use tonic::transport::server::TcpConnectInfo;
54 : use tracing::*;
55 : use utils::auth::{Claims, Scope, SwappableJwtAuth};
56 : use utils::id::{TenantId, TenantTimelineId, TimelineId};
57 : use utils::logging::log_slow;
58 : use utils::lsn::Lsn;
59 : use utils::shard::ShardIndex;
60 : use utils::simple_rcu::RcuReadGuard;
61 : use utils::sync::gate::{Gate, GateGuard};
62 : use utils::sync::spsc_fold;
63 : use utils::{failpoint_support, span_record};
64 :
65 : use crate::auth::check_permission;
66 : use crate::basebackup::{self, BasebackupError};
67 : use crate::config::PageServerConf;
68 : use crate::context::{
69 : DownloadBehavior, PerfInstrumentFutureExt, RequestContext, RequestContextBuilder,
70 : };
71 : use crate::metrics::{
72 : self, COMPUTE_COMMANDS_COUNTERS, ComputeCommandKind, GetPageBatchBreakReason, LIVE_CONNECTIONS,
73 : SmgrOpTimer, TimelineMetrics,
74 : };
75 : use crate::pgdatadir_mapping::{LsnRange, Version};
76 : use crate::span::{
77 : debug_assert_current_span_has_tenant_and_timeline_id,
78 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id,
79 : };
80 : use crate::task_mgr::{self, COMPUTE_REQUEST_RUNTIME, TaskKind};
81 : use crate::tenant::mgr::{
82 : GetActiveTenantError, GetTenantError, ShardResolveResult, ShardSelector, TenantManager,
83 : };
84 : use crate::tenant::storage_layer::IoConcurrency;
85 : use crate::tenant::timeline::handle::{Handle, HandleUpgradeError, WeakHandle};
86 : use crate::tenant::timeline::{self, WaitLsnError, WaitLsnTimeout, WaitLsnWaiter};
87 : use crate::tenant::{GetTimelineError, PageReconstructError, Timeline};
88 : use crate::{CancellableTask, PERF_TRACE_TARGET, timed_after_cancellation};
89 :
90 : /// How long we may wait for a [`crate::tenant::mgr::TenantSlot::InProgress`]` and/or a [`crate::tenant::TenantShard`] which
91 : /// is not yet in state [`TenantState::Active`].
92 : ///
93 : /// NB: this is a different value than [`crate::http::routes::ACTIVE_TENANT_TIMEOUT`].
94 : const ACTIVE_TENANT_TIMEOUT: Duration = Duration::from_millis(30000);
95 :
96 : /// Threshold at which to log slow GetPage requests.
97 : const LOG_SLOW_GETPAGE_THRESHOLD: Duration = Duration::from_secs(30);
98 :
99 : /// The idle time before sending TCP keepalive probes for gRPC connections. The
100 : /// interval and timeout between each probe is configured via sysctl. This
101 : /// allows detecting dead connections sooner.
102 : const GRPC_TCP_KEEPALIVE_TIME: Duration = Duration::from_secs(60);
103 :
104 : /// Whether to enable TCP nodelay for gRPC connections. This disables Nagle's
105 : /// algorithm, which can cause latency spikes for small messages.
106 : const GRPC_TCP_NODELAY: bool = true;
107 :
108 : /// The interval between HTTP2 keepalive pings. This allows shutting down server
109 : /// tasks when clients are unresponsive.
110 : const GRPC_HTTP2_KEEPALIVE_INTERVAL: Duration = Duration::from_secs(30);
111 :
112 : /// The timeout for HTTP2 keepalive pings. Should be <= GRPC_KEEPALIVE_INTERVAL.
113 : const GRPC_HTTP2_KEEPALIVE_TIMEOUT: Duration = Duration::from_secs(20);
114 :
115 : /// Number of concurrent gRPC streams per TCP connection. We expect something
116 : /// like 8 GetPage streams per connections, plus any unary requests.
117 : const GRPC_MAX_CONCURRENT_STREAMS: u32 = 256;
118 :
119 : ///////////////////////////////////////////////////////////////////////////////
120 :
121 : pub struct Listener {
122 : cancel: CancellationToken,
123 : /// Cancel the listener task through `listen_cancel` to shut down the listener
124 : /// and get a handle on the existing connections.
125 : task: JoinHandle<Connections>,
126 : }
127 :
128 : pub struct Connections {
129 : cancel: CancellationToken,
130 : tasks: tokio::task::JoinSet<ConnectionHandlerResult>,
131 : gate: Gate,
132 : }
133 :
134 0 : pub fn spawn(
135 0 : conf: &'static PageServerConf,
136 0 : tenant_manager: Arc<TenantManager>,
137 0 : pg_auth: Option<Arc<SwappableJwtAuth>>,
138 0 : perf_trace_dispatch: Option<Dispatch>,
139 0 : tcp_listener: tokio::net::TcpListener,
140 0 : tls_config: Option<Arc<rustls::ServerConfig>>,
141 0 : ) -> Listener {
142 0 : let cancel = CancellationToken::new();
143 0 : let libpq_ctx = RequestContext::todo_child(
144 0 : TaskKind::LibpqEndpointListener,
145 : // listener task shouldn't need to download anything. (We will
146 : // create a separate sub-contexts for each connection, with their
147 : // own download behavior. This context is used only to listen and
148 : // accept connections.)
149 0 : DownloadBehavior::Error,
150 : );
151 0 : let task = COMPUTE_REQUEST_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
152 0 : "libpq listener",
153 0 : libpq_listener_main(
154 0 : conf,
155 0 : tenant_manager,
156 0 : pg_auth,
157 0 : perf_trace_dispatch,
158 0 : tcp_listener,
159 0 : conf.pg_auth_type,
160 0 : tls_config,
161 0 : conf.page_service_pipelining.clone(),
162 0 : libpq_ctx,
163 0 : cancel.clone(),
164 0 : )
165 0 : .map(anyhow::Ok),
166 0 : ));
167 :
168 0 : Listener { cancel, task }
169 0 : }
170 :
171 : impl Listener {
172 0 : pub async fn stop_accepting(self) -> Connections {
173 0 : self.cancel.cancel();
174 0 : self.task
175 0 : .await
176 0 : .expect("unreachable: we wrap the listener task in task_mgr::exit_on_panic_or_error")
177 0 : }
178 : }
179 : impl Connections {
180 0 : pub(crate) async fn shutdown(self) {
181 : let Self {
182 0 : cancel,
183 0 : mut tasks,
184 0 : gate,
185 0 : } = self;
186 0 : cancel.cancel();
187 0 : while let Some(res) = tasks.join_next().await {
188 0 : Self::handle_connection_completion(res);
189 0 : }
190 0 : gate.close().await;
191 0 : }
192 :
193 0 : fn handle_connection_completion(res: Result<anyhow::Result<()>, tokio::task::JoinError>) {
194 0 : match res {
195 0 : Ok(Ok(())) => {}
196 0 : Ok(Err(e)) => error!("error in page_service connection task: {:?}", e),
197 0 : Err(e) => error!("page_service connection task panicked: {:?}", e),
198 : }
199 0 : }
200 : }
201 :
202 : ///
203 : /// Main loop of the page service.
204 : ///
205 : /// Listens for connections, and launches a new handler task for each.
206 : ///
207 : /// Returns Ok(()) upon cancellation via `cancel`, returning the set of
208 : /// open connections.
209 : ///
210 : #[allow(clippy::too_many_arguments)]
211 0 : pub async fn libpq_listener_main(
212 0 : conf: &'static PageServerConf,
213 0 : tenant_manager: Arc<TenantManager>,
214 0 : auth: Option<Arc<SwappableJwtAuth>>,
215 0 : perf_trace_dispatch: Option<Dispatch>,
216 0 : listener: tokio::net::TcpListener,
217 0 : auth_type: AuthType,
218 0 : tls_config: Option<Arc<rustls::ServerConfig>>,
219 0 : pipelining_config: PageServicePipeliningConfig,
220 0 : listener_ctx: RequestContext,
221 0 : listener_cancel: CancellationToken,
222 0 : ) -> Connections {
223 0 : let connections_cancel = CancellationToken::new();
224 0 : let connections_gate = Gate::default();
225 0 : let mut connection_handler_tasks = tokio::task::JoinSet::default();
226 :
227 : loop {
228 0 : let gate_guard = match connections_gate.enter() {
229 0 : Ok(guard) => guard,
230 0 : Err(_) => break,
231 : };
232 :
233 0 : let accepted = tokio::select! {
234 : biased;
235 0 : _ = listener_cancel.cancelled() => break,
236 0 : next = connection_handler_tasks.join_next(), if !connection_handler_tasks.is_empty() => {
237 0 : let res = next.expect("we dont poll while empty");
238 0 : Connections::handle_connection_completion(res);
239 0 : continue;
240 : }
241 0 : accepted = listener.accept() => accepted,
242 : };
243 :
244 0 : match accepted {
245 0 : Ok((socket, peer_addr)) => {
246 : // Connection established. Spawn a new task to handle it.
247 0 : debug!("accepted connection from {}", peer_addr);
248 0 : let local_auth = auth.clone();
249 0 : let connection_ctx = RequestContextBuilder::from(&listener_ctx)
250 0 : .task_kind(TaskKind::PageRequestHandler)
251 0 : .download_behavior(DownloadBehavior::Download)
252 0 : .perf_span_dispatch(perf_trace_dispatch.clone())
253 0 : .detached_child();
254 :
255 0 : connection_handler_tasks.spawn(page_service_conn_main(
256 0 : conf,
257 0 : tenant_manager.clone(),
258 0 : local_auth,
259 0 : socket,
260 0 : auth_type,
261 0 : tls_config.clone(),
262 0 : pipelining_config.clone(),
263 0 : connection_ctx,
264 0 : connections_cancel.child_token(),
265 0 : gate_guard,
266 : ));
267 : }
268 0 : Err(err) => {
269 : // accept() failed. Log the error, and loop back to retry on next connection.
270 0 : error!("accept() failed: {:?}", err);
271 : }
272 : }
273 : }
274 :
275 0 : debug!("page_service listener loop terminated");
276 :
277 0 : Connections {
278 0 : cancel: connections_cancel,
279 0 : tasks: connection_handler_tasks,
280 0 : gate: connections_gate,
281 0 : }
282 0 : }
283 :
284 : type ConnectionHandlerResult = anyhow::Result<()>;
285 :
286 : /// Perf root spans start at the per-request level, after shard routing.
287 : /// This struct carries connection-level information to the root perf span definition.
288 : #[derive(Clone, Default)]
289 : struct ConnectionPerfSpanFields {
290 : peer_addr: String,
291 : application_name: Option<String>,
292 : compute_mode: Option<String>,
293 : }
294 :
295 : #[instrument(skip_all, fields(peer_addr, application_name, compute_mode))]
296 : #[allow(clippy::too_many_arguments)]
297 : async fn page_service_conn_main(
298 : conf: &'static PageServerConf,
299 : tenant_manager: Arc<TenantManager>,
300 : auth: Option<Arc<SwappableJwtAuth>>,
301 : socket: tokio::net::TcpStream,
302 : auth_type: AuthType,
303 : tls_config: Option<Arc<rustls::ServerConfig>>,
304 : pipelining_config: PageServicePipeliningConfig,
305 : connection_ctx: RequestContext,
306 : cancel: CancellationToken,
307 : gate_guard: GateGuard,
308 : ) -> ConnectionHandlerResult {
309 : let _guard = LIVE_CONNECTIONS
310 : .with_label_values(&["page_service"])
311 : .guard();
312 :
313 : socket
314 : .set_nodelay(true)
315 : .context("could not set TCP_NODELAY")?;
316 :
317 : let socket_fd = socket.as_raw_fd();
318 :
319 : let peer_addr = socket.peer_addr().context("get peer address")?;
320 :
321 : let perf_span_fields = ConnectionPerfSpanFields {
322 : peer_addr: peer_addr.to_string(),
323 : application_name: None, // filled in later
324 : compute_mode: None, // filled in later
325 : };
326 : tracing::Span::current().record("peer_addr", field::display(peer_addr));
327 :
328 : // setup read timeout of 10 minutes. the timeout is rather arbitrary for requirements:
329 : // - long enough for most valid compute connections
330 : // - less than infinite to stop us from "leaking" connections to long-gone computes
331 : //
332 : // no write timeout is used, because the kernel is assumed to error writes after some time.
333 : let mut socket = tokio_io_timeout::TimeoutReader::new(socket);
334 :
335 : let default_timeout_ms = 10 * 60 * 1000; // 10 minutes by default
336 0 : let socket_timeout_ms = (|| {
337 0 : fail::fail_point!("simulated-bad-compute-connection", |avg_timeout_ms| {
338 : // Exponential distribution for simulating
339 : // poor network conditions, expect about avg_timeout_ms to be around 15
340 : // in tests
341 0 : if let Some(avg_timeout_ms) = avg_timeout_ms {
342 0 : let avg = avg_timeout_ms.parse::<i64>().unwrap() as f32;
343 0 : let u = rand::random::<f32>();
344 0 : ((1.0 - u).ln() / (-avg)) as u64
345 : } else {
346 0 : default_timeout_ms
347 : }
348 0 : });
349 0 : default_timeout_ms
350 : })();
351 :
352 : // A timeout here does not mean the client died, it can happen if it's just idle for
353 : // a while: we will tear down this PageServerHandler and instantiate a new one if/when
354 : // they reconnect.
355 : socket.set_timeout(Some(std::time::Duration::from_millis(socket_timeout_ms)));
356 : let socket = Box::pin(socket);
357 :
358 : fail::fail_point!("ps::connection-start::pre-login");
359 :
360 : // XXX: pgbackend.run() should take the connection_ctx,
361 : // and create a child per-query context when it invokes process_query.
362 : // But it's in a shared crate, so, we store connection_ctx inside PageServerHandler
363 : // and create the per-query context in process_query ourselves.
364 : let mut conn_handler = PageServerHandler::new(
365 : tenant_manager,
366 : auth,
367 : pipelining_config,
368 : conf.get_vectored_concurrent_io,
369 : perf_span_fields,
370 : connection_ctx,
371 : cancel.clone(),
372 : gate_guard,
373 : );
374 : let pgbackend =
375 : PostgresBackend::new_from_io(socket_fd, socket, peer_addr, auth_type, tls_config)?;
376 :
377 : match pgbackend.run(&mut conn_handler, &cancel).await {
378 : Ok(()) => {
379 : // we've been requested to shut down
380 : Ok(())
381 : }
382 : Err(QueryError::Disconnected(ConnectionError::Io(io_error))) => {
383 : if is_expected_io_error(&io_error) {
384 : info!("Postgres client disconnected ({io_error})");
385 : Ok(())
386 : } else {
387 : let tenant_id = conn_handler.timeline_handles.as_ref().unwrap().tenant_id();
388 : Err(io_error).context(format!(
389 : "Postgres connection error for tenant_id={tenant_id:?} client at peer_addr={peer_addr}"
390 : ))
391 : }
392 : }
393 : other => {
394 : let tenant_id = conn_handler.timeline_handles.as_ref().unwrap().tenant_id();
395 : other.context(format!(
396 : "Postgres query error for tenant_id={tenant_id:?} client peer_addr={peer_addr}"
397 : ))
398 : }
399 : }
400 : }
401 :
402 : /// Page service connection handler.
403 : struct PageServerHandler {
404 : auth: Option<Arc<SwappableJwtAuth>>,
405 : claims: Option<Claims>,
406 :
407 : /// The context created for the lifetime of the connection
408 : /// services by this PageServerHandler.
409 : /// For each query received over the connection,
410 : /// `process_query` creates a child context from this one.
411 : connection_ctx: RequestContext,
412 :
413 : perf_span_fields: ConnectionPerfSpanFields,
414 :
415 : cancel: CancellationToken,
416 :
417 : /// None only while pagestream protocol is being processed.
418 : timeline_handles: Option<TimelineHandles>,
419 :
420 : pipelining_config: PageServicePipeliningConfig,
421 : get_vectored_concurrent_io: GetVectoredConcurrentIo,
422 :
423 : gate_guard: GateGuard,
424 : }
425 :
426 : struct TimelineHandles {
427 : wrapper: TenantManagerWrapper,
428 : /// Note on size: the typical size of this map is 1. The largest size we expect
429 : /// to see is the number of shards divided by the number of pageservers (typically < 2),
430 : /// or the ratio used when splitting shards (i.e. how many children created from one)
431 : /// parent shard, where a "large" number might be ~8.
432 : handles: timeline::handle::Cache<TenantManagerTypes>,
433 : }
434 :
435 : impl TimelineHandles {
436 0 : fn new(tenant_manager: Arc<TenantManager>) -> Self {
437 0 : Self {
438 0 : wrapper: TenantManagerWrapper {
439 0 : tenant_manager,
440 0 : tenant_id: OnceCell::new(),
441 0 : },
442 0 : handles: Default::default(),
443 0 : }
444 0 : }
445 0 : async fn get(
446 0 : &mut self,
447 0 : tenant_id: TenantId,
448 0 : timeline_id: TimelineId,
449 0 : shard_selector: ShardSelector,
450 0 : ) -> Result<Handle<TenantManagerTypes>, GetActiveTimelineError> {
451 0 : if *self.wrapper.tenant_id.get_or_init(|| tenant_id) != tenant_id {
452 0 : return Err(GetActiveTimelineError::Tenant(
453 0 : GetActiveTenantError::SwitchedTenant,
454 0 : ));
455 0 : }
456 0 : self.handles
457 0 : .get(timeline_id, shard_selector, &self.wrapper)
458 0 : .await
459 0 : .map_err(|e| match e {
460 0 : timeline::handle::GetError::TenantManager(e) => e,
461 : timeline::handle::GetError::PerTimelineStateShutDown => {
462 0 : trace!("per-timeline state shut down");
463 0 : GetActiveTimelineError::Timeline(GetTimelineError::ShuttingDown)
464 : }
465 0 : })
466 0 : }
467 :
468 0 : fn tenant_id(&self) -> Option<TenantId> {
469 0 : self.wrapper.tenant_id.get().copied()
470 0 : }
471 : }
472 :
473 : pub(crate) struct TenantManagerWrapper {
474 : tenant_manager: Arc<TenantManager>,
475 : // We do not support switching tenant_id on a connection at this point.
476 : // We can can add support for this later if needed without changing
477 : // the protocol.
478 : tenant_id: once_cell::sync::OnceCell<TenantId>,
479 : }
480 :
481 : #[derive(Debug)]
482 : pub(crate) struct TenantManagerTypes;
483 :
484 : impl timeline::handle::Types for TenantManagerTypes {
485 : type TenantManagerError = GetActiveTimelineError;
486 : type TenantManager = TenantManagerWrapper;
487 : type Timeline = TenantManagerCacheItem;
488 : }
489 :
490 : pub(crate) struct TenantManagerCacheItem {
491 : pub(crate) timeline: Arc<Timeline>,
492 : // allow() for cheap propagation through RequestContext inside a task
493 : #[allow(clippy::redundant_allocation)]
494 : pub(crate) metrics: Arc<Arc<TimelineMetrics>>,
495 : #[allow(dead_code)] // we store it to keep the gate open
496 : pub(crate) gate_guard: GateGuard,
497 : }
498 :
499 : impl std::ops::Deref for TenantManagerCacheItem {
500 : type Target = Arc<Timeline>;
501 0 : fn deref(&self) -> &Self::Target {
502 0 : &self.timeline
503 0 : }
504 : }
505 :
506 : impl timeline::handle::Timeline<TenantManagerTypes> for TenantManagerCacheItem {
507 0 : fn shard_timeline_id(&self) -> timeline::handle::ShardTimelineId {
508 0 : Timeline::shard_timeline_id(&self.timeline)
509 0 : }
510 :
511 0 : fn per_timeline_state(&self) -> &timeline::handle::PerTimelineState<TenantManagerTypes> {
512 0 : &self.timeline.handles
513 0 : }
514 :
515 0 : fn get_shard_identity(&self) -> &pageserver_api::shard::ShardIdentity {
516 0 : Timeline::get_shard_identity(&self.timeline)
517 0 : }
518 : }
519 :
520 : impl timeline::handle::TenantManager<TenantManagerTypes> for TenantManagerWrapper {
521 0 : async fn resolve(
522 0 : &self,
523 0 : timeline_id: TimelineId,
524 0 : shard_selector: ShardSelector,
525 0 : ) -> Result<TenantManagerCacheItem, GetActiveTimelineError> {
526 0 : let tenant_id = self.tenant_id.get().expect("we set this in get()");
527 0 : let timeout = ACTIVE_TENANT_TIMEOUT;
528 0 : let wait_start = Instant::now();
529 0 : let deadline = wait_start + timeout;
530 0 : let tenant_shard = loop {
531 0 : let resolved = self
532 0 : .tenant_manager
533 0 : .resolve_attached_shard(tenant_id, shard_selector);
534 0 : match resolved {
535 0 : ShardResolveResult::Found(tenant_shard) => break tenant_shard,
536 : ShardResolveResult::NotFound => {
537 0 : return Err(GetActiveTimelineError::Tenant(
538 0 : GetActiveTenantError::NotFound(GetTenantError::NotFound(*tenant_id)),
539 0 : ));
540 : }
541 0 : ShardResolveResult::InProgress(barrier) => {
542 : // We can't authoritatively answer right now: wait for InProgress state
543 : // to end, then try again
544 0 : tokio::select! {
545 0 : _ = barrier.wait() => {
546 0 : // The barrier completed: proceed around the loop to try looking up again
547 0 : },
548 0 : _ = tokio::time::sleep(deadline.duration_since(Instant::now())) => {
549 0 : return Err(GetActiveTimelineError::Tenant(GetActiveTenantError::WaitForActiveTimeout {
550 0 : latest_state: None,
551 0 : wait_time: timeout,
552 0 : }));
553 : }
554 : }
555 : }
556 : };
557 : };
558 :
559 0 : tracing::debug!("Waiting for tenant to enter active state...");
560 0 : tenant_shard
561 0 : .wait_to_become_active(deadline.duration_since(Instant::now()))
562 0 : .await
563 0 : .map_err(GetActiveTimelineError::Tenant)?;
564 :
565 0 : let timeline = tenant_shard
566 0 : .get_timeline(timeline_id, true)
567 0 : .map_err(GetActiveTimelineError::Timeline)?;
568 :
569 0 : let gate_guard = match timeline.gate.enter() {
570 0 : Ok(guard) => guard,
571 : Err(_) => {
572 0 : return Err(GetActiveTimelineError::Timeline(
573 0 : GetTimelineError::ShuttingDown,
574 0 : ));
575 : }
576 : };
577 :
578 0 : let metrics = Arc::new(Arc::clone(&timeline.metrics));
579 :
580 0 : Ok(TenantManagerCacheItem {
581 0 : timeline,
582 0 : metrics,
583 0 : gate_guard,
584 0 : })
585 0 : }
586 : }
587 :
588 : #[derive(thiserror::Error, Debug)]
589 : enum PageStreamError {
590 : /// We encountered an error that should prompt the client to reconnect:
591 : /// in practice this means we drop the connection without sending a response.
592 : #[error("Reconnect required: {0}")]
593 : Reconnect(Cow<'static, str>),
594 :
595 : /// We were instructed to shutdown while processing the query
596 : #[error("Shutting down")]
597 : Shutdown,
598 :
599 : /// Something went wrong reading a page: this likely indicates a pageserver bug
600 : #[error("Read error")]
601 : Read(#[source] PageReconstructError),
602 :
603 : /// Ran out of time waiting for an LSN
604 : #[error("LSN timeout: {0}")]
605 : LsnTimeout(WaitLsnError),
606 :
607 : /// The entity required to serve the request (tenant or timeline) is not found,
608 : /// or is not found in a suitable state to serve a request.
609 : #[error("Not found: {0}")]
610 : NotFound(Cow<'static, str>),
611 :
612 : /// Request asked for something that doesn't make sense, like an invalid LSN
613 : #[error("Bad request: {0}")]
614 : BadRequest(Cow<'static, str>),
615 : }
616 :
617 : impl From<PageStreamError> for tonic::Status {
618 0 : fn from(err: PageStreamError) -> Self {
619 : use tonic::Code;
620 0 : let message = err.to_string();
621 0 : let code = match err {
622 0 : PageStreamError::Reconnect(_) => Code::Unavailable,
623 0 : PageStreamError::Shutdown => Code::Unavailable,
624 0 : PageStreamError::Read(err) => match err {
625 0 : PageReconstructError::Cancelled => Code::Unavailable,
626 0 : PageReconstructError::MissingKey(_) => Code::NotFound,
627 0 : PageReconstructError::AncestorLsnTimeout(err) => tonic::Status::from(err).code(),
628 0 : PageReconstructError::Other(_) => Code::Internal,
629 0 : PageReconstructError::WalRedo(_) => Code::Internal,
630 : },
631 0 : PageStreamError::LsnTimeout(err) => tonic::Status::from(err).code(),
632 0 : PageStreamError::NotFound(_) => Code::NotFound,
633 0 : PageStreamError::BadRequest(_) => Code::InvalidArgument,
634 : };
635 0 : tonic::Status::new(code, message)
636 0 : }
637 : }
638 :
639 : impl From<PageReconstructError> for PageStreamError {
640 0 : fn from(value: PageReconstructError) -> Self {
641 0 : match value {
642 0 : PageReconstructError::Cancelled => Self::Shutdown,
643 0 : e => Self::Read(e),
644 : }
645 0 : }
646 : }
647 :
648 : impl From<GetActiveTimelineError> for PageStreamError {
649 0 : fn from(value: GetActiveTimelineError) -> Self {
650 0 : match value {
651 : GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled)
652 : | GetActiveTimelineError::Tenant(GetActiveTenantError::WillNotBecomeActive(
653 : TenantState::Stopping { .. },
654 : ))
655 0 : | GetActiveTimelineError::Timeline(GetTimelineError::ShuttingDown) => Self::Shutdown,
656 0 : GetActiveTimelineError::Tenant(e) => Self::NotFound(format!("{e}").into()),
657 0 : GetActiveTimelineError::Timeline(e) => Self::NotFound(format!("{e}").into()),
658 : }
659 0 : }
660 : }
661 :
662 : impl From<WaitLsnError> for PageStreamError {
663 0 : fn from(value: WaitLsnError) -> Self {
664 0 : match value {
665 0 : e @ WaitLsnError::Timeout(_) => Self::LsnTimeout(e),
666 0 : WaitLsnError::Shutdown => Self::Shutdown,
667 0 : e @ WaitLsnError::BadState { .. } => Self::Reconnect(format!("{e}").into()),
668 : }
669 0 : }
670 : }
671 :
672 : impl From<WaitLsnError> for QueryError {
673 0 : fn from(value: WaitLsnError) -> Self {
674 0 : match value {
675 0 : e @ WaitLsnError::Timeout(_) => Self::Other(anyhow::Error::new(e)),
676 0 : WaitLsnError::Shutdown => Self::Shutdown,
677 0 : WaitLsnError::BadState { .. } => Self::Reconnect,
678 : }
679 0 : }
680 : }
681 :
682 : #[derive(thiserror::Error, Debug)]
683 : struct BatchedPageStreamError {
684 : req: PagestreamRequest,
685 : err: PageStreamError,
686 : }
687 :
688 : impl std::fmt::Display for BatchedPageStreamError {
689 0 : fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
690 0 : self.err.fmt(f)
691 0 : }
692 : }
693 :
694 : struct BatchedGetPageRequest {
695 : req: PagestreamGetPageRequest,
696 : timer: SmgrOpTimer,
697 : lsn_range: LsnRange,
698 : ctx: RequestContext,
699 : // If the request is perf enabled, this contains a context
700 : // with a perf span tracking the time spent waiting for the executor.
701 : batch_wait_ctx: Option<RequestContext>,
702 : }
703 :
704 : #[cfg(feature = "testing")]
705 : struct BatchedTestRequest {
706 : req: pagestream_api::PagestreamTestRequest,
707 : timer: SmgrOpTimer,
708 : }
709 :
710 : /// NB: we only hold [`timeline::handle::WeakHandle`] inside this enum,
711 : /// so that we don't keep the [`Timeline::gate`] open while the batch
712 : /// is being built up inside the [`spsc_fold`] (pagestream pipelining).
713 : #[derive(IntoStaticStr)]
714 : #[allow(clippy::large_enum_variant)]
715 : enum BatchedFeMessage {
716 : Exists {
717 : span: Span,
718 : timer: SmgrOpTimer,
719 : shard: WeakHandle<TenantManagerTypes>,
720 : req: PagestreamExistsRequest,
721 : },
722 : Nblocks {
723 : span: Span,
724 : timer: SmgrOpTimer,
725 : shard: WeakHandle<TenantManagerTypes>,
726 : req: PagestreamNblocksRequest,
727 : },
728 : GetPage {
729 : span: Span,
730 : shard: WeakHandle<TenantManagerTypes>,
731 : pages: SmallVec<[BatchedGetPageRequest; 1]>,
732 : batch_break_reason: GetPageBatchBreakReason,
733 : },
734 : DbSize {
735 : span: Span,
736 : timer: SmgrOpTimer,
737 : shard: WeakHandle<TenantManagerTypes>,
738 : req: PagestreamDbSizeRequest,
739 : },
740 : GetSlruSegment {
741 : span: Span,
742 : timer: SmgrOpTimer,
743 : shard: WeakHandle<TenantManagerTypes>,
744 : req: PagestreamGetSlruSegmentRequest,
745 : },
746 : #[cfg(feature = "testing")]
747 : Test {
748 : span: Span,
749 : shard: WeakHandle<TenantManagerTypes>,
750 : requests: Vec<BatchedTestRequest>,
751 : },
752 : RespondError {
753 : span: Span,
754 : error: BatchedPageStreamError,
755 : },
756 : }
757 :
758 : impl BatchedFeMessage {
759 0 : fn as_static_str(&self) -> &'static str {
760 0 : self.into()
761 0 : }
762 :
763 0 : fn observe_execution_start(&mut self, at: Instant) {
764 0 : match self {
765 0 : BatchedFeMessage::Exists { timer, .. }
766 0 : | BatchedFeMessage::Nblocks { timer, .. }
767 0 : | BatchedFeMessage::DbSize { timer, .. }
768 0 : | BatchedFeMessage::GetSlruSegment { timer, .. } => {
769 0 : timer.observe_execution_start(at);
770 0 : }
771 0 : BatchedFeMessage::GetPage { pages, .. } => {
772 0 : for page in pages {
773 0 : page.timer.observe_execution_start(at);
774 0 : }
775 : }
776 : #[cfg(feature = "testing")]
777 0 : BatchedFeMessage::Test { requests, .. } => {
778 0 : for req in requests {
779 0 : req.timer.observe_execution_start(at);
780 0 : }
781 : }
782 0 : BatchedFeMessage::RespondError { .. } => {}
783 : }
784 0 : }
785 :
786 0 : fn should_break_batch(
787 0 : &self,
788 0 : other: &BatchedFeMessage,
789 0 : max_batch_size: NonZeroUsize,
790 0 : batching_strategy: PageServiceProtocolPipelinedBatchingStrategy,
791 0 : ) -> Option<GetPageBatchBreakReason> {
792 0 : match (self, other) {
793 : (
794 : BatchedFeMessage::GetPage {
795 0 : shard: accum_shard,
796 0 : pages: accum_pages,
797 : ..
798 : },
799 : BatchedFeMessage::GetPage {
800 0 : shard: this_shard,
801 0 : pages: this_pages,
802 : ..
803 : },
804 : ) => {
805 0 : assert_eq!(this_pages.len(), 1);
806 0 : if accum_pages.len() >= max_batch_size.get() {
807 0 : trace!(%max_batch_size, "stopping batching because of batch size");
808 0 : assert_eq!(accum_pages.len(), max_batch_size.get());
809 :
810 0 : return Some(GetPageBatchBreakReason::BatchFull);
811 0 : }
812 0 : if !accum_shard.is_same_handle_as(this_shard) {
813 0 : trace!("stopping batching because timeline object mismatch");
814 : // TODO: we _could_ batch & execute each shard seperately (and in parallel).
815 : // But the current logic for keeping responses in order does not support that.
816 :
817 0 : return Some(GetPageBatchBreakReason::NonUniformTimeline);
818 0 : }
819 :
820 0 : match batching_strategy {
821 : PageServiceProtocolPipelinedBatchingStrategy::UniformLsn => {
822 0 : if let Some(last_in_batch) = accum_pages.last() {
823 0 : if last_in_batch.lsn_range.effective_lsn
824 0 : != this_pages[0].lsn_range.effective_lsn
825 : {
826 0 : trace!(
827 : accum_lsn = %last_in_batch.lsn_range.effective_lsn,
828 0 : this_lsn = %this_pages[0].lsn_range.effective_lsn,
829 0 : "stopping batching because LSN changed"
830 : );
831 :
832 0 : return Some(GetPageBatchBreakReason::NonUniformLsn);
833 0 : }
834 0 : }
835 : }
836 : PageServiceProtocolPipelinedBatchingStrategy::ScatteredLsn => {
837 : // The read path doesn't curently support serving the same page at different LSNs.
838 : // While technically possible, it's uncertain if the complexity is worth it.
839 : // Break the batch if such a case is encountered.
840 0 : let same_page_different_lsn = accum_pages.iter().any(|batched| {
841 0 : batched.req.rel == this_pages[0].req.rel
842 0 : && batched.req.blkno == this_pages[0].req.blkno
843 0 : && batched.lsn_range.effective_lsn
844 0 : != this_pages[0].lsn_range.effective_lsn
845 0 : });
846 :
847 0 : if same_page_different_lsn {
848 0 : trace!(
849 0 : rel=%this_pages[0].req.rel,
850 0 : blkno=%this_pages[0].req.blkno,
851 0 : lsn=%this_pages[0].lsn_range.effective_lsn,
852 0 : "stopping batching because same page was requested at different LSNs"
853 : );
854 :
855 0 : return Some(GetPageBatchBreakReason::SamePageAtDifferentLsn);
856 0 : }
857 : }
858 : }
859 :
860 0 : None
861 : }
862 : #[cfg(feature = "testing")]
863 : (
864 : BatchedFeMessage::Test {
865 0 : shard: accum_shard,
866 0 : requests: accum_requests,
867 : ..
868 : },
869 : BatchedFeMessage::Test {
870 0 : shard: this_shard,
871 0 : requests: this_requests,
872 : ..
873 : },
874 : ) => {
875 0 : assert!(this_requests.len() == 1);
876 0 : if accum_requests.len() >= max_batch_size.get() {
877 0 : trace!(%max_batch_size, "stopping batching because of batch size");
878 0 : assert_eq!(accum_requests.len(), max_batch_size.get());
879 0 : return Some(GetPageBatchBreakReason::BatchFull);
880 0 : }
881 0 : if !accum_shard.is_same_handle_as(this_shard) {
882 0 : trace!("stopping batching because timeline object mismatch");
883 : // TODO: we _could_ batch & execute each shard seperately (and in parallel).
884 : // But the current logic for keeping responses in order does not support that.
885 0 : return Some(GetPageBatchBreakReason::NonUniformTimeline);
886 0 : }
887 0 : let this_batch_key = this_requests[0].req.batch_key;
888 0 : let accum_batch_key = accum_requests[0].req.batch_key;
889 0 : if this_requests[0].req.batch_key != accum_requests[0].req.batch_key {
890 0 : trace!(%accum_batch_key, %this_batch_key, "stopping batching because batch key changed");
891 0 : return Some(GetPageBatchBreakReason::NonUniformKey);
892 0 : }
893 0 : None
894 : }
895 0 : (_, _) => Some(GetPageBatchBreakReason::NonBatchableRequest),
896 : }
897 0 : }
898 : }
899 :
900 : impl PageServerHandler {
901 : #[allow(clippy::too_many_arguments)]
902 0 : pub fn new(
903 0 : tenant_manager: Arc<TenantManager>,
904 0 : auth: Option<Arc<SwappableJwtAuth>>,
905 0 : pipelining_config: PageServicePipeliningConfig,
906 0 : get_vectored_concurrent_io: GetVectoredConcurrentIo,
907 0 : perf_span_fields: ConnectionPerfSpanFields,
908 0 : connection_ctx: RequestContext,
909 0 : cancel: CancellationToken,
910 0 : gate_guard: GateGuard,
911 0 : ) -> Self {
912 0 : PageServerHandler {
913 0 : auth,
914 0 : claims: None,
915 0 : connection_ctx,
916 0 : perf_span_fields,
917 0 : timeline_handles: Some(TimelineHandles::new(tenant_manager)),
918 0 : cancel,
919 0 : pipelining_config,
920 0 : get_vectored_concurrent_io,
921 0 : gate_guard,
922 0 : }
923 0 : }
924 :
925 : /// This function always respects cancellation of any timeline in `[Self::shard_timelines]`. Pass in
926 : /// a cancellation token at the next scope up (such as a tenant cancellation token) to ensure we respect
927 : /// cancellation if there aren't any timelines in the cache.
928 : ///
929 : /// If calling from a function that doesn't use the `[Self::shard_timelines]` cache, then pass in the
930 : /// timeline cancellation token.
931 0 : async fn flush_cancellable<IO>(
932 0 : &self,
933 0 : pgb: &mut PostgresBackend<IO>,
934 0 : cancel: &CancellationToken,
935 0 : ) -> Result<(), QueryError>
936 0 : where
937 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
938 0 : {
939 0 : tokio::select!(
940 0 : flush_r = pgb.flush() => {
941 0 : Ok(flush_r?)
942 : },
943 0 : _ = cancel.cancelled() => {
944 0 : Err(QueryError::Shutdown)
945 : }
946 : )
947 0 : }
948 :
949 : #[allow(clippy::too_many_arguments)]
950 0 : async fn pagestream_read_message<IO>(
951 0 : pgb: &mut PostgresBackendReader<IO>,
952 0 : tenant_id: TenantId,
953 0 : timeline_id: TimelineId,
954 0 : timeline_handles: &mut TimelineHandles,
955 0 : conn_perf_span_fields: &ConnectionPerfSpanFields,
956 0 : cancel: &CancellationToken,
957 0 : ctx: &RequestContext,
958 0 : protocol_version: PagestreamProtocolVersion,
959 0 : parent_span: Span,
960 0 : ) -> Result<Option<BatchedFeMessage>, QueryError>
961 0 : where
962 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin + 'static,
963 0 : {
964 0 : let msg = tokio::select! {
965 : biased;
966 0 : _ = cancel.cancelled() => {
967 0 : return Err(QueryError::Shutdown)
968 : }
969 0 : msg = pgb.read_message() => { msg }
970 : };
971 :
972 0 : let received_at = Instant::now();
973 :
974 0 : let copy_data_bytes = match msg? {
975 0 : Some(FeMessage::CopyData(bytes)) => bytes,
976 : Some(FeMessage::Terminate) => {
977 0 : return Ok(None);
978 : }
979 0 : Some(m) => {
980 0 : return Err(QueryError::Other(anyhow::anyhow!(
981 0 : "unexpected message: {m:?} during COPY"
982 0 : )));
983 : }
984 : None => {
985 0 : return Ok(None);
986 : } // client disconnected
987 : };
988 0 : trace!("query: {copy_data_bytes:?}");
989 :
990 0 : fail::fail_point!("ps::handle-pagerequest-message");
991 :
992 : // parse request
993 0 : let neon_fe_msg =
994 0 : PagestreamFeMessage::parse(&mut copy_data_bytes.reader(), protocol_version)?;
995 :
996 0 : let batched_msg = match neon_fe_msg {
997 0 : PagestreamFeMessage::Exists(req) => {
998 0 : let shard = timeline_handles
999 0 : .get(tenant_id, timeline_id, ShardSelector::Zero)
1000 0 : .await?;
1001 0 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
1002 0 : let span = tracing::info_span!(parent: &parent_span, "handle_get_rel_exists_request", rel = %req.rel, req_lsn = %req.hdr.request_lsn, shard_id = %shard.tenant_shard_id.shard_slug());
1003 0 : let timer = Self::record_op_start_and_throttle(
1004 0 : &shard,
1005 0 : metrics::SmgrQueryType::GetRelExists,
1006 0 : received_at,
1007 0 : )
1008 0 : .await?;
1009 0 : BatchedFeMessage::Exists {
1010 0 : span,
1011 0 : timer,
1012 0 : shard: shard.downgrade(),
1013 0 : req,
1014 0 : }
1015 : }
1016 0 : PagestreamFeMessage::Nblocks(req) => {
1017 0 : let shard = timeline_handles
1018 0 : .get(tenant_id, timeline_id, ShardSelector::Zero)
1019 0 : .await?;
1020 0 : let span = tracing::info_span!(parent: &parent_span, "handle_get_nblocks_request", rel = %req.rel, req_lsn = %req.hdr.request_lsn, shard_id = %shard.tenant_shard_id.shard_slug());
1021 0 : let timer = Self::record_op_start_and_throttle(
1022 0 : &shard,
1023 0 : metrics::SmgrQueryType::GetRelSize,
1024 0 : received_at,
1025 0 : )
1026 0 : .await?;
1027 0 : BatchedFeMessage::Nblocks {
1028 0 : span,
1029 0 : timer,
1030 0 : shard: shard.downgrade(),
1031 0 : req,
1032 0 : }
1033 : }
1034 0 : PagestreamFeMessage::DbSize(req) => {
1035 0 : let shard = timeline_handles
1036 0 : .get(tenant_id, timeline_id, ShardSelector::Zero)
1037 0 : .await?;
1038 0 : let span = tracing::info_span!(parent: &parent_span, "handle_db_size_request", dbnode = %req.dbnode, req_lsn = %req.hdr.request_lsn, shard_id = %shard.tenant_shard_id.shard_slug());
1039 0 : let timer = Self::record_op_start_and_throttle(
1040 0 : &shard,
1041 0 : metrics::SmgrQueryType::GetDbSize,
1042 0 : received_at,
1043 0 : )
1044 0 : .await?;
1045 0 : BatchedFeMessage::DbSize {
1046 0 : span,
1047 0 : timer,
1048 0 : shard: shard.downgrade(),
1049 0 : req,
1050 0 : }
1051 : }
1052 0 : PagestreamFeMessage::GetSlruSegment(req) => {
1053 0 : let shard = timeline_handles
1054 0 : .get(tenant_id, timeline_id, ShardSelector::Zero)
1055 0 : .await?;
1056 0 : let span = tracing::info_span!(parent: &parent_span, "handle_get_slru_segment_request", kind = %req.kind, segno = %req.segno, req_lsn = %req.hdr.request_lsn, shard_id = %shard.tenant_shard_id.shard_slug());
1057 0 : let timer = Self::record_op_start_and_throttle(
1058 0 : &shard,
1059 0 : metrics::SmgrQueryType::GetSlruSegment,
1060 0 : received_at,
1061 0 : )
1062 0 : .await?;
1063 0 : BatchedFeMessage::GetSlruSegment {
1064 0 : span,
1065 0 : timer,
1066 0 : shard: shard.downgrade(),
1067 0 : req,
1068 0 : }
1069 : }
1070 0 : PagestreamFeMessage::GetPage(req) => {
1071 : // avoid a somewhat costly Span::record() by constructing the entire span in one go.
1072 : macro_rules! mkspan {
1073 : (before shard routing) => {{
1074 : tracing::info_span!(
1075 : parent: &parent_span,
1076 : "handle_get_page_request",
1077 : request_id = %req.hdr.reqid,
1078 : rel = %req.rel,
1079 : blkno = %req.blkno,
1080 : req_lsn = %req.hdr.request_lsn,
1081 : not_modified_since_lsn = %req.hdr.not_modified_since,
1082 : )
1083 : }};
1084 : ($shard_id:expr) => {{
1085 : tracing::info_span!(
1086 : parent: &parent_span,
1087 : "handle_get_page_request",
1088 : request_id = %req.hdr.reqid,
1089 : rel = %req.rel,
1090 : blkno = %req.blkno,
1091 : req_lsn = %req.hdr.request_lsn,
1092 : not_modified_since_lsn = %req.hdr.not_modified_since,
1093 : shard_id = %$shard_id,
1094 : )
1095 : }};
1096 : }
1097 :
1098 : macro_rules! respond_error {
1099 : ($span:expr, $error:expr) => {{
1100 : let error = BatchedFeMessage::RespondError {
1101 : span: $span,
1102 : error: BatchedPageStreamError {
1103 : req: req.hdr,
1104 : err: $error,
1105 : },
1106 : };
1107 : Ok(Some(error))
1108 : }};
1109 : }
1110 :
1111 0 : let key = rel_block_to_key(req.rel, req.blkno);
1112 :
1113 0 : let res = timeline_handles
1114 0 : .get(tenant_id, timeline_id, ShardSelector::Page(key))
1115 0 : .await;
1116 :
1117 0 : let shard = match res {
1118 0 : Ok(tl) => tl,
1119 0 : Err(e) => {
1120 0 : let span = mkspan!(before shard routing);
1121 0 : match e {
1122 : GetActiveTimelineError::Tenant(GetActiveTenantError::NotFound(_)) => {
1123 : // We already know this tenant exists in general, because we resolved it at
1124 : // start of connection. Getting a NotFound here indicates that the shard containing
1125 : // the requested page is not present on this node: the client's knowledge of shard->pageserver
1126 : // mapping is out of date.
1127 : //
1128 : // Closing the connection by returning ``::Reconnect` has the side effect of rate-limiting above message, via
1129 : // client's reconnect backoff, as well as hopefully prompting the client to load its updated configuration
1130 : // and talk to a different pageserver.
1131 0 : return respond_error!(
1132 0 : span,
1133 0 : PageStreamError::Reconnect(
1134 0 : "getpage@lsn request routed to wrong shard".into()
1135 0 : )
1136 : );
1137 : }
1138 0 : e => {
1139 0 : return respond_error!(span, e.into());
1140 : }
1141 : }
1142 : }
1143 : };
1144 :
1145 0 : let ctx = if shard.is_get_page_request_sampled() {
1146 0 : RequestContextBuilder::from(ctx)
1147 0 : .root_perf_span(|| {
1148 0 : info_span!(
1149 : target: PERF_TRACE_TARGET,
1150 : "GET_PAGE",
1151 : peer_addr = conn_perf_span_fields.peer_addr,
1152 : application_name = conn_perf_span_fields.application_name,
1153 : compute_mode = conn_perf_span_fields.compute_mode,
1154 : tenant_id = %tenant_id,
1155 0 : shard_id = %shard.get_shard_identity().shard_slug(),
1156 : timeline_id = %timeline_id,
1157 : lsn = %req.hdr.request_lsn,
1158 : not_modified_since_lsn = %req.hdr.not_modified_since,
1159 : request_id = %req.hdr.reqid,
1160 : key = %key,
1161 : )
1162 0 : })
1163 0 : .attached_child()
1164 : } else {
1165 0 : ctx.attached_child()
1166 : };
1167 :
1168 : // This ctx travels as part of the BatchedFeMessage through
1169 : // batching into the request handler.
1170 : // The request handler needs to do some per-request work
1171 : // (relsize check) before dispatching the batch as a single
1172 : // get_vectored call to the Timeline.
1173 : // This ctx will be used for the reslize check, whereas the
1174 : // get_vectored call will be a different ctx with separate
1175 : // perf span.
1176 0 : let ctx = ctx.with_scope_page_service_pagestream(&shard);
1177 :
1178 : // Similar game for this `span`: we funnel it through so that
1179 : // request handler log messages contain the request-specific fields.
1180 0 : let span = mkspan!(shard.tenant_shard_id.shard_slug());
1181 :
1182 0 : let timer = Self::record_op_start_and_throttle(
1183 0 : &shard,
1184 0 : metrics::SmgrQueryType::GetPageAtLsn,
1185 0 : received_at,
1186 : )
1187 0 : .maybe_perf_instrument(&ctx, |current_perf_span| {
1188 0 : info_span!(
1189 : target: PERF_TRACE_TARGET,
1190 0 : parent: current_perf_span,
1191 : "THROTTLE",
1192 : )
1193 0 : })
1194 0 : .await?;
1195 :
1196 : // We're holding the Handle
1197 0 : let effective_lsn = match Self::effective_request_lsn(
1198 0 : &shard,
1199 0 : shard.get_last_record_lsn(),
1200 0 : req.hdr.request_lsn,
1201 0 : req.hdr.not_modified_since,
1202 0 : &shard.get_applied_gc_cutoff_lsn(),
1203 0 : ) {
1204 0 : Ok(lsn) => lsn,
1205 0 : Err(e) => {
1206 0 : return respond_error!(span, e);
1207 : }
1208 : };
1209 :
1210 0 : let batch_wait_ctx = if ctx.has_perf_span() {
1211 : Some(
1212 0 : RequestContextBuilder::from(&ctx)
1213 0 : .perf_span(|crnt_perf_span| {
1214 0 : info_span!(
1215 : target: PERF_TRACE_TARGET,
1216 0 : parent: crnt_perf_span,
1217 : "WAIT_EXECUTOR",
1218 : )
1219 0 : })
1220 0 : .attached_child(),
1221 : )
1222 : } else {
1223 0 : None
1224 : };
1225 :
1226 : BatchedFeMessage::GetPage {
1227 0 : span,
1228 0 : shard: shard.downgrade(),
1229 0 : pages: smallvec![BatchedGetPageRequest {
1230 0 : req,
1231 0 : timer,
1232 0 : lsn_range: LsnRange {
1233 0 : effective_lsn,
1234 0 : request_lsn: req.hdr.request_lsn
1235 0 : },
1236 0 : ctx,
1237 0 : batch_wait_ctx,
1238 0 : }],
1239 : // The executor grabs the batch when it becomes idle.
1240 : // Hence, [`GetPageBatchBreakReason::ExecutorSteal`] is the
1241 : // default reason for breaking the batch.
1242 0 : batch_break_reason: GetPageBatchBreakReason::ExecutorSteal,
1243 : }
1244 : }
1245 : #[cfg(feature = "testing")]
1246 0 : PagestreamFeMessage::Test(req) => {
1247 0 : let shard = timeline_handles
1248 0 : .get(tenant_id, timeline_id, ShardSelector::Zero)
1249 0 : .await?;
1250 0 : let span = tracing::info_span!(parent: &parent_span, "handle_test_request", shard_id = %shard.tenant_shard_id.shard_slug());
1251 0 : let timer = Self::record_op_start_and_throttle(
1252 0 : &shard,
1253 0 : metrics::SmgrQueryType::Test,
1254 0 : received_at,
1255 0 : )
1256 0 : .await?;
1257 0 : BatchedFeMessage::Test {
1258 0 : span,
1259 0 : shard: shard.downgrade(),
1260 0 : requests: vec![BatchedTestRequest { req, timer }],
1261 0 : }
1262 : }
1263 : };
1264 0 : Ok(Some(batched_msg))
1265 0 : }
1266 :
1267 : /// Starts a SmgrOpTimer at received_at and throttles the request.
1268 0 : async fn record_op_start_and_throttle(
1269 0 : shard: &Handle<TenantManagerTypes>,
1270 0 : op: metrics::SmgrQueryType,
1271 0 : received_at: Instant,
1272 0 : ) -> Result<SmgrOpTimer, QueryError> {
1273 : // It's important to start the smgr op metric recorder as early as possible
1274 : // so that the _started counters are incremented before we do
1275 : // any serious waiting, e.g., for throttle, batching, or actual request handling.
1276 0 : let mut timer = shard.query_metrics.start_smgr_op(op, received_at);
1277 0 : let now = Instant::now();
1278 0 : timer.observe_throttle_start(now);
1279 0 : let throttled = tokio::select! {
1280 0 : res = shard.pagestream_throttle.throttle(1, now) => res,
1281 0 : _ = shard.cancel.cancelled() => return Err(QueryError::Shutdown),
1282 : };
1283 0 : timer.observe_throttle_done(throttled);
1284 0 : Ok(timer)
1285 0 : }
1286 :
1287 : /// Post-condition: `batch` is Some()
1288 : #[instrument(skip_all, level = tracing::Level::TRACE)]
1289 : #[allow(clippy::boxed_local)]
1290 : fn pagestream_do_batch(
1291 : batching_strategy: PageServiceProtocolPipelinedBatchingStrategy,
1292 : max_batch_size: NonZeroUsize,
1293 : batch: &mut Result<BatchedFeMessage, QueryError>,
1294 : this_msg: Result<BatchedFeMessage, QueryError>,
1295 : ) -> Result<(), Result<BatchedFeMessage, QueryError>> {
1296 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
1297 :
1298 : let this_msg = match this_msg {
1299 : Ok(this_msg) => this_msg,
1300 : Err(e) => return Err(Err(e)),
1301 : };
1302 :
1303 : let eligible_batch = match batch {
1304 : Ok(b) => b,
1305 : Err(_) => {
1306 : return Err(Ok(this_msg));
1307 : }
1308 : };
1309 :
1310 : let batch_break =
1311 : eligible_batch.should_break_batch(&this_msg, max_batch_size, batching_strategy);
1312 :
1313 : match batch_break {
1314 : Some(reason) => {
1315 : if let BatchedFeMessage::GetPage {
1316 : batch_break_reason, ..
1317 : } = eligible_batch
1318 : {
1319 : *batch_break_reason = reason;
1320 : }
1321 :
1322 : Err(Ok(this_msg))
1323 : }
1324 : None => {
1325 : // ok to batch
1326 : match (eligible_batch, this_msg) {
1327 : (
1328 : BatchedFeMessage::GetPage {
1329 : pages: accum_pages, ..
1330 : },
1331 : BatchedFeMessage::GetPage {
1332 : pages: this_pages, ..
1333 : },
1334 : ) => {
1335 : accum_pages.extend(this_pages);
1336 : Ok(())
1337 : }
1338 : #[cfg(feature = "testing")]
1339 : (
1340 : BatchedFeMessage::Test {
1341 : requests: accum_requests,
1342 : ..
1343 : },
1344 : BatchedFeMessage::Test {
1345 : requests: this_requests,
1346 : ..
1347 : },
1348 : ) => {
1349 : accum_requests.extend(this_requests);
1350 : Ok(())
1351 : }
1352 : // Shape guaranteed by [`BatchedFeMessage::should_break_batch`]
1353 : _ => unreachable!(),
1354 : }
1355 : }
1356 : }
1357 : }
1358 :
1359 0 : #[instrument(level = tracing::Level::DEBUG, skip_all)]
1360 : async fn pagestream_handle_batched_message<IO>(
1361 : &mut self,
1362 : pgb_writer: &mut PostgresBackend<IO>,
1363 : batch: BatchedFeMessage,
1364 : io_concurrency: IoConcurrency,
1365 : cancel: &CancellationToken,
1366 : protocol_version: PagestreamProtocolVersion,
1367 : ctx: &RequestContext,
1368 : ) -> Result<(), QueryError>
1369 : where
1370 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
1371 : {
1372 : let started_at = Instant::now();
1373 : let batch = {
1374 : let mut batch = batch;
1375 : batch.observe_execution_start(started_at);
1376 : batch
1377 : };
1378 :
1379 : // Dispatch the batch to the appropriate request handler.
1380 : let log_slow_name = batch.as_static_str();
1381 : let (mut handler_results, span) = {
1382 : // TODO: we unfortunately have to pin the future on the heap, since GetPage futures are huge and
1383 : // won't fit on the stack.
1384 : let mut boxpinned = Box::pin(Self::pagestream_dispatch_batched_message(
1385 : batch,
1386 : io_concurrency,
1387 : ctx,
1388 : ));
1389 : log_slow(
1390 : log_slow_name,
1391 : LOG_SLOW_GETPAGE_THRESHOLD,
1392 : boxpinned.as_mut(),
1393 : )
1394 : .await?
1395 : };
1396 :
1397 : // We purposefully don't count flush time into the smgr operation timer.
1398 : //
1399 : // The reason is that current compute client will not perform protocol processing
1400 : // if the postgres backend process is doing things other than `->smgr_read()`.
1401 : // This is especially the case for prefetch.
1402 : //
1403 : // If the compute doesn't read from the connection, eventually TCP will backpressure
1404 : // all the way into our flush call below.
1405 : //
1406 : // The timer's underlying metric is used for a storage-internal latency SLO and
1407 : // we don't want to include latency in it that we can't control.
1408 : // And as pointed out above, in this case, we don't control the time that flush will take.
1409 : //
1410 : // We put each response in the batch onto the wire in a separate pgb_writer.flush()
1411 : // call, which (all unmeasured) adds syscall overhead but reduces time to first byte
1412 : // and avoids building up a "giant" contiguous userspace buffer to hold the entire response.
1413 : // TODO: vectored socket IO would be great, but pgb_writer doesn't support that.
1414 : let flush_timers = {
1415 : let flushing_start_time = Instant::now();
1416 : let mut flush_timers = Vec::with_capacity(handler_results.len());
1417 : for handler_result in &mut handler_results {
1418 : let flush_timer = match handler_result {
1419 : Ok((_response, timer, _ctx)) => Some(
1420 : timer
1421 : .observe_execution_end(flushing_start_time)
1422 : .expect("we are the first caller"),
1423 : ),
1424 : Err(_) => {
1425 : // TODO: measure errors
1426 : None
1427 : }
1428 : };
1429 : flush_timers.push(flush_timer);
1430 : }
1431 : assert_eq!(flush_timers.len(), handler_results.len());
1432 : flush_timers
1433 : };
1434 :
1435 : // Map handler result to protocol behavior.
1436 : // Some handler errors cause exit from pagestream protocol.
1437 : // Other handler errors are sent back as an error message and we stay in pagestream protocol.
1438 : for (handler_result, flushing_timer) in handler_results.into_iter().zip(flush_timers) {
1439 : let (response_msg, ctx) = match handler_result {
1440 : Err(e) => match &e.err {
1441 : PageStreamError::Shutdown => {
1442 : // If we fail to fulfil a request during shutdown, which may be _because_ of
1443 : // shutdown, then do not send the error to the client. Instead just drop the
1444 : // connection.
1445 0 : span.in_scope(|| info!("dropping connection due to shutdown"));
1446 : return Err(QueryError::Shutdown);
1447 : }
1448 : PageStreamError::Reconnect(reason) => {
1449 0 : span.in_scope(|| info!("handler requested reconnect: {reason}"));
1450 : return Err(QueryError::Reconnect);
1451 : }
1452 : PageStreamError::Read(_)
1453 : | PageStreamError::LsnTimeout(_)
1454 : | PageStreamError::NotFound(_)
1455 : | PageStreamError::BadRequest(_) => {
1456 : // print the all details to the log with {:#}, but for the client the
1457 : // error message is enough. Do not log if shutting down, as the anyhow::Error
1458 : // here includes cancellation which is not an error.
1459 : let full = utils::error::report_compact_sources(&e.err);
1460 0 : span.in_scope(|| {
1461 0 : error!("error reading relation or page version: {full:#}")
1462 0 : });
1463 :
1464 : (
1465 : PagestreamBeMessage::Error(PagestreamErrorResponse {
1466 : req: e.req,
1467 : message: e.err.to_string(),
1468 : }),
1469 : None,
1470 : )
1471 : }
1472 : },
1473 : Ok((response_msg, _op_timer_already_observed, ctx)) => (response_msg, Some(ctx)),
1474 : };
1475 :
1476 0 : let ctx = ctx.map(|req_ctx| {
1477 0 : RequestContextBuilder::from(&req_ctx)
1478 0 : .perf_span(|crnt_perf_span| {
1479 0 : info_span!(
1480 : target: PERF_TRACE_TARGET,
1481 0 : parent: crnt_perf_span,
1482 : "FLUSH_RESPONSE",
1483 : )
1484 0 : })
1485 0 : .attached_child()
1486 0 : });
1487 :
1488 : //
1489 : // marshal & transmit response message
1490 : //
1491 :
1492 : pgb_writer.write_message_noflush(&BeMessage::CopyData(
1493 : &response_msg.serialize(protocol_version),
1494 : ))?;
1495 :
1496 : failpoint_support::sleep_millis_async!("before-pagestream-msg-flush", cancel);
1497 :
1498 : // what we want to do
1499 : let socket_fd = pgb_writer.socket_fd;
1500 : let flush_fut = pgb_writer.flush();
1501 : // metric for how long flushing takes
1502 : let flush_fut = match flushing_timer {
1503 : Some(flushing_timer) => futures::future::Either::Left(flushing_timer.measure(
1504 : Instant::now(),
1505 : flush_fut,
1506 : socket_fd,
1507 : )),
1508 : None => futures::future::Either::Right(flush_fut),
1509 : };
1510 :
1511 : let flush_fut = if let Some(req_ctx) = ctx.as_ref() {
1512 : futures::future::Either::Left(
1513 0 : flush_fut.maybe_perf_instrument(req_ctx, |current_perf_span| {
1514 0 : current_perf_span.clone()
1515 0 : }),
1516 : )
1517 : } else {
1518 : futures::future::Either::Right(flush_fut)
1519 : };
1520 :
1521 : // do it while respecting cancellation
1522 0 : let _: () = async move {
1523 0 : tokio::select! {
1524 : biased;
1525 0 : _ = cancel.cancelled() => {
1526 : // We were requested to shut down.
1527 0 : info!("shutdown request received in page handler");
1528 0 : return Err(QueryError::Shutdown)
1529 : }
1530 0 : res = flush_fut => {
1531 0 : res?;
1532 : }
1533 : }
1534 0 : Ok(())
1535 0 : }
1536 : .await?;
1537 : }
1538 : Ok(())
1539 : }
1540 :
1541 : /// Helper which dispatches a batched message to the appropriate handler.
1542 : /// Returns a vec of results, along with the extracted trace span.
1543 0 : async fn pagestream_dispatch_batched_message(
1544 0 : batch: BatchedFeMessage,
1545 0 : io_concurrency: IoConcurrency,
1546 0 : ctx: &RequestContext,
1547 0 : ) -> Result<
1548 0 : (
1549 0 : Vec<Result<(PagestreamBeMessage, SmgrOpTimer, RequestContext), BatchedPageStreamError>>,
1550 0 : Span,
1551 0 : ),
1552 0 : QueryError,
1553 0 : > {
1554 : macro_rules! upgrade_handle_and_set_context {
1555 : ($shard:ident) => {{
1556 : let weak_handle = &$shard;
1557 : let handle = weak_handle.upgrade()?;
1558 : let ctx = ctx.with_scope_page_service_pagestream(&handle);
1559 : (handle, ctx)
1560 : }};
1561 : }
1562 0 : Ok(match batch {
1563 : BatchedFeMessage::Exists {
1564 0 : span,
1565 0 : timer,
1566 0 : shard,
1567 0 : req,
1568 : } => {
1569 0 : fail::fail_point!("ps::handle-pagerequest-message::exists");
1570 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1571 : (
1572 0 : vec![
1573 0 : Self::handle_get_rel_exists_request(&shard, &req, &ctx)
1574 0 : .instrument(span.clone())
1575 0 : .await
1576 0 : .map(|msg| (PagestreamBeMessage::Exists(msg), timer, ctx))
1577 0 : .map_err(|err| BatchedPageStreamError { err, req: req.hdr }),
1578 : ],
1579 0 : span,
1580 : )
1581 : }
1582 : BatchedFeMessage::Nblocks {
1583 0 : span,
1584 0 : timer,
1585 0 : shard,
1586 0 : req,
1587 : } => {
1588 0 : fail::fail_point!("ps::handle-pagerequest-message::nblocks");
1589 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1590 : (
1591 0 : vec![
1592 0 : Self::handle_get_nblocks_request(&shard, &req, &ctx)
1593 0 : .instrument(span.clone())
1594 0 : .await
1595 0 : .map(|msg| (PagestreamBeMessage::Nblocks(msg), timer, ctx))
1596 0 : .map_err(|err| BatchedPageStreamError { err, req: req.hdr }),
1597 : ],
1598 0 : span,
1599 : )
1600 : }
1601 : BatchedFeMessage::GetPage {
1602 0 : span,
1603 0 : shard,
1604 0 : pages,
1605 0 : batch_break_reason,
1606 : } => {
1607 0 : fail::fail_point!("ps::handle-pagerequest-message::getpage");
1608 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1609 : (
1610 : {
1611 0 : let npages = pages.len();
1612 0 : trace!(npages, "handling getpage request");
1613 0 : let res = Self::handle_get_page_at_lsn_request_batched(
1614 0 : &shard,
1615 0 : pages,
1616 0 : io_concurrency,
1617 0 : batch_break_reason,
1618 0 : &ctx,
1619 0 : )
1620 0 : .instrument(span.clone())
1621 0 : .await;
1622 0 : assert_eq!(res.len(), npages);
1623 0 : res
1624 : },
1625 0 : span,
1626 : )
1627 : }
1628 : BatchedFeMessage::DbSize {
1629 0 : span,
1630 0 : timer,
1631 0 : shard,
1632 0 : req,
1633 : } => {
1634 0 : fail::fail_point!("ps::handle-pagerequest-message::dbsize");
1635 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1636 : (
1637 0 : vec![
1638 0 : Self::handle_db_size_request(&shard, &req, &ctx)
1639 0 : .instrument(span.clone())
1640 0 : .await
1641 0 : .map(|msg| (PagestreamBeMessage::DbSize(msg), timer, ctx))
1642 0 : .map_err(|err| BatchedPageStreamError { err, req: req.hdr }),
1643 : ],
1644 0 : span,
1645 : )
1646 : }
1647 : BatchedFeMessage::GetSlruSegment {
1648 0 : span,
1649 0 : timer,
1650 0 : shard,
1651 0 : req,
1652 : } => {
1653 0 : fail::fail_point!("ps::handle-pagerequest-message::slrusegment");
1654 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1655 : (
1656 0 : vec![
1657 0 : Self::handle_get_slru_segment_request(&shard, &req, &ctx)
1658 0 : .instrument(span.clone())
1659 0 : .await
1660 0 : .map(|msg| (PagestreamBeMessage::GetSlruSegment(msg), timer, ctx))
1661 0 : .map_err(|err| BatchedPageStreamError { err, req: req.hdr }),
1662 : ],
1663 0 : span,
1664 : )
1665 : }
1666 : #[cfg(feature = "testing")]
1667 : BatchedFeMessage::Test {
1668 0 : span,
1669 0 : shard,
1670 0 : requests,
1671 : } => {
1672 0 : fail::fail_point!("ps::handle-pagerequest-message::test");
1673 0 : let (shard, ctx) = upgrade_handle_and_set_context!(shard);
1674 : (
1675 : {
1676 0 : let npages = requests.len();
1677 0 : trace!(npages, "handling getpage request");
1678 0 : let res = Self::handle_test_request_batch(&shard, requests, &ctx)
1679 0 : .instrument(span.clone())
1680 0 : .await;
1681 0 : assert_eq!(res.len(), npages);
1682 0 : res
1683 : },
1684 0 : span,
1685 : )
1686 : }
1687 0 : BatchedFeMessage::RespondError { span, error } => {
1688 : // We've already decided to respond with an error, so we don't need to
1689 : // call the handler.
1690 0 : (vec![Err(error)], span)
1691 : }
1692 : })
1693 0 : }
1694 :
1695 : /// Pagestream sub-protocol handler.
1696 : ///
1697 : /// It is a simple request-response protocol inside a COPYBOTH session.
1698 : ///
1699 : /// # Coding Discipline
1700 : ///
1701 : /// Coding discipline within this function: all interaction with the `pgb` connection
1702 : /// needs to be sensitive to connection shutdown, currently signalled via [`Self::cancel`].
1703 : /// This is so that we can shutdown page_service quickly.
1704 : #[instrument(skip_all)]
1705 : async fn handle_pagerequests<IO>(
1706 : &mut self,
1707 : pgb: &mut PostgresBackend<IO>,
1708 : tenant_id: TenantId,
1709 : timeline_id: TimelineId,
1710 : protocol_version: PagestreamProtocolVersion,
1711 : ctx: RequestContext,
1712 : ) -> Result<(), QueryError>
1713 : where
1714 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin + 'static,
1715 : {
1716 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
1717 :
1718 : // switch client to COPYBOTH
1719 : pgb.write_message_noflush(&BeMessage::CopyBothResponse)?;
1720 : tokio::select! {
1721 : biased;
1722 : _ = self.cancel.cancelled() => {
1723 : return Err(QueryError::Shutdown)
1724 : }
1725 : res = pgb.flush() => {
1726 : res?;
1727 : }
1728 : }
1729 :
1730 : let io_concurrency = IoConcurrency::spawn_from_conf(
1731 : self.get_vectored_concurrent_io,
1732 : match self.gate_guard.try_clone() {
1733 : Ok(guard) => guard,
1734 : Err(_) => {
1735 : info!("shutdown request received in page handler");
1736 : return Err(QueryError::Shutdown);
1737 : }
1738 : },
1739 : );
1740 :
1741 : let pgb_reader = pgb
1742 : .split()
1743 : .context("implementation error: split pgb into reader and writer")?;
1744 :
1745 : let timeline_handles = self
1746 : .timeline_handles
1747 : .take()
1748 : .expect("implementation error: timeline_handles should not be locked");
1749 :
1750 : let request_span = info_span!("request");
1751 : let ((pgb_reader, timeline_handles), result) = match self.pipelining_config.clone() {
1752 : PageServicePipeliningConfig::Pipelined(pipelining_config) => {
1753 : self.handle_pagerequests_pipelined(
1754 : pgb,
1755 : pgb_reader,
1756 : tenant_id,
1757 : timeline_id,
1758 : timeline_handles,
1759 : request_span,
1760 : pipelining_config,
1761 : protocol_version,
1762 : io_concurrency,
1763 : &ctx,
1764 : )
1765 : .await
1766 : }
1767 : PageServicePipeliningConfig::Serial => {
1768 : self.handle_pagerequests_serial(
1769 : pgb,
1770 : pgb_reader,
1771 : tenant_id,
1772 : timeline_id,
1773 : timeline_handles,
1774 : request_span,
1775 : protocol_version,
1776 : io_concurrency,
1777 : &ctx,
1778 : )
1779 : .await
1780 : }
1781 : };
1782 :
1783 : debug!("pagestream subprotocol shut down cleanly");
1784 :
1785 : pgb.unsplit(pgb_reader)
1786 : .context("implementation error: unsplit pgb")?;
1787 :
1788 : let replaced = self.timeline_handles.replace(timeline_handles);
1789 : assert!(replaced.is_none());
1790 :
1791 : result
1792 : }
1793 :
1794 : #[allow(clippy::too_many_arguments)]
1795 0 : async fn handle_pagerequests_serial<IO>(
1796 0 : &mut self,
1797 0 : pgb_writer: &mut PostgresBackend<IO>,
1798 0 : mut pgb_reader: PostgresBackendReader<IO>,
1799 0 : tenant_id: TenantId,
1800 0 : timeline_id: TimelineId,
1801 0 : mut timeline_handles: TimelineHandles,
1802 0 : request_span: Span,
1803 0 : protocol_version: PagestreamProtocolVersion,
1804 0 : io_concurrency: IoConcurrency,
1805 0 : ctx: &RequestContext,
1806 0 : ) -> (
1807 0 : (PostgresBackendReader<IO>, TimelineHandles),
1808 0 : Result<(), QueryError>,
1809 0 : )
1810 0 : where
1811 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin + 'static,
1812 0 : {
1813 0 : let cancel = self.cancel.clone();
1814 :
1815 0 : let err = loop {
1816 0 : let msg = Self::pagestream_read_message(
1817 0 : &mut pgb_reader,
1818 0 : tenant_id,
1819 0 : timeline_id,
1820 0 : &mut timeline_handles,
1821 0 : &self.perf_span_fields,
1822 0 : &cancel,
1823 0 : ctx,
1824 0 : protocol_version,
1825 0 : request_span.clone(),
1826 0 : )
1827 0 : .await;
1828 0 : let msg = match msg {
1829 0 : Ok(msg) => msg,
1830 0 : Err(e) => break e,
1831 : };
1832 0 : let msg = match msg {
1833 0 : Some(msg) => msg,
1834 : None => {
1835 0 : debug!("pagestream subprotocol end observed");
1836 0 : return ((pgb_reader, timeline_handles), Ok(()));
1837 : }
1838 : };
1839 :
1840 0 : let result = self
1841 0 : .pagestream_handle_batched_message(
1842 0 : pgb_writer,
1843 0 : msg,
1844 0 : io_concurrency.clone(),
1845 0 : &cancel,
1846 0 : protocol_version,
1847 0 : ctx,
1848 0 : )
1849 0 : .await;
1850 0 : match result {
1851 0 : Ok(()) => {}
1852 0 : Err(e) => break e,
1853 : }
1854 : };
1855 0 : ((pgb_reader, timeline_handles), Err(err))
1856 0 : }
1857 :
1858 : /// # Cancel-Safety
1859 : ///
1860 : /// May leak tokio tasks if not polled to completion.
1861 : #[allow(clippy::too_many_arguments)]
1862 0 : async fn handle_pagerequests_pipelined<IO>(
1863 0 : &mut self,
1864 0 : pgb_writer: &mut PostgresBackend<IO>,
1865 0 : pgb_reader: PostgresBackendReader<IO>,
1866 0 : tenant_id: TenantId,
1867 0 : timeline_id: TimelineId,
1868 0 : mut timeline_handles: TimelineHandles,
1869 0 : request_span: Span,
1870 0 : pipelining_config: PageServicePipeliningConfigPipelined,
1871 0 : protocol_version: PagestreamProtocolVersion,
1872 0 : io_concurrency: IoConcurrency,
1873 0 : ctx: &RequestContext,
1874 0 : ) -> (
1875 0 : (PostgresBackendReader<IO>, TimelineHandles),
1876 0 : Result<(), QueryError>,
1877 0 : )
1878 0 : where
1879 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin + 'static,
1880 0 : {
1881 : //
1882 : // Pipelined pagestream handling consists of
1883 : // - a Batcher that reads requests off the wire and
1884 : // and batches them if possible,
1885 : // - an Executor that processes the batched requests.
1886 : //
1887 : // The batch is built up inside an `spsc_fold` channel,
1888 : // shared betwen Batcher (Sender) and Executor (Receiver).
1889 : //
1890 : // The Batcher continously folds client requests into the batch,
1891 : // while the Executor can at any time take out what's in the batch
1892 : // in order to process it.
1893 : // This means the next batch builds up while the Executor
1894 : // executes the last batch.
1895 : //
1896 : // CANCELLATION
1897 : //
1898 : // We run both Batcher and Executor futures to completion before
1899 : // returning from this function.
1900 : //
1901 : // If Executor exits first, it signals cancellation to the Batcher
1902 : // via a CancellationToken that is child of `self.cancel`.
1903 : // If Batcher exits first, it signals cancellation to the Executor
1904 : // by dropping the spsc_fold channel Sender.
1905 : //
1906 : // CLEAN SHUTDOWN
1907 : //
1908 : // Clean shutdown means that the client ends the COPYBOTH session.
1909 : // In response to such a client message, the Batcher exits.
1910 : // The Executor continues to run, draining the spsc_fold channel.
1911 : // Once drained, the spsc_fold recv will fail with a distinct error
1912 : // indicating that the sender disconnected.
1913 : // The Executor exits with Ok(()) in response to that error.
1914 : //
1915 : // Server initiated shutdown is not clean shutdown, but instead
1916 : // is an error Err(QueryError::Shutdown) that is propagated through
1917 : // error propagation.
1918 : //
1919 : // ERROR PROPAGATION
1920 : //
1921 : // When the Batcher encounter an error, it sends it as a value
1922 : // through the spsc_fold channel and exits afterwards.
1923 : // When the Executor observes such an error in the channel,
1924 : // it exits returning that error value.
1925 : //
1926 : // This design ensures that the Executor stage will still process
1927 : // the batch that was in flight when the Batcher encountered an error,
1928 : // thereby beahving identical to a serial implementation.
1929 :
1930 : let PageServicePipeliningConfigPipelined {
1931 0 : max_batch_size,
1932 0 : execution,
1933 0 : batching: batching_strategy,
1934 0 : } = pipelining_config;
1935 :
1936 : // Macro to _define_ a pipeline stage.
1937 : macro_rules! pipeline_stage {
1938 : ($name:literal, $cancel:expr, $make_fut:expr) => {{
1939 : let cancel: CancellationToken = $cancel;
1940 : let stage_fut = $make_fut(cancel.clone());
1941 0 : async move {
1942 0 : scopeguard::defer! {
1943 : debug!("exiting");
1944 : }
1945 0 : timed_after_cancellation(stage_fut, $name, Duration::from_millis(100), &cancel)
1946 0 : .await
1947 0 : }
1948 : .instrument(tracing::info_span!($name))
1949 : }};
1950 : }
1951 :
1952 : //
1953 : // Batcher
1954 : //
1955 :
1956 0 : let perf_span_fields = self.perf_span_fields.clone();
1957 :
1958 0 : let cancel_batcher = self.cancel.child_token();
1959 0 : let (mut batch_tx, mut batch_rx) = spsc_fold::channel();
1960 0 : let batcher = pipeline_stage!("batcher", cancel_batcher.clone(), move |cancel_batcher| {
1961 0 : let ctx = ctx.attached_child();
1962 0 : async move {
1963 0 : let mut pgb_reader = pgb_reader;
1964 0 : let mut exit = false;
1965 0 : while !exit {
1966 0 : let read_res = Self::pagestream_read_message(
1967 0 : &mut pgb_reader,
1968 0 : tenant_id,
1969 0 : timeline_id,
1970 0 : &mut timeline_handles,
1971 0 : &perf_span_fields,
1972 0 : &cancel_batcher,
1973 0 : &ctx,
1974 0 : protocol_version,
1975 0 : request_span.clone(),
1976 0 : )
1977 0 : .await;
1978 0 : let Some(read_res) = read_res.transpose() else {
1979 0 : debug!("client-initiated shutdown");
1980 0 : break;
1981 : };
1982 0 : exit |= read_res.is_err();
1983 0 : let could_send = batch_tx
1984 0 : .send(read_res, |batch, res| {
1985 0 : Self::pagestream_do_batch(batching_strategy, max_batch_size, batch, res)
1986 0 : })
1987 0 : .await;
1988 0 : exit |= could_send.is_err();
1989 : }
1990 0 : (pgb_reader, timeline_handles)
1991 0 : }
1992 0 : });
1993 :
1994 : //
1995 : // Executor
1996 : //
1997 :
1998 0 : let executor = pipeline_stage!("executor", self.cancel.clone(), move |cancel| {
1999 0 : let ctx = ctx.attached_child();
2000 0 : async move {
2001 0 : let _cancel_batcher = cancel_batcher.drop_guard();
2002 : loop {
2003 0 : let maybe_batch = batch_rx.recv().await;
2004 0 : let batch = match maybe_batch {
2005 0 : Ok(batch) => batch,
2006 : Err(spsc_fold::RecvError::SenderGone) => {
2007 0 : debug!("upstream gone");
2008 0 : return Ok(());
2009 : }
2010 : };
2011 0 : let mut batch = match batch {
2012 0 : Ok(batch) => batch,
2013 0 : Err(e) => {
2014 0 : return Err(e);
2015 : }
2016 : };
2017 :
2018 : if let BatchedFeMessage::GetPage {
2019 0 : pages,
2020 : span: _,
2021 : shard: _,
2022 : batch_break_reason: _,
2023 0 : } = &mut batch
2024 : {
2025 0 : for req in pages {
2026 0 : req.batch_wait_ctx.take();
2027 0 : }
2028 0 : }
2029 :
2030 0 : self.pagestream_handle_batched_message(
2031 0 : pgb_writer,
2032 0 : batch,
2033 0 : io_concurrency.clone(),
2034 0 : &cancel,
2035 0 : protocol_version,
2036 0 : &ctx,
2037 0 : )
2038 0 : .await?;
2039 : }
2040 0 : }
2041 0 : });
2042 :
2043 : //
2044 : // Execute the stages.
2045 : //
2046 :
2047 0 : match execution {
2048 : PageServiceProtocolPipelinedExecutionStrategy::ConcurrentFutures => {
2049 0 : tokio::join!(batcher, executor)
2050 : }
2051 : PageServiceProtocolPipelinedExecutionStrategy::Tasks => {
2052 : // These tasks are not tracked anywhere.
2053 0 : let read_messages_task = tokio::spawn(batcher);
2054 0 : let (read_messages_task_res, executor_res_) =
2055 0 : tokio::join!(read_messages_task, executor,);
2056 0 : (
2057 0 : read_messages_task_res.expect("propagated panic from read_messages"),
2058 0 : executor_res_,
2059 0 : )
2060 : }
2061 : }
2062 0 : }
2063 :
2064 : /// Helper function to handle the LSN from client request.
2065 : ///
2066 : /// Each GetPage (and Exists and Nblocks) request includes information about
2067 : /// which version of the page is being requested. The primary compute node
2068 : /// will always request the latest page version, by setting 'request_lsn' to
2069 : /// the last inserted or flushed WAL position, while a standby will request
2070 : /// a version at the LSN that it's currently caught up to.
2071 : ///
2072 : /// In either case, if the page server hasn't received the WAL up to the
2073 : /// requested LSN yet, we will wait for it to arrive. The return value is
2074 : /// the LSN that should be used to look up the page versions.
2075 : ///
2076 : /// In addition to the request LSN, each request carries another LSN,
2077 : /// 'not_modified_since', which is a hint to the pageserver that the client
2078 : /// knows that the page has not been modified between 'not_modified_since'
2079 : /// and the request LSN. This allows skipping the wait, as long as the WAL
2080 : /// up to 'not_modified_since' has arrived. If the client doesn't have any
2081 : /// information about when the page was modified, it will use
2082 : /// not_modified_since == lsn. If the client lies and sends a too low
2083 : /// not_modified_hint such that there are in fact later page versions, the
2084 : /// behavior is undefined: the pageserver may return any of the page versions
2085 : /// or an error.
2086 0 : async fn wait_or_get_last_lsn(
2087 0 : timeline: &Timeline,
2088 0 : request_lsn: Lsn,
2089 0 : not_modified_since: Lsn,
2090 0 : latest_gc_cutoff_lsn: &RcuReadGuard<Lsn>,
2091 0 : ctx: &RequestContext,
2092 0 : ) -> Result<Lsn, PageStreamError> {
2093 0 : let last_record_lsn = timeline.get_last_record_lsn();
2094 0 : let effective_request_lsn = Self::effective_request_lsn(
2095 0 : timeline,
2096 0 : last_record_lsn,
2097 0 : request_lsn,
2098 0 : not_modified_since,
2099 0 : latest_gc_cutoff_lsn,
2100 0 : )?;
2101 :
2102 0 : if effective_request_lsn > last_record_lsn {
2103 0 : timeline
2104 0 : .wait_lsn(
2105 0 : not_modified_since,
2106 0 : crate::tenant::timeline::WaitLsnWaiter::PageService,
2107 0 : timeline::WaitLsnTimeout::Default,
2108 0 : ctx,
2109 0 : )
2110 0 : .await?;
2111 :
2112 : // Since we waited for 'effective_request_lsn' to arrive, that is now the last
2113 : // record LSN. (Or close enough for our purposes; the last-record LSN can
2114 : // advance immediately after we return anyway)
2115 0 : }
2116 :
2117 0 : Ok(effective_request_lsn)
2118 0 : }
2119 :
2120 0 : fn effective_request_lsn(
2121 0 : timeline: &Timeline,
2122 0 : last_record_lsn: Lsn,
2123 0 : request_lsn: Lsn,
2124 0 : not_modified_since: Lsn,
2125 0 : latest_gc_cutoff_lsn: &RcuReadGuard<Lsn>,
2126 0 : ) -> Result<Lsn, PageStreamError> {
2127 : // Sanity check the request
2128 0 : if request_lsn < not_modified_since {
2129 0 : return Err(PageStreamError::BadRequest(
2130 0 : format!(
2131 0 : "invalid request with request LSN {request_lsn} and not_modified_since {not_modified_since}",
2132 0 : )
2133 0 : .into(),
2134 0 : ));
2135 0 : }
2136 :
2137 : // Check explicitly for INVALID just to get a less scary error message if the request is obviously bogus
2138 0 : if request_lsn == Lsn::INVALID {
2139 0 : return Err(PageStreamError::BadRequest(
2140 0 : "invalid LSN(0) in request".into(),
2141 0 : ));
2142 0 : }
2143 :
2144 : // Clients should only read from recent LSNs on their timeline, or from locations holding an LSN lease.
2145 : //
2146 : // We may have older data available, but we make a best effort to detect this case and return an error,
2147 : // to distinguish a misbehaving client (asking for old LSN) from a storage issue (data missing at a legitimate LSN).
2148 0 : if request_lsn < **latest_gc_cutoff_lsn && !timeline.is_gc_blocked_by_lsn_lease_deadline() {
2149 0 : let gc_info = &timeline.gc_info.read().unwrap();
2150 0 : if !gc_info.lsn_covered_by_lease(request_lsn) {
2151 0 : return Err(
2152 0 : PageStreamError::BadRequest(format!(
2153 0 : "tried to request a page version that was garbage collected. requested at {} gc cutoff {}",
2154 0 : request_lsn, **latest_gc_cutoff_lsn
2155 0 : ).into())
2156 0 : );
2157 0 : }
2158 0 : }
2159 :
2160 0 : if not_modified_since > last_record_lsn {
2161 0 : Ok(not_modified_since)
2162 : } else {
2163 : // It might be better to use max(not_modified_since, latest_gc_cutoff_lsn)
2164 : // here instead. That would give the same result, since we know that there
2165 : // haven't been any modifications since 'not_modified_since'. Using an older
2166 : // LSN might be faster, because that could allow skipping recent layers when
2167 : // finding the page. However, we have historically used 'last_record_lsn', so
2168 : // stick to that for now.
2169 0 : Ok(std::cmp::min(last_record_lsn, request_lsn))
2170 : }
2171 0 : }
2172 :
2173 : /// Handles the lsn lease request.
2174 : /// If a lease cannot be obtained, the client will receive NULL.
2175 : #[instrument(skip_all, fields(shard_id, %lsn))]
2176 : async fn handle_make_lsn_lease<IO>(
2177 : &mut self,
2178 : pgb: &mut PostgresBackend<IO>,
2179 : tenant_shard_id: TenantShardId,
2180 : timeline_id: TimelineId,
2181 : lsn: Lsn,
2182 : ctx: &RequestContext,
2183 : ) -> Result<(), QueryError>
2184 : where
2185 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
2186 : {
2187 : let timeline = self
2188 : .timeline_handles
2189 : .as_mut()
2190 : .unwrap()
2191 : .get(
2192 : tenant_shard_id.tenant_id,
2193 : timeline_id,
2194 : ShardSelector::Known(tenant_shard_id.to_index()),
2195 : )
2196 : .await?;
2197 : set_tracing_field_shard_id(&timeline);
2198 :
2199 : let lease = timeline
2200 : .renew_lsn_lease(lsn, timeline.get_lsn_lease_length(), ctx)
2201 0 : .inspect_err(|e| {
2202 0 : warn!("{e}");
2203 0 : })
2204 : .ok();
2205 0 : let valid_until_str = lease.map(|l| {
2206 0 : l.valid_until
2207 0 : .duration_since(SystemTime::UNIX_EPOCH)
2208 0 : .expect("valid_until is earlier than UNIX_EPOCH")
2209 0 : .as_millis()
2210 0 : .to_string()
2211 0 : });
2212 :
2213 : info!(
2214 : "acquired lease for {} until {}",
2215 : lsn,
2216 : valid_until_str.as_deref().unwrap_or("<unknown>")
2217 : );
2218 :
2219 0 : let bytes = valid_until_str.as_ref().map(|x| x.as_bytes());
2220 :
2221 : pgb.write_message_noflush(&BeMessage::RowDescription(&[RowDescriptor::text_col(
2222 : b"valid_until",
2223 : )]))?
2224 : .write_message_noflush(&BeMessage::DataRow(&[bytes]))?;
2225 :
2226 : Ok(())
2227 : }
2228 :
2229 : #[instrument(skip_all, fields(shard_id))]
2230 : async fn handle_get_rel_exists_request(
2231 : timeline: &Timeline,
2232 : req: &PagestreamExistsRequest,
2233 : ctx: &RequestContext,
2234 : ) -> Result<PagestreamExistsResponse, PageStreamError> {
2235 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
2236 : let lsn = Self::wait_or_get_last_lsn(
2237 : timeline,
2238 : req.hdr.request_lsn,
2239 : req.hdr.not_modified_since,
2240 : &latest_gc_cutoff_lsn,
2241 : ctx,
2242 : )
2243 : .await?;
2244 :
2245 : let exists = timeline
2246 : .get_rel_exists(
2247 : req.rel,
2248 : Version::LsnRange(LsnRange {
2249 : effective_lsn: lsn,
2250 : request_lsn: req.hdr.request_lsn,
2251 : }),
2252 : ctx,
2253 : )
2254 : .await?;
2255 :
2256 : Ok(PagestreamExistsResponse { req: *req, exists })
2257 : }
2258 :
2259 : #[instrument(skip_all, fields(shard_id))]
2260 : async fn handle_get_nblocks_request(
2261 : timeline: &Timeline,
2262 : req: &PagestreamNblocksRequest,
2263 : ctx: &RequestContext,
2264 : ) -> Result<PagestreamNblocksResponse, PageStreamError> {
2265 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
2266 : let lsn = Self::wait_or_get_last_lsn(
2267 : timeline,
2268 : req.hdr.request_lsn,
2269 : req.hdr.not_modified_since,
2270 : &latest_gc_cutoff_lsn,
2271 : ctx,
2272 : )
2273 : .await?;
2274 :
2275 : let n_blocks = timeline
2276 : .get_rel_size(
2277 : req.rel,
2278 : Version::LsnRange(LsnRange {
2279 : effective_lsn: lsn,
2280 : request_lsn: req.hdr.request_lsn,
2281 : }),
2282 : ctx,
2283 : )
2284 : .await?;
2285 :
2286 : Ok(PagestreamNblocksResponse {
2287 : req: *req,
2288 : n_blocks,
2289 : })
2290 : }
2291 :
2292 : #[instrument(skip_all, fields(shard_id))]
2293 : async fn handle_db_size_request(
2294 : timeline: &Timeline,
2295 : req: &PagestreamDbSizeRequest,
2296 : ctx: &RequestContext,
2297 : ) -> Result<PagestreamDbSizeResponse, PageStreamError> {
2298 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
2299 : let lsn = Self::wait_or_get_last_lsn(
2300 : timeline,
2301 : req.hdr.request_lsn,
2302 : req.hdr.not_modified_since,
2303 : &latest_gc_cutoff_lsn,
2304 : ctx,
2305 : )
2306 : .await?;
2307 :
2308 : let total_blocks = timeline
2309 : .get_db_size(
2310 : DEFAULTTABLESPACE_OID,
2311 : req.dbnode,
2312 : Version::LsnRange(LsnRange {
2313 : effective_lsn: lsn,
2314 : request_lsn: req.hdr.request_lsn,
2315 : }),
2316 : ctx,
2317 : )
2318 : .await?;
2319 : let db_size = total_blocks as i64 * BLCKSZ as i64;
2320 :
2321 : Ok(PagestreamDbSizeResponse { req: *req, db_size })
2322 : }
2323 :
2324 : #[instrument(skip_all)]
2325 : async fn handle_get_page_at_lsn_request_batched(
2326 : timeline: &Timeline,
2327 : requests: SmallVec<[BatchedGetPageRequest; 1]>,
2328 : io_concurrency: IoConcurrency,
2329 : batch_break_reason: GetPageBatchBreakReason,
2330 : ctx: &RequestContext,
2331 : ) -> Vec<Result<(PagestreamBeMessage, SmgrOpTimer, RequestContext), BatchedPageStreamError>>
2332 : {
2333 : debug_assert_current_span_has_tenant_and_timeline_id();
2334 :
2335 : timeline
2336 : .query_metrics
2337 : .observe_getpage_batch_start(requests.len(), batch_break_reason);
2338 :
2339 : // If a page trace is running, submit an event for this request.
2340 : if let Some(page_trace) = timeline.page_trace.load().as_ref() {
2341 : let time = SystemTime::now();
2342 : for batch in &requests {
2343 : let key = rel_block_to_key(batch.req.rel, batch.req.blkno).to_compact();
2344 : // Ignore error (trace buffer may be full or tracer may have disconnected).
2345 : _ = page_trace.try_send(PageTraceEvent {
2346 : key,
2347 : effective_lsn: batch.lsn_range.effective_lsn,
2348 : time,
2349 : });
2350 : }
2351 : }
2352 :
2353 : // If any request in the batch needs to wait for LSN, then do so now.
2354 : let mut perf_instrument = false;
2355 : let max_effective_lsn = requests
2356 : .iter()
2357 0 : .map(|req| {
2358 0 : if req.ctx.has_perf_span() {
2359 0 : perf_instrument = true;
2360 0 : }
2361 :
2362 0 : req.lsn_range.effective_lsn
2363 0 : })
2364 : .max()
2365 : .expect("batch is never empty");
2366 :
2367 : let ctx = match perf_instrument {
2368 : true => RequestContextBuilder::from(ctx)
2369 0 : .root_perf_span(|| {
2370 0 : info_span!(
2371 : target: PERF_TRACE_TARGET,
2372 : "GET_VECTORED",
2373 : tenant_id = %timeline.tenant_shard_id.tenant_id,
2374 : timeline_id = %timeline.timeline_id,
2375 0 : shard = %timeline.tenant_shard_id.shard_slug(),
2376 : %max_effective_lsn
2377 : )
2378 0 : })
2379 : .attached_child(),
2380 : false => ctx.attached_child(),
2381 : };
2382 :
2383 : let last_record_lsn = timeline.get_last_record_lsn();
2384 : if max_effective_lsn > last_record_lsn {
2385 : if let Err(e) = timeline
2386 : .wait_lsn(
2387 : max_effective_lsn,
2388 : crate::tenant::timeline::WaitLsnWaiter::PageService,
2389 : timeline::WaitLsnTimeout::Default,
2390 : &ctx,
2391 : )
2392 0 : .maybe_perf_instrument(&ctx, |current_perf_span| {
2393 0 : info_span!(
2394 : target: PERF_TRACE_TARGET,
2395 0 : parent: current_perf_span,
2396 : "WAIT_LSN",
2397 : )
2398 0 : })
2399 : .await
2400 : {
2401 0 : return Vec::from_iter(requests.into_iter().map(|req| {
2402 0 : Err(BatchedPageStreamError {
2403 0 : err: PageStreamError::from(e.clone()),
2404 0 : req: req.req.hdr,
2405 0 : })
2406 0 : }));
2407 : }
2408 : }
2409 :
2410 : let results = timeline
2411 : .get_rel_page_at_lsn_batched(
2412 0 : requests.iter().map(|p| {
2413 0 : (
2414 0 : &p.req.rel,
2415 0 : &p.req.blkno,
2416 0 : p.lsn_range,
2417 0 : p.ctx.attached_child(),
2418 0 : )
2419 0 : }),
2420 : io_concurrency,
2421 : &ctx,
2422 : )
2423 : .await;
2424 : assert_eq!(results.len(), requests.len());
2425 :
2426 : // TODO: avoid creating the new Vec here
2427 : Vec::from_iter(
2428 : requests
2429 : .into_iter()
2430 : .zip(results.into_iter())
2431 0 : .map(|(req, res)| {
2432 0 : res.map(|page| {
2433 0 : (
2434 0 : PagestreamBeMessage::GetPage(
2435 0 : pagestream_api::PagestreamGetPageResponse { req: req.req, page },
2436 0 : ),
2437 0 : req.timer,
2438 0 : req.ctx,
2439 0 : )
2440 0 : })
2441 0 : .map_err(|e| BatchedPageStreamError {
2442 0 : err: PageStreamError::from(e),
2443 0 : req: req.req.hdr,
2444 0 : })
2445 0 : }),
2446 : )
2447 : }
2448 :
2449 : #[instrument(skip_all, fields(shard_id))]
2450 : async fn handle_get_slru_segment_request(
2451 : timeline: &Timeline,
2452 : req: &PagestreamGetSlruSegmentRequest,
2453 : ctx: &RequestContext,
2454 : ) -> Result<PagestreamGetSlruSegmentResponse, PageStreamError> {
2455 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
2456 : let lsn = Self::wait_or_get_last_lsn(
2457 : timeline,
2458 : req.hdr.request_lsn,
2459 : req.hdr.not_modified_since,
2460 : &latest_gc_cutoff_lsn,
2461 : ctx,
2462 : )
2463 : .await?;
2464 :
2465 : let kind = SlruKind::from_repr(req.kind)
2466 : .ok_or(PageStreamError::BadRequest("invalid SLRU kind".into()))?;
2467 : let segment = timeline.get_slru_segment(kind, req.segno, lsn, ctx).await?;
2468 :
2469 : Ok(PagestreamGetSlruSegmentResponse { req: *req, segment })
2470 : }
2471 :
2472 : // NB: this impl mimics what we do for batched getpage requests.
2473 : #[cfg(feature = "testing")]
2474 : #[instrument(skip_all, fields(shard_id))]
2475 : async fn handle_test_request_batch(
2476 : timeline: &Timeline,
2477 : requests: Vec<BatchedTestRequest>,
2478 : _ctx: &RequestContext,
2479 : ) -> Vec<Result<(PagestreamBeMessage, SmgrOpTimer, RequestContext), BatchedPageStreamError>>
2480 : {
2481 : // real requests would do something with the timeline
2482 : let mut results = Vec::with_capacity(requests.len());
2483 : for _req in requests.iter() {
2484 : tokio::task::yield_now().await;
2485 :
2486 : results.push({
2487 : if timeline.cancel.is_cancelled() {
2488 : Err(PageReconstructError::Cancelled)
2489 : } else {
2490 : Ok(())
2491 : }
2492 : });
2493 : }
2494 :
2495 : // TODO: avoid creating the new Vec here
2496 : Vec::from_iter(
2497 : requests
2498 : .into_iter()
2499 : .zip(results.into_iter())
2500 0 : .map(|(req, res)| {
2501 0 : res.map(|()| {
2502 0 : (
2503 0 : PagestreamBeMessage::Test(pagestream_api::PagestreamTestResponse {
2504 0 : req: req.req.clone(),
2505 0 : }),
2506 0 : req.timer,
2507 0 : RequestContext::new(
2508 0 : TaskKind::PageRequestHandler,
2509 0 : DownloadBehavior::Warn,
2510 0 : ),
2511 0 : )
2512 0 : })
2513 0 : .map_err(|e| BatchedPageStreamError {
2514 0 : err: PageStreamError::from(e),
2515 0 : req: req.req.hdr,
2516 0 : })
2517 0 : }),
2518 : )
2519 : }
2520 :
2521 : /// Note on "fullbackup":
2522 : /// Full basebackups should only be used for debugging purposes.
2523 : /// Originally, it was introduced to enable breaking storage format changes,
2524 : /// but that is not applicable anymore.
2525 : ///
2526 : /// # Coding Discipline
2527 : ///
2528 : /// Coding discipline within this function: all interaction with the `pgb` connection
2529 : /// needs to be sensitive to connection shutdown, currently signalled via [`Self::cancel`].
2530 : /// This is so that we can shutdown page_service quickly.
2531 : ///
2532 : /// TODO: wrap the pgb that we pass to the basebackup handler so that it's sensitive
2533 : /// to connection cancellation.
2534 : #[allow(clippy::too_many_arguments)]
2535 : #[instrument(skip_all, fields(shard_id, ?lsn, ?prev_lsn, %full_backup))]
2536 : async fn handle_basebackup_request<IO>(
2537 : &mut self,
2538 : pgb: &mut PostgresBackend<IO>,
2539 : tenant_id: TenantId,
2540 : timeline_id: TimelineId,
2541 : lsn: Option<Lsn>,
2542 : prev_lsn: Option<Lsn>,
2543 : full_backup: bool,
2544 : gzip: bool,
2545 : replica: bool,
2546 : ctx: &RequestContext,
2547 : ) -> Result<(), QueryError>
2548 : where
2549 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
2550 : {
2551 : let started = std::time::Instant::now();
2552 :
2553 : let timeline = self
2554 : .timeline_handles
2555 : .as_mut()
2556 : .unwrap()
2557 : .get(tenant_id, timeline_id, ShardSelector::Zero)
2558 : .await?;
2559 : set_tracing_field_shard_id(&timeline);
2560 : let ctx = ctx.with_scope_timeline(&timeline);
2561 :
2562 : if timeline.is_archived() == Some(true) {
2563 : tracing::info!(
2564 : "timeline {tenant_id}/{timeline_id} is archived, but got basebackup request for it."
2565 : );
2566 : return Err(QueryError::NotFound("timeline is archived".into()));
2567 : }
2568 :
2569 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
2570 : if let Some(lsn) = lsn {
2571 : // Backup was requested at a particular LSN. Wait for it to arrive.
2572 : info!("waiting for {}", lsn);
2573 : timeline
2574 : .wait_lsn(
2575 : lsn,
2576 : crate::tenant::timeline::WaitLsnWaiter::PageService,
2577 : crate::tenant::timeline::WaitLsnTimeout::Default,
2578 : &ctx,
2579 : )
2580 : .await?;
2581 : timeline
2582 : .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn)
2583 : .context("invalid basebackup lsn")?;
2584 : }
2585 :
2586 : let lsn_awaited_after = started.elapsed();
2587 :
2588 : // switch client to COPYOUT
2589 : pgb.write_message_noflush(&BeMessage::CopyOutResponse)
2590 : .map_err(QueryError::Disconnected)?;
2591 : self.flush_cancellable(pgb, &self.cancel).await?;
2592 :
2593 : let mut from_cache = false;
2594 :
2595 : // Send a tarball of the latest layer on the timeline. Compress if not
2596 : // fullbackup. TODO Compress in that case too (tests need to be updated)
2597 : if full_backup {
2598 : let mut writer = pgb.copyout_writer();
2599 : basebackup::send_basebackup_tarball(
2600 : &mut writer,
2601 : &timeline,
2602 : lsn,
2603 : prev_lsn,
2604 : full_backup,
2605 : replica,
2606 : None,
2607 : &ctx,
2608 : )
2609 : .await?;
2610 : } else {
2611 : let mut writer = BufWriter::new(pgb.copyout_writer());
2612 :
2613 : let cached = timeline
2614 : .get_cached_basebackup_if_enabled(lsn, prev_lsn, full_backup, replica, gzip)
2615 : .await;
2616 :
2617 : if let Some(mut cached) = cached {
2618 : from_cache = true;
2619 : tokio::io::copy(&mut cached, &mut writer)
2620 : .await
2621 0 : .map_err(|err| {
2622 0 : BasebackupError::Client(err, "handle_basebackup_request,cached,copy")
2623 0 : })?;
2624 : } else {
2625 : basebackup::send_basebackup_tarball(
2626 : &mut writer,
2627 : &timeline,
2628 : lsn,
2629 : prev_lsn,
2630 : full_backup,
2631 : replica,
2632 : // NB: using fast compression because it's on the critical path for compute
2633 : // startup. For an empty database, we get <100KB with this method. The
2634 : // Level::Best compression method gives us <20KB, but maybe we should add
2635 : // basebackup caching on compute shutdown first.
2636 : gzip.then_some(async_compression::Level::Fastest),
2637 : &ctx,
2638 : )
2639 : .await?;
2640 : }
2641 : writer
2642 : .flush()
2643 : .await
2644 0 : .map_err(|err| BasebackupError::Client(err, "handle_basebackup_request,flush"))?;
2645 : }
2646 :
2647 : pgb.write_message_noflush(&BeMessage::CopyDone)
2648 : .map_err(QueryError::Disconnected)?;
2649 : self.flush_cancellable(pgb, &timeline.cancel).await?;
2650 :
2651 : let basebackup_after = started
2652 : .elapsed()
2653 : .checked_sub(lsn_awaited_after)
2654 : .unwrap_or(Duration::ZERO);
2655 :
2656 : info!(
2657 : lsn_await_millis = lsn_awaited_after.as_millis(),
2658 : basebackup_millis = basebackup_after.as_millis(),
2659 : %from_cache,
2660 : "basebackup complete"
2661 : );
2662 :
2663 : Ok(())
2664 : }
2665 :
2666 : // when accessing management api supply None as an argument
2667 : // when using to authorize tenant pass corresponding tenant id
2668 0 : fn check_permission(&self, tenant_id: Option<TenantId>) -> Result<(), QueryError> {
2669 0 : if self.auth.is_none() {
2670 : // auth is set to Trust, nothing to check so just return ok
2671 0 : return Ok(());
2672 0 : }
2673 : // auth is some, just checked above, when auth is some
2674 : // then claims are always present because of checks during connection init
2675 : // so this expect won't trigger
2676 0 : let claims = self
2677 0 : .claims
2678 0 : .as_ref()
2679 0 : .expect("claims presence already checked");
2680 0 : check_permission(claims, tenant_id).map_err(|e| QueryError::Unauthorized(e.0))
2681 0 : }
2682 : }
2683 :
2684 : /// `basebackup tenant timeline [lsn] [--gzip] [--replica]`
2685 : #[derive(Debug, Clone, Eq, PartialEq)]
2686 : struct BaseBackupCmd {
2687 : tenant_id: TenantId,
2688 : timeline_id: TimelineId,
2689 : lsn: Option<Lsn>,
2690 : gzip: bool,
2691 : replica: bool,
2692 : }
2693 :
2694 : /// `fullbackup tenant timeline [lsn] [prev_lsn]`
2695 : #[derive(Debug, Clone, Eq, PartialEq)]
2696 : struct FullBackupCmd {
2697 : tenant_id: TenantId,
2698 : timeline_id: TimelineId,
2699 : lsn: Option<Lsn>,
2700 : prev_lsn: Option<Lsn>,
2701 : }
2702 :
2703 : /// `pagestream_v2 tenant timeline`
2704 : #[derive(Debug, Clone, Eq, PartialEq)]
2705 : struct PageStreamCmd {
2706 : tenant_id: TenantId,
2707 : timeline_id: TimelineId,
2708 : protocol_version: PagestreamProtocolVersion,
2709 : }
2710 :
2711 : /// `lease lsn tenant timeline lsn`
2712 : #[derive(Debug, Clone, Eq, PartialEq)]
2713 : struct LeaseLsnCmd {
2714 : tenant_shard_id: TenantShardId,
2715 : timeline_id: TimelineId,
2716 : lsn: Lsn,
2717 : }
2718 :
2719 : #[derive(Debug, Clone, Eq, PartialEq)]
2720 : enum PageServiceCmd {
2721 : Set,
2722 : PageStream(PageStreamCmd),
2723 : BaseBackup(BaseBackupCmd),
2724 : FullBackup(FullBackupCmd),
2725 : LeaseLsn(LeaseLsnCmd),
2726 : }
2727 :
2728 : impl PageStreamCmd {
2729 3 : fn parse(query: &str, protocol_version: PagestreamProtocolVersion) -> anyhow::Result<Self> {
2730 3 : let parameters = query.split_whitespace().collect_vec();
2731 3 : if parameters.len() != 2 {
2732 1 : bail!(
2733 1 : "invalid number of parameters for pagestream command: {}",
2734 : query
2735 : );
2736 2 : }
2737 2 : let tenant_id = TenantId::from_str(parameters[0])
2738 2 : .with_context(|| format!("Failed to parse tenant id from {}", parameters[0]))?;
2739 1 : let timeline_id = TimelineId::from_str(parameters[1])
2740 1 : .with_context(|| format!("Failed to parse timeline id from {}", parameters[1]))?;
2741 1 : Ok(Self {
2742 1 : tenant_id,
2743 1 : timeline_id,
2744 1 : protocol_version,
2745 1 : })
2746 3 : }
2747 : }
2748 :
2749 : impl FullBackupCmd {
2750 2 : fn parse(query: &str) -> anyhow::Result<Self> {
2751 2 : let parameters = query.split_whitespace().collect_vec();
2752 2 : if parameters.len() < 2 || parameters.len() > 4 {
2753 0 : bail!(
2754 0 : "invalid number of parameters for basebackup command: {}",
2755 : query
2756 : );
2757 2 : }
2758 2 : let tenant_id = TenantId::from_str(parameters[0])
2759 2 : .with_context(|| format!("Failed to parse tenant id from {}", parameters[0]))?;
2760 2 : let timeline_id = TimelineId::from_str(parameters[1])
2761 2 : .with_context(|| format!("Failed to parse timeline id from {}", parameters[1]))?;
2762 : // The caller is responsible for providing correct lsn and prev_lsn.
2763 2 : let lsn = if let Some(lsn_str) = parameters.get(2) {
2764 : Some(
2765 1 : Lsn::from_str(lsn_str)
2766 1 : .with_context(|| format!("Failed to parse Lsn from {lsn_str}"))?,
2767 : )
2768 : } else {
2769 1 : None
2770 : };
2771 2 : let prev_lsn = if let Some(prev_lsn_str) = parameters.get(3) {
2772 : Some(
2773 1 : Lsn::from_str(prev_lsn_str)
2774 1 : .with_context(|| format!("Failed to parse Lsn from {prev_lsn_str}"))?,
2775 : )
2776 : } else {
2777 1 : None
2778 : };
2779 2 : Ok(Self {
2780 2 : tenant_id,
2781 2 : timeline_id,
2782 2 : lsn,
2783 2 : prev_lsn,
2784 2 : })
2785 2 : }
2786 : }
2787 :
2788 : impl BaseBackupCmd {
2789 9 : fn parse(query: &str) -> anyhow::Result<Self> {
2790 9 : let parameters = query.split_whitespace().collect_vec();
2791 9 : if parameters.len() < 2 {
2792 0 : bail!(
2793 0 : "invalid number of parameters for basebackup command: {}",
2794 : query
2795 : );
2796 9 : }
2797 9 : let tenant_id = TenantId::from_str(parameters[0])
2798 9 : .with_context(|| format!("Failed to parse tenant id from {}", parameters[0]))?;
2799 9 : let timeline_id = TimelineId::from_str(parameters[1])
2800 9 : .with_context(|| format!("Failed to parse timeline id from {}", parameters[1]))?;
2801 : let lsn;
2802 : let flags_parse_from;
2803 9 : if let Some(maybe_lsn) = parameters.get(2) {
2804 8 : if *maybe_lsn == "latest" {
2805 1 : lsn = None;
2806 1 : flags_parse_from = 3;
2807 7 : } else if maybe_lsn.starts_with("--") {
2808 5 : lsn = None;
2809 5 : flags_parse_from = 2;
2810 5 : } else {
2811 : lsn = Some(
2812 2 : Lsn::from_str(maybe_lsn)
2813 2 : .with_context(|| format!("Failed to parse lsn from {maybe_lsn}"))?,
2814 : );
2815 2 : flags_parse_from = 3;
2816 : }
2817 1 : } else {
2818 1 : lsn = None;
2819 1 : flags_parse_from = 2;
2820 1 : }
2821 :
2822 9 : let mut gzip = false;
2823 9 : let mut replica = false;
2824 :
2825 11 : for ¶m in ¶meters[flags_parse_from..] {
2826 11 : match param {
2827 11 : "--gzip" => {
2828 7 : if gzip {
2829 1 : bail!("duplicate parameter for basebackup command: {param}")
2830 6 : }
2831 6 : gzip = true
2832 : }
2833 4 : "--replica" => {
2834 2 : if replica {
2835 0 : bail!("duplicate parameter for basebackup command: {param}")
2836 2 : }
2837 2 : replica = true
2838 : }
2839 2 : _ => bail!("invalid parameter for basebackup command: {param}"),
2840 : }
2841 : }
2842 6 : Ok(Self {
2843 6 : tenant_id,
2844 6 : timeline_id,
2845 6 : lsn,
2846 6 : gzip,
2847 6 : replica,
2848 6 : })
2849 9 : }
2850 : }
2851 :
2852 : impl LeaseLsnCmd {
2853 2 : fn parse(query: &str) -> anyhow::Result<Self> {
2854 2 : let parameters = query.split_whitespace().collect_vec();
2855 2 : if parameters.len() != 3 {
2856 0 : bail!(
2857 0 : "invalid number of parameters for lease lsn command: {}",
2858 : query
2859 : );
2860 2 : }
2861 2 : let tenant_shard_id = TenantShardId::from_str(parameters[0])
2862 2 : .with_context(|| format!("Failed to parse tenant id from {}", parameters[0]))?;
2863 2 : let timeline_id = TimelineId::from_str(parameters[1])
2864 2 : .with_context(|| format!("Failed to parse timeline id from {}", parameters[1]))?;
2865 2 : let lsn = Lsn::from_str(parameters[2])
2866 2 : .with_context(|| format!("Failed to parse lsn from {}", parameters[2]))?;
2867 2 : Ok(Self {
2868 2 : tenant_shard_id,
2869 2 : timeline_id,
2870 2 : lsn,
2871 2 : })
2872 2 : }
2873 : }
2874 :
2875 : impl PageServiceCmd {
2876 21 : fn parse(query: &str) -> anyhow::Result<Self> {
2877 21 : let query = query.trim();
2878 21 : let Some((cmd, other)) = query.split_once(' ') else {
2879 2 : bail!("cannot parse query: {query}")
2880 : };
2881 19 : match cmd.to_ascii_lowercase().as_str() {
2882 19 : "pagestream_v2" => Ok(Self::PageStream(PageStreamCmd::parse(
2883 3 : other,
2884 3 : PagestreamProtocolVersion::V2,
2885 2 : )?)),
2886 16 : "pagestream_v3" => Ok(Self::PageStream(PageStreamCmd::parse(
2887 0 : other,
2888 0 : PagestreamProtocolVersion::V3,
2889 0 : )?)),
2890 16 : "basebackup" => Ok(Self::BaseBackup(BaseBackupCmd::parse(other)?)),
2891 7 : "fullbackup" => Ok(Self::FullBackup(FullBackupCmd::parse(other)?)),
2892 5 : "lease" => {
2893 3 : let Some((cmd2, other)) = other.split_once(' ') else {
2894 0 : bail!("invalid lease command: {cmd}");
2895 : };
2896 3 : let cmd2 = cmd2.to_ascii_lowercase();
2897 3 : if cmd2 == "lsn" {
2898 2 : Ok(Self::LeaseLsn(LeaseLsnCmd::parse(other)?))
2899 : } else {
2900 1 : bail!("invalid lease command: {cmd}");
2901 : }
2902 : }
2903 2 : "set" => Ok(Self::Set),
2904 0 : _ => Err(anyhow::anyhow!("unsupported command {cmd} in {query}")),
2905 : }
2906 21 : }
2907 : }
2908 :
2909 : /// Parse the startup options from the postgres wire protocol startup packet.
2910 : ///
2911 : /// It takes a sequence of `-c option=X` or `-coption=X`. It parses the options string
2912 : /// by best effort and returns all the options parsed (key-value pairs) and a bool indicating
2913 : /// whether all options are successfully parsed. There could be duplicates in the options
2914 : /// if the caller passed such parameters.
2915 7 : fn parse_options(options: &str) -> (Vec<(String, String)>, bool) {
2916 7 : let mut parsing_config = false;
2917 7 : let mut has_error = false;
2918 7 : let mut config = Vec::new();
2919 16 : for item in options.split_whitespace() {
2920 16 : if item == "-c" {
2921 9 : if !parsing_config {
2922 8 : parsing_config = true;
2923 8 : } else {
2924 : // "-c" followed with another "-c"
2925 1 : tracing::warn!("failed to parse the startup options: {options}");
2926 1 : has_error = true;
2927 1 : break;
2928 : }
2929 7 : } else if item.starts_with("-c") || parsing_config {
2930 7 : let Some((mut key, value)) = item.split_once('=') else {
2931 : // "-c" followed with an invalid option
2932 1 : tracing::warn!("failed to parse the startup options: {options}");
2933 1 : has_error = true;
2934 1 : break;
2935 : };
2936 6 : if !parsing_config {
2937 : // Parse "-coptions=X"
2938 1 : let Some(stripped_key) = key.strip_prefix("-c") else {
2939 0 : tracing::warn!("failed to parse the startup options: {options}");
2940 0 : has_error = true;
2941 0 : break;
2942 : };
2943 1 : key = stripped_key;
2944 5 : }
2945 6 : config.push((key.to_string(), value.to_string()));
2946 6 : parsing_config = false;
2947 : } else {
2948 0 : tracing::warn!("failed to parse the startup options: {options}");
2949 0 : has_error = true;
2950 0 : break;
2951 : }
2952 : }
2953 7 : if parsing_config {
2954 : // "-c" without the option
2955 3 : tracing::warn!("failed to parse the startup options: {options}");
2956 3 : has_error = true;
2957 4 : }
2958 7 : (config, has_error)
2959 7 : }
2960 :
2961 : impl<IO> postgres_backend::Handler<IO> for PageServerHandler
2962 : where
2963 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin + 'static,
2964 : {
2965 0 : fn check_auth_jwt(
2966 0 : &mut self,
2967 0 : _pgb: &mut PostgresBackend<IO>,
2968 0 : jwt_response: &[u8],
2969 0 : ) -> Result<(), QueryError> {
2970 : // this unwrap is never triggered, because check_auth_jwt only called when auth_type is NeonJWT
2971 : // which requires auth to be present
2972 0 : let data: TokenData<Claims> = self
2973 0 : .auth
2974 0 : .as_ref()
2975 0 : .unwrap()
2976 0 : .decode(str::from_utf8(jwt_response).context("jwt response is not UTF-8")?)
2977 0 : .map_err(|e| QueryError::Unauthorized(e.0))?;
2978 :
2979 0 : if matches!(data.claims.scope, Scope::Tenant) && data.claims.tenant_id.is_none() {
2980 0 : return Err(QueryError::Unauthorized(
2981 0 : "jwt token scope is Tenant, but tenant id is missing".into(),
2982 0 : ));
2983 0 : }
2984 :
2985 0 : debug!(
2986 0 : "jwt scope check succeeded for scope: {:#?} by tenant id: {:?}",
2987 : data.claims.scope, data.claims.tenant_id,
2988 : );
2989 :
2990 0 : self.claims = Some(data.claims);
2991 0 : Ok(())
2992 0 : }
2993 :
2994 0 : fn startup(
2995 0 : &mut self,
2996 0 : _pgb: &mut PostgresBackend<IO>,
2997 0 : sm: &FeStartupPacket,
2998 0 : ) -> Result<(), QueryError> {
2999 0 : fail::fail_point!("ps::connection-start::startup-packet");
3000 :
3001 0 : if let FeStartupPacket::StartupMessage { params, .. } = sm {
3002 0 : if let Some(app_name) = params.get("application_name") {
3003 0 : self.perf_span_fields.application_name = Some(app_name.to_string());
3004 0 : Span::current().record("application_name", field::display(app_name));
3005 0 : }
3006 0 : if let Some(options) = params.get("options") {
3007 0 : let (config, _) = parse_options(options);
3008 0 : for (key, value) in config {
3009 0 : if key == "neon.compute_mode" {
3010 0 : self.perf_span_fields.compute_mode = Some(value.clone());
3011 0 : Span::current().record("compute_mode", field::display(value));
3012 0 : }
3013 : }
3014 0 : }
3015 0 : };
3016 :
3017 0 : Ok(())
3018 0 : }
3019 :
3020 : #[instrument(skip_all, fields(tenant_id, timeline_id))]
3021 : async fn process_query(
3022 : &mut self,
3023 : pgb: &mut PostgresBackend<IO>,
3024 : query_string: &str,
3025 : ) -> Result<(), QueryError> {
3026 0 : fail::fail_point!("simulated-bad-compute-connection", |_| {
3027 0 : info!("Hit failpoint for bad connection");
3028 0 : Err(QueryError::SimulatedConnectionError)
3029 0 : });
3030 :
3031 : fail::fail_point!("ps::connection-start::process-query");
3032 :
3033 : let ctx = self.connection_ctx.attached_child();
3034 : debug!("process query {query_string}");
3035 : let query = PageServiceCmd::parse(query_string)?;
3036 : match query {
3037 : PageServiceCmd::PageStream(PageStreamCmd {
3038 : tenant_id,
3039 : timeline_id,
3040 : protocol_version,
3041 : }) => {
3042 : tracing::Span::current()
3043 : .record("tenant_id", field::display(tenant_id))
3044 : .record("timeline_id", field::display(timeline_id));
3045 :
3046 : self.check_permission(Some(tenant_id))?;
3047 : let command_kind = match protocol_version {
3048 : PagestreamProtocolVersion::V2 => ComputeCommandKind::PageStreamV2,
3049 : PagestreamProtocolVersion::V3 => ComputeCommandKind::PageStreamV3,
3050 : };
3051 : COMPUTE_COMMANDS_COUNTERS.for_command(command_kind).inc();
3052 :
3053 : self.handle_pagerequests(pgb, tenant_id, timeline_id, protocol_version, ctx)
3054 : .await?;
3055 : }
3056 : PageServiceCmd::BaseBackup(BaseBackupCmd {
3057 : tenant_id,
3058 : timeline_id,
3059 : lsn,
3060 : gzip,
3061 : replica,
3062 : }) => {
3063 : tracing::Span::current()
3064 : .record("tenant_id", field::display(tenant_id))
3065 : .record("timeline_id", field::display(timeline_id));
3066 :
3067 : self.check_permission(Some(tenant_id))?;
3068 :
3069 : COMPUTE_COMMANDS_COUNTERS
3070 : .for_command(ComputeCommandKind::Basebackup)
3071 : .inc();
3072 : let metric_recording = metrics::BASEBACKUP_QUERY_TIME.start_recording();
3073 0 : let res = async {
3074 0 : self.handle_basebackup_request(
3075 0 : pgb,
3076 0 : tenant_id,
3077 0 : timeline_id,
3078 0 : lsn,
3079 0 : None,
3080 0 : false,
3081 0 : gzip,
3082 0 : replica,
3083 0 : &ctx,
3084 0 : )
3085 0 : .await?;
3086 0 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
3087 0 : Result::<(), QueryError>::Ok(())
3088 0 : }
3089 : .await;
3090 : metric_recording.observe(&res);
3091 : res?;
3092 : }
3093 : // same as basebackup, but result includes relational data as well
3094 : PageServiceCmd::FullBackup(FullBackupCmd {
3095 : tenant_id,
3096 : timeline_id,
3097 : lsn,
3098 : prev_lsn,
3099 : }) => {
3100 : tracing::Span::current()
3101 : .record("tenant_id", field::display(tenant_id))
3102 : .record("timeline_id", field::display(timeline_id));
3103 :
3104 : self.check_permission(Some(tenant_id))?;
3105 :
3106 : COMPUTE_COMMANDS_COUNTERS
3107 : .for_command(ComputeCommandKind::Fullbackup)
3108 : .inc();
3109 :
3110 : // Check that the timeline exists
3111 : self.handle_basebackup_request(
3112 : pgb,
3113 : tenant_id,
3114 : timeline_id,
3115 : lsn,
3116 : prev_lsn,
3117 : true,
3118 : false,
3119 : false,
3120 : &ctx,
3121 : )
3122 : .await?;
3123 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
3124 : }
3125 : PageServiceCmd::Set => {
3126 : // important because psycopg2 executes "SET datestyle TO 'ISO'"
3127 : // on connect
3128 : // TODO: allow setting options, i.e., application_name/compute_mode via SET commands
3129 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
3130 : }
3131 : PageServiceCmd::LeaseLsn(LeaseLsnCmd {
3132 : tenant_shard_id,
3133 : timeline_id,
3134 : lsn,
3135 : }) => {
3136 : tracing::Span::current()
3137 : .record("tenant_id", field::display(tenant_shard_id))
3138 : .record("timeline_id", field::display(timeline_id));
3139 :
3140 : self.check_permission(Some(tenant_shard_id.tenant_id))?;
3141 :
3142 : COMPUTE_COMMANDS_COUNTERS
3143 : .for_command(ComputeCommandKind::LeaseLsn)
3144 : .inc();
3145 :
3146 : match self
3147 : .handle_make_lsn_lease(pgb, tenant_shard_id, timeline_id, lsn, &ctx)
3148 : .await
3149 : {
3150 : Ok(()) => {
3151 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?
3152 : }
3153 : Err(e) => {
3154 : error!("error obtaining lsn lease for {lsn}: {e:?}");
3155 : pgb.write_message_noflush(&BeMessage::ErrorResponse(
3156 : &e.to_string(),
3157 : Some(e.pg_error_code()),
3158 : ))?
3159 : }
3160 : };
3161 : }
3162 : }
3163 :
3164 : Ok(())
3165 : }
3166 : }
3167 :
3168 : /// Serves the page service over gRPC. Dispatches to PageServerHandler for request processing.
3169 : ///
3170 : /// TODO: rename to PageServiceHandler when libpq impl is removed.
3171 : pub struct GrpcPageServiceHandler {
3172 : tenant_manager: Arc<TenantManager>,
3173 : ctx: RequestContext,
3174 : gate_guard: GateGuard,
3175 : get_vectored_concurrent_io: GetVectoredConcurrentIo,
3176 : }
3177 :
3178 : impl GrpcPageServiceHandler {
3179 : /// Spawns a gRPC server for the page service.
3180 : ///
3181 : /// TODO: this doesn't support TLS. We need TLS reloading via ReloadingCertificateResolver, so we
3182 : /// need to reimplement the TCP+TLS accept loop ourselves.
3183 0 : pub fn spawn(
3184 0 : tenant_manager: Arc<TenantManager>,
3185 0 : auth: Option<Arc<SwappableJwtAuth>>,
3186 0 : perf_trace_dispatch: Option<Dispatch>,
3187 0 : get_vectored_concurrent_io: GetVectoredConcurrentIo,
3188 0 : listener: std::net::TcpListener,
3189 0 : ) -> anyhow::Result<CancellableTask> {
3190 0 : let cancel = CancellationToken::new();
3191 0 : let ctx = RequestContextBuilder::new(TaskKind::PageRequestHandler)
3192 0 : .download_behavior(DownloadBehavior::Download)
3193 0 : .perf_span_dispatch(perf_trace_dispatch)
3194 0 : .detached_child();
3195 0 : let gate = Gate::default();
3196 :
3197 : // Set up the TCP socket. We take a preconfigured TcpListener to bind the
3198 : // port early during startup.
3199 0 : let incoming = {
3200 0 : let _runtime = COMPUTE_REQUEST_RUNTIME.enter(); // required by TcpListener::from_std
3201 0 : listener.set_nonblocking(true)?;
3202 0 : tonic::transport::server::TcpIncoming::from(tokio::net::TcpListener::from_std(
3203 0 : listener,
3204 0 : )?)
3205 0 : .with_nodelay(Some(GRPC_TCP_NODELAY))
3206 0 : .with_keepalive(Some(GRPC_TCP_KEEPALIVE_TIME))
3207 : };
3208 :
3209 : // Set up the gRPC server.
3210 : //
3211 : // TODO: consider tuning window sizes.
3212 0 : let mut server = tonic::transport::Server::builder()
3213 0 : .http2_keepalive_interval(Some(GRPC_HTTP2_KEEPALIVE_INTERVAL))
3214 0 : .http2_keepalive_timeout(Some(GRPC_HTTP2_KEEPALIVE_TIMEOUT))
3215 0 : .max_concurrent_streams(Some(GRPC_MAX_CONCURRENT_STREAMS));
3216 :
3217 : // Main page service stack. Uses a mix of Tonic interceptors and Tower layers:
3218 : //
3219 : // * Interceptors: can inspect and modify the gRPC request. Sync code only, runs before service.
3220 : //
3221 : // * Layers: allow async code, can run code after the service response. However, only has access
3222 : // to the raw HTTP request/response, not the gRPC types.
3223 0 : let page_service_handler = GrpcPageServiceHandler {
3224 0 : tenant_manager,
3225 0 : ctx,
3226 0 : gate_guard: gate.enter().expect("gate was just created"),
3227 0 : get_vectored_concurrent_io,
3228 0 : };
3229 :
3230 0 : let observability_layer = ObservabilityLayer;
3231 0 : let mut tenant_interceptor = TenantMetadataInterceptor;
3232 0 : let mut auth_interceptor = TenantAuthInterceptor::new(auth);
3233 :
3234 0 : let page_service = tower::ServiceBuilder::new()
3235 : // Create tracing span and record request start time.
3236 0 : .layer(observability_layer)
3237 : // Intercept gRPC requests.
3238 0 : .layer(tonic::service::InterceptorLayer::new(move |mut req| {
3239 : // Extract tenant metadata.
3240 0 : req = tenant_interceptor.call(req)?;
3241 : // Authenticate tenant JWT token.
3242 0 : req = auth_interceptor.call(req)?;
3243 0 : Ok(req)
3244 0 : }))
3245 : // Run the page service.
3246 0 : .service(
3247 0 : proto::PageServiceServer::new(page_service_handler)
3248 : // Support both gzip and zstd compression. The client decides what to use.
3249 0 : .accept_compressed(tonic::codec::CompressionEncoding::Gzip)
3250 0 : .accept_compressed(tonic::codec::CompressionEncoding::Zstd)
3251 0 : .send_compressed(tonic::codec::CompressionEncoding::Gzip)
3252 0 : .send_compressed(tonic::codec::CompressionEncoding::Zstd),
3253 : );
3254 0 : let server = server.add_service(page_service);
3255 :
3256 : // Reflection service for use with e.g. grpcurl.
3257 0 : let reflection_service = tonic_reflection::server::Builder::configure()
3258 0 : .register_encoded_file_descriptor_set(proto::FILE_DESCRIPTOR_SET)
3259 0 : .build_v1()?;
3260 0 : let server = server.add_service(reflection_service);
3261 :
3262 : // Spawn server task.
3263 0 : let task_cancel = cancel.clone();
3264 0 : let task = COMPUTE_REQUEST_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
3265 : "grpc listener",
3266 0 : async move {
3267 0 : let result = server
3268 0 : .serve_with_incoming_shutdown(incoming, task_cancel.cancelled())
3269 0 : .await;
3270 0 : if result.is_ok() {
3271 : // TODO: revisit shutdown logic once page service is implemented.
3272 0 : gate.close().await;
3273 0 : }
3274 0 : result
3275 0 : },
3276 : ));
3277 :
3278 0 : Ok(CancellableTask { task, cancel })
3279 0 : }
3280 :
3281 : /// Errors if the request is executed on a non-zero shard. Only shard 0 has a complete view of
3282 : /// relations and their sizes, as well as SLRU segments and similar data.
3283 : #[allow(clippy::result_large_err)]
3284 0 : fn ensure_shard_zero(timeline: &Handle<TenantManagerTypes>) -> Result<(), tonic::Status> {
3285 0 : match timeline.get_shard_index().shard_number.0 {
3286 0 : 0 => Ok(()),
3287 0 : shard => Err(tonic::Status::invalid_argument(format!(
3288 0 : "request must execute on shard zero (is shard {shard})",
3289 0 : ))),
3290 : }
3291 0 : }
3292 :
3293 : /// Generates a PagestreamRequest header from a ReadLsn and request ID.
3294 0 : fn make_hdr(read_lsn: page_api::ReadLsn, req_id: u64) -> PagestreamRequest {
3295 0 : PagestreamRequest {
3296 0 : reqid: req_id,
3297 0 : request_lsn: read_lsn.request_lsn,
3298 0 : not_modified_since: read_lsn
3299 0 : .not_modified_since_lsn
3300 0 : .unwrap_or(read_lsn.request_lsn),
3301 0 : }
3302 0 : }
3303 :
3304 : /// Acquires a timeline handle for the given request.
3305 : ///
3306 : /// TODO: during shard splits, the compute may still be sending requests to the parent shard
3307 : /// until the entire split is committed and the compute is notified. Consider installing a
3308 : /// temporary shard router from the parent to the children while the split is in progress.
3309 : ///
3310 : /// TODO: consider moving this to a middleware layer; all requests need it. Needs to manage
3311 : /// the TimelineHandles lifecycle.
3312 : ///
3313 : /// TODO: untangle acquisition from TenantManagerWrapper::resolve() and Cache::get(), to avoid
3314 : /// the unnecessary overhead.
3315 0 : async fn get_request_timeline(
3316 0 : &self,
3317 0 : req: &tonic::Request<impl Any>,
3318 0 : ) -> Result<Handle<TenantManagerTypes>, GetActiveTimelineError> {
3319 0 : let ttid = *extract::<TenantTimelineId>(req);
3320 0 : let shard_index = *extract::<ShardIndex>(req);
3321 0 : let shard_selector = ShardSelector::Known(shard_index);
3322 :
3323 0 : TimelineHandles::new(self.tenant_manager.clone())
3324 0 : .get(ttid.tenant_id, ttid.timeline_id, shard_selector)
3325 0 : .await
3326 0 : }
3327 :
3328 : /// Starts a SmgrOpTimer at received_at, throttles the request, and records execution start.
3329 : /// Only errors if the timeline is shutting down.
3330 : ///
3331 : /// TODO: move timer construction to ObservabilityLayer (see TODO there).
3332 : /// TODO: decouple rate limiting (middleware?), and return SlowDown errors instead.
3333 0 : async fn record_op_start_and_throttle(
3334 0 : timeline: &Handle<TenantManagerTypes>,
3335 0 : op: metrics::SmgrQueryType,
3336 0 : received_at: Instant,
3337 0 : ) -> Result<SmgrOpTimer, tonic::Status> {
3338 0 : let mut timer = PageServerHandler::record_op_start_and_throttle(timeline, op, received_at)
3339 0 : .await
3340 0 : .map_err(|err| match err {
3341 : // record_op_start_and_throttle() only returns Shutdown.
3342 0 : QueryError::Shutdown => tonic::Status::unavailable(format!("{err}")),
3343 0 : err => tonic::Status::internal(format!("unexpected error: {err}")),
3344 0 : })?;
3345 0 : timer.observe_execution_start(Instant::now());
3346 0 : Ok(timer)
3347 0 : }
3348 :
3349 : /// Processes a GetPage batch request, via the GetPages bidirectional streaming RPC.
3350 : ///
3351 : /// NB: errors returned from here are intercepted in get_pages(), and may be converted to a
3352 : /// GetPageResponse with an appropriate status code to avoid terminating the stream.
3353 : ///
3354 : /// TODO: get_vectored() currently enforces a batch limit of 32. Postgres will typically send
3355 : /// batches up to effective_io_concurrency = 100. Either we have to accept large batches, or
3356 : /// split them up in the client or server.
3357 : #[instrument(skip_all, fields(req_id, rel, blkno, blks, req_lsn, mod_lsn))]
3358 : async fn get_page(
3359 : ctx: &RequestContext,
3360 : timeline: &WeakHandle<TenantManagerTypes>,
3361 : req: proto::GetPageRequest,
3362 : io_concurrency: IoConcurrency,
3363 : ) -> Result<proto::GetPageResponse, tonic::Status> {
3364 : let received_at = Instant::now();
3365 : let timeline = timeline.upgrade()?;
3366 : let ctx = ctx.with_scope_page_service_pagestream(&timeline);
3367 :
3368 : // Validate the request, decorate the span, and convert it to a Pagestream request.
3369 : let req = page_api::GetPageRequest::try_from(req)?;
3370 :
3371 : span_record!(
3372 : req_id = %req.request_id,
3373 : rel = %req.rel,
3374 : blkno = %req.block_numbers[0],
3375 : blks = %req.block_numbers.len(),
3376 : lsn = %req.read_lsn,
3377 : );
3378 :
3379 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn(); // hold guard
3380 : let effective_lsn = PageServerHandler::effective_request_lsn(
3381 : &timeline,
3382 : timeline.get_last_record_lsn(),
3383 : req.read_lsn.request_lsn,
3384 : req.read_lsn
3385 : .not_modified_since_lsn
3386 : .unwrap_or(req.read_lsn.request_lsn),
3387 : &latest_gc_cutoff_lsn,
3388 : )?;
3389 :
3390 : let mut batch = SmallVec::with_capacity(req.block_numbers.len());
3391 : for blkno in req.block_numbers {
3392 : // TODO: this creates one timer per page and throttles it. We should have a timer for
3393 : // the entire batch, and throttle only the batch, but this is equivalent to what
3394 : // PageServerHandler does already so we keep it for now.
3395 : let timer = Self::record_op_start_and_throttle(
3396 : &timeline,
3397 : metrics::SmgrQueryType::GetPageAtLsn,
3398 : received_at,
3399 : )
3400 : .await?;
3401 :
3402 : batch.push(BatchedGetPageRequest {
3403 : req: PagestreamGetPageRequest {
3404 : hdr: Self::make_hdr(req.read_lsn, req.request_id),
3405 : rel: req.rel,
3406 : blkno,
3407 : },
3408 : lsn_range: LsnRange {
3409 : effective_lsn,
3410 : request_lsn: req.read_lsn.request_lsn,
3411 : },
3412 : timer,
3413 : ctx: ctx.attached_child(),
3414 : batch_wait_ctx: None, // TODO: add tracing
3415 : });
3416 : }
3417 :
3418 : // TODO: this does a relation size query for every page in the batch. Since this batch is
3419 : // all for one relation, we could do this only once. However, this is not the case for the
3420 : // libpq implementation.
3421 : let results = PageServerHandler::handle_get_page_at_lsn_request_batched(
3422 : &timeline,
3423 : batch,
3424 : io_concurrency,
3425 : GetPageBatchBreakReason::BatchFull, // TODO: not relevant for gRPC batches
3426 : &ctx,
3427 : )
3428 : .await;
3429 :
3430 : let mut resp = page_api::GetPageResponse {
3431 : request_id: req.request_id,
3432 : status_code: page_api::GetPageStatusCode::Ok,
3433 : reason: None,
3434 : page_images: Vec::with_capacity(results.len()),
3435 : };
3436 :
3437 : for result in results {
3438 : match result {
3439 : Ok((PagestreamBeMessage::GetPage(r), _, _)) => resp.page_images.push(r.page),
3440 : Ok((resp, _, _)) => {
3441 : return Err(tonic::Status::internal(format!(
3442 : "unexpected response: {resp:?}"
3443 : )));
3444 : }
3445 : Err(err) => return Err(err.err.into()),
3446 : };
3447 : }
3448 :
3449 : Ok(resp.into())
3450 : }
3451 : }
3452 :
3453 : /// Implements the gRPC page service.
3454 : ///
3455 : /// TODO: cancellation.
3456 : /// TODO: when the libpq impl is removed, remove the Pagestream types and inline the handler code.
3457 : #[tonic::async_trait]
3458 : impl proto::PageService for GrpcPageServiceHandler {
3459 : type GetBaseBackupStream = Pin<
3460 : Box<dyn Stream<Item = Result<proto::GetBaseBackupResponseChunk, tonic::Status>> + Send>,
3461 : >;
3462 :
3463 : type GetPagesStream =
3464 : Pin<Box<dyn Stream<Item = Result<proto::GetPageResponse, tonic::Status>> + Send>>;
3465 :
3466 : #[instrument(skip_all, fields(rel, lsn))]
3467 : async fn check_rel_exists(
3468 : &self,
3469 : req: tonic::Request<proto::CheckRelExistsRequest>,
3470 0 : ) -> Result<tonic::Response<proto::CheckRelExistsResponse>, tonic::Status> {
3471 0 : let received_at = extract::<ReceivedAt>(&req).0;
3472 0 : let timeline = self.get_request_timeline(&req).await?;
3473 0 : let ctx = self.ctx.with_scope_page_service_pagestream(&timeline);
3474 :
3475 : // Validate the request, decorate the span, and convert it to a Pagestream request.
3476 0 : Self::ensure_shard_zero(&timeline)?;
3477 0 : let req: page_api::CheckRelExistsRequest = req.into_inner().try_into()?;
3478 :
3479 0 : span_record!(rel=%req.rel, lsn=%req.read_lsn);
3480 :
3481 0 : let req = PagestreamExistsRequest {
3482 0 : hdr: Self::make_hdr(req.read_lsn, 0),
3483 0 : rel: req.rel,
3484 0 : };
3485 :
3486 : // Execute the request and convert the response.
3487 0 : let _timer = Self::record_op_start_and_throttle(
3488 0 : &timeline,
3489 0 : metrics::SmgrQueryType::GetRelExists,
3490 0 : received_at,
3491 0 : )
3492 0 : .await?;
3493 :
3494 0 : let resp = PageServerHandler::handle_get_rel_exists_request(&timeline, &req, &ctx).await?;
3495 0 : let resp: page_api::CheckRelExistsResponse = resp.exists;
3496 0 : Ok(tonic::Response::new(resp.into()))
3497 0 : }
3498 :
3499 : #[instrument(skip_all, fields(lsn))]
3500 : async fn get_base_backup(
3501 : &self,
3502 : req: tonic::Request<proto::GetBaseBackupRequest>,
3503 0 : ) -> Result<tonic::Response<Self::GetBaseBackupStream>, tonic::Status> {
3504 : // Send chunks of 256 KB to avoid large memory allocations. pagebench basebackup shows this
3505 : // to be the sweet spot where throughput is saturated.
3506 : const CHUNK_SIZE: usize = 256 * 1024;
3507 :
3508 0 : let timeline = self.get_request_timeline(&req).await?;
3509 0 : let ctx = self.ctx.with_scope_timeline(&timeline);
3510 :
3511 : // Validate the request and decorate the span.
3512 0 : Self::ensure_shard_zero(&timeline)?;
3513 0 : if timeline.is_archived() == Some(true) {
3514 0 : return Err(tonic::Status::failed_precondition("timeline is archived"));
3515 0 : }
3516 0 : let req: page_api::GetBaseBackupRequest = req.into_inner().try_into()?;
3517 :
3518 0 : span_record!(lsn=?req.lsn);
3519 :
3520 : // Wait for the LSN to arrive, if given.
3521 0 : if let Some(lsn) = req.lsn {
3522 0 : let latest_gc_cutoff_lsn = timeline.get_applied_gc_cutoff_lsn();
3523 0 : timeline
3524 0 : .wait_lsn(
3525 0 : lsn,
3526 0 : WaitLsnWaiter::PageService,
3527 0 : WaitLsnTimeout::Default,
3528 0 : &ctx,
3529 0 : )
3530 0 : .await?;
3531 0 : timeline
3532 0 : .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn)
3533 0 : .map_err(|err| {
3534 0 : tonic::Status::invalid_argument(format!("invalid basebackup LSN: {err}"))
3535 0 : })?;
3536 0 : }
3537 :
3538 : // Spawn a task to run the basebackup.
3539 0 : let span = Span::current();
3540 0 : let (mut simplex_read, mut simplex_write) = tokio::io::simplex(CHUNK_SIZE);
3541 0 : let jh = tokio::spawn(async move {
3542 0 : let gzip_level = match req.compression {
3543 0 : page_api::BaseBackupCompression::None => None,
3544 : // NB: using fast compression because it's on the critical path for compute
3545 : // startup. For an empty database, we get <100KB with this method. The
3546 : // Level::Best compression method gives us <20KB, but maybe we should add
3547 : // basebackup caching on compute shutdown first.
3548 0 : page_api::BaseBackupCompression::Gzip => Some(async_compression::Level::Fastest),
3549 : };
3550 :
3551 : // Check for a cached basebackup.
3552 0 : let cached = timeline
3553 0 : .get_cached_basebackup_if_enabled(
3554 0 : req.lsn,
3555 0 : None,
3556 0 : req.full,
3557 0 : req.replica,
3558 0 : gzip_level.is_some(),
3559 0 : )
3560 0 : .await;
3561 :
3562 0 : let result = if let Some(mut cached) = cached {
3563 : // If we have a cached basebackup, send it.
3564 0 : tokio::io::copy(&mut cached, &mut simplex_write)
3565 0 : .await
3566 0 : .map(|_| ())
3567 0 : .map_err(|err| BasebackupError::Client(err, "cached,copy"))
3568 : } else {
3569 0 : basebackup::send_basebackup_tarball(
3570 0 : &mut simplex_write,
3571 0 : &timeline,
3572 0 : req.lsn,
3573 0 : None,
3574 0 : req.full,
3575 0 : req.replica,
3576 0 : gzip_level,
3577 0 : &ctx,
3578 0 : )
3579 0 : .instrument(span) // propagate request span
3580 0 : .await
3581 : };
3582 0 : simplex_write
3583 0 : .shutdown()
3584 0 : .await
3585 0 : .map_err(|err| BasebackupError::Client(err, "simplex_write"))?;
3586 0 : result
3587 0 : });
3588 :
3589 : // Emit chunks of size CHUNK_SIZE.
3590 0 : let chunks = async_stream::try_stream! {
3591 : loop {
3592 : let mut chunk = BytesMut::with_capacity(CHUNK_SIZE).limit(CHUNK_SIZE);
3593 : loop {
3594 0 : let n = simplex_read.read_buf(&mut chunk).await.map_err(|err| {
3595 0 : tonic::Status::internal(format!("failed to read basebackup chunk: {err}"))
3596 0 : })?;
3597 : if n == 0 {
3598 : break; // full chunk or closed stream
3599 : }
3600 : }
3601 : let chunk = chunk.into_inner().freeze();
3602 : if chunk.is_empty() {
3603 : break;
3604 : }
3605 : yield proto::GetBaseBackupResponseChunk::from(chunk);
3606 : }
3607 : // Wait for the basebackup task to exit and check for errors.
3608 0 : jh.await.map_err(|err| {
3609 0 : tonic::Status::internal(format!("basebackup failed: {err}"))
3610 0 : })??;
3611 : };
3612 :
3613 0 : Ok(tonic::Response::new(Box::pin(chunks)))
3614 0 : }
3615 :
3616 : #[instrument(skip_all, fields(db_oid, lsn))]
3617 : async fn get_db_size(
3618 : &self,
3619 : req: tonic::Request<proto::GetDbSizeRequest>,
3620 0 : ) -> Result<tonic::Response<proto::GetDbSizeResponse>, tonic::Status> {
3621 0 : let received_at = extract::<ReceivedAt>(&req).0;
3622 0 : let timeline = self.get_request_timeline(&req).await?;
3623 0 : let ctx = self.ctx.with_scope_page_service_pagestream(&timeline);
3624 :
3625 : // Validate the request, decorate the span, and convert it to a Pagestream request.
3626 0 : Self::ensure_shard_zero(&timeline)?;
3627 0 : let req: page_api::GetDbSizeRequest = req.into_inner().try_into()?;
3628 :
3629 0 : span_record!(db_oid=%req.db_oid, lsn=%req.read_lsn);
3630 :
3631 0 : let req = PagestreamDbSizeRequest {
3632 0 : hdr: Self::make_hdr(req.read_lsn, 0),
3633 0 : dbnode: req.db_oid,
3634 0 : };
3635 :
3636 : // Execute the request and convert the response.
3637 0 : let _timer = Self::record_op_start_and_throttle(
3638 0 : &timeline,
3639 0 : metrics::SmgrQueryType::GetDbSize,
3640 0 : received_at,
3641 0 : )
3642 0 : .await?;
3643 :
3644 0 : let resp = PageServerHandler::handle_db_size_request(&timeline, &req, &ctx).await?;
3645 0 : let resp = resp.db_size as page_api::GetDbSizeResponse;
3646 0 : Ok(tonic::Response::new(resp.into()))
3647 0 : }
3648 :
3649 : // NB: don't instrument this, instrument each streamed request.
3650 0 : async fn get_pages(
3651 : &self,
3652 : req: tonic::Request<tonic::Streaming<proto::GetPageRequest>>,
3653 0 : ) -> Result<tonic::Response<Self::GetPagesStream>, tonic::Status> {
3654 : // Extract the timeline from the request and check that it exists.
3655 0 : let ttid = *extract::<TenantTimelineId>(&req);
3656 0 : let shard_index = *extract::<ShardIndex>(&req);
3657 0 : let shard_selector = ShardSelector::Known(shard_index);
3658 :
3659 0 : let mut handles = TimelineHandles::new(self.tenant_manager.clone());
3660 0 : handles
3661 0 : .get(ttid.tenant_id, ttid.timeline_id, shard_selector)
3662 0 : .await?;
3663 :
3664 : // Spawn an IoConcurrency sidecar, if enabled.
3665 0 : let Ok(gate_guard) = self.gate_guard.try_clone() else {
3666 0 : return Err(tonic::Status::unavailable("shutting down"));
3667 : };
3668 0 : let io_concurrency =
3669 0 : IoConcurrency::spawn_from_conf(self.get_vectored_concurrent_io, gate_guard);
3670 :
3671 : // Spawn a task to handle the GetPageRequest stream.
3672 0 : let span = Span::current();
3673 0 : let ctx = self.ctx.attached_child();
3674 0 : let mut reqs = req.into_inner();
3675 :
3676 0 : let resps = async_stream::try_stream! {
3677 : let timeline = handles
3678 : .get(ttid.tenant_id, ttid.timeline_id, shard_selector)
3679 : .await?
3680 : .downgrade();
3681 : while let Some(req) = reqs.message().await? {
3682 : let req_id = req.request_id;
3683 : let result = Self::get_page(&ctx, &timeline, req, io_concurrency.clone())
3684 : .instrument(span.clone()) // propagate request span
3685 : .await;
3686 : yield match result {
3687 : Ok(resp) => resp,
3688 : // Convert per-request errors to GetPageResponses as appropriate, or terminate
3689 : // the stream with a tonic::Status. Log the error regardless, since
3690 : // ObservabilityLayer can't automatically log stream errors.
3691 : Err(status) => {
3692 : // TODO: it would be nice if we could propagate the get_page() fields here.
3693 0 : span.in_scope(|| {
3694 0 : warn!("request failed with {:?}: {}", status.code(), status.message());
3695 0 : });
3696 : page_api::GetPageResponse::try_from_status(status, req_id)?.into()
3697 : }
3698 : }
3699 : }
3700 : };
3701 :
3702 0 : Ok(tonic::Response::new(Box::pin(resps)))
3703 0 : }
3704 :
3705 : #[instrument(skip_all, fields(rel, lsn))]
3706 : async fn get_rel_size(
3707 : &self,
3708 : req: tonic::Request<proto::GetRelSizeRequest>,
3709 0 : ) -> Result<tonic::Response<proto::GetRelSizeResponse>, tonic::Status> {
3710 0 : let received_at = extract::<ReceivedAt>(&req).0;
3711 0 : let timeline = self.get_request_timeline(&req).await?;
3712 0 : let ctx = self.ctx.with_scope_page_service_pagestream(&timeline);
3713 :
3714 : // Validate the request, decorate the span, and convert it to a Pagestream request.
3715 0 : Self::ensure_shard_zero(&timeline)?;
3716 0 : let req: page_api::GetRelSizeRequest = req.into_inner().try_into()?;
3717 :
3718 0 : span_record!(rel=%req.rel, lsn=%req.read_lsn);
3719 :
3720 0 : let req = PagestreamNblocksRequest {
3721 0 : hdr: Self::make_hdr(req.read_lsn, 0),
3722 0 : rel: req.rel,
3723 0 : };
3724 :
3725 : // Execute the request and convert the response.
3726 0 : let _timer = Self::record_op_start_and_throttle(
3727 0 : &timeline,
3728 0 : metrics::SmgrQueryType::GetRelSize,
3729 0 : received_at,
3730 0 : )
3731 0 : .await?;
3732 :
3733 0 : let resp = PageServerHandler::handle_get_nblocks_request(&timeline, &req, &ctx).await?;
3734 0 : let resp: page_api::GetRelSizeResponse = resp.n_blocks;
3735 0 : Ok(tonic::Response::new(resp.into()))
3736 0 : }
3737 :
3738 : #[instrument(skip_all, fields(kind, segno, lsn))]
3739 : async fn get_slru_segment(
3740 : &self,
3741 : req: tonic::Request<proto::GetSlruSegmentRequest>,
3742 0 : ) -> Result<tonic::Response<proto::GetSlruSegmentResponse>, tonic::Status> {
3743 0 : let received_at = extract::<ReceivedAt>(&req).0;
3744 0 : let timeline = self.get_request_timeline(&req).await?;
3745 0 : let ctx = self.ctx.with_scope_page_service_pagestream(&timeline);
3746 :
3747 : // Validate the request, decorate the span, and convert it to a Pagestream request.
3748 0 : Self::ensure_shard_zero(&timeline)?;
3749 0 : let req: page_api::GetSlruSegmentRequest = req.into_inner().try_into()?;
3750 :
3751 0 : span_record!(kind=%req.kind, segno=%req.segno, lsn=%req.read_lsn);
3752 :
3753 0 : let req = PagestreamGetSlruSegmentRequest {
3754 0 : hdr: Self::make_hdr(req.read_lsn, 0),
3755 0 : kind: req.kind as u8,
3756 0 : segno: req.segno,
3757 0 : };
3758 :
3759 : // Execute the request and convert the response.
3760 0 : let _timer = Self::record_op_start_and_throttle(
3761 0 : &timeline,
3762 0 : metrics::SmgrQueryType::GetSlruSegment,
3763 0 : received_at,
3764 0 : )
3765 0 : .await?;
3766 :
3767 0 : let resp =
3768 0 : PageServerHandler::handle_get_slru_segment_request(&timeline, &req, &ctx).await?;
3769 0 : let resp: page_api::GetSlruSegmentResponse = resp.segment;
3770 0 : Ok(tonic::Response::new(resp.into()))
3771 0 : }
3772 :
3773 : #[instrument(skip_all, fields(lsn))]
3774 : async fn lease_lsn(
3775 : &self,
3776 : req: tonic::Request<proto::LeaseLsnRequest>,
3777 0 : ) -> Result<tonic::Response<proto::LeaseLsnResponse>, tonic::Status> {
3778 0 : let timeline = self.get_request_timeline(&req).await?;
3779 0 : let ctx = self.ctx.with_scope_timeline(&timeline);
3780 :
3781 : // Validate and convert the request, and decorate the span.
3782 0 : let req: page_api::LeaseLsnRequest = req.into_inner().try_into()?;
3783 :
3784 0 : span_record!(lsn=%req.lsn);
3785 :
3786 : // Attempt to acquire a lease. Return FailedPrecondition if the lease could not be granted.
3787 0 : let lease_length = timeline.get_lsn_lease_length();
3788 0 : let expires = match timeline.renew_lsn_lease(req.lsn, lease_length, &ctx) {
3789 0 : Ok(lease) => lease.valid_until,
3790 0 : Err(err) => return Err(tonic::Status::failed_precondition(format!("{err}"))),
3791 : };
3792 :
3793 : // TODO: is this spammy? Move it compute-side?
3794 0 : info!(
3795 0 : "acquired lease for {} until {}",
3796 : req.lsn,
3797 0 : chrono::DateTime::<Utc>::from(expires).to_rfc3339()
3798 : );
3799 :
3800 0 : Ok(tonic::Response::new(expires.into()))
3801 0 : }
3802 : }
3803 :
3804 : /// gRPC middleware layer that handles observability concerns:
3805 : ///
3806 : /// * Creates and enters a tracing span.
3807 : /// * Records the request start time as a ReceivedAt request extension.
3808 : ///
3809 : /// TODO: add perf tracing.
3810 : /// TODO: add timing and metrics.
3811 : /// TODO: add logging.
3812 : #[derive(Clone)]
3813 : struct ObservabilityLayer;
3814 :
3815 : impl<S: tonic::server::NamedService> tower::Layer<S> for ObservabilityLayer {
3816 : type Service = ObservabilityLayerService<S>;
3817 :
3818 0 : fn layer(&self, inner: S) -> Self::Service {
3819 0 : Self::Service { inner }
3820 0 : }
3821 : }
3822 :
3823 : #[derive(Clone)]
3824 : struct ObservabilityLayerService<S> {
3825 : inner: S,
3826 : }
3827 :
3828 : #[derive(Clone, Copy)]
3829 : struct ReceivedAt(Instant);
3830 :
3831 : impl<S: tonic::server::NamedService> tonic::server::NamedService for ObservabilityLayerService<S> {
3832 : const NAME: &'static str = S::NAME; // propagate inner service name
3833 : }
3834 :
3835 : impl<S, Req, Resp> tower::Service<http::Request<Req>> for ObservabilityLayerService<S>
3836 : where
3837 : S: tower::Service<http::Request<Req>, Response = http::Response<Resp>> + Send,
3838 : S::Future: Send + 'static,
3839 : {
3840 : type Response = S::Response;
3841 : type Error = S::Error;
3842 : type Future = BoxFuture<'static, Result<Self::Response, Self::Error>>;
3843 :
3844 0 : fn call(&mut self, mut req: http::Request<Req>) -> Self::Future {
3845 : // Record the request start time as a request extension.
3846 : //
3847 : // TODO: we should start a timer here instead, but it currently requires a timeline handle
3848 : // and SmgrQueryType, which we don't have yet. Refactor it to provide it later.
3849 0 : req.extensions_mut().insert(ReceivedAt(Instant::now()));
3850 :
3851 : // Extract the peer address and gRPC method.
3852 0 : let peer = req
3853 0 : .extensions()
3854 0 : .get::<TcpConnectInfo>()
3855 0 : .and_then(|info| info.remote_addr())
3856 0 : .map(|addr| addr.to_string())
3857 0 : .unwrap_or_default();
3858 :
3859 0 : let method = req
3860 0 : .uri()
3861 0 : .path()
3862 0 : .split('/')
3863 0 : .nth(2)
3864 0 : .unwrap_or(req.uri().path())
3865 0 : .to_string();
3866 :
3867 : // Create a basic tracing span.
3868 : //
3869 : // Enter the span for the current thread and instrument the future. It is not sufficient to
3870 : // only instrument the future, since it only takes effect after the future is returned and
3871 : // polled, not when the inner service is called below (e.g. during interceptor execution).
3872 0 : let span = info_span!(
3873 : "grpc:pageservice",
3874 : // These will be populated by TenantMetadataInterceptor.
3875 : tenant_id = field::Empty,
3876 : timeline_id = field::Empty,
3877 : shard_id = field::Empty,
3878 : // NB: empty fields must be listed first above. Otherwise, the field names will be
3879 : // clobbered when the empty fields are populated. They will be output last regardless.
3880 : %peer,
3881 : %method,
3882 : );
3883 0 : let _guard = span.enter();
3884 :
3885 : // Construct a future for calling the inner service, but don't await it. This avoids having
3886 : // to clone the inner service into the future below.
3887 0 : let call = self.inner.call(req);
3888 :
3889 0 : async move {
3890 : // Await the inner service call.
3891 0 : let result = call.await;
3892 :
3893 : // Log gRPC error statuses. This won't include request info from handler spans, but it
3894 : // will catch all errors (even those emitted before handler spans are constructed). Only
3895 : // unary request errors are logged here, not streaming response errors.
3896 0 : if let Ok(ref resp) = result
3897 0 : && let Some(status) = tonic::Status::from_header_map(resp.headers())
3898 0 : && status.code() != tonic::Code::Ok
3899 : {
3900 : // TODO: it would be nice if we could propagate the handler span's request fields
3901 : // here. This could e.g. be done by attaching the request fields to
3902 : // tonic::Status::metadata via a proc macro.
3903 0 : warn!(
3904 0 : "request failed with {:?}: {}",
3905 0 : status.code(),
3906 0 : status.message()
3907 : );
3908 0 : }
3909 :
3910 0 : result
3911 0 : }
3912 0 : .instrument(span.clone())
3913 0 : .boxed()
3914 0 : }
3915 :
3916 0 : fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
3917 0 : self.inner.poll_ready(cx)
3918 0 : }
3919 : }
3920 :
3921 : /// gRPC interceptor that decodes tenant metadata and stores it as request extensions of type
3922 : /// TenantTimelineId and ShardIndex.
3923 : #[derive(Clone)]
3924 : struct TenantMetadataInterceptor;
3925 :
3926 : impl tonic::service::Interceptor for TenantMetadataInterceptor {
3927 0 : fn call(&mut self, mut req: tonic::Request<()>) -> Result<tonic::Request<()>, tonic::Status> {
3928 : // Decode the tenant ID.
3929 0 : let tenant_id = req
3930 0 : .metadata()
3931 0 : .get("neon-tenant-id")
3932 0 : .ok_or_else(|| tonic::Status::invalid_argument("missing neon-tenant-id"))?
3933 0 : .to_str()
3934 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-tenant-id"))?;
3935 0 : let tenant_id = TenantId::from_str(tenant_id)
3936 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-tenant-id"))?;
3937 :
3938 : // Decode the timeline ID.
3939 0 : let timeline_id = req
3940 0 : .metadata()
3941 0 : .get("neon-timeline-id")
3942 0 : .ok_or_else(|| tonic::Status::invalid_argument("missing neon-timeline-id"))?
3943 0 : .to_str()
3944 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-timeline-id"))?;
3945 0 : let timeline_id = TimelineId::from_str(timeline_id)
3946 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-timeline-id"))?;
3947 :
3948 : // Decode the shard ID.
3949 0 : let shard_id = req
3950 0 : .metadata()
3951 0 : .get("neon-shard-id")
3952 0 : .ok_or_else(|| tonic::Status::invalid_argument("missing neon-shard-id"))?
3953 0 : .to_str()
3954 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-shard-id"))?;
3955 0 : let shard_id = ShardIndex::from_str(shard_id)
3956 0 : .map_err(|_| tonic::Status::invalid_argument("invalid neon-shard-id"))?;
3957 :
3958 : // Stash them in the request.
3959 0 : let extensions = req.extensions_mut();
3960 0 : extensions.insert(TenantTimelineId::new(tenant_id, timeline_id));
3961 0 : extensions.insert(shard_id);
3962 :
3963 : // Decorate the tracing span.
3964 0 : span_record!(%tenant_id, %timeline_id, %shard_id);
3965 :
3966 0 : Ok(req)
3967 0 : }
3968 : }
3969 :
3970 : /// Authenticates gRPC page service requests.
3971 : #[derive(Clone)]
3972 : struct TenantAuthInterceptor {
3973 : auth: Option<Arc<SwappableJwtAuth>>,
3974 : }
3975 :
3976 : impl TenantAuthInterceptor {
3977 0 : fn new(auth: Option<Arc<SwappableJwtAuth>>) -> Self {
3978 0 : Self { auth }
3979 0 : }
3980 : }
3981 :
3982 : impl tonic::service::Interceptor for TenantAuthInterceptor {
3983 0 : fn call(&mut self, req: tonic::Request<()>) -> Result<tonic::Request<()>, tonic::Status> {
3984 : // Do nothing if auth is disabled.
3985 0 : let Some(auth) = self.auth.as_ref() else {
3986 0 : return Ok(req);
3987 : };
3988 :
3989 : // Fetch the tenant ID from the request extensions (set by TenantMetadataInterceptor).
3990 0 : let TenantTimelineId { tenant_id, .. } = *extract::<TenantTimelineId>(&req);
3991 :
3992 : // Fetch and decode the JWT token.
3993 0 : let jwt = req
3994 0 : .metadata()
3995 0 : .get("authorization")
3996 0 : .ok_or_else(|| tonic::Status::unauthenticated("no authorization header"))?
3997 0 : .to_str()
3998 0 : .map_err(|_| tonic::Status::invalid_argument("invalid authorization header"))?
3999 0 : .strip_prefix("Bearer ")
4000 0 : .ok_or_else(|| tonic::Status::invalid_argument("invalid authorization header"))?
4001 0 : .trim();
4002 0 : let jwtdata: TokenData<Claims> = auth
4003 0 : .decode(jwt)
4004 0 : .map_err(|err| tonic::Status::invalid_argument(format!("invalid JWT token: {err}")))?;
4005 0 : let claims = jwtdata.claims;
4006 :
4007 : // Check if the token is valid for this tenant.
4008 0 : check_permission(&claims, Some(tenant_id))
4009 0 : .map_err(|err| tonic::Status::permission_denied(err.to_string()))?;
4010 :
4011 : // TODO: consider stashing the claims in the request extensions, if needed.
4012 :
4013 0 : Ok(req)
4014 0 : }
4015 : }
4016 :
4017 : /// Extracts the given type from the request extensions, or panics if it is missing.
4018 0 : fn extract<T: Send + Sync + 'static>(req: &tonic::Request<impl Any>) -> &T {
4019 0 : extract_from(req.extensions())
4020 0 : }
4021 :
4022 : /// Extract the given type from the request extensions, or panics if it is missing. This variant
4023 : /// can extract both from a tonic::Request and http::Request.
4024 0 : fn extract_from<T: Send + Sync + 'static>(ext: &http::Extensions) -> &T {
4025 0 : let Some(value) = ext.get::<T>() else {
4026 0 : let name = std::any::type_name::<T>();
4027 0 : panic!("extension {name} should be set by middleware");
4028 : };
4029 0 : value
4030 0 : }
4031 :
4032 : #[derive(Debug, thiserror::Error)]
4033 : pub(crate) enum GetActiveTimelineError {
4034 : #[error(transparent)]
4035 : Tenant(GetActiveTenantError),
4036 : #[error(transparent)]
4037 : Timeline(#[from] GetTimelineError),
4038 : }
4039 :
4040 : impl From<GetActiveTimelineError> for QueryError {
4041 0 : fn from(e: GetActiveTimelineError) -> Self {
4042 0 : match e {
4043 0 : GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled) => QueryError::Shutdown,
4044 0 : GetActiveTimelineError::Tenant(e) => e.into(),
4045 0 : GetActiveTimelineError::Timeline(e) => QueryError::NotFound(format!("{e}").into()),
4046 : }
4047 0 : }
4048 : }
4049 :
4050 : impl From<GetActiveTimelineError> for tonic::Status {
4051 0 : fn from(err: GetActiveTimelineError) -> Self {
4052 0 : let message = err.to_string();
4053 0 : let code = match err {
4054 0 : GetActiveTimelineError::Tenant(err) => tonic::Status::from(err).code(),
4055 0 : GetActiveTimelineError::Timeline(err) => tonic::Status::from(err).code(),
4056 : };
4057 0 : tonic::Status::new(code, message)
4058 0 : }
4059 : }
4060 :
4061 : impl From<GetTimelineError> for tonic::Status {
4062 0 : fn from(err: GetTimelineError) -> Self {
4063 : use tonic::Code;
4064 0 : let code = match &err {
4065 0 : GetTimelineError::NotFound { .. } => Code::NotFound,
4066 0 : GetTimelineError::NotActive { .. } => Code::Unavailable,
4067 0 : GetTimelineError::ShuttingDown => Code::Unavailable,
4068 : };
4069 0 : tonic::Status::new(code, err.to_string())
4070 0 : }
4071 : }
4072 :
4073 : impl From<GetActiveTenantError> for QueryError {
4074 0 : fn from(e: GetActiveTenantError) -> Self {
4075 0 : match e {
4076 0 : GetActiveTenantError::WaitForActiveTimeout { .. } => QueryError::Disconnected(
4077 0 : ConnectionError::Io(io::Error::new(io::ErrorKind::TimedOut, e.to_string())),
4078 0 : ),
4079 : GetActiveTenantError::Cancelled
4080 : | GetActiveTenantError::WillNotBecomeActive(TenantState::Stopping { .. }) => {
4081 0 : QueryError::Shutdown
4082 : }
4083 0 : e @ GetActiveTenantError::NotFound(_) => QueryError::NotFound(format!("{e}").into()),
4084 0 : e => QueryError::Other(anyhow::anyhow!(e)),
4085 : }
4086 0 : }
4087 : }
4088 :
4089 : impl From<GetActiveTenantError> for tonic::Status {
4090 0 : fn from(err: GetActiveTenantError) -> Self {
4091 : use tonic::Code;
4092 0 : let code = match &err {
4093 0 : GetActiveTenantError::Broken(_) => Code::Internal,
4094 0 : GetActiveTenantError::Cancelled => Code::Unavailable,
4095 0 : GetActiveTenantError::NotFound(_) => Code::NotFound,
4096 0 : GetActiveTenantError::SwitchedTenant => Code::Unavailable,
4097 0 : GetActiveTenantError::WaitForActiveTimeout { .. } => Code::Unavailable,
4098 0 : GetActiveTenantError::WillNotBecomeActive(_) => Code::Unavailable,
4099 : };
4100 0 : tonic::Status::new(code, err.to_string())
4101 0 : }
4102 : }
4103 :
4104 : impl From<HandleUpgradeError> for QueryError {
4105 0 : fn from(e: HandleUpgradeError) -> Self {
4106 0 : match e {
4107 0 : HandleUpgradeError::ShutDown => QueryError::Shutdown,
4108 : }
4109 0 : }
4110 : }
4111 :
4112 : impl From<HandleUpgradeError> for tonic::Status {
4113 0 : fn from(err: HandleUpgradeError) -> Self {
4114 0 : match err {
4115 0 : HandleUpgradeError::ShutDown => tonic::Status::unavailable("timeline is shutting down"),
4116 : }
4117 0 : }
4118 : }
4119 :
4120 0 : fn set_tracing_field_shard_id(timeline: &Timeline) {
4121 0 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
4122 0 : tracing::Span::current().record(
4123 0 : "shard_id",
4124 0 : tracing::field::display(timeline.tenant_shard_id.shard_slug()),
4125 : );
4126 0 : debug_assert_current_span_has_tenant_and_timeline_id();
4127 0 : }
4128 :
4129 : struct WaitedForLsn(Lsn);
4130 : impl From<WaitedForLsn> for Lsn {
4131 0 : fn from(WaitedForLsn(lsn): WaitedForLsn) -> Self {
4132 0 : lsn
4133 0 : }
4134 : }
4135 :
4136 : #[cfg(test)]
4137 : mod tests {
4138 : use utils::shard::ShardCount;
4139 :
4140 : use super::*;
4141 :
4142 : #[test]
4143 1 : fn pageservice_cmd_parse() {
4144 1 : let tenant_id = TenantId::generate();
4145 1 : let timeline_id = TimelineId::generate();
4146 1 : let cmd =
4147 1 : PageServiceCmd::parse(&format!("pagestream_v2 {tenant_id} {timeline_id}")).unwrap();
4148 1 : assert_eq!(
4149 : cmd,
4150 1 : PageServiceCmd::PageStream(PageStreamCmd {
4151 1 : tenant_id,
4152 1 : timeline_id,
4153 1 : protocol_version: PagestreamProtocolVersion::V2,
4154 1 : })
4155 : );
4156 1 : let cmd = PageServiceCmd::parse(&format!("basebackup {tenant_id} {timeline_id}")).unwrap();
4157 1 : assert_eq!(
4158 : cmd,
4159 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4160 1 : tenant_id,
4161 1 : timeline_id,
4162 1 : lsn: None,
4163 1 : gzip: false,
4164 1 : replica: false
4165 1 : })
4166 : );
4167 1 : let cmd =
4168 1 : PageServiceCmd::parse(&format!("basebackup {tenant_id} {timeline_id} --gzip")).unwrap();
4169 1 : assert_eq!(
4170 : cmd,
4171 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4172 1 : tenant_id,
4173 1 : timeline_id,
4174 1 : lsn: None,
4175 1 : gzip: true,
4176 1 : replica: false
4177 1 : })
4178 : );
4179 1 : let cmd =
4180 1 : PageServiceCmd::parse(&format!("basebackup {tenant_id} {timeline_id} latest")).unwrap();
4181 1 : assert_eq!(
4182 : cmd,
4183 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4184 1 : tenant_id,
4185 1 : timeline_id,
4186 1 : lsn: None,
4187 1 : gzip: false,
4188 1 : replica: false
4189 1 : })
4190 : );
4191 1 : let cmd = PageServiceCmd::parse(&format!("basebackup {tenant_id} {timeline_id} 0/16ABCDE"))
4192 1 : .unwrap();
4193 1 : assert_eq!(
4194 : cmd,
4195 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4196 1 : tenant_id,
4197 1 : timeline_id,
4198 1 : lsn: Some(Lsn::from_str("0/16ABCDE").unwrap()),
4199 1 : gzip: false,
4200 1 : replica: false
4201 1 : })
4202 : );
4203 1 : let cmd = PageServiceCmd::parse(&format!(
4204 1 : "basebackup {tenant_id} {timeline_id} --replica --gzip"
4205 1 : ))
4206 1 : .unwrap();
4207 1 : assert_eq!(
4208 : cmd,
4209 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4210 1 : tenant_id,
4211 1 : timeline_id,
4212 1 : lsn: None,
4213 1 : gzip: true,
4214 1 : replica: true
4215 1 : })
4216 : );
4217 1 : let cmd = PageServiceCmd::parse(&format!(
4218 1 : "basebackup {tenant_id} {timeline_id} 0/16ABCDE --replica --gzip"
4219 1 : ))
4220 1 : .unwrap();
4221 1 : assert_eq!(
4222 : cmd,
4223 1 : PageServiceCmd::BaseBackup(BaseBackupCmd {
4224 1 : tenant_id,
4225 1 : timeline_id,
4226 1 : lsn: Some(Lsn::from_str("0/16ABCDE").unwrap()),
4227 1 : gzip: true,
4228 1 : replica: true
4229 1 : })
4230 : );
4231 1 : let cmd = PageServiceCmd::parse(&format!("fullbackup {tenant_id} {timeline_id}")).unwrap();
4232 1 : assert_eq!(
4233 : cmd,
4234 1 : PageServiceCmd::FullBackup(FullBackupCmd {
4235 1 : tenant_id,
4236 1 : timeline_id,
4237 1 : lsn: None,
4238 1 : prev_lsn: None
4239 1 : })
4240 : );
4241 1 : let cmd = PageServiceCmd::parse(&format!(
4242 1 : "fullbackup {tenant_id} {timeline_id} 0/16ABCDE 0/16ABCDF"
4243 1 : ))
4244 1 : .unwrap();
4245 1 : assert_eq!(
4246 : cmd,
4247 1 : PageServiceCmd::FullBackup(FullBackupCmd {
4248 1 : tenant_id,
4249 1 : timeline_id,
4250 1 : lsn: Some(Lsn::from_str("0/16ABCDE").unwrap()),
4251 1 : prev_lsn: Some(Lsn::from_str("0/16ABCDF").unwrap()),
4252 1 : })
4253 : );
4254 1 : let tenant_shard_id = TenantShardId::unsharded(tenant_id);
4255 1 : let cmd = PageServiceCmd::parse(&format!(
4256 1 : "lease lsn {tenant_shard_id} {timeline_id} 0/16ABCDE"
4257 1 : ))
4258 1 : .unwrap();
4259 1 : assert_eq!(
4260 : cmd,
4261 1 : PageServiceCmd::LeaseLsn(LeaseLsnCmd {
4262 1 : tenant_shard_id,
4263 1 : timeline_id,
4264 1 : lsn: Lsn::from_str("0/16ABCDE").unwrap(),
4265 1 : })
4266 : );
4267 1 : let tenant_shard_id = TenantShardId::split(&tenant_shard_id, ShardCount(8))[1];
4268 1 : let cmd = PageServiceCmd::parse(&format!(
4269 1 : "lease lsn {tenant_shard_id} {timeline_id} 0/16ABCDE"
4270 1 : ))
4271 1 : .unwrap();
4272 1 : assert_eq!(
4273 : cmd,
4274 1 : PageServiceCmd::LeaseLsn(LeaseLsnCmd {
4275 1 : tenant_shard_id,
4276 1 : timeline_id,
4277 1 : lsn: Lsn::from_str("0/16ABCDE").unwrap(),
4278 1 : })
4279 : );
4280 1 : let cmd = PageServiceCmd::parse("set a = b").unwrap();
4281 1 : assert_eq!(cmd, PageServiceCmd::Set);
4282 1 : let cmd = PageServiceCmd::parse("SET foo").unwrap();
4283 1 : assert_eq!(cmd, PageServiceCmd::Set);
4284 1 : }
4285 :
4286 : #[test]
4287 1 : fn pageservice_cmd_err_handling() {
4288 1 : let tenant_id = TenantId::generate();
4289 1 : let timeline_id = TimelineId::generate();
4290 1 : let cmd = PageServiceCmd::parse("unknown_command");
4291 1 : assert!(cmd.is_err());
4292 1 : let cmd = PageServiceCmd::parse("pagestream_v2");
4293 1 : assert!(cmd.is_err());
4294 1 : let cmd = PageServiceCmd::parse(&format!("pagestream_v2 {tenant_id}xxx"));
4295 1 : assert!(cmd.is_err());
4296 1 : let cmd = PageServiceCmd::parse(&format!("pagestream_v2 {tenant_id}xxx {timeline_id}xxx"));
4297 1 : assert!(cmd.is_err());
4298 1 : let cmd = PageServiceCmd::parse(&format!(
4299 1 : "basebackup {tenant_id} {timeline_id} --gzip --gzip"
4300 1 : ));
4301 1 : assert!(cmd.is_err());
4302 1 : let cmd = PageServiceCmd::parse(&format!(
4303 1 : "basebackup {tenant_id} {timeline_id} --gzip --unknown"
4304 1 : ));
4305 1 : assert!(cmd.is_err());
4306 1 : let cmd = PageServiceCmd::parse(&format!(
4307 1 : "basebackup {tenant_id} {timeline_id} --gzip 0/16ABCDE"
4308 1 : ));
4309 1 : assert!(cmd.is_err());
4310 1 : let cmd = PageServiceCmd::parse(&format!("lease {tenant_id} {timeline_id} gzip 0/16ABCDE"));
4311 1 : assert!(cmd.is_err());
4312 1 : }
4313 :
4314 : #[test]
4315 1 : fn test_parse_options() {
4316 1 : let (config, has_error) = parse_options(" -c neon.compute_mode=primary ");
4317 1 : assert!(!has_error);
4318 1 : assert_eq!(
4319 : config,
4320 1 : vec![("neon.compute_mode".to_string(), "primary".to_string())]
4321 : );
4322 :
4323 1 : let (config, has_error) = parse_options(" -c neon.compute_mode=primary -c foo=bar ");
4324 1 : assert!(!has_error);
4325 1 : assert_eq!(
4326 : config,
4327 1 : vec![
4328 1 : ("neon.compute_mode".to_string(), "primary".to_string()),
4329 1 : ("foo".to_string(), "bar".to_string()),
4330 : ]
4331 : );
4332 :
4333 1 : let (config, has_error) = parse_options(" -c neon.compute_mode=primary -cfoo=bar");
4334 1 : assert!(!has_error);
4335 1 : assert_eq!(
4336 : config,
4337 1 : vec![
4338 1 : ("neon.compute_mode".to_string(), "primary".to_string()),
4339 1 : ("foo".to_string(), "bar".to_string()),
4340 : ]
4341 : );
4342 :
4343 1 : let (_, has_error) = parse_options("-c");
4344 1 : assert!(has_error);
4345 :
4346 1 : let (_, has_error) = parse_options("-c foo=bar -c -c");
4347 1 : assert!(has_error);
4348 :
4349 1 : let (_, has_error) = parse_options(" ");
4350 1 : assert!(!has_error);
4351 :
4352 1 : let (_, has_error) = parse_options(" -c neon.compute_mode");
4353 1 : assert!(has_error);
4354 1 : }
4355 : }
|