mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-08 14:02:55 +00:00
refactor(pageserver) remove task_mgr for most global tasks (#8449)
## Motivation & Context We want to move away from `task_mgr` towards explicit tracking of child tasks. This PR is extracted from https://github.com/neondatabase/neon/pull/8339 where I refactor `PageRequestHandler` to not depend on task_mgr anymore. ## Changes This PR refactors all global tasks but `PageRequestHandler` to use some combination of `JoinHandle`/`JoinSet` + `CancellationToken`. The `task_mgr::spawn(.., shutdown_process_on_error)` functionality is preserved through the new `exit_on_panic_or_error` wrapper. Some global tasks were not using it before, but as of this PR, they are. The rationale is that all global tasks are relevant for correct operation of the overall Neon system in one way or another. ## Future Work After #8339, we can make `task_mgr::spawn` require a `TenantId` instead of an `Option<TenantId>` which concludes this step of cleanup work and will help discourage future usage of task_mgr for global tasks.
This commit is contained in:
committed by
GitHub
parent
631a9c372f
commit
e8523014d4
@@ -18,9 +18,13 @@ use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_evicti
|
||||
use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING};
|
||||
use pageserver::task_mgr::WALRECEIVER_RUNTIME;
|
||||
use pageserver::tenant::{secondary, TenantSharedResources};
|
||||
use pageserver::{
|
||||
CancellableTask, ConsumptionMetricsTasks, HttpEndpointListener, LibpqEndpointListener,
|
||||
};
|
||||
use remote_storage::GenericRemoteStorage;
|
||||
use tokio::signal::unix::SignalKind;
|
||||
use tokio::time::Instant;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::*;
|
||||
|
||||
use metrics::set_build_info_metric;
|
||||
@@ -430,8 +434,10 @@ fn start_pageserver(
|
||||
|
||||
// Scan the local 'tenants/' directory and start loading the tenants
|
||||
let deletion_queue_client = deletion_queue.new_client();
|
||||
let background_purges = mgr::BackgroundPurges::default();
|
||||
let tenant_manager = BACKGROUND_RUNTIME.block_on(mgr::init_tenant_mgr(
|
||||
conf,
|
||||
background_purges.clone(),
|
||||
TenantSharedResources {
|
||||
broker_client: broker_client.clone(),
|
||||
remote_storage: remote_storage.clone(),
|
||||
@@ -523,7 +529,7 @@ fn start_pageserver(
|
||||
}
|
||||
});
|
||||
|
||||
let secondary_controller = secondary::spawn_tasks(
|
||||
let (secondary_controller, secondary_controller_tasks) = secondary::spawn_tasks(
|
||||
tenant_manager.clone(),
|
||||
remote_storage.clone(),
|
||||
background_jobs_barrier.clone(),
|
||||
@@ -536,18 +542,19 @@ fn start_pageserver(
|
||||
// been configured.
|
||||
let disk_usage_eviction_state: Arc<disk_usage_eviction_task::State> = Arc::default();
|
||||
|
||||
launch_disk_usage_global_eviction_task(
|
||||
let disk_usage_eviction_task = launch_disk_usage_global_eviction_task(
|
||||
conf,
|
||||
remote_storage.clone(),
|
||||
disk_usage_eviction_state.clone(),
|
||||
tenant_manager.clone(),
|
||||
background_jobs_barrier.clone(),
|
||||
)?;
|
||||
);
|
||||
|
||||
// Start up the service to handle HTTP mgmt API request. We created the
|
||||
// listener earlier already.
|
||||
{
|
||||
let _rt_guard = MGMT_REQUEST_RUNTIME.enter();
|
||||
let http_endpoint_listener = {
|
||||
let _rt_guard = MGMT_REQUEST_RUNTIME.enter(); // for hyper
|
||||
let cancel = CancellationToken::new();
|
||||
|
||||
let router_state = Arc::new(
|
||||
http::routes::State::new(
|
||||
@@ -568,77 +575,44 @@ fn start_pageserver(
|
||||
let service = utils::http::RouterService::new(router).unwrap();
|
||||
let server = hyper::Server::from_tcp(http_listener)?
|
||||
.serve(service)
|
||||
.with_graceful_shutdown(task_mgr::shutdown_watcher());
|
||||
.with_graceful_shutdown({
|
||||
let cancel = cancel.clone();
|
||||
async move { cancel.clone().cancelled().await }
|
||||
});
|
||||
|
||||
task_mgr::spawn(
|
||||
MGMT_REQUEST_RUNTIME.handle(),
|
||||
TaskKind::HttpEndpointListener,
|
||||
None,
|
||||
None,
|
||||
let task = MGMT_REQUEST_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"http endpoint listener",
|
||||
true,
|
||||
async {
|
||||
server.await?;
|
||||
Ok(())
|
||||
},
|
||||
);
|
||||
}
|
||||
server,
|
||||
));
|
||||
HttpEndpointListener(CancellableTask { task, cancel })
|
||||
};
|
||||
|
||||
if let Some(metric_collection_endpoint) = &conf.metric_collection_endpoint {
|
||||
let metrics_ctx = RequestContext::todo_child(
|
||||
TaskKind::MetricsCollection,
|
||||
// This task itself shouldn't download anything.
|
||||
// The actual size calculation does need downloads, and
|
||||
// creates a child context with the right DownloadBehavior.
|
||||
DownloadBehavior::Error,
|
||||
);
|
||||
let consumption_metrics_tasks = {
|
||||
let cancel = shutdown_pageserver.child_token();
|
||||
let task = crate::BACKGROUND_RUNTIME.spawn({
|
||||
let tenant_manager = tenant_manager.clone();
|
||||
let cancel = cancel.clone();
|
||||
async move {
|
||||
// first wait until background jobs are cleared to launch.
|
||||
//
|
||||
// this is because we only process active tenants and timelines, and the
|
||||
// Timeline::get_current_logical_size will spawn the logical size calculation,
|
||||
// which will not be rate-limited.
|
||||
tokio::select! {
|
||||
_ = cancel.cancelled() => { return; },
|
||||
_ = background_jobs_barrier.wait() => {}
|
||||
};
|
||||
|
||||
let local_disk_storage = conf.workdir.join("last_consumption_metrics.json");
|
||||
|
||||
task_mgr::spawn(
|
||||
crate::BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::MetricsCollection,
|
||||
None,
|
||||
None,
|
||||
"consumption metrics collection",
|
||||
true,
|
||||
{
|
||||
let tenant_manager = tenant_manager.clone();
|
||||
async move {
|
||||
// first wait until background jobs are cleared to launch.
|
||||
//
|
||||
// this is because we only process active tenants and timelines, and the
|
||||
// Timeline::get_current_logical_size will spawn the logical size calculation,
|
||||
// which will not be rate-limited.
|
||||
let cancel = task_mgr::shutdown_token();
|
||||
|
||||
tokio::select! {
|
||||
_ = cancel.cancelled() => { return Ok(()); },
|
||||
_ = background_jobs_barrier.wait() => {}
|
||||
};
|
||||
|
||||
pageserver::consumption_metrics::collect_metrics(
|
||||
tenant_manager,
|
||||
metric_collection_endpoint,
|
||||
&conf.metric_collection_bucket,
|
||||
conf.metric_collection_interval,
|
||||
conf.synthetic_size_calculation_interval,
|
||||
conf.id,
|
||||
local_disk_storage,
|
||||
cancel,
|
||||
metrics_ctx,
|
||||
)
|
||||
.instrument(info_span!("metrics_collection"))
|
||||
.await?;
|
||||
Ok(())
|
||||
}
|
||||
},
|
||||
);
|
||||
}
|
||||
pageserver::consumption_metrics::run(conf, tenant_manager, cancel).await;
|
||||
}
|
||||
});
|
||||
ConsumptionMetricsTasks(CancellableTask { task, cancel })
|
||||
};
|
||||
|
||||
// Spawn a task to listen for libpq connections. It will spawn further tasks
|
||||
// for each connection. We created the listener earlier already.
|
||||
{
|
||||
let libpq_listener = {
|
||||
let cancel = CancellationToken::new();
|
||||
let libpq_ctx = RequestContext::todo_child(
|
||||
TaskKind::LibpqEndpointListener,
|
||||
// listener task shouldn't need to download anything. (We will
|
||||
@@ -647,29 +621,20 @@ fn start_pageserver(
|
||||
// accept connections.)
|
||||
DownloadBehavior::Error,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
COMPUTE_REQUEST_RUNTIME.handle(),
|
||||
TaskKind::LibpqEndpointListener,
|
||||
None,
|
||||
None,
|
||||
"libpq endpoint listener",
|
||||
true,
|
||||
{
|
||||
let tenant_manager = tenant_manager.clone();
|
||||
async move {
|
||||
page_service::libpq_listener_main(
|
||||
tenant_manager,
|
||||
pg_auth,
|
||||
pageserver_listener,
|
||||
conf.pg_auth_type,
|
||||
libpq_ctx,
|
||||
task_mgr::shutdown_token(),
|
||||
)
|
||||
.await
|
||||
}
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
let task = COMPUTE_REQUEST_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"libpq listener",
|
||||
page_service::libpq_listener_main(
|
||||
tenant_manager.clone(),
|
||||
pg_auth,
|
||||
pageserver_listener,
|
||||
conf.pg_auth_type,
|
||||
libpq_ctx,
|
||||
cancel.clone(),
|
||||
),
|
||||
));
|
||||
LibpqEndpointListener(CancellableTask { task, cancel })
|
||||
};
|
||||
|
||||
let mut shutdown_pageserver = Some(shutdown_pageserver.drop_guard());
|
||||
|
||||
@@ -695,7 +660,18 @@ fn start_pageserver(
|
||||
// Right now that tree doesn't reach very far, and `task_mgr` is used instead.
|
||||
// The plan is to change that over time.
|
||||
shutdown_pageserver.take();
|
||||
pageserver::shutdown_pageserver(&tenant_manager, deletion_queue.clone(), 0).await;
|
||||
pageserver::shutdown_pageserver(
|
||||
http_endpoint_listener,
|
||||
libpq_listener,
|
||||
consumption_metrics_tasks,
|
||||
disk_usage_eviction_task,
|
||||
&tenant_manager,
|
||||
background_purges,
|
||||
deletion_queue.clone(),
|
||||
secondary_controller_tasks,
|
||||
0,
|
||||
)
|
||||
.await;
|
||||
unreachable!()
|
||||
})
|
||||
}
|
||||
|
||||
@@ -1,5 +1,6 @@
|
||||
//! Periodically collect consumption metrics for all active tenants
|
||||
//! and push them to a HTTP endpoint.
|
||||
use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME};
|
||||
use crate::tenant::size::CalculateSyntheticSizeError;
|
||||
@@ -39,49 +40,74 @@ type RawMetric = (MetricsKey, (EventType, u64));
|
||||
/// for deduplication, but that is no longer needed.
|
||||
type Cache = HashMap<MetricsKey, (EventType, u64)>;
|
||||
|
||||
pub async fn run(
|
||||
conf: &'static PageServerConf,
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
cancel: CancellationToken,
|
||||
) {
|
||||
let Some(metric_collection_endpoint) = conf.metric_collection_endpoint.as_ref() else {
|
||||
return;
|
||||
};
|
||||
|
||||
let local_disk_storage = conf.workdir.join("last_consumption_metrics.json");
|
||||
|
||||
let metrics_ctx = RequestContext::todo_child(
|
||||
TaskKind::MetricsCollection,
|
||||
// This task itself shouldn't download anything.
|
||||
// The actual size calculation does need downloads, and
|
||||
// creates a child context with the right DownloadBehavior.
|
||||
DownloadBehavior::Error,
|
||||
);
|
||||
let collect_metrics = BACKGROUND_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"consumption metrics collection",
|
||||
collect_metrics(
|
||||
tenant_manager.clone(),
|
||||
metric_collection_endpoint,
|
||||
&conf.metric_collection_bucket,
|
||||
conf.metric_collection_interval,
|
||||
conf.id,
|
||||
local_disk_storage,
|
||||
cancel.clone(),
|
||||
metrics_ctx,
|
||||
)
|
||||
.instrument(info_span!("metrics_collection")),
|
||||
));
|
||||
|
||||
let worker_ctx =
|
||||
RequestContext::todo_child(TaskKind::CalculateSyntheticSize, DownloadBehavior::Download);
|
||||
let synthetic_size_worker = BACKGROUND_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"synthetic size calculation",
|
||||
calculate_synthetic_size_worker(
|
||||
tenant_manager.clone(),
|
||||
conf.synthetic_size_calculation_interval,
|
||||
cancel.clone(),
|
||||
worker_ctx,
|
||||
)
|
||||
.instrument(info_span!("synthetic_size_worker")),
|
||||
));
|
||||
|
||||
let (collect_metrics, synthetic_size_worker) =
|
||||
futures::future::join(collect_metrics, synthetic_size_worker).await;
|
||||
collect_metrics
|
||||
.expect("unreachable: exit_on_panic_or_error would catch the panic and exit the process");
|
||||
synthetic_size_worker
|
||||
.expect("unreachable: exit_on_panic_or_error would catch the panic and exit the process");
|
||||
}
|
||||
|
||||
/// Main thread that serves metrics collection
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn collect_metrics(
|
||||
async fn collect_metrics(
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
metric_collection_endpoint: &Url,
|
||||
metric_collection_bucket: &Option<RemoteStorageConfig>,
|
||||
metric_collection_interval: Duration,
|
||||
synthetic_size_calculation_interval: Duration,
|
||||
node_id: NodeId,
|
||||
local_disk_storage: Utf8PathBuf,
|
||||
cancel: CancellationToken,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
// spin up background worker that caclulates tenant sizes
|
||||
let worker_ctx =
|
||||
ctx.detached_child(TaskKind::CalculateSyntheticSize, DownloadBehavior::Download);
|
||||
task_mgr::spawn(
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::CalculateSyntheticSize,
|
||||
None,
|
||||
None,
|
||||
"synthetic size calculation",
|
||||
false,
|
||||
{
|
||||
let tenant_manager = tenant_manager.clone();
|
||||
async move {
|
||||
calculate_synthetic_size_worker(
|
||||
tenant_manager,
|
||||
synthetic_size_calculation_interval,
|
||||
&cancel,
|
||||
&worker_ctx,
|
||||
)
|
||||
.instrument(info_span!("synthetic_size_worker"))
|
||||
.await?;
|
||||
Ok(())
|
||||
}
|
||||
},
|
||||
);
|
||||
|
||||
let path: Arc<Utf8PathBuf> = Arc::new(local_disk_storage);
|
||||
|
||||
let cancel = task_mgr::shutdown_token();
|
||||
|
||||
let restore_and_reschedule = restore_and_reschedule(&path, metric_collection_interval);
|
||||
|
||||
let mut cached_metrics = tokio::select! {
|
||||
@@ -168,11 +194,9 @@ pub async fn collect_metrics(
|
||||
BackgroundLoopKind::ConsumptionMetricsCollectMetrics,
|
||||
);
|
||||
|
||||
let res = tokio::time::timeout_at(
|
||||
started_at + metric_collection_interval,
|
||||
task_mgr::shutdown_token().cancelled(),
|
||||
)
|
||||
.await;
|
||||
let res =
|
||||
tokio::time::timeout_at(started_at + metric_collection_interval, cancel.cancelled())
|
||||
.await;
|
||||
if res.is_ok() {
|
||||
return Ok(());
|
||||
}
|
||||
@@ -272,8 +296,8 @@ async fn reschedule(
|
||||
async fn calculate_synthetic_size_worker(
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
synthetic_size_calculation_interval: Duration,
|
||||
cancel: &CancellationToken,
|
||||
ctx: &RequestContext,
|
||||
cancel: CancellationToken,
|
||||
ctx: RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
info!("starting calculate_synthetic_size_worker");
|
||||
scopeguard::defer! {
|
||||
@@ -313,7 +337,7 @@ async fn calculate_synthetic_size_worker(
|
||||
// there is never any reason to exit calculate_synthetic_size_worker following any
|
||||
// return value -- we don't need to care about shutdown because no tenant is found when
|
||||
// pageserver is shut down.
|
||||
calculate_and_log(&tenant, cancel, ctx).await;
|
||||
calculate_and_log(&tenant, &cancel, &ctx).await;
|
||||
}
|
||||
|
||||
crate::tenant::tasks::warn_when_period_overrun(
|
||||
|
||||
@@ -59,13 +59,14 @@ use utils::{completion, id::TimelineId};
|
||||
use crate::{
|
||||
config::PageServerConf,
|
||||
metrics::disk_usage_based_eviction::METRICS,
|
||||
task_mgr::{self, TaskKind, BACKGROUND_RUNTIME},
|
||||
task_mgr::{self, BACKGROUND_RUNTIME},
|
||||
tenant::{
|
||||
mgr::TenantManager,
|
||||
remote_timeline_client::LayerFileMetadata,
|
||||
secondary::SecondaryTenant,
|
||||
storage_layer::{AsLayerDesc, EvictionError, Layer, LayerName},
|
||||
},
|
||||
CancellableTask, DiskUsageEvictionTask,
|
||||
};
|
||||
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
|
||||
@@ -186,36 +187,34 @@ pub fn launch_disk_usage_global_eviction_task(
|
||||
state: Arc<State>,
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
background_jobs_barrier: completion::Barrier,
|
||||
) -> anyhow::Result<()> {
|
||||
) -> Option<DiskUsageEvictionTask> {
|
||||
let Some(task_config) = &conf.disk_usage_based_eviction else {
|
||||
info!("disk usage based eviction task not configured");
|
||||
return Ok(());
|
||||
return None;
|
||||
};
|
||||
|
||||
info!("launching disk usage based eviction task");
|
||||
|
||||
task_mgr::spawn(
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::DiskUsageEviction,
|
||||
None,
|
||||
None,
|
||||
let cancel = CancellationToken::new();
|
||||
let task = BACKGROUND_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"disk usage based eviction",
|
||||
false,
|
||||
async move {
|
||||
let cancel = task_mgr::shutdown_token();
|
||||
{
|
||||
let cancel = cancel.clone();
|
||||
async move {
|
||||
// wait until initial load is complete, because we cannot evict from loading tenants.
|
||||
tokio::select! {
|
||||
_ = cancel.cancelled() => { return anyhow::Ok(()); },
|
||||
_ = background_jobs_barrier.wait() => { }
|
||||
};
|
||||
|
||||
// wait until initial load is complete, because we cannot evict from loading tenants.
|
||||
tokio::select! {
|
||||
_ = cancel.cancelled() => { return Ok(()); },
|
||||
_ = background_jobs_barrier.wait() => { }
|
||||
};
|
||||
|
||||
disk_usage_eviction_task(&state, task_config, &storage, tenant_manager, cancel).await;
|
||||
Ok(())
|
||||
disk_usage_eviction_task(&state, task_config, &storage, tenant_manager, cancel)
|
||||
.await;
|
||||
anyhow::Ok(())
|
||||
}
|
||||
},
|
||||
);
|
||||
));
|
||||
|
||||
Ok(())
|
||||
Some(DiskUsageEvictionTask(CancellableTask { cancel, task }))
|
||||
}
|
||||
|
||||
#[instrument(skip_all)]
|
||||
|
||||
@@ -13,6 +13,7 @@ pub mod http;
|
||||
pub mod import_datadir;
|
||||
pub mod l0_flush;
|
||||
pub use pageserver_api::keyspace;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
pub mod aux_file;
|
||||
pub mod metrics;
|
||||
pub mod page_cache;
|
||||
@@ -32,7 +33,10 @@ pub mod walredo;
|
||||
use crate::task_mgr::TaskKind;
|
||||
use camino::Utf8Path;
|
||||
use deletion_queue::DeletionQueue;
|
||||
use tenant::mgr::TenantManager;
|
||||
use tenant::{
|
||||
mgr::{BackgroundPurges, TenantManager},
|
||||
secondary,
|
||||
};
|
||||
use tracing::info;
|
||||
|
||||
/// Current storage format version
|
||||
@@ -54,17 +58,39 @@ static ZERO_PAGE: bytes::Bytes = bytes::Bytes::from_static(&[0u8; 8192]);
|
||||
|
||||
pub use crate::metrics::preinitialize_metrics;
|
||||
|
||||
pub struct CancellableTask {
|
||||
pub task: tokio::task::JoinHandle<()>,
|
||||
pub cancel: CancellationToken,
|
||||
}
|
||||
pub struct HttpEndpointListener(pub CancellableTask);
|
||||
pub struct LibpqEndpointListener(pub CancellableTask);
|
||||
pub struct ConsumptionMetricsTasks(pub CancellableTask);
|
||||
pub struct DiskUsageEvictionTask(pub CancellableTask);
|
||||
impl CancellableTask {
|
||||
pub async fn shutdown(self) {
|
||||
self.cancel.cancel();
|
||||
self.task.await.unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
#[tracing::instrument(skip_all, fields(%exit_code))]
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn shutdown_pageserver(
|
||||
http_listener: HttpEndpointListener,
|
||||
libpq_listener: LibpqEndpointListener,
|
||||
consumption_metrics_worker: ConsumptionMetricsTasks,
|
||||
disk_usage_eviction_task: Option<DiskUsageEvictionTask>,
|
||||
tenant_manager: &TenantManager,
|
||||
background_purges: BackgroundPurges,
|
||||
mut deletion_queue: DeletionQueue,
|
||||
secondary_controller_tasks: secondary::GlobalTasks,
|
||||
exit_code: i32,
|
||||
) {
|
||||
use std::time::Duration;
|
||||
// Shut down the libpq endpoint task. This prevents new connections from
|
||||
// being accepted.
|
||||
timed(
|
||||
task_mgr::shutdown_tasks(Some(TaskKind::LibpqEndpointListener), None, None),
|
||||
libpq_listener.0.shutdown(),
|
||||
"shutdown LibpqEndpointListener",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
@@ -91,16 +117,44 @@ pub async fn shutdown_pageserver(
|
||||
// Best effort to persist any outstanding deletions, to avoid leaking objects
|
||||
deletion_queue.shutdown(Duration::from_secs(5)).await;
|
||||
|
||||
timed(
|
||||
consumption_metrics_worker.0.shutdown(),
|
||||
"shutdown consumption metrics",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
.await;
|
||||
|
||||
timed(
|
||||
futures::future::OptionFuture::from(disk_usage_eviction_task.map(|t| t.0.shutdown())),
|
||||
"shutdown disk usage eviction",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
.await;
|
||||
|
||||
timed(
|
||||
background_purges.shutdown(),
|
||||
"shutdown background purges",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
.await;
|
||||
|
||||
// Shut down the HTTP endpoint last, so that you can still check the server's
|
||||
// status while it's shutting down.
|
||||
// FIXME: We should probably stop accepting commands like attach/detach earlier.
|
||||
timed(
|
||||
task_mgr::shutdown_tasks(Some(TaskKind::HttpEndpointListener), None, None),
|
||||
http_listener.0.shutdown(),
|
||||
"shutdown http",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
.await;
|
||||
|
||||
timed(
|
||||
secondary_controller_tasks.wait(), // cancellation happened in caller
|
||||
"secondary controller wait",
|
||||
Duration::from_secs(1),
|
||||
)
|
||||
.await;
|
||||
|
||||
// There should be nothing left, but let's be sure
|
||||
timed(
|
||||
task_mgr::shutdown_tasks(None, None, None),
|
||||
|
||||
@@ -124,7 +124,6 @@ pub async fn libpq_listener_main(
|
||||
None,
|
||||
None,
|
||||
"serving compute connection task",
|
||||
false,
|
||||
page_service_conn_main(
|
||||
tenant_manager.clone(),
|
||||
local_auth,
|
||||
|
||||
@@ -408,7 +408,6 @@ pub fn spawn<F>(
|
||||
tenant_shard_id: Option<TenantShardId>,
|
||||
timeline_id: Option<TimelineId>,
|
||||
name: &str,
|
||||
shutdown_process_on_error: bool,
|
||||
future: F,
|
||||
) -> PageserverTaskId
|
||||
where
|
||||
@@ -437,7 +436,6 @@ where
|
||||
task_id,
|
||||
task_cloned,
|
||||
cancel,
|
||||
shutdown_process_on_error,
|
||||
future,
|
||||
));
|
||||
task_mut.join_handle = Some(join_handle);
|
||||
@@ -454,82 +452,78 @@ async fn task_wrapper<F>(
|
||||
task_id: u64,
|
||||
task: Arc<PageServerTask>,
|
||||
shutdown_token: CancellationToken,
|
||||
shutdown_process_on_error: bool,
|
||||
future: F,
|
||||
) where
|
||||
F: Future<Output = anyhow::Result<()>> + Send + 'static,
|
||||
{
|
||||
debug!("Starting task '{}'", task_name);
|
||||
|
||||
let result = SHUTDOWN_TOKEN
|
||||
.scope(
|
||||
shutdown_token,
|
||||
CURRENT_TASK.scope(task, {
|
||||
// We use AssertUnwindSafe here so that the payload function
|
||||
// doesn't need to be UnwindSafe. We don't do anything after the
|
||||
// unwinding that would expose us to unwind-unsafe behavior.
|
||||
AssertUnwindSafe(future).catch_unwind()
|
||||
}),
|
||||
)
|
||||
.await;
|
||||
task_finish(result, task_name, task_id, shutdown_process_on_error).await;
|
||||
}
|
||||
|
||||
async fn task_finish(
|
||||
result: std::result::Result<
|
||||
anyhow::Result<()>,
|
||||
std::boxed::Box<dyn std::any::Any + std::marker::Send>,
|
||||
>,
|
||||
task_name: String,
|
||||
task_id: u64,
|
||||
shutdown_process_on_error: bool,
|
||||
) {
|
||||
// Remove our entry from the global hashmap.
|
||||
let task = TASKS
|
||||
.lock()
|
||||
.unwrap()
|
||||
.remove(&task_id)
|
||||
.expect("no task in registry");
|
||||
|
||||
let mut shutdown_process = false;
|
||||
{
|
||||
// wrap the future so we log panics and errors
|
||||
let tenant_shard_id = task.tenant_shard_id;
|
||||
let timeline_id = task.timeline_id;
|
||||
let fut = async move {
|
||||
// We use AssertUnwindSafe here so that the payload function
|
||||
// doesn't need to be UnwindSafe. We don't do anything after the
|
||||
// unwinding that would expose us to unwind-unsafe behavior.
|
||||
let result = AssertUnwindSafe(future).catch_unwind().await;
|
||||
match result {
|
||||
Ok(Ok(())) => {
|
||||
debug!("Task '{}' exited normally", task_name);
|
||||
}
|
||||
Ok(Err(err)) => {
|
||||
if shutdown_process_on_error {
|
||||
error!(
|
||||
"Shutting down: task '{}' tenant_shard_id: {:?}, timeline_id: {:?} exited with error: {:?}",
|
||||
task_name, task.tenant_shard_id, task.timeline_id, err
|
||||
);
|
||||
shutdown_process = true;
|
||||
} else {
|
||||
error!(
|
||||
"Task '{}' tenant_shard_id: {:?}, timeline_id: {:?} exited with error: {:?}",
|
||||
task_name, task.tenant_shard_id, task.timeline_id, err
|
||||
);
|
||||
}
|
||||
error!(
|
||||
"Task '{}' tenant_shard_id: {:?}, timeline_id: {:?} exited with error: {:?}",
|
||||
task_name, tenant_shard_id, timeline_id, err
|
||||
);
|
||||
}
|
||||
Err(err) => {
|
||||
if shutdown_process_on_error {
|
||||
error!(
|
||||
"Shutting down: task '{}' tenant_shard_id: {:?}, timeline_id: {:?} panicked: {:?}",
|
||||
task_name, task.tenant_shard_id, task.timeline_id, err
|
||||
);
|
||||
shutdown_process = true;
|
||||
} else {
|
||||
error!(
|
||||
"Task '{}' tenant_shard_id: {:?}, timeline_id: {:?} panicked: {:?}",
|
||||
task_name, task.tenant_shard_id, task.timeline_id, err
|
||||
);
|
||||
}
|
||||
error!(
|
||||
"Task '{}' tenant_shard_id: {:?}, timeline_id: {:?} panicked: {:?}",
|
||||
task_name, tenant_shard_id, timeline_id, err
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if shutdown_process {
|
||||
std::process::exit(1);
|
||||
// add the task-locals
|
||||
let fut = CURRENT_TASK.scope(task, fut);
|
||||
let fut = SHUTDOWN_TOKEN.scope(shutdown_token, fut);
|
||||
|
||||
// poll future to completion
|
||||
fut.await;
|
||||
|
||||
// Remove our entry from the global hashmap.
|
||||
TASKS
|
||||
.lock()
|
||||
.unwrap()
|
||||
.remove(&task_id)
|
||||
.expect("no task in registry");
|
||||
}
|
||||
|
||||
pub async fn exit_on_panic_or_error<T, E>(
|
||||
task_name: &'static str,
|
||||
future: impl Future<Output = Result<T, E>>,
|
||||
) -> T
|
||||
where
|
||||
E: std::fmt::Debug,
|
||||
{
|
||||
// We use AssertUnwindSafe here so that the payload function
|
||||
// doesn't need to be UnwindSafe. We don't do anything after the
|
||||
// unwinding that would expose us to unwind-unsafe behavior.
|
||||
let result = AssertUnwindSafe(future).catch_unwind().await;
|
||||
match result {
|
||||
Ok(Ok(val)) => val,
|
||||
Ok(Err(err)) => {
|
||||
error!(
|
||||
task_name,
|
||||
"Task exited with error, exiting process: {err:?}"
|
||||
);
|
||||
std::process::exit(1);
|
||||
}
|
||||
Err(panic_obj) => {
|
||||
error!(task_name, "Task panicked, exiting process: {panic_obj:?}");
|
||||
std::process::exit(1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -721,7 +721,6 @@ impl Tenant {
|
||||
Some(tenant_shard_id),
|
||||
None,
|
||||
"attach tenant",
|
||||
false,
|
||||
async move {
|
||||
|
||||
info!(
|
||||
|
||||
@@ -36,7 +36,7 @@ use crate::control_plane_client::{
|
||||
use crate::deletion_queue::DeletionQueueClient;
|
||||
use crate::http::routes::ACTIVE_TENANT_TIMEOUT;
|
||||
use crate::metrics::{TENANT, TENANT_MANAGER as METRICS};
|
||||
use crate::task_mgr::{self, TaskKind};
|
||||
use crate::task_mgr::{TaskKind, BACKGROUND_RUNTIME};
|
||||
use crate::tenant::config::{
|
||||
AttachedLocationConfig, AttachmentMode, LocationConf, LocationMode, SecondaryLocationConfig,
|
||||
};
|
||||
@@ -225,26 +225,98 @@ async fn safe_rename_tenant_dir(path: impl AsRef<Utf8Path>) -> std::io::Result<U
|
||||
Ok(tmp_path)
|
||||
}
|
||||
|
||||
/// When we have moved a tenant's content to a temporary directory, we may delete it lazily in
|
||||
/// the background, and thereby avoid blocking any API requests on this deletion completing.
|
||||
fn spawn_background_purge(tmp_path: Utf8PathBuf) {
|
||||
// Although we are cleaning up the tenant, this task is not meant to be bound by the lifetime of the tenant in memory.
|
||||
// After a tenant is detached, there are no more task_mgr tasks for that tenant_id.
|
||||
let task_tenant_id = None;
|
||||
/// See [`Self::spawn`].
|
||||
#[derive(Clone)]
|
||||
pub struct BackgroundPurges(Arc<std::sync::Mutex<BackgroundPurgesInner>>);
|
||||
enum BackgroundPurgesInner {
|
||||
Open(tokio::task::JoinSet<()>),
|
||||
// we use the async mutex for coalescing
|
||||
ShuttingDown(Arc<tokio::sync::Mutex<tokio::task::JoinSet<()>>>),
|
||||
}
|
||||
|
||||
task_mgr::spawn(
|
||||
task_mgr::BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::MgmtRequest,
|
||||
task_tenant_id,
|
||||
None,
|
||||
"tenant_files_delete",
|
||||
false,
|
||||
async move {
|
||||
fs::remove_dir_all(tmp_path.as_path())
|
||||
.await
|
||||
.with_context(|| format!("tenant directory {:?} deletion", tmp_path))
|
||||
},
|
||||
);
|
||||
impl Default for BackgroundPurges {
|
||||
fn default() -> Self {
|
||||
Self(Arc::new(std::sync::Mutex::new(
|
||||
BackgroundPurgesInner::Open(JoinSet::new()),
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
impl BackgroundPurges {
|
||||
/// When we have moved a tenant's content to a temporary directory, we may delete it lazily in
|
||||
/// the background, and thereby avoid blocking any API requests on this deletion completing.
|
||||
///
|
||||
/// Although we are cleaning up the tenant, this task is not meant to be bound by the lifetime of the tenant in memory.
|
||||
/// Thus the [`BackgroundPurges`] type to keep track of these tasks.
|
||||
pub fn spawn(&self, tmp_path: Utf8PathBuf) {
|
||||
let mut guard = self.0.lock().unwrap();
|
||||
let jset = match &mut *guard {
|
||||
BackgroundPurgesInner::Open(ref mut jset) => jset,
|
||||
BackgroundPurgesInner::ShuttingDown(_) => {
|
||||
warn!("trying to spawn background purge during shutdown, ignoring");
|
||||
return;
|
||||
}
|
||||
};
|
||||
jset.spawn_on(
|
||||
async move {
|
||||
if let Err(error) = fs::remove_dir_all(tmp_path.as_path()).await {
|
||||
// should we fatal_io_error here?
|
||||
warn!(%error, path=%tmp_path, "failed to purge tenant directory");
|
||||
}
|
||||
}
|
||||
.instrument(info_span!(parent: None, "background_purge")),
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
);
|
||||
}
|
||||
|
||||
/// When this future completes, all background purges have completed.
|
||||
/// The first poll of the future will already lock out new background purges spawned via [`Self::spawn`].
|
||||
///
|
||||
/// Concurrent calls will coalesce.
|
||||
///
|
||||
/// # Cancellation-Safety
|
||||
///
|
||||
/// If this future is dropped before polled to completion, concurrent and subsequent
|
||||
/// instances of this future will continue to be correct.
|
||||
#[instrument(skip_all)]
|
||||
pub async fn shutdown(&self) {
|
||||
let jset = {
|
||||
let mut guard = self.0.lock().unwrap();
|
||||
match &mut *guard {
|
||||
BackgroundPurgesInner::Open(jset) => {
|
||||
*guard = BackgroundPurgesInner::ShuttingDown(Arc::new(tokio::sync::Mutex::new(
|
||||
std::mem::take(jset),
|
||||
)))
|
||||
}
|
||||
BackgroundPurgesInner::ShuttingDown(_) => {
|
||||
// calling shutdown multiple times is most likely a bug in pageserver shutdown code
|
||||
warn!("already shutting down");
|
||||
}
|
||||
};
|
||||
match &mut *guard {
|
||||
BackgroundPurgesInner::ShuttingDown(ref mut jset) => jset.clone(),
|
||||
BackgroundPurgesInner::Open(_) => {
|
||||
unreachable!("above code transitions into shut down state");
|
||||
}
|
||||
}
|
||||
};
|
||||
let mut jset = jset.lock().await; // concurrent callers coalesce here
|
||||
while let Some(res) = jset.join_next().await {
|
||||
match res {
|
||||
Ok(()) => {}
|
||||
Err(e) if e.is_panic() => {
|
||||
// If it panicked, the error is already logged by the panic hook.
|
||||
}
|
||||
Err(e) if e.is_cancelled() => {
|
||||
unreachable!("we don't cancel the joinset or runtime")
|
||||
}
|
||||
Err(e) => {
|
||||
// No idea when this can happen, but let's log it.
|
||||
warn!(%e, "background purge task failed or panicked");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static TENANTS: Lazy<std::sync::RwLock<TenantsMap>> =
|
||||
@@ -270,6 +342,8 @@ pub struct TenantManager {
|
||||
// tenants have their own cancellation tokens, which we fire individually in [`Self::shutdown`], or
|
||||
// when the tenant detaches.
|
||||
cancel: CancellationToken,
|
||||
|
||||
background_purges: BackgroundPurges,
|
||||
}
|
||||
|
||||
fn emergency_generations(
|
||||
@@ -447,6 +521,7 @@ pub(crate) enum DeleteTenantError {
|
||||
#[instrument(skip_all)]
|
||||
pub async fn init_tenant_mgr(
|
||||
conf: &'static PageServerConf,
|
||||
background_purges: BackgroundPurges,
|
||||
resources: TenantSharedResources,
|
||||
init_order: InitializationOrder,
|
||||
cancel: CancellationToken,
|
||||
@@ -512,7 +587,7 @@ pub async fn init_tenant_mgr(
|
||||
|
||||
match safe_rename_tenant_dir(&tenant_dir_path).await {
|
||||
Ok(tmp_path) => {
|
||||
spawn_background_purge(tmp_path);
|
||||
background_purges.spawn(tmp_path);
|
||||
}
|
||||
Err(e) => {
|
||||
error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(),
|
||||
@@ -634,6 +709,7 @@ pub async fn init_tenant_mgr(
|
||||
tenants: &TENANTS,
|
||||
resources,
|
||||
cancel: CancellationToken::new(),
|
||||
background_purges,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -1353,6 +1429,7 @@ impl TenantManager {
|
||||
|
||||
async fn delete_local(
|
||||
conf: &PageServerConf,
|
||||
background_purges: &BackgroundPurges,
|
||||
tenant_shard_id: &TenantShardId,
|
||||
) -> anyhow::Result<()> {
|
||||
let local_tenant_directory = conf.tenant_path(tenant_shard_id);
|
||||
@@ -1361,7 +1438,7 @@ impl TenantManager {
|
||||
.with_context(|| {
|
||||
format!("local tenant directory {local_tenant_directory:?} rename")
|
||||
})?;
|
||||
spawn_background_purge(tmp_dir);
|
||||
background_purges.spawn(tmp_dir);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1379,12 +1456,12 @@ impl TenantManager {
|
||||
barrier.wait().await;
|
||||
}
|
||||
}
|
||||
delete_local(self.conf, &tenant_shard_id).await?;
|
||||
delete_local(self.conf, &self.background_purges, &tenant_shard_id).await?;
|
||||
}
|
||||
Some(TenantSlot::Secondary(secondary_tenant)) => {
|
||||
secondary_tenant.shutdown().await;
|
||||
|
||||
delete_local(self.conf, &tenant_shard_id).await?;
|
||||
delete_local(self.conf, &self.background_purges, &tenant_shard_id).await?;
|
||||
}
|
||||
Some(TenantSlot::InProgress(_)) => unreachable!(),
|
||||
None => {}
|
||||
@@ -1655,7 +1732,7 @@ impl TenantManager {
|
||||
let tmp_path = safe_rename_tenant_dir(&local_tenant_directory)
|
||||
.await
|
||||
.with_context(|| format!("local tenant directory {local_tenant_directory:?} rename"))?;
|
||||
spawn_background_purge(tmp_path);
|
||||
self.background_purges.spawn(tmp_path);
|
||||
|
||||
fail::fail_point!("shard-split-pre-finish", |_| Err(anyhow::anyhow!(
|
||||
"failpoint"
|
||||
@@ -1831,7 +1908,7 @@ impl TenantManager {
|
||||
let tmp_path = self
|
||||
.detach_tenant0(conf, tenant_shard_id, deletion_queue_client)
|
||||
.await?;
|
||||
spawn_background_purge(tmp_path);
|
||||
self.background_purges.spawn(tmp_path);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1525,7 +1525,6 @@ impl RemoteTimelineClient {
|
||||
Some(self.tenant_shard_id),
|
||||
Some(self.timeline_id),
|
||||
"remote upload",
|
||||
false,
|
||||
async move {
|
||||
self_rc.perform_upload_task(task).await;
|
||||
Ok(())
|
||||
|
||||
@@ -31,6 +31,7 @@ use pageserver_api::{
|
||||
};
|
||||
use remote_storage::GenericRemoteStorage;
|
||||
|
||||
use tokio::task::JoinHandle;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::instrument;
|
||||
use utils::{completion::Barrier, id::TimelineId, sync::gate::Gate};
|
||||
@@ -293,15 +294,50 @@ impl SecondaryController {
|
||||
}
|
||||
}
|
||||
|
||||
pub struct GlobalTasks {
|
||||
cancel: CancellationToken,
|
||||
uploader: JoinHandle<()>,
|
||||
downloader: JoinHandle<()>,
|
||||
}
|
||||
|
||||
impl GlobalTasks {
|
||||
/// Caller is responsible for requesting shutdown via the cancellation token that was
|
||||
/// passed to [`spawn_tasks`].
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// This method panics if that token is not cancelled.
|
||||
/// This is low-risk because we're calling this during process shutdown, so, a panic
|
||||
/// will be informative but not cause undue downtime.
|
||||
pub async fn wait(self) {
|
||||
let Self {
|
||||
cancel,
|
||||
uploader,
|
||||
downloader,
|
||||
} = self;
|
||||
assert!(
|
||||
cancel.is_cancelled(),
|
||||
"must cancel cancellation token, otherwise the tasks will not shut down"
|
||||
);
|
||||
|
||||
let (uploader, downloader) = futures::future::join(uploader, downloader).await;
|
||||
uploader.expect(
|
||||
"unreachable: exit_on_panic_or_error would catch the panic and exit the process",
|
||||
);
|
||||
downloader.expect(
|
||||
"unreachable: exit_on_panic_or_error would catch the panic and exit the process",
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
pub fn spawn_tasks(
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
remote_storage: GenericRemoteStorage,
|
||||
background_jobs_can_start: Barrier,
|
||||
cancel: CancellationToken,
|
||||
) -> SecondaryController {
|
||||
) -> (SecondaryController, GlobalTasks) {
|
||||
let mgr_clone = tenant_manager.clone();
|
||||
let storage_clone = remote_storage.clone();
|
||||
let cancel_clone = cancel.clone();
|
||||
let bg_jobs_clone = background_jobs_can_start.clone();
|
||||
|
||||
let (download_req_tx, download_req_rx) =
|
||||
@@ -309,17 +345,9 @@ pub fn spawn_tasks(
|
||||
let (upload_req_tx, upload_req_rx) =
|
||||
tokio::sync::mpsc::channel::<CommandRequest<UploadCommand>>(16);
|
||||
|
||||
let downloader_task_ctx = RequestContext::new(
|
||||
TaskKind::SecondaryDownloads,
|
||||
crate::context::DownloadBehavior::Download,
|
||||
);
|
||||
task_mgr::spawn(
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
downloader_task_ctx.task_kind(),
|
||||
None,
|
||||
None,
|
||||
let cancel_clone = cancel.clone();
|
||||
let downloader = BACKGROUND_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"secondary tenant downloads",
|
||||
false,
|
||||
async move {
|
||||
downloader_task(
|
||||
mgr_clone,
|
||||
@@ -327,49 +355,41 @@ pub fn spawn_tasks(
|
||||
download_req_rx,
|
||||
bg_jobs_clone,
|
||||
cancel_clone,
|
||||
downloader_task_ctx,
|
||||
RequestContext::new(
|
||||
TaskKind::SecondaryDownloads,
|
||||
crate::context::DownloadBehavior::Download,
|
||||
),
|
||||
)
|
||||
.await;
|
||||
|
||||
Ok(())
|
||||
anyhow::Ok(())
|
||||
},
|
||||
);
|
||||
));
|
||||
|
||||
task_mgr::spawn(
|
||||
BACKGROUND_RUNTIME.handle(),
|
||||
TaskKind::SecondaryUploads,
|
||||
None,
|
||||
None,
|
||||
let cancel_clone = cancel.clone();
|
||||
let uploader = BACKGROUND_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
|
||||
"heatmap uploads",
|
||||
false,
|
||||
async move {
|
||||
heatmap_uploader_task(
|
||||
tenant_manager,
|
||||
remote_storage,
|
||||
upload_req_rx,
|
||||
background_jobs_can_start,
|
||||
cancel,
|
||||
cancel_clone,
|
||||
)
|
||||
.await;
|
||||
|
||||
Ok(())
|
||||
anyhow::Ok(())
|
||||
},
|
||||
);
|
||||
));
|
||||
|
||||
SecondaryController {
|
||||
download_req_tx,
|
||||
upload_req_tx,
|
||||
}
|
||||
}
|
||||
|
||||
/// For running with remote storage disabled: a SecondaryController that is connected to nothing.
|
||||
pub fn null_controller() -> SecondaryController {
|
||||
let (download_req_tx, _download_req_rx) =
|
||||
tokio::sync::mpsc::channel::<CommandRequest<DownloadCommand>>(16);
|
||||
let (upload_req_tx, _upload_req_rx) =
|
||||
tokio::sync::mpsc::channel::<CommandRequest<UploadCommand>>(16);
|
||||
SecondaryController {
|
||||
upload_req_tx,
|
||||
download_req_tx,
|
||||
}
|
||||
(
|
||||
SecondaryController {
|
||||
upload_req_tx,
|
||||
download_req_tx,
|
||||
},
|
||||
GlobalTasks {
|
||||
cancel,
|
||||
uploader,
|
||||
downloader,
|
||||
},
|
||||
)
|
||||
}
|
||||
|
||||
@@ -101,7 +101,6 @@ pub fn start_background_loops(
|
||||
Some(tenant_shard_id),
|
||||
None,
|
||||
&format!("compactor for tenant {tenant_shard_id}"),
|
||||
false,
|
||||
{
|
||||
let tenant = Arc::clone(tenant);
|
||||
let background_jobs_can_start = background_jobs_can_start.cloned();
|
||||
@@ -125,7 +124,6 @@ pub fn start_background_loops(
|
||||
Some(tenant_shard_id),
|
||||
None,
|
||||
&format!("garbage collector for tenant {tenant_shard_id}"),
|
||||
false,
|
||||
{
|
||||
let tenant = Arc::clone(tenant);
|
||||
let background_jobs_can_start = background_jobs_can_start.cloned();
|
||||
@@ -149,7 +147,6 @@ pub fn start_background_loops(
|
||||
Some(tenant_shard_id),
|
||||
None,
|
||||
&format!("ingest housekeeping for tenant {tenant_shard_id}"),
|
||||
false,
|
||||
{
|
||||
let tenant = Arc::clone(tenant);
|
||||
let background_jobs_can_start = background_jobs_can_start.cloned();
|
||||
|
||||
@@ -2477,7 +2477,6 @@ impl Timeline {
|
||||
Some(self.tenant_shard_id),
|
||||
Some(self.timeline_id),
|
||||
"layer flush task",
|
||||
false,
|
||||
async move {
|
||||
let _guard = guard;
|
||||
let background_ctx = RequestContext::todo_child(TaskKind::LayerFlushTask, DownloadBehavior::Error);
|
||||
@@ -2822,7 +2821,6 @@ impl Timeline {
|
||||
Some(self.tenant_shard_id),
|
||||
Some(self.timeline_id),
|
||||
"initial size calculation",
|
||||
false,
|
||||
// NB: don't log errors here, task_mgr will do that.
|
||||
async move {
|
||||
let cancel = task_mgr::shutdown_token();
|
||||
@@ -2991,7 +2989,6 @@ impl Timeline {
|
||||
Some(self.tenant_shard_id),
|
||||
Some(self.timeline_id),
|
||||
"ondemand logical size calculation",
|
||||
false,
|
||||
async move {
|
||||
let res = self_clone
|
||||
.logical_size_calculation_task(lsn, cause, &ctx)
|
||||
@@ -5435,7 +5432,6 @@ impl Timeline {
|
||||
Some(self.tenant_shard_id),
|
||||
Some(self.timeline_id),
|
||||
"download all remote layers task",
|
||||
false,
|
||||
async move {
|
||||
self_clone.download_all_remote_layers(request).await;
|
||||
let mut status_guard = self_clone.download_all_remote_layers_task_info.write().unwrap();
|
||||
|
||||
@@ -391,7 +391,6 @@ impl DeleteTimelineFlow {
|
||||
Some(tenant_shard_id),
|
||||
Some(timeline_id),
|
||||
"timeline_delete",
|
||||
false,
|
||||
async move {
|
||||
if let Err(err) = Self::background(guard, conf, &tenant, &timeline).await {
|
||||
error!("Error: {err:#}");
|
||||
|
||||
@@ -65,7 +65,6 @@ impl Timeline {
|
||||
"layer eviction for {}/{}",
|
||||
self.tenant_shard_id, self.timeline_id
|
||||
),
|
||||
false,
|
||||
async move {
|
||||
tokio::select! {
|
||||
_ = self_clone.cancel.cancelled() => { return Ok(()); }
|
||||
|
||||
Reference in New Issue
Block a user