Skip to content

Commit

Permalink
refactor(pageserver): move queue logic to compaction.rs (#10330)
Browse files Browse the repository at this point in the history
## Problem

close #10031, part of
#9114

## Summary of changes

Move the compaction job generation to `compaction.rs`, thus making the
code more readable and debuggable. We now also return running job
through the get compaction job API, versus before we only return
scheduled jobs.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
  • Loading branch information
skyzh authored Jan 10, 2025
1 parent 58332cb commit b5d54ba
Show file tree
Hide file tree
Showing 4 changed files with 324 additions and 160 deletions.
2 changes: 2 additions & 0 deletions libs/pageserver_api/src/models.rs
Original file line number Diff line number Diff line change
Expand Up @@ -272,6 +272,8 @@ pub struct CompactInfoResponse {
pub compact_key_range: Option<CompactKeyRange>,
pub compact_lsn_range: Option<CompactLsnRange>,
pub sub_compaction: bool,
pub running: bool,
pub job_id: usize,
}

#[derive(Serialize, Deserialize, Clone)]
Expand Down
14 changes: 3 additions & 11 deletions pageserver/src/http/routes.rs
Original file line number Diff line number Diff line change
Expand Up @@ -97,8 +97,8 @@ use crate::tenant::{LogicalSizeCalculationCause, PageReconstructError};
use crate::DEFAULT_PG_VERSION;
use crate::{disk_usage_eviction_task, tenant};
use pageserver_api::models::{
CompactInfoResponse, StatusResponse, TenantConfigRequest, TenantInfo, TimelineCreateRequest,
TimelineGcRequest, TimelineInfo,
StatusResponse, TenantConfigRequest, TenantInfo, TimelineCreateRequest, TimelineGcRequest,
TimelineInfo,
};
use utils::{
auth::SwappableJwtAuth,
Expand Down Expand Up @@ -2052,15 +2052,7 @@ async fn timeline_compact_info_handler(
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
let res = tenant.get_scheduled_compaction_tasks(timeline_id);
let mut resp = Vec::new();
for item in res {
resp.push(CompactInfoResponse {
compact_key_range: item.compact_key_range,
compact_lsn_range: item.compact_lsn_range,
sub_compaction: item.sub_compaction,
});
}
let resp = tenant.get_scheduled_compaction_tasks(timeline_id);
json_response(StatusCode::OK, resp)
}
.instrument(info_span!("timeline_compact_info", tenant_id = %tenant_shard_id.tenant_id, shard_id = %tenant_shard_id.shard_slug(), %timeline_id))
Expand Down
175 changes: 37 additions & 138 deletions pageserver/src/tenant.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ use enumset::EnumSet;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
use pageserver_api::models;
use pageserver_api::models::CompactInfoResponse;
use pageserver_api::models::LsnLease;
use pageserver_api::models::TimelineArchivalState;
use pageserver_api::models::TimelineState;
Expand All @@ -37,21 +38,17 @@ use remote_timeline_client::manifest::{
};
use remote_timeline_client::UploadQueueNotReadyError;
use std::collections::BTreeMap;
use std::collections::VecDeque;
use std::fmt;
use std::future::Future;
use std::sync::atomic::AtomicBool;
use std::sync::Weak;
use std::time::SystemTime;
use storage_broker::BrokerClientChannel;
use timeline::compaction::GcCompactJob;
use timeline::compaction::ScheduledCompactionTask;
use timeline::compaction::GcCompactionQueue;
use timeline::import_pgdata;
use timeline::offload::offload_timeline;
use timeline::offload::OffloadError;
use timeline::CompactFlags;
use timeline::CompactOptions;
use timeline::CompactionError;
use timeline::ShutdownMode;
use tokio::io::BufReader;
use tokio::sync::watch;
Expand Down Expand Up @@ -347,10 +344,8 @@ pub struct Tenant {
/// Overhead of mutex is acceptable because compaction is done with a multi-second period.
compaction_circuit_breaker: std::sync::Mutex<CircuitBreaker>,

/// Scheduled compaction tasks. Currently, this can only be populated by triggering
/// a manual gc-compaction from the manual compaction API.
scheduled_compaction_tasks:
std::sync::Mutex<HashMap<TimelineId, VecDeque<ScheduledCompactionTask>>>,
/// Scheduled gc-compaction tasks.
scheduled_compaction_tasks: std::sync::Mutex<HashMap<TimelineId, Arc<GcCompactionQueue>>>,

/// If the tenant is in Activating state, notify this to encourage it
/// to proceed to Active as soon as possible, rather than waiting for lazy
Expand Down Expand Up @@ -2997,104 +2992,18 @@ impl Tenant {
if has_pending_l0_compaction_task {
Some(true)
} else {
let mut has_pending_scheduled_compaction_task;
let next_scheduled_compaction_task = {
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
if let Some(tline_pending_tasks) = guard.get_mut(timeline_id) {
if !tline_pending_tasks.is_empty() {
info!(
"{} tasks left in the compaction schedule queue",
tline_pending_tasks.len()
);
}
let next_task = tline_pending_tasks.pop_front();
has_pending_scheduled_compaction_task = !tline_pending_tasks.is_empty();
next_task
} else {
has_pending_scheduled_compaction_task = false;
None
}
let queue = {
let guard = self.scheduled_compaction_tasks.lock().unwrap();
guard.get(timeline_id).cloned()
};
if let Some(mut next_scheduled_compaction_task) = next_scheduled_compaction_task
{
if !next_scheduled_compaction_task
.options
.flags
.contains(CompactFlags::EnhancedGcBottomMostCompaction)
{
warn!("ignoring scheduled compaction task: scheduled task must be gc compaction: {:?}", next_scheduled_compaction_task.options);
} else if next_scheduled_compaction_task.options.sub_compaction {
info!("running scheduled enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs");
let jobs: Vec<GcCompactJob> = timeline
.gc_compaction_split_jobs(
GcCompactJob::from_compact_options(
next_scheduled_compaction_task.options.clone(),
),
next_scheduled_compaction_task
.options
.sub_compaction_max_job_size_mb,
)
.await
.map_err(CompactionError::Other)?;
if jobs.is_empty() {
info!("no jobs to run, skipping scheduled compaction task");
} else {
has_pending_scheduled_compaction_task = true;
let jobs_len = jobs.len();
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
let tline_pending_tasks = guard.entry(*timeline_id).or_default();
for (idx, job) in jobs.into_iter().enumerate() {
// Unfortunately we need to convert the `GcCompactJob` back to `CompactionOptions`
// until we do further refactors to allow directly call `compact_with_gc`.
let mut flags: EnumSet<CompactFlags> = EnumSet::default();
flags |= CompactFlags::EnhancedGcBottomMostCompaction;
if job.dry_run {
flags |= CompactFlags::DryRun;
}
let options = CompactOptions {
flags,
sub_compaction: false,
compact_key_range: Some(job.compact_key_range.into()),
compact_lsn_range: Some(job.compact_lsn_range.into()),
sub_compaction_max_job_size_mb: None,
};
tline_pending_tasks.push_back(if idx == jobs_len - 1 {
ScheduledCompactionTask {
options,
// The last job in the queue sends the signal and releases the gc guard
result_tx: next_scheduled_compaction_task
.result_tx
.take(),
gc_block: next_scheduled_compaction_task
.gc_block
.take(),
}
} else {
ScheduledCompactionTask {
options,
result_tx: None,
gc_block: None,
}
});
}
info!("scheduled enhanced gc bottom-most compaction with sub-compaction, split into {} jobs", jobs_len);
}
} else {
let _ = timeline
.compact_with_options(
cancel,
next_scheduled_compaction_task.options,
ctx,
)
.instrument(info_span!("scheduled_compact_timeline", %timeline_id))
.await?;
if let Some(tx) = next_scheduled_compaction_task.result_tx.take() {
// TODO: we can send compaction statistics in the future
tx.send(()).ok();
}
}
if let Some(queue) = queue {
let has_pending_tasks = queue
.iteration(cancel, ctx, &self.gc_block, timeline)
.await?;
Some(has_pending_tasks)
} else {
Some(false)
}
Some(has_pending_scheduled_compaction_task)
}
} else {
None
Expand Down Expand Up @@ -3124,34 +3033,32 @@ impl Tenant {
}

/// Cancel scheduled compaction tasks
pub(crate) fn cancel_scheduled_compaction(
&self,
timeline_id: TimelineId,
) -> Vec<ScheduledCompactionTask> {
pub(crate) fn cancel_scheduled_compaction(&self, timeline_id: TimelineId) {
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
if let Some(tline_pending_tasks) = guard.get_mut(&timeline_id) {
let current_tline_pending_tasks = std::mem::take(tline_pending_tasks);
current_tline_pending_tasks.into_iter().collect()
} else {
Vec::new()
if let Some(q) = guard.get_mut(&timeline_id) {
q.cancel_scheduled();
}
}

pub(crate) fn get_scheduled_compaction_tasks(
&self,
timeline_id: TimelineId,
) -> Vec<CompactOptions> {
use itertools::Itertools;
let guard = self.scheduled_compaction_tasks.lock().unwrap();
guard
.get(&timeline_id)
.map(|tline_pending_tasks| {
tline_pending_tasks
.iter()
.map(|x| x.options.clone())
.collect_vec()
})
.unwrap_or_default()
) -> Vec<CompactInfoResponse> {
let res = {
let guard = self.scheduled_compaction_tasks.lock().unwrap();
guard.get(&timeline_id).map(|q| q.remaining_jobs())
};
let Some((running, remaining)) = res else {
return Vec::new();
};
let mut result = Vec::new();
if let Some((id, running)) = running {
result.extend(running.into_compact_info_resp(id, true));
}
for (id, job) in remaining {
result.extend(job.into_compact_info_resp(id, false));
}
result
}

/// Schedule a compaction task for a timeline.
Expand All @@ -3160,20 +3067,12 @@ impl Tenant {
timeline_id: TimelineId,
options: CompactOptions,
) -> anyhow::Result<tokio::sync::oneshot::Receiver<()>> {
let gc_guard = match self.gc_block.start().await {
Ok(guard) => guard,
Err(e) => {
bail!("cannot run gc-compaction because gc is blocked: {}", e);
}
};
let (tx, rx) = tokio::sync::oneshot::channel();
let mut guard = self.scheduled_compaction_tasks.lock().unwrap();
let tline_pending_tasks = guard.entry(timeline_id).or_default();
tline_pending_tasks.push_back(ScheduledCompactionTask {
options,
result_tx: Some(tx),
gc_block: Some(gc_guard),
});
let q = guard
.entry(timeline_id)
.or_insert_with(|| Arc::new(GcCompactionQueue::new()));
q.schedule_manual_compaction(options, Some(tx));
Ok(rx)
}

Expand Down
Loading

1 comment on commit b5d54ba

@github-actions
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

7433 tests run: 7058 passed, 1 failed, 374 skipped (full report)


Failures on Postgres 16

  • test_storage_controller_many_tenants[github-actions-selfhosted]: release-x86-64
# Run all failed tests locally:
scripts/pytest -vv -n $(nproc) -k "test_storage_controller_many_tenants[release-pg16-github-actions-selfhosted]"

Code coverage* (full report)

  • functions: 32.7% (8051 of 24640 functions)
  • lines: 47.7% (66839 of 140043 lines)

* collected from Rust tests only


The comment gets automatically updated with the latest test results
b5d54ba at 2025-01-10T23:12:57.012Z :recycle:

Please sign in to comment.