storage controller: scheduling optimization for sharded tenants (#7181)

## Problem

- When we scheduled locations, we were doing it without any context
about other shards in the same tenant
- After a shard split, there wasn't an automatic mechanism to migrate
the attachments away from the split location
- After a shard split and the migration away from the split location,
there wasn't an automatic mechanism to pick new secondary locations so
that the end state has no concentration of locations on the nodes where
the split happened.

Partially completes: https://github.com/neondatabase/neon/issues/7139

## Summary of changes

- Scheduler now takes a `ScheduleContext` object that can be populated
with information about other shards
- During tenant creation and shard split, we incrementally build up the
ScheduleContext, updating it for each shard as we proceed.
- When scheduling new locations, the ScheduleContext is used to apply a
soft anti-affinity to nodes where a tenant already has shards.
- The background reconciler task now has an extra phase `optimize_all`,
which runs only if the primary `reconcile_all` phase didn't generate any
work. The separation is that `reconcile_all` is needed for availability,
but optimize_all is purely "nice to have" work to balance work across
the nodes better.
- optimize_all calls into two new TenantState methods called
optimize_attachment and optimize_secondary, which seek out opportunities
to improve placment:
- optimize_attachment: if the node where we're currently attached has an
excess of attached shard locations for this tenant compared with the
node where we have a secondary location, then cut over to the secondary
location.
- optimize_secondary: if the node holding our secondary location has an
excessive number of locations for this tenant compared with some other
node where we don't currently have a location, then create a new
secondary location on that other node.
- a new debug API endpoint is provided to run background tasks
on-demand. This returns a number of reconciliations in progress, so
callers can keep calling until they get a `0` to advance the system to
its final state without waiting for many iterations of the background
task.

Optimization is run at an implicitly low priority by:
- Omitting the phase entirely if reconcile_all has work to do
- Skipping optimization of any tenant that has reconciles in flight
- Limiting the total number of optimizations that will be run from one
call to optimize_all to a constant (currently 2).

The idea of that low priority execution is to minimize the operational
risk that optimization work overloads any part of the system. It happens
to also make the system easier to observe and debug, as we avoid running
large numbers of concurrent changes. Eventually we may relax these
limitations: there is no correctness problem with optimizing lots of
tenants concurrently, and optimizing multiple shards in one tenant just
requires housekeeping changes to update ShardContext with the result of
one optimization before proceeding to the next shard.
This commit is contained in:
John Spray
2024-03-28 18:48:52 +00:00
committed by GitHub
parent 090123a429
commit 63213fc814
6 changed files with 780 additions and 64 deletions

View File

@@ -37,6 +37,9 @@ pub(crate) struct StorageControllerMetricGroup {
pub(crate) storage_controller_reconcile_complete:
measured::CounterVec<ReconcileCompleteLabelGroupSet>,
/// Count of how many times we make an optimization change to a tenant's scheduling
pub(crate) storage_controller_schedule_optimization: measured::Counter,
/// HTTP request status counters for handled requests
pub(crate) storage_controller_http_request_status:
measured::CounterVec<HttpRequestStatusLabelGroupSet>,
@@ -101,6 +104,7 @@ impl StorageControllerMetricGroup {
status: StaticLabelSet::new(),
},
),
storage_controller_schedule_optimization: measured::Counter::new(),
storage_controller_http_request_status: measured::CounterVec::new(
HttpRequestStatusLabelGroupSet {
path: lasso::ThreadedRodeo::new(),

View File

@@ -487,6 +487,7 @@ impl Reconciler {
while let Err(e) = self.compute_notify().await {
match e {
NotifyError::Fatal(_) => return Err(ReconcileError::Notify(e)),
NotifyError::ShuttingDown => return Err(ReconcileError::Cancel),
_ => {
tracing::warn!(
"Live migration blocked by compute notification error, retrying: {e}"

View File

@@ -58,6 +58,70 @@ pub(crate) struct Scheduler {
nodes: HashMap<NodeId, SchedulerNode>,
}
/// Score for soft constraint scheduling: lower scores are preferred to higher scores.
///
/// For example, we may set an affinity score based on the number of shards from the same
/// tenant already on a node, to implicitly prefer to balance out shards.
#[derive(Copy, Clone, Debug, Eq, PartialEq, PartialOrd, Ord)]
pub(crate) struct AffinityScore(pub(crate) usize);
impl AffinityScore {
/// If we have no anti-affinity at all toward a node, this is its score. It means
/// the scheduler has a free choice amongst nodes with this score, and may pick a node
/// based on other information such as total utilization.
pub(crate) const FREE: Self = Self(0);
pub(crate) fn inc(&mut self) {
self.0 += 1;
}
}
impl std::ops::Add for AffinityScore {
type Output = Self;
fn add(self, rhs: Self) -> Self::Output {
Self(self.0 + rhs.0)
}
}
// For carrying state between multiple calls to [`TenantState::schedule`], e.g. when calling
// it for many shards in the same tenant.
#[derive(Debug, Default)]
pub(crate) struct ScheduleContext {
/// Sparse map of nodes: omitting a node implicitly makes its affinity [`AffinityScore::FREE`]
pub(crate) nodes: HashMap<NodeId, AffinityScore>,
/// Specifically how many _attached_ locations are on each node
pub(crate) attached_nodes: HashMap<NodeId, usize>,
}
impl ScheduleContext {
/// Input is a list of nodes we would like to avoid using again within this context. The more
/// times a node is passed into this call, the less inclined we are to use it.
pub(crate) fn avoid(&mut self, nodes: &[NodeId]) {
for node_id in nodes {
let entry = self.nodes.entry(*node_id).or_insert(AffinityScore::FREE);
entry.inc()
}
}
pub(crate) fn push_attached(&mut self, node_id: NodeId) {
let entry = self.attached_nodes.entry(node_id).or_default();
*entry += 1;
}
pub(crate) fn get_node_affinity(&self, node_id: NodeId) -> AffinityScore {
self.nodes
.get(&node_id)
.copied()
.unwrap_or(AffinityScore::FREE)
}
pub(crate) fn get_node_attachments(&self, node_id: NodeId) -> usize {
self.attached_nodes.get(&node_id).copied().unwrap_or(0)
}
}
impl Scheduler {
pub(crate) fn new<'a>(nodes: impl Iterator<Item = &'a Node>) -> Self {
let mut scheduler_nodes = HashMap::new();
@@ -224,27 +288,47 @@ impl Scheduler {
node.and_then(|(node_id, may_schedule)| if may_schedule { Some(node_id) } else { None })
}
pub(crate) fn schedule_shard(&self, hard_exclude: &[NodeId]) -> Result<NodeId, ScheduleError> {
/// hard_exclude: it is forbidden to use nodes in this list, typically becacuse they
/// are already in use by this shard -- we use this to avoid picking the same node
/// as both attached and secondary location. This is a hard constraint: if we cannot
/// find any nodes that aren't in this list, then we will return a [`ScheduleError::ImpossibleConstraint`].
///
/// context: we prefer to avoid using nodes identified in the context, according
/// to their anti-affinity score. We use this to prefeer to avoid placing shards in
/// the same tenant on the same node. This is a soft constraint: the context will never
/// cause us to fail to schedule a shard.
pub(crate) fn schedule_shard(
&self,
hard_exclude: &[NodeId],
context: &ScheduleContext,
) -> Result<NodeId, ScheduleError> {
if self.nodes.is_empty() {
return Err(ScheduleError::NoPageservers);
}
let mut tenant_counts: Vec<(NodeId, usize)> = self
let mut scores: Vec<(NodeId, AffinityScore, usize)> = self
.nodes
.iter()
.filter_map(|(k, v)| {
if hard_exclude.contains(k) || v.may_schedule == MaySchedule::No {
None
} else {
Some((*k, v.shard_count))
Some((
*k,
context.nodes.get(k).copied().unwrap_or(AffinityScore::FREE),
v.shard_count,
))
}
})
.collect();
// Sort by tenant count. Nodes with the same tenant count are sorted by ID.
tenant_counts.sort_by_key(|i| (i.1, i.0));
// Sort by, in order of precedence:
// 1st: Affinity score. We should never pick a higher-score node if a lower-score node is available
// 2nd: Utilization. Within nodes with the same affinity, use the least loaded nodes.
// 3rd: Node ID. This is a convenience to make selection deterministic in tests and empty systems.
scores.sort_by_key(|i| (i.1, i.2, i.0));
if tenant_counts.is_empty() {
if scores.is_empty() {
// After applying constraints, no pageservers were left. We log some detail about
// the state of nodes to help understand why this happened. This is not logged as an error because
// it is legitimately possible for enough nodes to be Offline to prevent scheduling a shard.
@@ -260,10 +344,11 @@ impl Scheduler {
return Err(ScheduleError::ImpossibleConstraint);
}
let node_id = tenant_counts.first().unwrap().0;
// Lowest score wins
let node_id = scores.first().unwrap().0;
tracing::info!(
"scheduler selected node {node_id} (elegible nodes {:?}, exclude: {hard_exclude:?})",
tenant_counts.iter().map(|i| i.0 .0).collect::<Vec<_>>()
"scheduler selected node {node_id} (elegible nodes {:?}, hard exclude: {hard_exclude:?}, soft exclude: {context:?})",
scores.iter().map(|i| i.0 .0).collect::<Vec<_>>()
);
// Note that we do not update shard count here to reflect the scheduling: that
@@ -271,6 +356,12 @@ impl Scheduler {
Ok(node_id)
}
/// Unit test access to internal state
#[cfg(test)]
pub(crate) fn get_node_shard_count(&self, node_id: NodeId) -> usize {
self.nodes.get(&node_id).unwrap().shard_count
}
}
#[cfg(test)]
@@ -316,15 +407,17 @@ mod tests {
let mut t1_intent = IntentState::new();
let mut t2_intent = IntentState::new();
let scheduled = scheduler.schedule_shard(&[])?;
let context = ScheduleContext::default();
let scheduled = scheduler.schedule_shard(&[], &context)?;
t1_intent.set_attached(&mut scheduler, Some(scheduled));
let scheduled = scheduler.schedule_shard(&[])?;
let scheduled = scheduler.schedule_shard(&[], &context)?;
t2_intent.set_attached(&mut scheduler, Some(scheduled));
assert_eq!(scheduler.nodes.get(&NodeId(1)).unwrap().shard_count, 1);
assert_eq!(scheduler.nodes.get(&NodeId(2)).unwrap().shard_count, 1);
let scheduled = scheduler.schedule_shard(&t1_intent.all_pageservers())?;
let scheduled = scheduler.schedule_shard(&t1_intent.all_pageservers(), &context)?;
t1_intent.push_secondary(&mut scheduler, scheduled);
assert_eq!(scheduler.nodes.get(&NodeId(1)).unwrap().shard_count, 1);

View File

@@ -11,6 +11,7 @@ use crate::{
id_lock_map::IdLockMap,
persistence::{AbortShardSplitStatus, TenantFilter},
reconciler::ReconcileError,
scheduler::ScheduleContext,
};
use anyhow::Context;
use control_plane::storage_controller::{
@@ -345,9 +346,15 @@ impl Service {
}
// Populate each tenant's intent state
let mut schedule_context = ScheduleContext::default();
for (tenant_shard_id, tenant_state) in tenants.iter_mut() {
if tenant_shard_id.shard_number == ShardNumber(0) {
// Reset scheduling context each time we advance to the next Tenant
schedule_context = ScheduleContext::default();
}
tenant_state.intent_from_observed(scheduler);
if let Err(e) = tenant_state.schedule(scheduler) {
if let Err(e) = tenant_state.schedule(scheduler, &mut schedule_context) {
// Non-fatal error: we are unable to properly schedule the tenant, perhaps because
// not enough pageservers are available. The tenant may well still be available
// to clients.
@@ -671,7 +678,13 @@ impl Service {
let mut interval = tokio::time::interval(BACKGROUND_RECONCILE_PERIOD);
while !self.cancel.is_cancelled() {
tokio::select! {
_ = interval.tick() => { self.reconcile_all(); }
_ = interval.tick() => {
let reconciles_spawned = self.reconcile_all();
if reconciles_spawned == 0 {
// Run optimizer only when we didn't find any other work to do
self.optimize_all();
}
}
_ = self.cancel.cancelled() => return
}
}
@@ -1627,6 +1640,8 @@ impl Service {
Err(e) => return Err(ApiError::InternalServerError(anyhow::anyhow!(e))),
};
let mut schedule_context = ScheduleContext::default();
let (waiters, response_shards) = {
let mut locked = self.inner.write().unwrap();
let (nodes, tenants, scheduler) = locked.parts_mut();
@@ -1648,11 +1663,14 @@ impl Service {
// attached and secondary locations (independently) away frorm those
// pageservers also holding a shard for this tenant.
entry.get_mut().schedule(scheduler).map_err(|e| {
ApiError::Conflict(format!(
"Failed to schedule shard {tenant_shard_id}: {e}"
))
})?;
entry
.get_mut()
.schedule(scheduler, &mut schedule_context)
.map_err(|e| {
ApiError::Conflict(format!(
"Failed to schedule shard {tenant_shard_id}: {e}"
))
})?;
if let Some(node_id) = entry.get().intent.get_attached() {
let generation = entry
@@ -1680,7 +1698,7 @@ impl Service {
state.generation = initial_generation;
state.config = create_req.config.clone();
if let Err(e) = state.schedule(scheduler) {
if let Err(e) = state.schedule(scheduler, &mut schedule_context) {
schcedule_error = Some(e);
}
@@ -1888,6 +1906,7 @@ impl Service {
// Persist updates
// Ordering: write to the database before applying changes in-memory, so that
// we will not appear time-travel backwards on a restart.
let mut schedule_context = ScheduleContext::default();
for ShardUpdate {
tenant_shard_id,
placement_policy,
@@ -1935,7 +1954,7 @@ impl Service {
shard.generation = Some(generation);
}
shard.schedule(scheduler)?;
shard.schedule(scheduler, &mut schedule_context)?;
let maybe_waiter = self.maybe_reconcile_shard(shard, nodes);
if let Some(waiter) = maybe_waiter {
@@ -2095,7 +2114,7 @@ impl Service {
let scheduler = &locked.scheduler;
// Right now we only perform the operation on a single node without parallelization
// TODO fan out the operation to multiple nodes for better performance
let node_id = scheduler.schedule_shard(&[])?;
let node_id = scheduler.schedule_shard(&[], &ScheduleContext::default())?;
let node = locked
.nodes
.get(&node_id)
@@ -2364,6 +2383,7 @@ impl Service {
)
.await?;
let mut schedule_context = ScheduleContext::default();
let mut locked = self.inner.write().unwrap();
let (nodes, tenants, scheduler) = locked.parts_mut();
for (shard_id, shard) in tenants.range_mut(TenantShardId::tenant_range(tenant_id)) {
@@ -2382,7 +2402,7 @@ impl Service {
}
// In case scheduling is being switched back on, try it now.
shard.schedule(scheduler).ok();
shard.schedule(scheduler, &mut schedule_context).ok();
self.maybe_reconcile_shard(shard, nodes);
}
@@ -2846,7 +2866,7 @@ impl Service {
tracing::info!("Restoring parent shard {tenant_shard_id}");
shard.splitting = SplitState::Idle;
if let Err(e) = shard.schedule(scheduler) {
if let Err(e) = shard.schedule(scheduler, &mut ScheduleContext::default()) {
// If this shard can't be scheduled now (perhaps due to offline nodes or
// capacity issues), that must not prevent us rolling back a split. In this
// case it should be eventually scheduled in the background.
@@ -2970,6 +2990,7 @@ impl Service {
)
};
let mut schedule_context = ScheduleContext::default();
for child in child_ids {
let mut child_shard = parent_ident;
child_shard.number = child.shard_number;
@@ -3005,7 +3026,7 @@ impl Service {
child_locations.push((child, pageserver, child_shard.stripe_size));
if let Err(e) = child_state.schedule(scheduler) {
if let Err(e) = child_state.schedule(scheduler, &mut schedule_context) {
// This is not fatal, because we've implicitly already got an attached
// location for the child shard. Failure here just means we couldn't
// find a secondary (e.g. because cluster is overloaded).
@@ -3869,6 +3890,7 @@ impl Service {
AvailabilityTransition::ToOffline => {
tracing::info!("Node {} transition to offline", node_id);
let mut tenants_affected: usize = 0;
for (tenant_shard_id, tenant_state) in tenants {
if let Some(observed_loc) = tenant_state.observed.locations.get_mut(&node_id) {
// When a node goes offline, we set its observed configuration to None, indicating unknown: we will
@@ -3885,7 +3907,13 @@ impl Service {
if tenant_state.intent.demote_attached(node_id) {
tenant_state.sequence = tenant_state.sequence.next();
match tenant_state.schedule(scheduler) {
// TODO: populate a ScheduleContext including all shards in the same tenant_id (only matters
// for tenants without secondary locations: if they have a secondary location, then this
// schedule() call is just promoting an existing secondary)
let mut schedule_context = ScheduleContext::default();
match tenant_state.schedule(scheduler, &mut schedule_context) {
Err(e) => {
// It is possible that some tenants will become unschedulable when too many pageservers
// go offline: in this case there isn't much we can do other than make the issue observable.
@@ -3944,8 +3972,9 @@ impl Service {
let mut waiters = Vec::new();
let (nodes, tenants, scheduler) = locked.parts_mut();
let mut schedule_context = ScheduleContext::default();
for (tenant_shard_id, shard) in tenants.range_mut(TenantShardId::tenant_range(tenant_id)) {
shard.schedule(scheduler)?;
shard.schedule(scheduler, &mut schedule_context)?;
// The shard's policies may not result in an attached location being scheduled: this
// is an error because our caller needs it attached somewhere.
@@ -4025,8 +4054,144 @@ impl Service {
let (nodes, tenants, _scheduler) = locked.parts_mut();
let pageservers = nodes.clone();
let mut schedule_context = ScheduleContext::default();
let mut reconciles_spawned = 0;
for (_tenant_shard_id, shard) in tenants.iter_mut() {
for (tenant_shard_id, shard) in tenants.iter_mut() {
if tenant_shard_id.is_zero() {
schedule_context = ScheduleContext::default();
}
// Eventual consistency: if an earlier reconcile job failed, and the shard is still
// dirty, spawn another rone
if self.maybe_reconcile_shard(shard, &pageservers).is_some() {
reconciles_spawned += 1;
}
schedule_context.avoid(&shard.intent.all_pageservers());
}
reconciles_spawned
}
/// `optimize` in this context means identifying shards which have valid scheduled locations, but
/// could be scheduled somewhere better:
/// - Cutting over to a secondary if the node with the secondary is more lightly loaded
/// * e.g. after a node fails then recovers, to move some work back to it
/// - Cutting over to a secondary if it improves the spread of shard attachments within a tenant
/// * e.g. after a shard split, the initial attached locations will all be on the node where
/// we did the split, but are probably better placed elsewhere.
/// - Creating new secondary locations if it improves the spreading of a sharded tenant
/// * e.g. after a shard split, some locations will be on the same node (where the split
/// happened), and will probably be better placed elsewhere.
///
/// To put it more briefly: whereas the scheduler respects soft constraints in a ScheduleContext at
/// the time of scheduling, this function looks for cases where a better-scoring location is available
/// according to those same soft constraints.
fn optimize_all(&self) -> usize {
let mut locked = self.inner.write().unwrap();
let (nodes, tenants, scheduler) = locked.parts_mut();
let pageservers = nodes.clone();
let mut schedule_context = ScheduleContext::default();
let mut reconciles_spawned = 0;
let mut tenant_shards: Vec<&TenantState> = Vec::new();
// Limit on how many shards' optmizations each call to this function will execute. Combined
// with the frequency of background calls, this acts as an implicit rate limit that runs a small
// trickle of optimizations in the background, rather than executing a large number in parallel
// when a change occurs.
const MAX_OPTIMIZATIONS_PER_PASS: usize = 2;
let mut work = Vec::new();
for (tenant_shard_id, shard) in tenants.iter() {
if tenant_shard_id.is_zero() {
// Reset accumulators on the first shard in a tenant
schedule_context = ScheduleContext::default();
tenant_shards.clear();
}
if work.len() >= MAX_OPTIMIZATIONS_PER_PASS {
break;
}
match shard.get_scheduling_policy() {
ShardSchedulingPolicy::Active => {
// Ok to do optimization
}
ShardSchedulingPolicy::Essential
| ShardSchedulingPolicy::Pause
| ShardSchedulingPolicy::Stop => {
// Policy prevents optimizing this shard.
continue;
}
}
// Accumulate the schedule context for all the shards in a tenant: we must have
// the total view of all shards before we can try to optimize any of them.
schedule_context.avoid(&shard.intent.all_pageservers());
if let Some(attached) = shard.intent.get_attached() {
schedule_context.push_attached(*attached);
}
tenant_shards.push(shard);
// Once we have seen the last shard in the tenant, proceed to search across all shards
// in the tenant for optimizations
if shard.shard.number.0 == shard.shard.count.count() - 1 {
if tenant_shards.iter().any(|s| s.reconciler.is_some()) {
// Do not start any optimizations while another change to the tenant is ongoing: this
// is not necessary for correctness, but simplifies operations and implicitly throttles
// optimization changes to happen in a "trickle" over time.
continue;
}
if tenant_shards.iter().any(|s| {
!matches!(s.splitting, SplitState::Idle)
|| matches!(s.policy, PlacementPolicy::Detached)
}) {
// Never attempt to optimize a tenant that is currently being split, or
// a tenant that is meant to be detached
continue;
}
// TODO: optimization calculations are relatively expensive: create some fast-path for
// the common idle case (avoiding the search on tenants that we have recently checked)
for shard in &tenant_shards {
if let Some(optimization) =
// If idle, maybe ptimize attachments: if a shard has a secondary location that is preferable to
// its primary location based on soft constraints, cut it over.
shard.optimize_attachment(nodes, &schedule_context)
{
work.push((shard.tenant_shard_id, optimization));
break;
} else if let Some(optimization) =
// If idle, maybe optimize secondary locations: if a shard has a secondary location that would be
// better placed on another node, based on ScheduleContext, then adjust it. This
// covers cases like after a shard split, where we might have too many shards
// in the same tenant with secondary locations on the node where they originally split.
shard.optimize_secondary(scheduler, &schedule_context)
{
work.push((shard.tenant_shard_id, optimization));
break;
}
// TODO: extend this mechanism to prefer attaching on nodes with fewer attached
// tenants (i.e. extend schedule state to distinguish attached from secondary counts),
// for the total number of attachments on a node (not just within a tenant.)
}
}
}
for (tenant_shard_id, optimization) in work {
let shard = tenants
.get_mut(&tenant_shard_id)
.expect("We held lock from place we got this ID");
shard.apply_optimization(scheduler, optimization);
if self.maybe_reconcile_shard(shard, &pageservers).is_some() {
reconciles_spawned += 1;
}
@@ -4039,7 +4204,11 @@ impl Service {
/// also wait for any generated Reconcilers to complete. Calling this until it returns zero should
/// put the system into a quiescent state where future background reconciliations won't do anything.
pub(crate) async fn reconcile_all_now(&self) -> Result<usize, ReconcileWaitError> {
self.reconcile_all();
let reconciles_spawned = self.reconcile_all();
if reconciles_spawned == 0 {
// Only optimize when we are otherwise idle
self.optimize_all();
}
let waiters = {
let mut waiters = Vec::new();

View File

@@ -7,6 +7,7 @@ use std::{
use crate::{
metrics::{self, ReconcileCompleteLabelGroup, ReconcileOutcome},
persistence::TenantShardPersistence,
scheduler::{AffinityScore, MaySchedule, ScheduleContext},
};
use pageserver_api::controller_api::{PlacementPolicy, ShardSchedulingPolicy};
use pageserver_api::{
@@ -250,8 +251,13 @@ impl IntentState {
impl Drop for IntentState {
fn drop(&mut self) {
// Must clear before dropping, to avoid leaving stale refcounts in the Scheduler
debug_assert!(self.attached.is_none() && self.secondary.is_empty());
// Must clear before dropping, to avoid leaving stale refcounts in the Scheduler.
// We do not check this while panicking, to avoid polluting unit test failures or
// other assertions with this assertion's output. It's still wrong to leak these,
// but if we already have a panic then we don't need to independently flag this case.
if !(std::thread::panicking()) {
debug_assert!(self.attached.is_none() && self.secondary.is_empty());
}
}
}
@@ -296,6 +302,26 @@ pub enum ReconcileWaitError {
Failed(TenantShardId, String),
}
#[derive(Eq, PartialEq, Debug)]
pub(crate) struct ReplaceSecondary {
old_node_id: NodeId,
new_node_id: NodeId,
}
#[derive(Eq, PartialEq, Debug)]
pub(crate) struct MigrateAttachment {
old_attached_node_id: NodeId,
new_attached_node_id: NodeId,
}
#[derive(Eq, PartialEq, Debug)]
pub(crate) enum ScheduleOptimization {
// Replace one of our secondary locations with a different node
ReplaceSecondary(ReplaceSecondary),
// Migrate attachment to an existing secondary location
MigrateAttachment(MigrateAttachment),
}
impl ReconcilerWaiter {
pub(crate) async fn wait_timeout(&self, timeout: Duration) -> Result<(), ReconcileWaitError> {
tokio::select! {
@@ -430,6 +456,7 @@ impl TenantState {
fn schedule_attached(
&mut self,
scheduler: &mut Scheduler,
context: &ScheduleContext,
) -> Result<(bool, NodeId), ScheduleError> {
// No work to do if we already have an attached tenant
if let Some(node_id) = self.intent.attached {
@@ -443,14 +470,33 @@ impl TenantState {
Ok((true, promote_secondary))
} else {
// Pick a fresh node: either we had no secondaries or none were schedulable
let node_id = scheduler.schedule_shard(&self.intent.secondary)?;
let node_id = scheduler.schedule_shard(&self.intent.secondary, context)?;
tracing::debug!("Selected {} as attached", node_id);
self.intent.set_attached(scheduler, Some(node_id));
Ok((true, node_id))
}
}
pub(crate) fn schedule(&mut self, scheduler: &mut Scheduler) -> Result<(), ScheduleError> {
pub(crate) fn schedule(
&mut self,
scheduler: &mut Scheduler,
context: &mut ScheduleContext,
) -> Result<(), ScheduleError> {
let r = self.do_schedule(scheduler, context);
context.avoid(&self.intent.all_pageservers());
if let Some(attached) = self.intent.get_attached() {
context.push_attached(*attached);
}
r
}
pub(crate) fn do_schedule(
&mut self,
scheduler: &mut Scheduler,
context: &ScheduleContext,
) -> Result<(), ScheduleError> {
// TODO: before scheduling new nodes, check if any existing content in
// self.intent refers to pageservers that are offline, and pick other
// pageservers if so.
@@ -494,12 +540,13 @@ impl TenantState {
}
// Should have exactly one attached, and N secondaries
let (modified_attached, attached_node_id) = self.schedule_attached(scheduler)?;
let (modified_attached, attached_node_id) =
self.schedule_attached(scheduler, context)?;
modified |= modified_attached;
let mut used_pageservers = vec![attached_node_id];
while self.intent.secondary.len() < secondary_count {
let node_id = scheduler.schedule_shard(&used_pageservers)?;
let node_id = scheduler.schedule_shard(&used_pageservers, context)?;
self.intent.push_secondary(scheduler, node_id);
used_pageservers.push(node_id);
modified = true;
@@ -512,7 +559,7 @@ impl TenantState {
modified = true;
} else if self.intent.secondary.is_empty() {
// Populate secondary by scheduling a fresh node
let node_id = scheduler.schedule_shard(&[])?;
let node_id = scheduler.schedule_shard(&[], context)?;
self.intent.push_secondary(scheduler, node_id);
modified = true;
}
@@ -539,6 +586,167 @@ impl TenantState {
Ok(())
}
/// Optimize attachments: if a shard has a secondary location that is preferable to
/// its primary location based on soft constraints, switch that secondary location
/// to be attached.
#[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()))]
pub(crate) fn optimize_attachment(
&self,
nodes: &HashMap<NodeId, Node>,
schedule_context: &ScheduleContext,
) -> Option<ScheduleOptimization> {
let attached = (*self.intent.get_attached())?;
if self.intent.secondary.is_empty() {
// We can only do useful work if we have both attached and secondary locations: this
// function doesn't schedule new locations, only swaps between attached and secondaries.
return None;
}
let current_affinity_score = schedule_context.get_node_affinity(attached);
let current_attachment_count = schedule_context.get_node_attachments(attached);
// Generate score for each node, dropping any un-schedulable nodes.
let all_pageservers = self.intent.all_pageservers();
let mut scores = all_pageservers
.iter()
.flat_map(|node_id| {
if matches!(
nodes
.get(node_id)
.map(|n| n.may_schedule())
.unwrap_or(MaySchedule::No),
MaySchedule::No
) {
None
} else {
let affinity_score = schedule_context.get_node_affinity(*node_id);
let attachment_count = schedule_context.get_node_attachments(*node_id);
Some((*node_id, affinity_score, attachment_count))
}
})
.collect::<Vec<_>>();
// Sort precedence:
// 1st - prefer nodes with the lowest total affinity score
// 2nd - prefer nodes with the lowest number of attachments in this context
// 3rd - if all else is equal, sort by node ID for determinism in tests.
scores.sort_by_key(|i| (i.1, i.2, i.0));
if let Some((preferred_node, preferred_affinity_score, preferred_attachment_count)) =
scores.first()
{
if attached != *preferred_node {
// The best alternative must be more than 1 better than us, otherwise we could end
// up flapping back next time we're called (e.g. there's no point migrating from
// a location with score 1 to a score zero, because on next location the situation
// would be the same, but in reverse).
if current_affinity_score > *preferred_affinity_score + AffinityScore(1)
|| current_attachment_count > *preferred_attachment_count + 1
{
tracing::info!(
"Identified optimization: migrate attachment {attached}->{preferred_node} (secondaries {:?})",
self.intent.get_secondary()
);
return Some(ScheduleOptimization::MigrateAttachment(MigrateAttachment {
old_attached_node_id: attached,
new_attached_node_id: *preferred_node,
}));
}
} else {
tracing::debug!(
"Node {} is already preferred (score {:?})",
preferred_node,
preferred_affinity_score
);
}
}
// Fall-through: we didn't find an optimization
None
}
#[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()))]
pub(crate) fn optimize_secondary(
&self,
scheduler: &Scheduler,
schedule_context: &ScheduleContext,
) -> Option<ScheduleOptimization> {
if self.intent.secondary.is_empty() {
// We can only do useful work if we have both attached and secondary locations: this
// function doesn't schedule new locations, only swaps between attached and secondaries.
return None;
}
for secondary in self.intent.get_secondary() {
let Some(affinity_score) = schedule_context.nodes.get(secondary) else {
// We're already on a node unaffected any affinity constraints,
// so we won't change it.
continue;
};
// Let the scheduler suggest a node, where it would put us if we were scheduling afresh
// This implicitly limits the choice to nodes that are available, and prefers nodes
// with lower utilization.
let Ok(candidate_node) =
scheduler.schedule_shard(&self.intent.all_pageservers(), schedule_context)
else {
// A scheduling error means we have no possible candidate replacements
continue;
};
let candidate_affinity_score = schedule_context
.nodes
.get(&candidate_node)
.unwrap_or(&AffinityScore::FREE);
// The best alternative must be more than 1 better than us, otherwise we could end
// up flapping back next time we're called.
if *candidate_affinity_score + AffinityScore(1) < *affinity_score {
// If some other node is available and has a lower score than this node, then
// that other node is a good place to migrate to.
tracing::info!(
"Identified optimization: replace secondary {secondary}->{candidate_node} (current secondaries {:?})",
self.intent.get_secondary()
);
return Some(ScheduleOptimization::ReplaceSecondary(ReplaceSecondary {
old_node_id: *secondary,
new_node_id: candidate_node,
}));
}
}
None
}
pub(crate) fn apply_optimization(
&mut self,
scheduler: &mut Scheduler,
optimization: ScheduleOptimization,
) {
metrics::METRICS_REGISTRY
.metrics_group
.storage_controller_schedule_optimization
.inc();
match optimization {
ScheduleOptimization::MigrateAttachment(MigrateAttachment {
old_attached_node_id,
new_attached_node_id,
}) => {
self.intent.demote_attached(old_attached_node_id);
self.intent
.promote_attached(scheduler, new_attached_node_id);
}
ScheduleOptimization::ReplaceSecondary(ReplaceSecondary {
old_node_id,
new_node_id,
}) => {
self.intent.remove_secondary(scheduler, old_node_id);
self.intent.push_secondary(scheduler, new_node_id);
}
}
}
/// Query whether the tenant's observed state for attached node matches its intent state, and if so,
/// yield the node ID. This is appropriate for emitting compute hook notifications: we are checking that
/// the node in question is not only where we intend to attach, but that the tenant is indeed already attached there.
@@ -877,6 +1085,10 @@ impl TenantState {
self.scheduling_policy = p;
}
pub(crate) fn get_scheduling_policy(&self) -> &ShardSchedulingPolicy {
&self.scheduling_policy
}
pub(crate) fn from_persistent(
tsp: TenantShardPersistence,
intent: IntentState,
@@ -953,6 +1165,32 @@ pub(crate) mod tests {
)
}
fn make_test_tenant(policy: PlacementPolicy, shard_count: ShardCount) -> Vec<TenantState> {
let tenant_id = TenantId::generate();
(0..shard_count.count())
.map(|i| {
let shard_number = ShardNumber(i);
let tenant_shard_id = TenantShardId {
tenant_id,
shard_number,
shard_count,
};
TenantState::new(
tenant_shard_id,
ShardIdentity::new(
shard_number,
shard_count,
pageserver_api::shard::ShardStripeSize(32768),
)
.unwrap(),
policy.clone(),
)
})
.collect()
}
/// Test the scheduling behaviors used when a tenant configured for HA is subject
/// to nodes being marked offline.
#[test]
@@ -962,10 +1200,11 @@ pub(crate) mod tests {
let mut nodes = make_test_nodes(3);
let mut scheduler = Scheduler::new(nodes.values());
let mut context = ScheduleContext::default();
let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Attached(1));
tenant_state
.schedule(&mut scheduler)
.schedule(&mut scheduler, &mut context)
.expect("we have enough nodes, scheduling should work");
// Expect to initially be schedule on to different nodes
@@ -991,7 +1230,7 @@ pub(crate) mod tests {
// Scheduling the node should promote the still-available secondary node to attached
tenant_state
.schedule(&mut scheduler)
.schedule(&mut scheduler, &mut context)
.expect("active nodes are available");
assert_eq!(tenant_state.intent.attached.unwrap(), secondary_node_id);
@@ -1065,15 +1304,209 @@ pub(crate) mod tests {
// In pause mode, schedule() shouldn't do anything
tenant_state.scheduling_policy = ShardSchedulingPolicy::Pause;
assert!(tenant_state.schedule(&mut scheduler).is_ok());
assert!(tenant_state
.schedule(&mut scheduler, &mut ScheduleContext::default())
.is_ok());
assert!(tenant_state.intent.all_pageservers().is_empty());
// In active mode, schedule() works
tenant_state.scheduling_policy = ShardSchedulingPolicy::Active;
assert!(tenant_state.schedule(&mut scheduler).is_ok());
assert!(tenant_state
.schedule(&mut scheduler, &mut ScheduleContext::default())
.is_ok());
assert!(!tenant_state.intent.all_pageservers().is_empty());
tenant_state.intent.clear(&mut scheduler);
Ok(())
}
#[test]
fn optimize_attachment() -> anyhow::Result<()> {
let nodes = make_test_nodes(3);
let mut scheduler = Scheduler::new(nodes.values());
let mut shard_a = make_test_tenant_shard(PlacementPolicy::Attached(1));
let mut shard_b = make_test_tenant_shard(PlacementPolicy::Attached(1));
// Initially: both nodes attached on shard 1, and both have secondary locations
// on different nodes.
shard_a.intent.set_attached(&mut scheduler, Some(NodeId(1)));
shard_a.intent.push_secondary(&mut scheduler, NodeId(2));
shard_b.intent.set_attached(&mut scheduler, Some(NodeId(1)));
shard_b.intent.push_secondary(&mut scheduler, NodeId(3));
let mut schedule_context = ScheduleContext::default();
schedule_context.avoid(&shard_a.intent.all_pageservers());
schedule_context.push_attached(shard_a.intent.get_attached().unwrap());
schedule_context.avoid(&shard_b.intent.all_pageservers());
schedule_context.push_attached(shard_b.intent.get_attached().unwrap());
let optimization_a = shard_a.optimize_attachment(&nodes, &schedule_context);
// Either shard should recognize that it has the option to switch to a secondary location where there
// would be no other shards from the same tenant, and request to do so.
assert_eq!(
optimization_a,
Some(ScheduleOptimization::MigrateAttachment(MigrateAttachment {
old_attached_node_id: NodeId(1),
new_attached_node_id: NodeId(2)
}))
);
// Note that these optimizing two shards in the same tenant with the same ScheduleContext is
// mutually exclusive (the optimization of one invalidates the stats) -- it is the responsibility
// of [`Service::optimize_all`] to avoid trying
// to do optimizations for multiple shards in the same tenant at the same time. Generating
// both optimizations is just done for test purposes
let optimization_b = shard_b.optimize_attachment(&nodes, &schedule_context);
assert_eq!(
optimization_b,
Some(ScheduleOptimization::MigrateAttachment(MigrateAttachment {
old_attached_node_id: NodeId(1),
new_attached_node_id: NodeId(3)
}))
);
// Applying these optimizations should result in the end state proposed
shard_a.apply_optimization(&mut scheduler, optimization_a.unwrap());
assert_eq!(shard_a.intent.get_attached(), &Some(NodeId(2)));
assert_eq!(shard_a.intent.get_secondary(), &vec![NodeId(1)]);
shard_b.apply_optimization(&mut scheduler, optimization_b.unwrap());
assert_eq!(shard_b.intent.get_attached(), &Some(NodeId(3)));
assert_eq!(shard_b.intent.get_secondary(), &vec![NodeId(1)]);
shard_a.intent.clear(&mut scheduler);
shard_b.intent.clear(&mut scheduler);
Ok(())
}
#[test]
fn optimize_secondary() -> anyhow::Result<()> {
let nodes = make_test_nodes(4);
let mut scheduler = Scheduler::new(nodes.values());
let mut shard_a = make_test_tenant_shard(PlacementPolicy::Attached(1));
let mut shard_b = make_test_tenant_shard(PlacementPolicy::Attached(1));
// Initially: both nodes attached on shard 1, and both have secondary locations
// on different nodes.
shard_a.intent.set_attached(&mut scheduler, Some(NodeId(1)));
shard_a.intent.push_secondary(&mut scheduler, NodeId(3));
shard_b.intent.set_attached(&mut scheduler, Some(NodeId(2)));
shard_b.intent.push_secondary(&mut scheduler, NodeId(3));
let mut schedule_context = ScheduleContext::default();
schedule_context.avoid(&shard_a.intent.all_pageservers());
schedule_context.push_attached(shard_a.intent.get_attached().unwrap());
schedule_context.avoid(&shard_b.intent.all_pageservers());
schedule_context.push_attached(shard_b.intent.get_attached().unwrap());
let optimization_a = shard_a.optimize_secondary(&scheduler, &schedule_context);
// Since there is a node with no locations available, the node with two locations for the
// same tenant should generate an optimization to move one away
assert_eq!(
optimization_a,
Some(ScheduleOptimization::ReplaceSecondary(ReplaceSecondary {
old_node_id: NodeId(3),
new_node_id: NodeId(4)
}))
);
shard_a.apply_optimization(&mut scheduler, optimization_a.unwrap());
assert_eq!(shard_a.intent.get_attached(), &Some(NodeId(1)));
assert_eq!(shard_a.intent.get_secondary(), &vec![NodeId(4)]);
shard_a.intent.clear(&mut scheduler);
shard_b.intent.clear(&mut scheduler);
Ok(())
}
// Optimize til quiescent: this emulates what Service::optimize_all does, when
// called repeatedly in the background.
fn optimize_til_idle(
nodes: &HashMap<NodeId, Node>,
scheduler: &mut Scheduler,
shards: &mut [TenantState],
) {
let mut loop_n = 0;
loop {
let mut schedule_context = ScheduleContext::default();
let mut any_changed = false;
for shard in shards.iter() {
schedule_context.avoid(&shard.intent.all_pageservers());
if let Some(attached) = shard.intent.get_attached() {
schedule_context.push_attached(*attached);
}
}
for shard in shards.iter_mut() {
let optimization = shard.optimize_attachment(nodes, &schedule_context);
if let Some(optimization) = optimization {
shard.apply_optimization(scheduler, optimization);
any_changed = true;
break;
}
let optimization = shard.optimize_secondary(scheduler, &schedule_context);
if let Some(optimization) = optimization {
shard.apply_optimization(scheduler, optimization);
any_changed = true;
break;
}
}
if !any_changed {
break;
}
// Assert no infinite loop
loop_n += 1;
assert!(loop_n < 1000);
}
}
/// Test the balancing behavior of shard scheduling: that it achieves a balance, and
/// that it converges.
#[test]
fn optimize_add_nodes() -> anyhow::Result<()> {
let nodes = make_test_nodes(4);
// Only show the scheduler a couple of nodes
let mut scheduler = Scheduler::new([].iter());
scheduler.node_upsert(nodes.get(&NodeId(1)).unwrap());
scheduler.node_upsert(nodes.get(&NodeId(2)).unwrap());
let mut shards = make_test_tenant(PlacementPolicy::Attached(1), ShardCount::new(4));
let mut schedule_context = ScheduleContext::default();
for shard in &mut shards {
assert!(shard
.schedule(&mut scheduler, &mut schedule_context)
.is_ok());
}
// We should see equal number of locations on the two nodes.
assert_eq!(scheduler.get_node_shard_count(NodeId(1)), 4);
assert_eq!(scheduler.get_node_shard_count(NodeId(2)), 4);
// Add another two nodes: we should see the shards spread out when their optimize
// methods are called
scheduler.node_upsert(nodes.get(&NodeId(3)).unwrap());
scheduler.node_upsert(nodes.get(&NodeId(4)).unwrap());
optimize_til_idle(&nodes, &mut scheduler, &mut shards);
assert_eq!(scheduler.get_node_shard_count(NodeId(1)), 2);
assert_eq!(scheduler.get_node_shard_count(NodeId(2)), 2);
assert_eq!(scheduler.get_node_shard_count(NodeId(3)), 2);
assert_eq!(scheduler.get_node_shard_count(NodeId(4)), 2);
for shard in shards.iter_mut() {
shard.intent.clear(&mut scheduler);
}
Ok(())
}
}

View File

@@ -146,7 +146,7 @@ def test_sharding_split_smoke(
# 8 shards onto separate pageservers
shard_count = 4
split_shard_count = 8
neon_env_builder.num_pageservers = split_shard_count
neon_env_builder.num_pageservers = split_shard_count * 2
# 1MiB stripes: enable getting some meaningful data distribution without
# writing large quantities of data in this test. The stripe size is given
@@ -174,6 +174,7 @@ def test_sharding_split_smoke(
placement_policy='{"Attached": 1}',
conf=non_default_tenant_config,
)
workload = Workload(env, tenant_id, timeline_id, branch_name="main")
workload.init()
@@ -252,6 +253,10 @@ def test_sharding_split_smoke(
# The old parent shards should no longer exist on disk
assert not shards_on_disk(old_shard_ids)
# Enough background reconciliations should result in the shards being properly distributed.
# Run this before the workload, because its LSN-waiting code presumes stable locations.
env.storage_controller.reconcile_until_idle()
workload.validate()
workload.churn_rows(256)
@@ -265,27 +270,6 @@ def test_sharding_split_smoke(
pageserver.http_client().timeline_gc(tenant_shard_id, timeline_id, None)
workload.validate()
migrate_to_pageserver_ids = list(
set(p.id for p in env.pageservers) - set(pre_split_pageserver_ids)
)
assert len(migrate_to_pageserver_ids) == split_shard_count - shard_count
# Migrate shards away from the node where the split happened
for ps_id in pre_split_pageserver_ids:
shards_here = [
tenant_shard_id
for (tenant_shard_id, pageserver) in all_shards
if pageserver.id == ps_id
]
assert len(shards_here) == 2
migrate_shard = shards_here[0]
destination = migrate_to_pageserver_ids.pop()
log.info(f"Migrating shard {migrate_shard} from {ps_id} to {destination}")
env.storage_controller.tenant_shard_migrate(migrate_shard, destination)
workload.validate()
# Assert on how many reconciles happened during the process. This is something of an
# implementation detail, but it is useful to detect any bugs that might generate spurious
# extra reconcile iterations.
@@ -294,8 +278,9 @@ def test_sharding_split_smoke(
# - shard_count reconciles for the original setup of the tenant
# - shard_count reconciles for detaching the original secondary locations during split
# - split_shard_count reconciles during shard splitting, for setting up secondaries.
# - shard_count reconciles for the migrations we did to move child shards away from their split location
expect_reconciles = shard_count * 2 + split_shard_count + shard_count
# - shard_count of the child shards will need to fail over to their secondaries
# - shard_count of the child shard secondary locations will get moved to emptier nodes
expect_reconciles = shard_count * 2 + split_shard_count + shard_count * 2
reconcile_ok = env.storage_controller.get_metric_value(
"storage_controller_reconcile_complete_total", filter={"status": "ok"}
)
@@ -343,6 +328,31 @@ def test_sharding_split_smoke(
assert sum(total.values()) == split_shard_count * 2
check_effective_tenant_config()
# More specific check: that we are fully balanced. This is deterministic because
# the order in which we consider shards for optimization is deterministic, and the
# order of preference of nodes is also deterministic (lower node IDs win).
log.info(f"total: {total}")
assert total == {
1: 1,
2: 1,
3: 1,
4: 1,
5: 1,
6: 1,
7: 1,
8: 1,
9: 1,
10: 1,
11: 1,
12: 1,
13: 1,
14: 1,
15: 1,
16: 1,
}
log.info(f"attached: {attached}")
assert attached == {1: 1, 2: 1, 3: 1, 5: 1, 6: 1, 7: 1, 9: 1, 11: 1}
# Ensure post-split pageserver locations survive a restart (i.e. the child shards
# correctly wrote config to disk, and the storage controller responds correctly
# to /re-attach)
@@ -401,6 +411,7 @@ def test_sharding_split_stripe_size(
env.storage_controller.tenant_shard_split(
tenant_id, shard_count=2, shard_stripe_size=new_stripe_size
)
env.storage_controller.reconcile_until_idle()
# Check that we ended up with the stripe size that we expected, both on the pageserver
# and in the notifications to compute
@@ -869,6 +880,7 @@ def test_sharding_split_failures(
# Having failed+rolled back, we should be able to split again
# No failures this time; it will succeed
env.storage_controller.tenant_shard_split(tenant_id, shard_count=split_shard_count)
env.storage_controller.reconcile_until_idle(timeout_secs=30)
workload.churn_rows(10)
workload.validate()
@@ -922,6 +934,10 @@ def test_sharding_split_failures(
finish_split()
assert_split_done()
# Having completed the split, pump the background reconciles to ensure that
# the scheduler reaches an idle state
env.storage_controller.reconcile_until_idle(timeout_secs=30)
env.storage_controller.consistency_check()