Line data Source code
1 : //! The Page Service listens for client connections and serves their GetPage@LSN
2 : //! requests.
3 :
4 : use anyhow::Context;
5 : use async_compression::tokio::write::GzipEncoder;
6 : use bytes::Buf;
7 : use bytes::Bytes;
8 : use futures::stream::FuturesUnordered;
9 : use futures::Stream;
10 : use futures::StreamExt;
11 : use pageserver_api::key::Key;
12 : use pageserver_api::models::TenantState;
13 : use pageserver_api::models::{
14 : PagestreamBeMessage, PagestreamDbSizeRequest, PagestreamDbSizeResponse,
15 : PagestreamErrorResponse, PagestreamExistsRequest, PagestreamExistsResponse,
16 : PagestreamFeMessage, PagestreamGetPageRequest, PagestreamGetPageResponse,
17 : PagestreamGetSlruSegmentRequest, PagestreamGetSlruSegmentResponse, PagestreamNblocksRequest,
18 : PagestreamNblocksResponse, PagestreamProtocolVersion,
19 : };
20 : use pageserver_api::shard::ShardIndex;
21 : use pageserver_api::shard::ShardNumber;
22 : use postgres_backend::{is_expected_io_error, AuthType, PostgresBackend, QueryError};
23 : use pq_proto::framed::ConnectionError;
24 : use pq_proto::FeStartupPacket;
25 : use pq_proto::{BeMessage, FeMessage, RowDescriptor};
26 : use std::borrow::Cow;
27 : use std::collections::HashMap;
28 : use std::io;
29 : use std::net::TcpListener;
30 : use std::pin::pin;
31 : use std::str;
32 : use std::str::FromStr;
33 : use std::sync::Arc;
34 : use std::time::Duration;
35 : use std::time::Instant;
36 : use tokio::io::AsyncWriteExt;
37 : use tokio::io::{AsyncRead, AsyncWrite};
38 : use tokio_util::io::StreamReader;
39 : use tokio_util::sync::CancellationToken;
40 : use tracing::*;
41 : use utils::id::ConnectionId;
42 : use utils::sync::gate::GateGuard;
43 : use utils::{
44 : auth::{Claims, Scope, SwappableJwtAuth},
45 : id::{TenantId, TimelineId},
46 : lsn::Lsn,
47 : simple_rcu::RcuReadGuard,
48 : };
49 :
50 : use crate::auth::check_permission;
51 : use crate::basebackup;
52 : use crate::basebackup::BasebackupError;
53 : use crate::context::{DownloadBehavior, RequestContext};
54 : use crate::import_datadir::import_wal_from_tar;
55 : use crate::metrics;
56 : use crate::metrics::LIVE_CONNECTIONS_COUNT;
57 : use crate::pgdatadir_mapping::Version;
58 : use crate::span::debug_assert_current_span_has_tenant_and_timeline_id;
59 : use crate::span::debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id;
60 : use crate::task_mgr;
61 : use crate::task_mgr::TaskKind;
62 : use crate::tenant::mgr::GetActiveTenantError;
63 : use crate::tenant::mgr::GetTenantError;
64 : use crate::tenant::mgr::ShardResolveResult;
65 : use crate::tenant::mgr::ShardSelector;
66 : use crate::tenant::mgr::TenantManager;
67 : use crate::tenant::timeline::WaitLsnError;
68 : use crate::tenant::GetTimelineError;
69 : use crate::tenant::PageReconstructError;
70 : use crate::tenant::Tenant;
71 : use crate::tenant::Timeline;
72 : use crate::trace::Tracer;
73 : use pageserver_api::key::rel_block_to_key;
74 : use pageserver_api::reltag::SlruKind;
75 : use postgres_ffi::pg_constants::DEFAULTTABLESPACE_OID;
76 : use postgres_ffi::BLCKSZ;
77 :
78 : // How long we may wait for a [`TenantSlot::InProgress`]` and/or a [`Tenant`] which
79 : // is not yet in state [`TenantState::Active`].
80 : const ACTIVE_TENANT_TIMEOUT: Duration = Duration::from_millis(30000);
81 :
82 : /// Read the end of a tar archive.
83 : ///
84 : /// A tar archive normally ends with two consecutive blocks of zeros, 512 bytes each.
85 : /// `tokio_tar` already read the first such block. Read the second all-zeros block,
86 : /// and check that there is no more data after the EOF marker.
87 : ///
88 : /// 'tar' command can also write extra blocks of zeros, up to a record
89 : /// size, controlled by the --record-size argument. Ignore them too.
90 0 : async fn read_tar_eof(mut reader: (impl AsyncRead + Unpin)) -> anyhow::Result<()> {
91 0 : use tokio::io::AsyncReadExt;
92 0 : let mut buf = [0u8; 512];
93 0 :
94 0 : // Read the all-zeros block, and verify it
95 0 : let mut total_bytes = 0;
96 0 : while total_bytes < 512 {
97 0 : let nbytes = reader.read(&mut buf[total_bytes..]).await?;
98 0 : total_bytes += nbytes;
99 0 : if nbytes == 0 {
100 0 : break;
101 0 : }
102 : }
103 0 : if total_bytes < 512 {
104 0 : anyhow::bail!("incomplete or invalid tar EOF marker");
105 0 : }
106 0 : if !buf.iter().all(|&x| x == 0) {
107 0 : anyhow::bail!("invalid tar EOF marker");
108 0 : }
109 0 :
110 0 : // Drain any extra zero-blocks after the EOF marker
111 0 : let mut trailing_bytes = 0;
112 0 : let mut seen_nonzero_bytes = false;
113 : loop {
114 0 : let nbytes = reader.read(&mut buf).await?;
115 0 : trailing_bytes += nbytes;
116 0 : if !buf.iter().all(|&x| x == 0) {
117 0 : seen_nonzero_bytes = true;
118 0 : }
119 0 : if nbytes == 0 {
120 0 : break;
121 0 : }
122 : }
123 0 : if seen_nonzero_bytes {
124 0 : anyhow::bail!("unexpected non-zero bytes after the tar archive");
125 0 : }
126 0 : if trailing_bytes % 512 != 0 {
127 0 : anyhow::bail!("unexpected number of zeros ({trailing_bytes}), not divisible by tar block size (512 bytes), after the tar archive");
128 0 : }
129 0 : Ok(())
130 0 : }
131 :
132 : ///////////////////////////////////////////////////////////////////////////////
133 :
134 : ///
135 : /// Main loop of the page service.
136 : ///
137 : /// Listens for connections, and launches a new handler task for each.
138 : ///
139 0 : pub async fn libpq_listener_main(
140 0 : tenant_manager: Arc<TenantManager>,
141 0 : broker_client: storage_broker::BrokerClientChannel,
142 0 : auth: Option<Arc<SwappableJwtAuth>>,
143 0 : listener: TcpListener,
144 0 : auth_type: AuthType,
145 0 : listener_ctx: RequestContext,
146 0 : cancel: CancellationToken,
147 0 : ) -> anyhow::Result<()> {
148 0 : listener.set_nonblocking(true)?;
149 0 : let tokio_listener = tokio::net::TcpListener::from_std(listener)?;
150 :
151 : // Wait for a new connection to arrive, or for server shutdown.
152 0 : while let Some(res) = tokio::select! {
153 : biased;
154 :
155 : _ = cancel.cancelled() => {
156 : // We were requested to shut down.
157 : None
158 : }
159 :
160 : res = tokio_listener.accept() => {
161 : Some(res)
162 : }
163 : } {
164 0 : match res {
165 0 : Ok((socket, peer_addr)) => {
166 0 : // Connection established. Spawn a new task to handle it.
167 0 : debug!("accepted connection from {}", peer_addr);
168 0 : let local_auth = auth.clone();
169 0 :
170 0 : let connection_ctx = listener_ctx
171 0 : .detached_child(TaskKind::PageRequestHandler, DownloadBehavior::Download);
172 0 :
173 0 : // PageRequestHandler tasks are not associated with any particular
174 0 : // timeline in the task manager. In practice most connections will
175 0 : // only deal with a particular timeline, but we don't know which one
176 0 : // yet.
177 0 : task_mgr::spawn(
178 0 : &tokio::runtime::Handle::current(),
179 0 : TaskKind::PageRequestHandler,
180 0 : None,
181 0 : None,
182 0 : "serving compute connection task",
183 0 : false,
184 0 : page_service_conn_main(
185 0 : tenant_manager.clone(),
186 0 : broker_client.clone(),
187 0 : local_auth,
188 0 : socket,
189 0 : auth_type,
190 0 : connection_ctx,
191 0 : ),
192 0 : );
193 : }
194 0 : Err(err) => {
195 0 : // accept() failed. Log the error, and loop back to retry on next connection.
196 0 : error!("accept() failed: {:?}", err);
197 : }
198 : }
199 : }
200 :
201 0 : debug!("page_service loop terminated");
202 :
203 0 : Ok(())
204 0 : }
205 :
206 0 : #[instrument(skip_all, fields(peer_addr))]
207 : async fn page_service_conn_main(
208 : tenant_manager: Arc<TenantManager>,
209 : broker_client: storage_broker::BrokerClientChannel,
210 : auth: Option<Arc<SwappableJwtAuth>>,
211 : socket: tokio::net::TcpStream,
212 : auth_type: AuthType,
213 : connection_ctx: RequestContext,
214 : ) -> anyhow::Result<()> {
215 : // Immediately increment the gauge, then create a job to decrement it on task exit.
216 : // One of the pros of `defer!` is that this will *most probably*
217 : // get called, even in presence of panics.
218 : let gauge = LIVE_CONNECTIONS_COUNT.with_label_values(&["page_service"]);
219 : gauge.inc();
220 : scopeguard::defer! {
221 : gauge.dec();
222 : }
223 :
224 : socket
225 : .set_nodelay(true)
226 : .context("could not set TCP_NODELAY")?;
227 :
228 : let peer_addr = socket.peer_addr().context("get peer address")?;
229 : tracing::Span::current().record("peer_addr", field::display(peer_addr));
230 :
231 : // setup read timeout of 10 minutes. the timeout is rather arbitrary for requirements:
232 : // - long enough for most valid compute connections
233 : // - less than infinite to stop us from "leaking" connections to long-gone computes
234 : //
235 : // no write timeout is used, because the kernel is assumed to error writes after some time.
236 : let mut socket = tokio_io_timeout::TimeoutReader::new(socket);
237 :
238 : let default_timeout_ms = 10 * 60 * 1000; // 10 minutes by default
239 0 : let socket_timeout_ms = (|| {
240 0 : fail::fail_point!("simulated-bad-compute-connection", |avg_timeout_ms| {
241 : // Exponential distribution for simulating
242 : // poor network conditions, expect about avg_timeout_ms to be around 15
243 : // in tests
244 0 : if let Some(avg_timeout_ms) = avg_timeout_ms {
245 0 : let avg = avg_timeout_ms.parse::<i64>().unwrap() as f32;
246 0 : let u = rand::random::<f32>();
247 0 : ((1.0 - u).ln() / (-avg)) as u64
248 : } else {
249 0 : default_timeout_ms
250 : }
251 0 : });
252 0 : default_timeout_ms
253 : })();
254 :
255 : // A timeout here does not mean the client died, it can happen if it's just idle for
256 : // a while: we will tear down this PageServerHandler and instantiate a new one if/when
257 : // they reconnect.
258 : socket.set_timeout(Some(std::time::Duration::from_millis(socket_timeout_ms)));
259 : let socket = std::pin::pin!(socket);
260 :
261 : // XXX: pgbackend.run() should take the connection_ctx,
262 : // and create a child per-query context when it invokes process_query.
263 : // But it's in a shared crate, so, we store connection_ctx inside PageServerHandler
264 : // and create the per-query context in process_query ourselves.
265 : let mut conn_handler =
266 : PageServerHandler::new(tenant_manager, broker_client, auth, connection_ctx);
267 : let pgbackend = PostgresBackend::new_from_io(socket, peer_addr, auth_type, None)?;
268 :
269 : match pgbackend
270 : .run(&mut conn_handler, task_mgr::shutdown_watcher)
271 : .await
272 : {
273 : Ok(()) => {
274 : // we've been requested to shut down
275 : Ok(())
276 : }
277 : Err(QueryError::Disconnected(ConnectionError::Io(io_error))) => {
278 : if is_expected_io_error(&io_error) {
279 : info!("Postgres client disconnected ({io_error})");
280 : Ok(())
281 : } else {
282 : Err(io_error).context("Postgres connection error")
283 : }
284 : }
285 : other => other.context("Postgres query error"),
286 : }
287 : }
288 :
289 : /// While a handler holds a reference to a Timeline, it also holds a the
290 : /// timeline's Gate open.
291 : struct HandlerTimeline {
292 : timeline: Arc<Timeline>,
293 : _guard: GateGuard,
294 : }
295 :
296 : struct PageServerHandler {
297 : broker_client: storage_broker::BrokerClientChannel,
298 : auth: Option<Arc<SwappableJwtAuth>>,
299 : claims: Option<Claims>,
300 :
301 : tenant_manager: Arc<TenantManager>,
302 :
303 : /// The context created for the lifetime of the connection
304 : /// services by this PageServerHandler.
305 : /// For each query received over the connection,
306 : /// `process_query` creates a child context from this one.
307 : connection_ctx: RequestContext,
308 :
309 : /// See [`Self::cache_timeline`] for usage.
310 : ///
311 : /// Note on size: the typical size of this map is 1. The largest size we expect
312 : /// to see is the number of shards divided by the number of pageservers (typically < 2),
313 : /// or the ratio used when splitting shards (i.e. how many children created from one)
314 : /// parent shard, where a "large" number might be ~8.
315 : shard_timelines: HashMap<ShardIndex, HandlerTimeline>,
316 : }
317 :
318 0 : #[derive(thiserror::Error, Debug)]
319 : enum PageStreamError {
320 : /// We encountered an error that should prompt the client to reconnect:
321 : /// in practice this means we drop the connection without sending a response.
322 : #[error("Reconnect required: {0}")]
323 : Reconnect(Cow<'static, str>),
324 :
325 : /// We were instructed to shutdown while processing the query
326 : #[error("Shutting down")]
327 : Shutdown,
328 :
329 : /// Something went wrong reading a page: this likely indicates a pageserver bug
330 : #[error("Read error")]
331 : Read(#[source] PageReconstructError),
332 :
333 : /// Ran out of time waiting for an LSN
334 : #[error("LSN timeout: {0}")]
335 : LsnTimeout(WaitLsnError),
336 :
337 : /// The entity required to serve the request (tenant or timeline) is not found,
338 : /// or is not found in a suitable state to serve a request.
339 : #[error("Not found: {0}")]
340 : NotFound(Cow<'static, str>),
341 :
342 : /// Request asked for something that doesn't make sense, like an invalid LSN
343 : #[error("Bad request: {0}")]
344 : BadRequest(Cow<'static, str>),
345 : }
346 :
347 : impl From<PageReconstructError> for PageStreamError {
348 0 : fn from(value: PageReconstructError) -> Self {
349 0 : match value {
350 0 : PageReconstructError::Cancelled => Self::Shutdown,
351 0 : e => Self::Read(e),
352 : }
353 0 : }
354 : }
355 :
356 : impl From<GetActiveTimelineError> for PageStreamError {
357 0 : fn from(value: GetActiveTimelineError) -> Self {
358 0 : match value {
359 0 : GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled) => Self::Shutdown,
360 0 : GetActiveTimelineError::Tenant(e) => Self::NotFound(format!("{e}").into()),
361 0 : GetActiveTimelineError::Timeline(e) => Self::NotFound(format!("{e}").into()),
362 : }
363 0 : }
364 : }
365 :
366 : impl From<WaitLsnError> for PageStreamError {
367 0 : fn from(value: WaitLsnError) -> Self {
368 0 : match value {
369 0 : e @ WaitLsnError::Timeout(_) => Self::LsnTimeout(e),
370 0 : WaitLsnError::Shutdown => Self::Shutdown,
371 0 : WaitLsnError::BadState => Self::Reconnect("Timeline is not active".into()),
372 : }
373 0 : }
374 : }
375 :
376 : impl From<WaitLsnError> for QueryError {
377 0 : fn from(value: WaitLsnError) -> Self {
378 0 : match value {
379 0 : e @ WaitLsnError::Timeout(_) => Self::Other(anyhow::Error::new(e)),
380 0 : WaitLsnError::Shutdown => Self::Shutdown,
381 0 : WaitLsnError::BadState => Self::Reconnect,
382 : }
383 0 : }
384 : }
385 :
386 : impl PageServerHandler {
387 0 : pub fn new(
388 0 : tenant_manager: Arc<TenantManager>,
389 0 : broker_client: storage_broker::BrokerClientChannel,
390 0 : auth: Option<Arc<SwappableJwtAuth>>,
391 0 : connection_ctx: RequestContext,
392 0 : ) -> Self {
393 0 : PageServerHandler {
394 0 : tenant_manager,
395 0 : broker_client,
396 0 : auth,
397 0 : claims: None,
398 0 : connection_ctx,
399 0 : shard_timelines: HashMap::new(),
400 0 : }
401 0 : }
402 :
403 : /// Future that completes when we need to shut down the connection.
404 : ///
405 : /// We currently need to shut down when any of the following happens:
406 : /// 1. any of the timelines we hold GateGuards for in `shard_timelines` is cancelled
407 : /// 2. task_mgr requests shutdown of the connection
408 : ///
409 : /// NB on (1): the connection's lifecycle is not actually tied to any of the
410 : /// `shard_timelines`s' lifecycles. But it's _necessary_ in the current
411 : /// implementation to be responsive to timeline cancellation because
412 : /// the connection holds their `GateGuards` open (sored in `shard_timelines`).
413 : /// We currently do the easy thing and terminate the connection if any of the
414 : /// shard_timelines gets cancelled. But really, we cuold spend more effort
415 : /// and simply remove the cancelled timeline from the `shard_timelines`, thereby
416 : /// dropping the guard.
417 : ///
418 : /// NB: keep in sync with [`Self::is_connection_cancelled`]
419 0 : async fn await_connection_cancelled(&self) {
420 0 : // A short wait before we expend the cycles to walk our timeline map. This avoids incurring
421 0 : // that cost every time we check for cancellation.
422 0 : tokio::time::sleep(Duration::from_millis(10)).await;
423 :
424 : // This function is never called concurrently with code that adds timelines to shard_timelines,
425 : // which is enforced by the borrow checker (the future returned by this function carries the
426 : // immutable &self). So it's fine to evaluate shard_timelines after the sleep, we don't risk
427 : // missing any inserts to the map.
428 :
429 0 : let mut cancellation_sources = Vec::with_capacity(1 + self.shard_timelines.len());
430 0 : use futures::future::Either;
431 0 : cancellation_sources.push(Either::Left(task_mgr::shutdown_watcher()));
432 0 : cancellation_sources.extend(
433 0 : self.shard_timelines
434 0 : .values()
435 0 : .map(|ht| Either::Right(ht.timeline.cancel.cancelled())),
436 0 : );
437 0 : FuturesUnordered::from_iter(cancellation_sources)
438 0 : .next()
439 0 : .await;
440 0 : }
441 :
442 : /// Checking variant of [`Self::await_connection_cancelled`].
443 0 : fn is_connection_cancelled(&self) -> bool {
444 0 : task_mgr::is_shutdown_requested()
445 0 : || self
446 0 : .shard_timelines
447 0 : .values()
448 0 : .any(|ht| ht.timeline.cancel.is_cancelled() || ht.timeline.is_stopping())
449 0 : }
450 :
451 : /// This function always respects cancellation of any timeline in `[Self::shard_timelines]`. Pass in
452 : /// a cancellation token at the next scope up (such as a tenant cancellation token) to ensure we respect
453 : /// cancellation if there aren't any timelines in the cache.
454 : ///
455 : /// If calling from a function that doesn't use the `[Self::shard_timelines]` cache, then pass in the
456 : /// timeline cancellation token.
457 0 : async fn flush_cancellable<IO>(
458 0 : &self,
459 0 : pgb: &mut PostgresBackend<IO>,
460 0 : cancel: &CancellationToken,
461 0 : ) -> Result<(), QueryError>
462 0 : where
463 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
464 0 : {
465 : tokio::select!(
466 : flush_r = pgb.flush() => {
467 : Ok(flush_r?)
468 : },
469 : _ = self.await_connection_cancelled() => {
470 : Err(QueryError::Shutdown)
471 : }
472 : _ = cancel.cancelled() => {
473 : Err(QueryError::Shutdown)
474 : }
475 : )
476 0 : }
477 :
478 0 : fn copyin_stream<'a, IO>(
479 0 : &'a self,
480 0 : pgb: &'a mut PostgresBackend<IO>,
481 0 : cancel: &'a CancellationToken,
482 0 : ) -> impl Stream<Item = io::Result<Bytes>> + 'a
483 0 : where
484 0 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
485 0 : {
486 : async_stream::try_stream! {
487 : loop {
488 : let msg = tokio::select! {
489 : biased;
490 :
491 : _ = cancel.cancelled() => {
492 : // We were requested to shut down.
493 : let msg = "pageserver is shutting down";
494 : let _ = pgb.write_message_noflush(&BeMessage::ErrorResponse(msg, None));
495 : Err(QueryError::Shutdown)
496 : }
497 :
498 : msg = pgb.read_message() => { msg.map_err(QueryError::from)}
499 : };
500 :
501 : match msg {
502 : Ok(Some(message)) => {
503 : let copy_data_bytes = match message {
504 : FeMessage::CopyData(bytes) => bytes,
505 : FeMessage::CopyDone => { break },
506 : FeMessage::Sync => continue,
507 : FeMessage::Terminate => {
508 : let msg = "client terminated connection with Terminate message during COPY";
509 : let query_error = QueryError::Disconnected(ConnectionError::Io(io::Error::new(io::ErrorKind::ConnectionReset, msg)));
510 : // error can't happen here, ErrorResponse serialization should be always ok
511 0 : pgb.write_message_noflush(&BeMessage::ErrorResponse(msg, Some(query_error.pg_error_code()))).map_err(|e| e.into_io_error())?;
512 : Err(io::Error::new(io::ErrorKind::ConnectionReset, msg))?;
513 : break;
514 : }
515 : m => {
516 : let msg = format!("unexpected message {m:?}");
517 : // error can't happen here, ErrorResponse serialization should be always ok
518 0 : pgb.write_message_noflush(&BeMessage::ErrorResponse(&msg, None)).map_err(|e| e.into_io_error())?;
519 : Err(io::Error::new(io::ErrorKind::Other, msg))?;
520 : break;
521 : }
522 : };
523 :
524 : yield copy_data_bytes;
525 : }
526 : Ok(None) => {
527 : let msg = "client closed connection during COPY";
528 : let query_error = QueryError::Disconnected(ConnectionError::Io(io::Error::new(io::ErrorKind::ConnectionReset, msg)));
529 : // error can't happen here, ErrorResponse serialization should be always ok
530 0 : pgb.write_message_noflush(&BeMessage::ErrorResponse(msg, Some(query_error.pg_error_code()))).map_err(|e| e.into_io_error())?;
531 0 : self.flush_cancellable(pgb, cancel).await.map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
532 : Err(io::Error::new(io::ErrorKind::ConnectionReset, msg))?;
533 : }
534 : Err(QueryError::Disconnected(ConnectionError::Io(io_error))) => {
535 : Err(io_error)?;
536 : }
537 : Err(other) => {
538 : Err(io::Error::new(io::ErrorKind::Other, other.to_string()))?;
539 : }
540 : };
541 : }
542 : }
543 0 : }
544 :
545 0 : #[instrument(skip_all)]
546 : async fn handle_pagerequests<IO>(
547 : &mut self,
548 : pgb: &mut PostgresBackend<IO>,
549 : tenant_id: TenantId,
550 : timeline_id: TimelineId,
551 : protocol_version: PagestreamProtocolVersion,
552 : ctx: RequestContext,
553 : ) -> Result<(), QueryError>
554 : where
555 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
556 : {
557 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
558 :
559 : let tenant = self
560 : .get_active_tenant_with_timeout(tenant_id, ShardSelector::First, ACTIVE_TENANT_TIMEOUT)
561 : .await?;
562 :
563 : // Make request tracer if needed
564 : let mut tracer = if tenant.get_trace_read_requests() {
565 : let connection_id = ConnectionId::generate();
566 : let path =
567 : tenant
568 : .conf
569 : .trace_path(&tenant.tenant_shard_id(), &timeline_id, &connection_id);
570 : Some(Tracer::new(path))
571 : } else {
572 : None
573 : };
574 :
575 : // switch client to COPYBOTH
576 : pgb.write_message_noflush(&BeMessage::CopyBothResponse)?;
577 : self.flush_cancellable(pgb, &tenant.cancel).await?;
578 :
579 : loop {
580 : let msg = tokio::select! {
581 : biased;
582 :
583 : _ = self.await_connection_cancelled() => {
584 : // We were requested to shut down.
585 : info!("shutdown request received in page handler");
586 : return Err(QueryError::Shutdown)
587 : }
588 :
589 : msg = pgb.read_message() => { msg }
590 : };
591 :
592 : let copy_data_bytes = match msg? {
593 : Some(FeMessage::CopyData(bytes)) => bytes,
594 : Some(FeMessage::Terminate) => break,
595 : Some(m) => {
596 : return Err(QueryError::Other(anyhow::anyhow!(
597 : "unexpected message: {m:?} during COPY"
598 : )));
599 : }
600 : None => break, // client disconnected
601 : };
602 :
603 : trace!("query: {copy_data_bytes:?}");
604 :
605 : // Trace request if needed
606 : if let Some(t) = tracer.as_mut() {
607 : t.trace(©_data_bytes)
608 : }
609 :
610 : let neon_fe_msg =
611 : PagestreamFeMessage::parse(&mut copy_data_bytes.reader(), protocol_version)?;
612 :
613 : // TODO: We could create a new per-request context here, with unique ID.
614 : // Currently we use the same per-timeline context for all requests
615 :
616 : let (response, span) = match neon_fe_msg {
617 : PagestreamFeMessage::Exists(req) => {
618 : let span = tracing::info_span!("handle_get_rel_exists_request", rel = %req.rel, req_lsn = %req.request_lsn);
619 : (
620 : self.handle_get_rel_exists_request(tenant_id, timeline_id, &req, &ctx)
621 : .instrument(span.clone())
622 : .await,
623 : span,
624 : )
625 : }
626 : PagestreamFeMessage::Nblocks(req) => {
627 : let span = tracing::info_span!("handle_get_nblocks_request", rel = %req.rel, req_lsn = %req.request_lsn);
628 : (
629 : self.handle_get_nblocks_request(tenant_id, timeline_id, &req, &ctx)
630 : .instrument(span.clone())
631 : .await,
632 : span,
633 : )
634 : }
635 : PagestreamFeMessage::GetPage(req) => {
636 : // shard_id is filled in by the handler
637 : let span = tracing::info_span!("handle_get_page_at_lsn_request", rel = %req.rel, blkno = %req.blkno, req_lsn = %req.request_lsn);
638 : (
639 : self.handle_get_page_at_lsn_request(tenant_id, timeline_id, &req, &ctx)
640 : .instrument(span.clone())
641 : .await,
642 : span,
643 : )
644 : }
645 : PagestreamFeMessage::DbSize(req) => {
646 : let span = tracing::info_span!("handle_db_size_request", dbnode = %req.dbnode, req_lsn = %req.request_lsn);
647 : (
648 : self.handle_db_size_request(tenant_id, timeline_id, &req, &ctx)
649 : .instrument(span.clone())
650 : .await,
651 : span,
652 : )
653 : }
654 : PagestreamFeMessage::GetSlruSegment(req) => {
655 : let span = tracing::info_span!("handle_get_slru_segment_request", kind = %req.kind, segno = %req.segno, req_lsn = %req.request_lsn);
656 : (
657 : self.handle_get_slru_segment_request(tenant_id, timeline_id, &req, &ctx)
658 : .instrument(span.clone())
659 : .await,
660 : span,
661 : )
662 : }
663 : };
664 :
665 : match response {
666 : Err(PageStreamError::Shutdown) => {
667 : // If we fail to fulfil a request during shutdown, which may be _because_ of
668 : // shutdown, then do not send the error to the client. Instead just drop the
669 : // connection.
670 0 : span.in_scope(|| info!("dropping connection due to shutdown"));
671 : return Err(QueryError::Shutdown);
672 : }
673 : Err(PageStreamError::Reconnect(reason)) => {
674 0 : span.in_scope(|| info!("handler requested reconnect: {reason}"));
675 : return Err(QueryError::Reconnect);
676 : }
677 : Err(e) if self.is_connection_cancelled() => {
678 : // This branch accomodates code within request handlers that returns an anyhow::Error instead of a clean
679 : // shutdown error, this may be buried inside a PageReconstructError::Other for example.
680 : //
681 : // Requests may fail as soon as we are Stopping, even if the Timeline's cancellation token wasn't fired yet,
682 : // because wait_lsn etc will drop out
683 : // is_stopping(): [`Timeline::flush_and_shutdown`] has entered
684 : // is_canceled(): [`Timeline::shutdown`]` has entered
685 0 : span.in_scope(|| info!("dropped error response during shutdown: {e:#}"));
686 : return Err(QueryError::Shutdown);
687 : }
688 : r => {
689 0 : let response_msg = r.unwrap_or_else(|e| {
690 0 : // print the all details to the log with {:#}, but for the client the
691 0 : // error message is enough. Do not log if shutting down, as the anyhow::Error
692 0 : // here includes cancellation which is not an error.
693 0 : let full = utils::error::report_compact_sources(&e);
694 0 : span.in_scope(|| {
695 0 : error!("error reading relation or page version: {full:#}")
696 0 : });
697 0 : PagestreamBeMessage::Error(PagestreamErrorResponse {
698 0 : message: e.to_string(),
699 0 : })
700 0 : });
701 :
702 : pgb.write_message_noflush(&BeMessage::CopyData(&response_msg.serialize()))?;
703 : self.flush_cancellable(pgb, &tenant.cancel).await?;
704 : }
705 : }
706 : }
707 : Ok(())
708 : }
709 :
710 : #[allow(clippy::too_many_arguments)]
711 0 : #[instrument(skip_all, fields(%base_lsn, end_lsn=%_end_lsn, %pg_version))]
712 : async fn handle_import_basebackup<IO>(
713 : &self,
714 : pgb: &mut PostgresBackend<IO>,
715 : tenant_id: TenantId,
716 : timeline_id: TimelineId,
717 : base_lsn: Lsn,
718 : _end_lsn: Lsn,
719 : pg_version: u32,
720 : ctx: RequestContext,
721 : ) -> Result<(), QueryError>
722 : where
723 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
724 : {
725 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
726 :
727 : // Create empty timeline
728 : info!("creating new timeline");
729 : let tenant = self
730 : .get_active_tenant_with_timeout(tenant_id, ShardSelector::Zero, ACTIVE_TENANT_TIMEOUT)
731 : .await?;
732 : let timeline = tenant
733 : .create_empty_timeline(timeline_id, base_lsn, pg_version, &ctx)
734 : .await?;
735 :
736 : // TODO mark timeline as not ready until it reaches end_lsn.
737 : // We might have some wal to import as well, and we should prevent compute
738 : // from connecting before that and writing conflicting wal.
739 : //
740 : // This is not relevant for pageserver->pageserver migrations, since there's
741 : // no wal to import. But should be fixed if we want to import from postgres.
742 :
743 : // TODO leave clean state on error. For now you can use detach to clean
744 : // up broken state from a failed import.
745 :
746 : // Import basebackup provided via CopyData
747 : info!("importing basebackup");
748 : pgb.write_message_noflush(&BeMessage::CopyInResponse)?;
749 : self.flush_cancellable(pgb, &tenant.cancel).await?;
750 :
751 : let mut copyin_reader = pin!(StreamReader::new(self.copyin_stream(pgb, &tenant.cancel)));
752 : timeline
753 : .import_basebackup_from_tar(
754 : tenant.clone(),
755 : &mut copyin_reader,
756 : base_lsn,
757 : self.broker_client.clone(),
758 : &ctx,
759 : )
760 : .await?;
761 :
762 : // Read the end of the tar archive.
763 : read_tar_eof(copyin_reader).await?;
764 :
765 : // TODO check checksum
766 : // Meanwhile you can verify client-side by taking fullbackup
767 : // and checking that it matches in size with what was imported.
768 : // It wouldn't work if base came from vanilla postgres though,
769 : // since we discard some log files.
770 :
771 : info!("done");
772 : Ok(())
773 : }
774 :
775 0 : #[instrument(skip_all, fields(shard_id, %start_lsn, %end_lsn))]
776 : async fn handle_import_wal<IO>(
777 : &self,
778 : pgb: &mut PostgresBackend<IO>,
779 : tenant_id: TenantId,
780 : timeline_id: TimelineId,
781 : start_lsn: Lsn,
782 : end_lsn: Lsn,
783 : ctx: RequestContext,
784 : ) -> Result<(), QueryError>
785 : where
786 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
787 : {
788 : let timeline = self
789 : .get_active_tenant_timeline(tenant_id, timeline_id, ShardSelector::Zero)
790 : .await?;
791 : let last_record_lsn = timeline.get_last_record_lsn();
792 : if last_record_lsn != start_lsn {
793 : return Err(QueryError::Other(
794 : anyhow::anyhow!("Cannot import WAL from Lsn {start_lsn} because timeline does not start from the same lsn: {last_record_lsn}"))
795 : );
796 : }
797 :
798 : // TODO leave clean state on error. For now you can use detach to clean
799 : // up broken state from a failed import.
800 :
801 : // Import wal provided via CopyData
802 : info!("importing wal");
803 : pgb.write_message_noflush(&BeMessage::CopyInResponse)?;
804 : self.flush_cancellable(pgb, &timeline.cancel).await?;
805 : let mut copyin_reader = pin!(StreamReader::new(self.copyin_stream(pgb, &timeline.cancel)));
806 : import_wal_from_tar(&timeline, &mut copyin_reader, start_lsn, end_lsn, &ctx).await?;
807 : info!("wal import complete");
808 :
809 : // Read the end of the tar archive.
810 : read_tar_eof(copyin_reader).await?;
811 :
812 : // TODO Does it make sense to overshoot?
813 : if timeline.get_last_record_lsn() < end_lsn {
814 : return Err(QueryError::Other(
815 : anyhow::anyhow!("Cannot import WAL from Lsn {start_lsn} because timeline does not start from the same lsn: {last_record_lsn}"))
816 : );
817 : }
818 :
819 : // Flush data to disk, then upload to s3. No need for a forced checkpoint.
820 : // We only want to persist the data, and it doesn't matter if it's in the
821 : // shape of deltas or images.
822 : info!("flushing layers");
823 : timeline.freeze_and_flush().await?;
824 :
825 : info!("done");
826 : Ok(())
827 : }
828 :
829 : /// Helper function to handle the LSN from client request.
830 : ///
831 : /// Each GetPage (and Exists and Nblocks) request includes information about
832 : /// which version of the page is being requested. The primary compute node
833 : /// will always request the latest page version, by setting 'request_lsn' to
834 : /// the last inserted or flushed WAL position, while a standby will request
835 : /// a version at the LSN that it's currently caught up to.
836 : ///
837 : /// In either case, if the page server hasn't received the WAL up to the
838 : /// requested LSN yet, we will wait for it to arrive. The return value is
839 : /// the LSN that should be used to look up the page versions.
840 : ///
841 : /// In addition to the request LSN, each request carries another LSN,
842 : /// 'not_modified_since', which is a hint to the pageserver that the client
843 : /// knows that the page has not been modified between 'not_modified_since'
844 : /// and the request LSN. This allows skipping the wait, as long as the WAL
845 : /// up to 'not_modified_since' has arrived. If the client doesn't have any
846 : /// information about when the page was modified, it will use
847 : /// not_modified_since == lsn. If the client lies and sends a too low
848 : /// not_modified_hint such that there are in fact later page versions, the
849 : /// behavior is undefined: the pageserver may return any of the page versions
850 : /// or an error.
851 0 : async fn wait_or_get_last_lsn(
852 0 : timeline: &Timeline,
853 0 : request_lsn: Lsn,
854 0 : not_modified_since: Lsn,
855 0 : latest_gc_cutoff_lsn: &RcuReadGuard<Lsn>,
856 0 : ctx: &RequestContext,
857 0 : ) -> Result<Lsn, PageStreamError> {
858 0 : let last_record_lsn = timeline.get_last_record_lsn();
859 0 :
860 0 : // Sanity check the request
861 0 : if request_lsn < not_modified_since {
862 0 : return Err(PageStreamError::BadRequest(
863 0 : format!(
864 0 : "invalid request with request LSN {} and not_modified_since {}",
865 0 : request_lsn, not_modified_since,
866 0 : )
867 0 : .into(),
868 0 : ));
869 0 : }
870 0 :
871 0 : if request_lsn < **latest_gc_cutoff_lsn {
872 : // Check explicitly for INVALID just to get a less scary error message if the
873 : // request is obviously bogus
874 0 : return Err(if request_lsn == Lsn::INVALID {
875 0 : PageStreamError::BadRequest("invalid LSN(0) in request".into())
876 : } else {
877 0 : PageStreamError::BadRequest(format!(
878 0 : "tried to request a page version that was garbage collected. requested at {} gc cutoff {}",
879 0 : request_lsn, **latest_gc_cutoff_lsn
880 0 : ).into())
881 : });
882 0 : }
883 0 :
884 0 : // Wait for WAL up to 'not_modified_since' to arrive, if necessary
885 0 : if not_modified_since > last_record_lsn {
886 0 : timeline
887 0 : .wait_lsn(
888 0 : not_modified_since,
889 0 : crate::tenant::timeline::WaitLsnWaiter::PageService,
890 0 : ctx,
891 0 : )
892 0 : .await?;
893 : // Since we waited for 'not_modified_since' to arrive, that is now the last
894 : // record LSN. (Or close enough for our purposes; the last-record LSN can
895 : // advance immediately after we return anyway)
896 0 : Ok(not_modified_since)
897 : } else {
898 : // It might be better to use max(not_modified_since, latest_gc_cutoff_lsn)
899 : // here instead. That would give the same result, since we know that there
900 : // haven't been any modifications since 'not_modified_since'. Using an older
901 : // LSN might be faster, because that could allow skipping recent layers when
902 : // finding the page. However, we have historically used 'last_record_lsn', so
903 : // stick to that for now.
904 0 : Ok(std::cmp::min(last_record_lsn, request_lsn))
905 : }
906 0 : }
907 :
908 0 : #[instrument(skip_all, fields(shard_id))]
909 : async fn handle_get_rel_exists_request(
910 : &mut self,
911 : tenant_id: TenantId,
912 : timeline_id: TimelineId,
913 : req: &PagestreamExistsRequest,
914 : ctx: &RequestContext,
915 : ) -> Result<PagestreamBeMessage, PageStreamError> {
916 : let timeline = self.get_timeline_shard_zero(tenant_id, timeline_id).await?;
917 : let _timer = timeline
918 : .query_metrics
919 : .start_timer(metrics::SmgrQueryType::GetRelExists, ctx);
920 :
921 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
922 : let lsn = Self::wait_or_get_last_lsn(
923 : timeline,
924 : req.request_lsn,
925 : req.not_modified_since,
926 : &latest_gc_cutoff_lsn,
927 : ctx,
928 : )
929 : .await?;
930 :
931 : let exists = timeline
932 : .get_rel_exists(req.rel, Version::Lsn(lsn), ctx)
933 : .await?;
934 :
935 : Ok(PagestreamBeMessage::Exists(PagestreamExistsResponse {
936 : exists,
937 : }))
938 : }
939 :
940 0 : #[instrument(skip_all, fields(shard_id))]
941 : async fn handle_get_nblocks_request(
942 : &mut self,
943 : tenant_id: TenantId,
944 : timeline_id: TimelineId,
945 : req: &PagestreamNblocksRequest,
946 : ctx: &RequestContext,
947 : ) -> Result<PagestreamBeMessage, PageStreamError> {
948 : let timeline = self.get_timeline_shard_zero(tenant_id, timeline_id).await?;
949 :
950 : let _timer = timeline
951 : .query_metrics
952 : .start_timer(metrics::SmgrQueryType::GetRelSize, ctx);
953 :
954 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
955 : let lsn = Self::wait_or_get_last_lsn(
956 : timeline,
957 : req.request_lsn,
958 : req.not_modified_since,
959 : &latest_gc_cutoff_lsn,
960 : ctx,
961 : )
962 : .await?;
963 :
964 : let n_blocks = timeline
965 : .get_rel_size(req.rel, Version::Lsn(lsn), ctx)
966 : .await?;
967 :
968 : Ok(PagestreamBeMessage::Nblocks(PagestreamNblocksResponse {
969 : n_blocks,
970 : }))
971 : }
972 :
973 0 : #[instrument(skip_all, fields(shard_id))]
974 : async fn handle_db_size_request(
975 : &mut self,
976 : tenant_id: TenantId,
977 : timeline_id: TimelineId,
978 : req: &PagestreamDbSizeRequest,
979 : ctx: &RequestContext,
980 : ) -> Result<PagestreamBeMessage, PageStreamError> {
981 : let timeline = self.get_timeline_shard_zero(tenant_id, timeline_id).await?;
982 :
983 : let _timer = timeline
984 : .query_metrics
985 : .start_timer(metrics::SmgrQueryType::GetDbSize, ctx);
986 :
987 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
988 : let lsn = Self::wait_or_get_last_lsn(
989 : timeline,
990 : req.request_lsn,
991 : req.not_modified_since,
992 : &latest_gc_cutoff_lsn,
993 : ctx,
994 : )
995 : .await?;
996 :
997 : let total_blocks = timeline
998 : .get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, Version::Lsn(lsn), ctx)
999 : .await?;
1000 : let db_size = total_blocks as i64 * BLCKSZ as i64;
1001 :
1002 : Ok(PagestreamBeMessage::DbSize(PagestreamDbSizeResponse {
1003 : db_size,
1004 : }))
1005 : }
1006 :
1007 : /// For most getpage requests, we will already have a Timeline to serve the request: this function
1008 : /// looks up such a Timeline synchronously and without touching any global state.
1009 0 : fn get_cached_timeline_for_page(
1010 0 : &mut self,
1011 0 : req: &PagestreamGetPageRequest,
1012 0 : ) -> Result<&Arc<Timeline>, Key> {
1013 0 : let key = if let Some((first_idx, first_timeline)) = self.shard_timelines.iter().next() {
1014 : // Fastest path: single sharded case
1015 0 : if first_idx.shard_count.count() == 1 {
1016 0 : return Ok(&first_timeline.timeline);
1017 0 : }
1018 0 :
1019 0 : let key = rel_block_to_key(req.rel, req.blkno);
1020 0 : let shard_num = first_timeline
1021 0 : .timeline
1022 0 : .get_shard_identity()
1023 0 : .get_shard_number(&key);
1024 0 :
1025 0 : // Fast path: matched the first timeline in our local handler map. This case is common if
1026 0 : // only one shard per tenant is attached to this pageserver.
1027 0 : if first_timeline.timeline.get_shard_identity().number == shard_num {
1028 0 : return Ok(&first_timeline.timeline);
1029 0 : }
1030 0 :
1031 0 : let shard_index = ShardIndex {
1032 0 : shard_number: shard_num,
1033 0 : shard_count: first_timeline.timeline.get_shard_identity().count,
1034 0 : };
1035 :
1036 : // Fast-ish path: timeline is in the connection handler's local cache
1037 0 : if let Some(found) = self.shard_timelines.get(&shard_index) {
1038 0 : return Ok(&found.timeline);
1039 0 : }
1040 0 :
1041 0 : key
1042 : } else {
1043 0 : rel_block_to_key(req.rel, req.blkno)
1044 : };
1045 :
1046 0 : Err(key)
1047 0 : }
1048 :
1049 : /// Having looked up the [`Timeline`] instance for a particular shard, cache it to enable
1050 : /// use in future requests without having to traverse [`crate::tenant::mgr::TenantManager`]
1051 : /// again.
1052 : ///
1053 : /// Note that all the Timelines in this cache are for the same timeline_id: they're differ
1054 : /// in which shard they belong to. When we serve a getpage@lsn request, we choose a shard
1055 : /// based on key.
1056 : ///
1057 : /// The typical size of this cache is 1, as we generally create shards to distribute work
1058 : /// across pageservers, so don't tend to have multiple shards for the same tenant on the
1059 : /// same pageserver.
1060 0 : fn cache_timeline(
1061 0 : &mut self,
1062 0 : timeline: Arc<Timeline>,
1063 0 : ) -> Result<&Arc<Timeline>, GetActiveTimelineError> {
1064 0 : let gate_guard = timeline
1065 0 : .gate
1066 0 : .enter()
1067 0 : .map_err(|_| GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled))?;
1068 :
1069 0 : let shard_index = timeline.tenant_shard_id.to_index();
1070 0 : let entry = self
1071 0 : .shard_timelines
1072 0 : .entry(shard_index)
1073 0 : .or_insert(HandlerTimeline {
1074 0 : timeline,
1075 0 : _guard: gate_guard,
1076 0 : });
1077 0 :
1078 0 : Ok(&entry.timeline)
1079 0 : }
1080 :
1081 : /// If [`Self::get_cached_timeline_for_page`] missed, then this function is used to populate the cache with
1082 : /// a Timeline to serve requests for this key, if such a Timeline is present on this pageserver. If no such
1083 : /// Timeline is found, then we will return an error (this indicates that the client is talking to the wrong node).
1084 0 : async fn load_timeline_for_page(
1085 0 : &mut self,
1086 0 : tenant_id: TenantId,
1087 0 : timeline_id: TimelineId,
1088 0 : key: Key,
1089 0 : ) -> anyhow::Result<&Arc<Timeline>, GetActiveTimelineError> {
1090 : // Slow path: we must call out to the TenantManager to find the timeline for this Key
1091 0 : let timeline = self
1092 0 : .get_active_tenant_timeline(tenant_id, timeline_id, ShardSelector::Page(key))
1093 0 : .await?;
1094 :
1095 0 : self.cache_timeline(timeline)
1096 0 : }
1097 :
1098 0 : async fn get_timeline_shard_zero(
1099 0 : &mut self,
1100 0 : tenant_id: TenantId,
1101 0 : timeline_id: TimelineId,
1102 0 : ) -> anyhow::Result<&Arc<Timeline>, GetActiveTimelineError> {
1103 : // This is a borrow-checker workaround: we can't return from inside of the `if let Some` because
1104 : // that would be an immutable-borrow-self return, whereas later in the function we will use a mutable
1105 : // ref to salf. So instead, we first build a bool, and then return while not borrowing self.
1106 0 : let have_cached = if let Some((idx, _tl)) = self.shard_timelines.iter().next() {
1107 0 : idx.shard_number == ShardNumber(0)
1108 : } else {
1109 0 : false
1110 : };
1111 :
1112 0 : if have_cached {
1113 0 : let entry = self.shard_timelines.iter().next().unwrap();
1114 0 : Ok(&entry.1.timeline)
1115 : } else {
1116 0 : let timeline = self
1117 0 : .get_active_tenant_timeline(tenant_id, timeline_id, ShardSelector::Zero)
1118 0 : .await?;
1119 0 : Ok(self.cache_timeline(timeline)?)
1120 : }
1121 0 : }
1122 :
1123 0 : #[instrument(skip_all, fields(shard_id))]
1124 : async fn handle_get_page_at_lsn_request(
1125 : &mut self,
1126 : tenant_id: TenantId,
1127 : timeline_id: TimelineId,
1128 : req: &PagestreamGetPageRequest,
1129 : ctx: &RequestContext,
1130 : ) -> Result<PagestreamBeMessage, PageStreamError> {
1131 : let timeline = match self.get_cached_timeline_for_page(req) {
1132 : Ok(tl) => {
1133 : set_tracing_field_shard_id(tl);
1134 : tl
1135 : }
1136 : Err(key) => {
1137 : match self
1138 : .load_timeline_for_page(tenant_id, timeline_id, key)
1139 : .await
1140 : {
1141 : Ok(t) => t,
1142 : Err(GetActiveTimelineError::Tenant(GetActiveTenantError::NotFound(_))) => {
1143 : // We already know this tenant exists in general, because we resolved it at
1144 : // start of connection. Getting a NotFound here indicates that the shard containing
1145 : // the requested page is not present on this node: the client's knowledge of shard->pageserver
1146 : // mapping is out of date.
1147 : //
1148 : // Closing the connection by returning ``::Reconnect` has the side effect of rate-limiting above message, via
1149 : // client's reconnect backoff, as well as hopefully prompting the client to load its updated configuration
1150 : // and talk to a different pageserver.
1151 : return Err(PageStreamError::Reconnect(
1152 : "getpage@lsn request routed to wrong shard".into(),
1153 : ));
1154 : }
1155 : Err(e) => return Err(e.into()),
1156 : }
1157 : }
1158 : };
1159 :
1160 : let _timer = timeline
1161 : .query_metrics
1162 : .start_timer(metrics::SmgrQueryType::GetPageAtLsn, ctx);
1163 :
1164 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
1165 : let lsn = Self::wait_or_get_last_lsn(
1166 : timeline,
1167 : req.request_lsn,
1168 : req.not_modified_since,
1169 : &latest_gc_cutoff_lsn,
1170 : ctx,
1171 : )
1172 : .await?;
1173 :
1174 : let page = timeline
1175 : .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), ctx)
1176 : .await?;
1177 :
1178 : Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse {
1179 : page,
1180 : }))
1181 : }
1182 :
1183 0 : #[instrument(skip_all, fields(shard_id))]
1184 : async fn handle_get_slru_segment_request(
1185 : &mut self,
1186 : tenant_id: TenantId,
1187 : timeline_id: TimelineId,
1188 : req: &PagestreamGetSlruSegmentRequest,
1189 : ctx: &RequestContext,
1190 : ) -> Result<PagestreamBeMessage, PageStreamError> {
1191 : let timeline = self.get_timeline_shard_zero(tenant_id, timeline_id).await?;
1192 :
1193 : let _timer = timeline
1194 : .query_metrics
1195 : .start_timer(metrics::SmgrQueryType::GetSlruSegment, ctx);
1196 :
1197 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
1198 : let lsn = Self::wait_or_get_last_lsn(
1199 : timeline,
1200 : req.request_lsn,
1201 : req.not_modified_since,
1202 : &latest_gc_cutoff_lsn,
1203 : ctx,
1204 : )
1205 : .await?;
1206 :
1207 : let kind = SlruKind::from_repr(req.kind)
1208 : .ok_or(PageStreamError::BadRequest("invalid SLRU kind".into()))?;
1209 : let segment = timeline.get_slru_segment(kind, req.segno, lsn, ctx).await?;
1210 :
1211 : Ok(PagestreamBeMessage::GetSlruSegment(
1212 : PagestreamGetSlruSegmentResponse { segment },
1213 : ))
1214 : }
1215 :
1216 : /// Note on "fullbackup":
1217 : /// Full basebackups should only be used for debugging purposes.
1218 : /// Originally, it was introduced to enable breaking storage format changes,
1219 : /// but that is not applicable anymore.
1220 : #[allow(clippy::too_many_arguments)]
1221 0 : #[instrument(skip_all, fields(shard_id, ?lsn, ?prev_lsn, %full_backup))]
1222 : async fn handle_basebackup_request<IO>(
1223 : &mut self,
1224 : pgb: &mut PostgresBackend<IO>,
1225 : tenant_id: TenantId,
1226 : timeline_id: TimelineId,
1227 : lsn: Option<Lsn>,
1228 : prev_lsn: Option<Lsn>,
1229 : full_backup: bool,
1230 : gzip: bool,
1231 : ctx: &RequestContext,
1232 : ) -> Result<(), QueryError>
1233 : where
1234 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
1235 : {
1236 0 : fn map_basebackup_error(err: BasebackupError) -> QueryError {
1237 0 : match err {
1238 0 : BasebackupError::Client(e) => QueryError::Disconnected(ConnectionError::Io(e)),
1239 0 : BasebackupError::Server(e) => QueryError::Other(e),
1240 : }
1241 0 : }
1242 :
1243 : let started = std::time::Instant::now();
1244 :
1245 : // check that the timeline exists
1246 : let timeline = self
1247 : .get_active_tenant_timeline(tenant_id, timeline_id, ShardSelector::Zero)
1248 : .await?;
1249 : let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn();
1250 : if let Some(lsn) = lsn {
1251 : // Backup was requested at a particular LSN. Wait for it to arrive.
1252 : info!("waiting for {}", lsn);
1253 : timeline
1254 : .wait_lsn(
1255 : lsn,
1256 : crate::tenant::timeline::WaitLsnWaiter::PageService,
1257 : ctx,
1258 : )
1259 : .await?;
1260 : timeline
1261 : .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn)
1262 : .context("invalid basebackup lsn")?;
1263 : }
1264 :
1265 : let lsn_awaited_after = started.elapsed();
1266 :
1267 : // switch client to COPYOUT
1268 : pgb.write_message_noflush(&BeMessage::CopyOutResponse)
1269 : .map_err(QueryError::Disconnected)?;
1270 : self.flush_cancellable(pgb, &timeline.cancel).await?;
1271 :
1272 : // Send a tarball of the latest layer on the timeline. Compress if not
1273 : // fullbackup. TODO Compress in that case too (tests need to be updated)
1274 : if full_backup {
1275 : let mut writer = pgb.copyout_writer();
1276 : basebackup::send_basebackup_tarball(
1277 : &mut writer,
1278 : &timeline,
1279 : lsn,
1280 : prev_lsn,
1281 : full_backup,
1282 : ctx,
1283 : )
1284 : .await
1285 : .map_err(map_basebackup_error)?;
1286 : } else {
1287 : let mut writer = pgb.copyout_writer();
1288 : if gzip {
1289 : let mut encoder = GzipEncoder::with_quality(
1290 : writer,
1291 : // NOTE using fast compression because it's on the critical path
1292 : // for compute startup. For an empty database, we get
1293 : // <100KB with this method. The Level::Best compression method
1294 : // gives us <20KB, but maybe we should add basebackup caching
1295 : // on compute shutdown first.
1296 : async_compression::Level::Fastest,
1297 : );
1298 : basebackup::send_basebackup_tarball(
1299 : &mut encoder,
1300 : &timeline,
1301 : lsn,
1302 : prev_lsn,
1303 : full_backup,
1304 : ctx,
1305 : )
1306 : .await
1307 : .map_err(map_basebackup_error)?;
1308 : // shutdown the encoder to ensure the gzip footer is written
1309 : encoder
1310 : .shutdown()
1311 : .await
1312 0 : .map_err(|e| QueryError::Disconnected(ConnectionError::Io(e)))?;
1313 : } else {
1314 : basebackup::send_basebackup_tarball(
1315 : &mut writer,
1316 : &timeline,
1317 : lsn,
1318 : prev_lsn,
1319 : full_backup,
1320 : ctx,
1321 : )
1322 : .await
1323 : .map_err(map_basebackup_error)?;
1324 : }
1325 : }
1326 :
1327 : pgb.write_message_noflush(&BeMessage::CopyDone)
1328 : .map_err(QueryError::Disconnected)?;
1329 : self.flush_cancellable(pgb, &timeline.cancel).await?;
1330 :
1331 : let basebackup_after = started
1332 : .elapsed()
1333 : .checked_sub(lsn_awaited_after)
1334 : .unwrap_or(Duration::ZERO);
1335 :
1336 : info!(
1337 : lsn_await_millis = lsn_awaited_after.as_millis(),
1338 : basebackup_millis = basebackup_after.as_millis(),
1339 : "basebackup complete"
1340 : );
1341 :
1342 : Ok(())
1343 : }
1344 :
1345 : // when accessing management api supply None as an argument
1346 : // when using to authorize tenant pass corresponding tenant id
1347 0 : fn check_permission(&self, tenant_id: Option<TenantId>) -> Result<(), QueryError> {
1348 0 : if self.auth.is_none() {
1349 : // auth is set to Trust, nothing to check so just return ok
1350 0 : return Ok(());
1351 0 : }
1352 0 : // auth is some, just checked above, when auth is some
1353 0 : // then claims are always present because of checks during connection init
1354 0 : // so this expect won't trigger
1355 0 : let claims = self
1356 0 : .claims
1357 0 : .as_ref()
1358 0 : .expect("claims presence already checked");
1359 0 : check_permission(claims, tenant_id).map_err(|e| QueryError::Unauthorized(e.0))
1360 0 : }
1361 :
1362 : /// Shorthand for getting a reference to a Timeline of an Active tenant.
1363 0 : async fn get_active_tenant_timeline(
1364 0 : &self,
1365 0 : tenant_id: TenantId,
1366 0 : timeline_id: TimelineId,
1367 0 : selector: ShardSelector,
1368 0 : ) -> Result<Arc<Timeline>, GetActiveTimelineError> {
1369 0 : let tenant = self
1370 0 : .get_active_tenant_with_timeout(tenant_id, selector, ACTIVE_TENANT_TIMEOUT)
1371 0 : .await
1372 0 : .map_err(GetActiveTimelineError::Tenant)?;
1373 0 : let timeline = tenant.get_timeline(timeline_id, true)?;
1374 0 : set_tracing_field_shard_id(&timeline);
1375 0 : Ok(timeline)
1376 0 : }
1377 :
1378 : /// Get a shard's [`Tenant`] in its active state, if present. If we don't find the shard and some
1379 : /// slots for this tenant are `InProgress` then we will wait.
1380 : /// If we find the [`Tenant`] and it's not yet in state [`TenantState::Active`], we will wait.
1381 : ///
1382 : /// `timeout` is used as a total timeout for the whole wait operation.
1383 0 : async fn get_active_tenant_with_timeout(
1384 0 : &self,
1385 0 : tenant_id: TenantId,
1386 0 : shard_selector: ShardSelector,
1387 0 : timeout: Duration,
1388 0 : ) -> Result<Arc<Tenant>, GetActiveTenantError> {
1389 0 : let wait_start = Instant::now();
1390 0 : let deadline = wait_start + timeout;
1391 :
1392 : // Resolve TenantId to TenantShardId. This is usually a quick one-shot thing, the loop is
1393 : // for handling the rare case that the slot we're accessing is InProgress.
1394 0 : let tenant_shard = loop {
1395 0 : let resolved = self
1396 0 : .tenant_manager
1397 0 : .resolve_attached_shard(&tenant_id, shard_selector);
1398 0 : match resolved {
1399 0 : ShardResolveResult::Found(tenant_shard) => break tenant_shard,
1400 : ShardResolveResult::NotFound => {
1401 0 : return Err(GetActiveTenantError::NotFound(GetTenantError::NotFound(
1402 0 : tenant_id,
1403 0 : )));
1404 : }
1405 0 : ShardResolveResult::InProgress(barrier) => {
1406 : // We can't authoritatively answer right now: wait for InProgress state
1407 : // to end, then try again
1408 : tokio::select! {
1409 : _ = self.await_connection_cancelled() => {
1410 : return Err(GetActiveTenantError::Cancelled)
1411 : },
1412 : _ = barrier.wait() => {
1413 : // The barrier completed: proceed around the loop to try looking up again
1414 : },
1415 : _ = tokio::time::sleep(deadline.duration_since(Instant::now())) => {
1416 : return Err(GetActiveTenantError::WaitForActiveTimeout {
1417 : latest_state: None,
1418 : wait_time: timeout,
1419 : });
1420 : }
1421 : }
1422 : }
1423 : };
1424 : };
1425 :
1426 0 : tracing::debug!("Waiting for tenant to enter active state...");
1427 0 : tenant_shard
1428 0 : .wait_to_become_active(deadline.duration_since(Instant::now()))
1429 0 : .await?;
1430 0 : Ok(tenant_shard)
1431 0 : }
1432 : }
1433 :
1434 : #[async_trait::async_trait]
1435 : impl<IO> postgres_backend::Handler<IO> for PageServerHandler
1436 : where
1437 : IO: AsyncRead + AsyncWrite + Send + Sync + Unpin,
1438 : {
1439 0 : fn check_auth_jwt(
1440 0 : &mut self,
1441 0 : _pgb: &mut PostgresBackend<IO>,
1442 0 : jwt_response: &[u8],
1443 0 : ) -> Result<(), QueryError> {
1444 : // this unwrap is never triggered, because check_auth_jwt only called when auth_type is NeonJWT
1445 : // which requires auth to be present
1446 0 : let data = self
1447 0 : .auth
1448 0 : .as_ref()
1449 0 : .unwrap()
1450 0 : .decode(str::from_utf8(jwt_response).context("jwt response is not UTF-8")?)
1451 0 : .map_err(|e| QueryError::Unauthorized(e.0))?;
1452 :
1453 0 : if matches!(data.claims.scope, Scope::Tenant) && data.claims.tenant_id.is_none() {
1454 0 : return Err(QueryError::Unauthorized(
1455 0 : "jwt token scope is Tenant, but tenant id is missing".into(),
1456 0 : ));
1457 0 : }
1458 0 :
1459 0 : debug!(
1460 0 : "jwt scope check succeeded for scope: {:#?} by tenant id: {:?}",
1461 : data.claims.scope, data.claims.tenant_id,
1462 : );
1463 :
1464 0 : self.claims = Some(data.claims);
1465 0 : Ok(())
1466 0 : }
1467 :
1468 0 : fn startup(
1469 0 : &mut self,
1470 0 : _pgb: &mut PostgresBackend<IO>,
1471 0 : _sm: &FeStartupPacket,
1472 0 : ) -> Result<(), QueryError> {
1473 0 : Ok(())
1474 0 : }
1475 :
1476 0 : #[instrument(skip_all, fields(tenant_id, timeline_id))]
1477 : async fn process_query(
1478 : &mut self,
1479 : pgb: &mut PostgresBackend<IO>,
1480 : query_string: &str,
1481 0 : ) -> Result<(), QueryError> {
1482 0 : fail::fail_point!("simulated-bad-compute-connection", |_| {
1483 0 : info!("Hit failpoint for bad connection");
1484 0 : Err(QueryError::SimulatedConnectionError)
1485 0 : });
1486 :
1487 0 : let ctx = self.connection_ctx.attached_child();
1488 0 : debug!("process query {query_string:?}");
1489 0 : if query_string.starts_with("pagestream_v2 ") {
1490 0 : let (_, params_raw) = query_string.split_at("pagestream_v2 ".len());
1491 0 : let params = params_raw.split(' ').collect::<Vec<_>>();
1492 0 : if params.len() != 2 {
1493 0 : return Err(QueryError::Other(anyhow::anyhow!(
1494 0 : "invalid param number for pagestream command"
1495 0 : )));
1496 0 : }
1497 0 : let tenant_id = TenantId::from_str(params[0])
1498 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1499 0 : let timeline_id = TimelineId::from_str(params[1])
1500 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1501 :
1502 0 : tracing::Span::current()
1503 0 : .record("tenant_id", field::display(tenant_id))
1504 0 : .record("timeline_id", field::display(timeline_id));
1505 0 :
1506 0 : self.check_permission(Some(tenant_id))?;
1507 :
1508 0 : self.handle_pagerequests(
1509 0 : pgb,
1510 0 : tenant_id,
1511 0 : timeline_id,
1512 0 : PagestreamProtocolVersion::V2,
1513 0 : ctx,
1514 0 : )
1515 0 : .await?;
1516 0 : } else if query_string.starts_with("pagestream ") {
1517 0 : let (_, params_raw) = query_string.split_at("pagestream ".len());
1518 0 : let params = params_raw.split(' ').collect::<Vec<_>>();
1519 0 : if params.len() != 2 {
1520 0 : return Err(QueryError::Other(anyhow::anyhow!(
1521 0 : "invalid param number for pagestream command"
1522 0 : )));
1523 0 : }
1524 0 : let tenant_id = TenantId::from_str(params[0])
1525 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1526 0 : let timeline_id = TimelineId::from_str(params[1])
1527 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1528 :
1529 0 : tracing::Span::current()
1530 0 : .record("tenant_id", field::display(tenant_id))
1531 0 : .record("timeline_id", field::display(timeline_id));
1532 0 :
1533 0 : self.check_permission(Some(tenant_id))?;
1534 :
1535 0 : self.handle_pagerequests(
1536 0 : pgb,
1537 0 : tenant_id,
1538 0 : timeline_id,
1539 0 : PagestreamProtocolVersion::V1,
1540 0 : ctx,
1541 0 : )
1542 0 : .await?;
1543 0 : } else if query_string.starts_with("basebackup ") {
1544 0 : let (_, params_raw) = query_string.split_at("basebackup ".len());
1545 0 : let params = params_raw.split_whitespace().collect::<Vec<_>>();
1546 0 :
1547 0 : if params.len() < 2 {
1548 0 : return Err(QueryError::Other(anyhow::anyhow!(
1549 0 : "invalid param number for basebackup command"
1550 0 : )));
1551 0 : }
1552 :
1553 0 : let tenant_id = TenantId::from_str(params[0])
1554 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1555 0 : let timeline_id = TimelineId::from_str(params[1])
1556 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1557 :
1558 0 : tracing::Span::current()
1559 0 : .record("tenant_id", field::display(tenant_id))
1560 0 : .record("timeline_id", field::display(timeline_id));
1561 0 :
1562 0 : self.check_permission(Some(tenant_id))?;
1563 :
1564 0 : let lsn = if params.len() >= 3 {
1565 : Some(
1566 0 : Lsn::from_str(params[2])
1567 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[2]))?,
1568 : )
1569 : } else {
1570 0 : None
1571 : };
1572 :
1573 0 : let gzip = if params.len() >= 4 {
1574 0 : if params[3] == "--gzip" {
1575 0 : true
1576 : } else {
1577 0 : return Err(QueryError::Other(anyhow::anyhow!(
1578 0 : "Parameter in position 3 unknown {}",
1579 0 : params[3],
1580 0 : )));
1581 : }
1582 : } else {
1583 0 : false
1584 : };
1585 :
1586 0 : let metric_recording = metrics::BASEBACKUP_QUERY_TIME.start_recording(&ctx);
1587 0 : let res = async {
1588 0 : self.handle_basebackup_request(
1589 0 : pgb,
1590 0 : tenant_id,
1591 0 : timeline_id,
1592 0 : lsn,
1593 0 : None,
1594 0 : false,
1595 0 : gzip,
1596 0 : &ctx,
1597 0 : )
1598 0 : .await?;
1599 0 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
1600 0 : Result::<(), QueryError>::Ok(())
1601 0 : }
1602 0 : .await;
1603 0 : metric_recording.observe(&res);
1604 0 : res?;
1605 : }
1606 : // return pair of prev_lsn and last_lsn
1607 0 : else if query_string.starts_with("get_last_record_rlsn ") {
1608 0 : let (_, params_raw) = query_string.split_at("get_last_record_rlsn ".len());
1609 0 : let params = params_raw.split_whitespace().collect::<Vec<_>>();
1610 0 :
1611 0 : if params.len() != 2 {
1612 0 : return Err(QueryError::Other(anyhow::anyhow!(
1613 0 : "invalid param number for get_last_record_rlsn command"
1614 0 : )));
1615 0 : }
1616 :
1617 0 : let tenant_id = TenantId::from_str(params[0])
1618 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1619 0 : let timeline_id = TimelineId::from_str(params[1])
1620 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1621 :
1622 0 : tracing::Span::current()
1623 0 : .record("tenant_id", field::display(tenant_id))
1624 0 : .record("timeline_id", field::display(timeline_id));
1625 0 :
1626 0 : self.check_permission(Some(tenant_id))?;
1627 0 : async {
1628 0 : let timeline = self
1629 0 : .get_active_tenant_timeline(tenant_id, timeline_id, ShardSelector::Zero)
1630 0 : .await?;
1631 :
1632 0 : let end_of_timeline = timeline.get_last_record_rlsn();
1633 0 :
1634 0 : pgb.write_message_noflush(&BeMessage::RowDescription(&[
1635 0 : RowDescriptor::text_col(b"prev_lsn"),
1636 0 : RowDescriptor::text_col(b"last_lsn"),
1637 0 : ]))?
1638 0 : .write_message_noflush(&BeMessage::DataRow(&[
1639 0 : Some(end_of_timeline.prev.to_string().as_bytes()),
1640 0 : Some(end_of_timeline.last.to_string().as_bytes()),
1641 0 : ]))?
1642 0 : .write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
1643 0 : anyhow::Ok(())
1644 0 : }
1645 0 : .instrument(info_span!(
1646 0 : "handle_get_last_record_lsn",
1647 0 : shard_id = tracing::field::Empty
1648 0 : ))
1649 0 : .await?;
1650 : }
1651 : // same as basebackup, but result includes relational data as well
1652 0 : else if query_string.starts_with("fullbackup ") {
1653 0 : let (_, params_raw) = query_string.split_at("fullbackup ".len());
1654 0 : let params = params_raw.split_whitespace().collect::<Vec<_>>();
1655 0 :
1656 0 : if params.len() < 2 {
1657 0 : return Err(QueryError::Other(anyhow::anyhow!(
1658 0 : "invalid param number for fullbackup command"
1659 0 : )));
1660 0 : }
1661 :
1662 0 : let tenant_id = TenantId::from_str(params[0])
1663 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1664 0 : let timeline_id = TimelineId::from_str(params[1])
1665 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1666 :
1667 0 : tracing::Span::current()
1668 0 : .record("tenant_id", field::display(tenant_id))
1669 0 : .record("timeline_id", field::display(timeline_id));
1670 :
1671 : // The caller is responsible for providing correct lsn and prev_lsn.
1672 0 : let lsn = if params.len() > 2 {
1673 : Some(
1674 0 : Lsn::from_str(params[2])
1675 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[2]))?,
1676 : )
1677 : } else {
1678 0 : None
1679 : };
1680 0 : let prev_lsn = if params.len() > 3 {
1681 : Some(
1682 0 : Lsn::from_str(params[3])
1683 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[3]))?,
1684 : )
1685 : } else {
1686 0 : None
1687 : };
1688 :
1689 0 : self.check_permission(Some(tenant_id))?;
1690 :
1691 : // Check that the timeline exists
1692 0 : self.handle_basebackup_request(
1693 0 : pgb,
1694 0 : tenant_id,
1695 0 : timeline_id,
1696 0 : lsn,
1697 0 : prev_lsn,
1698 0 : true,
1699 0 : false,
1700 0 : &ctx,
1701 0 : )
1702 0 : .await?;
1703 0 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
1704 0 : } else if query_string.starts_with("import basebackup ") {
1705 : // Import the `base` section (everything but the wal) of a basebackup.
1706 : // Assumes the tenant already exists on this pageserver.
1707 : //
1708 : // Files are scheduled to be persisted to remote storage, and the
1709 : // caller should poll the http api to check when that is done.
1710 : //
1711 : // Example import command:
1712 : // 1. Get start/end LSN from backup_manifest file
1713 : // 2. Run:
1714 : // cat my_backup/base.tar | psql -h $PAGESERVER \
1715 : // -c "import basebackup $TENANT $TIMELINE $START_LSN $END_LSN $PG_VERSION"
1716 0 : let (_, params_raw) = query_string.split_at("import basebackup ".len());
1717 0 : let params = params_raw.split_whitespace().collect::<Vec<_>>();
1718 0 : if params.len() != 5 {
1719 0 : return Err(QueryError::Other(anyhow::anyhow!(
1720 0 : "invalid param number for import basebackup command"
1721 0 : )));
1722 0 : }
1723 0 : let tenant_id = TenantId::from_str(params[0])
1724 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1725 0 : let timeline_id = TimelineId::from_str(params[1])
1726 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1727 0 : let base_lsn = Lsn::from_str(params[2])
1728 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[2]))?;
1729 0 : let end_lsn = Lsn::from_str(params[3])
1730 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[3]))?;
1731 0 : let pg_version = u32::from_str(params[4])
1732 0 : .with_context(|| format!("Failed to parse pg_version from {}", params[4]))?;
1733 :
1734 0 : tracing::Span::current()
1735 0 : .record("tenant_id", field::display(tenant_id))
1736 0 : .record("timeline_id", field::display(timeline_id));
1737 0 :
1738 0 : self.check_permission(Some(tenant_id))?;
1739 :
1740 0 : match self
1741 0 : .handle_import_basebackup(
1742 0 : pgb,
1743 0 : tenant_id,
1744 0 : timeline_id,
1745 0 : base_lsn,
1746 0 : end_lsn,
1747 0 : pg_version,
1748 0 : ctx,
1749 0 : )
1750 0 : .await
1751 : {
1752 0 : Ok(()) => pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?,
1753 0 : Err(e) => {
1754 0 : error!("error importing base backup between {base_lsn} and {end_lsn}: {e:?}");
1755 0 : pgb.write_message_noflush(&BeMessage::ErrorResponse(
1756 0 : &e.to_string(),
1757 0 : Some(e.pg_error_code()),
1758 0 : ))?
1759 : }
1760 : };
1761 0 : } else if query_string.starts_with("import wal ") {
1762 : // Import the `pg_wal` section of a basebackup.
1763 : //
1764 : // Files are scheduled to be persisted to remote storage, and the
1765 : // caller should poll the http api to check when that is done.
1766 0 : let (_, params_raw) = query_string.split_at("import wal ".len());
1767 0 : let params = params_raw.split_whitespace().collect::<Vec<_>>();
1768 0 : if params.len() != 4 {
1769 0 : return Err(QueryError::Other(anyhow::anyhow!(
1770 0 : "invalid param number for import wal command"
1771 0 : )));
1772 0 : }
1773 0 : let tenant_id = TenantId::from_str(params[0])
1774 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1775 0 : let timeline_id = TimelineId::from_str(params[1])
1776 0 : .with_context(|| format!("Failed to parse timeline id from {}", params[1]))?;
1777 0 : let start_lsn = Lsn::from_str(params[2])
1778 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[2]))?;
1779 0 : let end_lsn = Lsn::from_str(params[3])
1780 0 : .with_context(|| format!("Failed to parse Lsn from {}", params[3]))?;
1781 :
1782 0 : tracing::Span::current()
1783 0 : .record("tenant_id", field::display(tenant_id))
1784 0 : .record("timeline_id", field::display(timeline_id));
1785 0 :
1786 0 : self.check_permission(Some(tenant_id))?;
1787 :
1788 0 : match self
1789 0 : .handle_import_wal(pgb, tenant_id, timeline_id, start_lsn, end_lsn, ctx)
1790 0 : .await
1791 : {
1792 0 : Ok(()) => pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?,
1793 0 : Err(e) => {
1794 0 : error!("error importing WAL between {start_lsn} and {end_lsn}: {e:?}");
1795 0 : pgb.write_message_noflush(&BeMessage::ErrorResponse(
1796 0 : &e.to_string(),
1797 0 : Some(e.pg_error_code()),
1798 0 : ))?
1799 : }
1800 : };
1801 0 : } else if query_string.to_ascii_lowercase().starts_with("set ") {
1802 : // important because psycopg2 executes "SET datestyle TO 'ISO'"
1803 : // on connect
1804 0 : pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
1805 0 : } else if query_string.starts_with("show ") {
1806 : // show <tenant_id>
1807 0 : let (_, params_raw) = query_string.split_at("show ".len());
1808 0 : let params = params_raw.split(' ').collect::<Vec<_>>();
1809 0 : if params.len() != 1 {
1810 0 : return Err(QueryError::Other(anyhow::anyhow!(
1811 0 : "invalid param number for config command"
1812 0 : )));
1813 0 : }
1814 0 : let tenant_id = TenantId::from_str(params[0])
1815 0 : .with_context(|| format!("Failed to parse tenant id from {}", params[0]))?;
1816 :
1817 0 : tracing::Span::current().record("tenant_id", field::display(tenant_id));
1818 0 :
1819 0 : self.check_permission(Some(tenant_id))?;
1820 :
1821 0 : let tenant = self
1822 0 : .get_active_tenant_with_timeout(
1823 0 : tenant_id,
1824 0 : ShardSelector::Zero,
1825 0 : ACTIVE_TENANT_TIMEOUT,
1826 0 : )
1827 0 : .await?;
1828 0 : pgb.write_message_noflush(&BeMessage::RowDescription(&[
1829 0 : RowDescriptor::int8_col(b"checkpoint_distance"),
1830 0 : RowDescriptor::int8_col(b"checkpoint_timeout"),
1831 0 : RowDescriptor::int8_col(b"compaction_target_size"),
1832 0 : RowDescriptor::int8_col(b"compaction_period"),
1833 0 : RowDescriptor::int8_col(b"compaction_threshold"),
1834 0 : RowDescriptor::int8_col(b"gc_horizon"),
1835 0 : RowDescriptor::int8_col(b"gc_period"),
1836 0 : RowDescriptor::int8_col(b"image_creation_threshold"),
1837 0 : RowDescriptor::int8_col(b"pitr_interval"),
1838 0 : ]))?
1839 0 : .write_message_noflush(&BeMessage::DataRow(&[
1840 0 : Some(tenant.get_checkpoint_distance().to_string().as_bytes()),
1841 0 : Some(
1842 0 : tenant
1843 0 : .get_checkpoint_timeout()
1844 0 : .as_secs()
1845 0 : .to_string()
1846 0 : .as_bytes(),
1847 0 : ),
1848 0 : Some(tenant.get_compaction_target_size().to_string().as_bytes()),
1849 0 : Some(
1850 0 : tenant
1851 0 : .get_compaction_period()
1852 0 : .as_secs()
1853 0 : .to_string()
1854 0 : .as_bytes(),
1855 0 : ),
1856 0 : Some(tenant.get_compaction_threshold().to_string().as_bytes()),
1857 0 : Some(tenant.get_gc_horizon().to_string().as_bytes()),
1858 0 : Some(tenant.get_gc_period().as_secs().to_string().as_bytes()),
1859 0 : Some(tenant.get_image_creation_threshold().to_string().as_bytes()),
1860 0 : Some(tenant.get_pitr_interval().as_secs().to_string().as_bytes()),
1861 0 : ]))?
1862 0 : .write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?;
1863 : } else {
1864 0 : return Err(QueryError::Other(anyhow::anyhow!(
1865 0 : "unknown command {query_string}"
1866 0 : )));
1867 : }
1868 :
1869 0 : Ok(())
1870 0 : }
1871 : }
1872 :
1873 : impl From<GetActiveTenantError> for QueryError {
1874 0 : fn from(e: GetActiveTenantError) -> Self {
1875 0 : match e {
1876 0 : GetActiveTenantError::WaitForActiveTimeout { .. } => QueryError::Disconnected(
1877 0 : ConnectionError::Io(io::Error::new(io::ErrorKind::TimedOut, e.to_string())),
1878 0 : ),
1879 : GetActiveTenantError::Cancelled
1880 : | GetActiveTenantError::WillNotBecomeActive(TenantState::Stopping { .. }) => {
1881 0 : QueryError::Shutdown
1882 : }
1883 0 : e @ GetActiveTenantError::NotFound(_) => QueryError::NotFound(format!("{e}").into()),
1884 0 : e => QueryError::Other(anyhow::anyhow!(e)),
1885 : }
1886 0 : }
1887 : }
1888 :
1889 0 : #[derive(Debug, thiserror::Error)]
1890 : enum GetActiveTimelineError {
1891 : #[error(transparent)]
1892 : Tenant(GetActiveTenantError),
1893 : #[error(transparent)]
1894 : Timeline(#[from] GetTimelineError),
1895 : }
1896 :
1897 : impl From<GetActiveTimelineError> for QueryError {
1898 0 : fn from(e: GetActiveTimelineError) -> Self {
1899 0 : match e {
1900 0 : GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled) => QueryError::Shutdown,
1901 0 : GetActiveTimelineError::Tenant(e) => e.into(),
1902 0 : GetActiveTimelineError::Timeline(e) => QueryError::NotFound(format!("{e}").into()),
1903 : }
1904 0 : }
1905 : }
1906 :
1907 0 : fn set_tracing_field_shard_id(timeline: &Timeline) {
1908 0 : debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
1909 0 : tracing::Span::current().record(
1910 0 : "shard_id",
1911 0 : tracing::field::display(timeline.tenant_shard_id.shard_slug()),
1912 0 : );
1913 0 : debug_assert_current_span_has_tenant_and_timeline_id();
1914 0 : }
|