diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index e627fc530e4..16c46138f42 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -6545,6 +6545,7 @@ version = "0.8.0" dependencies = [ "anyhow", "async-trait", + "base64 0.22.1", "bytesize", "fnv", "futures", @@ -6553,6 +6554,7 @@ dependencies = [ "mockall", "once_cell", "proptest", + "prost 0.14.1", "quickwit-actors", "quickwit-cluster", "quickwit-common", @@ -6565,6 +6567,7 @@ dependencies = [ "serde", "serde_json", "smallvec", + "time", "tokio", "tracing", "ulid", diff --git a/quickwit/quickwit-cli/src/cli.rs b/quickwit/quickwit-cli/src/cli.rs index 91bb338ae89..9532ad230bc 100644 --- a/quickwit/quickwit-cli/src/cli.rs +++ b/quickwit/quickwit-cli/src/cli.rs @@ -18,6 +18,7 @@ use quickwit_serve::EnvFilterReloadFn; use tracing::Level; use crate::index::{IndexCliCommand, build_index_command}; +use crate::maintenance::{MaintenanceCliCommand, build_maintenance_command}; use crate::service::{RunCliCommand, build_run_command}; use crate::source::{SourceCliCommand, build_source_command}; use crate::split::{SplitCliCommand, build_split_command}; @@ -47,6 +48,7 @@ pub fn build_cli() -> Command { .subcommand(build_source_command().display_order(3)) .subcommand(build_split_command().display_order(4)) .subcommand(build_tool_command().display_order(5)) + .subcommand(build_maintenance_command().display_order(6)) .arg_required_else_help(true) .disable_help_subcommand(true) .subcommand_required(true) @@ -59,6 +61,7 @@ pub enum CliCommand { Split(SplitCliCommand), Source(SourceCliCommand), Tool(ToolCliCommand), + Maintenance(MaintenanceCliCommand), } impl CliCommand { @@ -69,6 +72,7 @@ impl CliCommand { CliCommand::Source(_) => Level::ERROR, CliCommand::Split(_) => Level::ERROR, CliCommand::Tool(_) => Level::ERROR, + CliCommand::Maintenance(_) => Level::ERROR, } } @@ -82,6 +86,9 @@ impl CliCommand { "source" => SourceCliCommand::parse_cli_args(submatches).map(CliCommand::Source), "split" => SplitCliCommand::parse_cli_args(submatches).map(CliCommand::Split), "tool" => ToolCliCommand::parse_cli_args(submatches).map(CliCommand::Tool), + "maintenance" => { + MaintenanceCliCommand::parse_cli_args(submatches).map(CliCommand::Maintenance) + } _ => bail!("unknown command `{subcommand}`"), } } @@ -93,6 +100,7 @@ impl CliCommand { CliCommand::Source(subcommand) => subcommand.execute().await, CliCommand::Split(subcommand) => subcommand.execute().await, CliCommand::Tool(subcommand) => subcommand.execute().await, + CliCommand::Maintenance(subcommand) => subcommand.execute().await, } } } diff --git a/quickwit/quickwit-cli/src/lib.rs b/quickwit/quickwit-cli/src/lib.rs index aaeb4da7e9d..e05d5dc25aa 100644 --- a/quickwit/quickwit-cli/src/lib.rs +++ b/quickwit/quickwit-cli/src/lib.rs @@ -50,6 +50,7 @@ pub mod index; #[cfg(feature = "jemalloc")] pub mod jemalloc; pub mod logger; +pub mod maintenance; pub mod metrics; pub mod service; pub mod source; diff --git a/quickwit/quickwit-cli/src/maintenance.rs b/quickwit/quickwit-cli/src/maintenance.rs new file mode 100644 index 00000000000..d639d19b4d0 --- /dev/null +++ b/quickwit/quickwit-cli/src/maintenance.rs @@ -0,0 +1,149 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use anyhow::{Context, bail}; +use clap::{ArgMatches, Command}; +use colored::Colorize; +use tracing::debug; + +use crate::checklist::{GREEN_COLOR, RED_COLOR}; +use crate::{ClientArgs, client_args}; + +pub fn build_maintenance_command() -> Command { + Command::new("maintenance") + .about("Manages cluster maintenance mode for safe rolling upgrades.") + .args(client_args()) + .subcommand(Command::new("enable").about( + "Enables maintenance mode. Freezes the indexing plan; metadata mutations are accepted \ + but the plan is not rebuilt.", + )) + .subcommand( + Command::new("disable") + .about("Disables maintenance mode and triggers a full indexing plan rebuild."), + ) + .subcommand(Command::new("status").about("Shows the current maintenance mode status.")) + .subcommand_required(true) + .arg_required_else_help(true) +} + +#[derive(Debug, PartialEq)] +pub struct EnableMaintenanceArgs { + pub client_args: ClientArgs, +} + +#[derive(Debug, PartialEq)] +pub struct DisableMaintenanceArgs { + pub client_args: ClientArgs, +} + +#[derive(Debug, PartialEq)] +pub struct MaintenanceStatusArgs { + pub client_args: ClientArgs, +} + +#[derive(Debug, PartialEq)] +pub enum MaintenanceCliCommand { + Enable(EnableMaintenanceArgs), + Disable(DisableMaintenanceArgs), + Status(MaintenanceStatusArgs), +} + +impl MaintenanceCliCommand { + pub fn parse_cli_args(mut matches: ArgMatches) -> anyhow::Result { + let (subcommand, submatches) = matches + .remove_subcommand() + .context("failed to parse maintenance subcommand")?; + match subcommand.as_str() { + "enable" => Self::parse_enable_args(submatches), + "disable" => Self::parse_disable_args(submatches), + "status" => Self::parse_status_args(submatches), + _ => bail!("unknown maintenance subcommand `{subcommand}`"), + } + } + + fn parse_enable_args(mut matches: ArgMatches) -> anyhow::Result { + let client_args = ClientArgs::parse(&mut matches)?; + Ok(Self::Enable(EnableMaintenanceArgs { client_args })) + } + + fn parse_disable_args(mut matches: ArgMatches) -> anyhow::Result { + let client_args = ClientArgs::parse(&mut matches)?; + Ok(Self::Disable(DisableMaintenanceArgs { client_args })) + } + + fn parse_status_args(mut matches: ArgMatches) -> anyhow::Result { + let client_args = ClientArgs::parse(&mut matches)?; + Ok(Self::Status(MaintenanceStatusArgs { client_args })) + } + + pub fn default_log_level(&self) -> tracing::Level { + tracing::Level::ERROR + } + + pub async fn execute(self) -> anyhow::Result<()> { + match self { + Self::Enable(args) => enable_maintenance_cli(args).await, + Self::Disable(args) => disable_maintenance_cli(args).await, + Self::Status(args) => maintenance_status_cli(args).await, + } + } +} + +async fn enable_maintenance_cli(args: EnableMaintenanceArgs) -> anyhow::Result<()> { + debug!(args=?args, "enable-maintenance"); + println!("❯ Enabling maintenance mode..."); + let qw_client = args.client_args.client(); + let response = qw_client.maintenance().enable().await?; + println!( + "{} Maintenance mode enabled. Indexing plan frozen.", + "✔".color(GREEN_COLOR) + ); + debug!(frozen_plan_json_len = response.frozen_plan_json.len()); + Ok(()) +} + +async fn disable_maintenance_cli(args: DisableMaintenanceArgs) -> anyhow::Result<()> { + debug!(args=?args, "disable-maintenance"); + println!("❯ Disabling maintenance mode..."); + let qw_client = args.client_args.client(); + qw_client.maintenance().disable().await?; + println!( + "{} Maintenance mode disabled. Indexing plan rebuild triggered.", + "✔".color(GREEN_COLOR) + ); + Ok(()) +} + +async fn maintenance_status_cli(args: MaintenanceStatusArgs) -> anyhow::Result<()> { + debug!(args=?args, "maintenance-status"); + let qw_client = args.client_args.client(); + let status = qw_client.maintenance().status().await?; + if status.is_maintenance_mode { + println!( + "{} Maintenance mode is {}", + "●".color(RED_COLOR), + "ENABLED".color(RED_COLOR).bold() + ); + if let Some(enabled_at) = status.enabled_at { + println!(" Enabled at: {enabled_at}"); + } + } else { + println!( + "{} Maintenance mode is {}", + "●".color(GREEN_COLOR), + "DISABLED".color(GREEN_COLOR).bold() + ); + } + Ok(()) +} diff --git a/quickwit/quickwit-control-plane/Cargo.toml b/quickwit/quickwit-control-plane/Cargo.toml index 2957c9858c4..63cd6138af7 100644 --- a/quickwit/quickwit-control-plane/Cargo.toml +++ b/quickwit/quickwit-control-plane/Cargo.toml @@ -13,6 +13,7 @@ license.workspace = true [dependencies] anyhow = { workspace = true } async-trait = { workspace = true } +base64 = { workspace = true } bytesize = { workspace = true } fnv = { workspace = true } futures = { workspace = true } @@ -20,10 +21,12 @@ itertools = { workspace = true } lru = { workspace = true } mockall = { workspace = true, optional = true } once_cell = { workspace = true } +prost = { workspace = true } rand = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } smallvec = { workspace = true } +time = { workspace = true } tokio = { workspace = true } tracing = { workspace = true } ulid = { workspace = true } diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 58c1f64be0e..ca801f5a034 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -40,6 +40,8 @@ use quickwit_ingest::{IngesterPool, LocalShardsUpdate}; use quickwit_metastore::{CreateIndexRequestExt, CreateIndexResponseExt, IndexMetadataResponseExt}; use quickwit_proto::control_plane::{ AdviseResetShardsRequest, AdviseResetShardsResponse, ControlPlaneError, ControlPlaneResult, + DisableMaintenanceModeRequest, DisableMaintenanceModeResponse, EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, GetMaintenanceModeRequest, GetMaintenanceModeResponse, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, GetOrCreateOpenShardsSubrequest, SwapIndexingPipelinesRequest, SwapIndexingPipelinesResponse, }; @@ -63,6 +65,7 @@ use crate::debouncer::Debouncer; use crate::indexing_scheduler::{IndexingScheduler, IndexingSchedulerState}; use crate::ingest::IngestController; use crate::ingest::ingest_controller::{IngestControllerStats, RebalanceShardsCallback}; +use crate::maintenance::{MaintenanceState, MetastoreKvPersistence, serialize_frozen_plan}; use crate::model::ControlPlaneModel; /// Interval between two controls (or checks) of the desired plan VS running plan. @@ -103,6 +106,11 @@ pub struct ControlPlane { readiness_tx: watch::Sender, // Disables the control loop. This is useful for unit testing. disable_control_loop: bool, + /// Maintenance mode state. When active the indexing plan is frozen (not + /// rebuilt on topology changes). + maintenance: MaintenanceState, + /// Persistence backend for maintenance mode state (frozen plan + metadata). + maintenance_persistence: MetastoreKvPersistence, } impl fmt::Debug for ControlPlane { @@ -126,6 +134,7 @@ impl ControlPlane { watch::Receiver, ) { let disable_control_loop = false; + let maintenance_persistence = MetastoreKvPersistence::new(metastore.clone()); Self::spawn_inner( universe, cluster_config, @@ -135,6 +144,7 @@ impl ControlPlane { ingester_pool, metastore, disable_control_loop, + maintenance_persistence, ) } @@ -148,6 +158,7 @@ impl ControlPlane { ingester_pool: IngesterPool, metastore: MetastoreServiceClient, disable_control_loop: bool, + maintenance_persistence: MetastoreKvPersistence, ) -> ( Mailbox, ActorHandle>, @@ -187,6 +198,8 @@ impl ControlPlane { rebuild_plan_debouncer: Debouncer::new(REBUILD_PLAN_COOLDOWN_PERIOD), readiness_tx, disable_control_loop, + maintenance: MaintenanceState::default(), + maintenance_persistence: maintenance_persistence.clone(), } }); (control_plane_mailbox, control_plane_handle, readiness_rx) @@ -200,6 +213,7 @@ pub struct ControlPlaneObservableState { pub num_indexes: usize, pub num_sources: usize, pub readiness: bool, + pub maintenance_mode: bool, } #[async_trait] @@ -217,6 +231,7 @@ impl Actor for ControlPlane { num_indexes: self.model.num_indexes(), num_sources: self.model.num_sources(), readiness: *self.readiness_tx.borrow(), + maintenance_mode: self.maintenance.is_active(), } } @@ -228,7 +243,17 @@ impl Actor for ControlPlane { .await .context("failed to initialize control plane model")?; - let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + self.load_maintenance_state_from_persistence().await; + + if self.maintenance.is_active() { + // In maintenance mode: restore the frozen plan without triggering a rebuild. + info!( + enabled_at = self.maintenance.enabled_at().unwrap_or_default(), + "control plane starting in maintenance mode: indexing plan is frozen" + ); + } else { + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + } self.ingest_controller.sync_with_all_ingesters(&self.model); @@ -246,6 +271,37 @@ impl Actor for ControlPlane { } impl ControlPlane { + /// Loads maintenance state from the persistence backend. + /// Called during `initialize()`. + async fn load_maintenance_state_from_persistence(&mut self) { + match self.maintenance_persistence.load().await { + Some(persisted) => { + self.maintenance.load_from_metadata(persisted.metadata); + if self.maintenance.is_active() { + crate::metrics::CONTROL_PLANE_METRICS + .maintenance_mode + .set(1); + let num_indexers = persisted.frozen_plan.num_indexers(); + let num_pipelines: usize = persisted + .frozen_plan + .indexing_tasks_per_indexer() + .values() + .map(|tasks| tasks.len()) + .sum(); + info!( + num_indexers, + num_pipelines, "restored frozen indexing plan from persistence" + ); + self.indexing_scheduler + .load_frozen_plan(persisted.frozen_plan); + } + } + None => { + // No maintenance state persisted — normal operation. + } + } + } + async fn auto_create_indexes( &mut self, subrequests: &[GetOrCreateOpenShardsSubrequest], @@ -428,7 +484,8 @@ impl Handler for ControlPlane { _message: RebuildPlan, _ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - self.indexing_scheduler.rebuild_plan(&self.model); + self.indexing_scheduler + .rebuild_plan(&self.model, self.maintenance.is_active()); Ok(()) } } @@ -509,14 +566,21 @@ impl Handler for ControlPlane { if self.disable_control_loop { return Ok(()); } + let is_maintenance = self.maintenance.is_active(); if let Err(metastore_error) = self .ingest_controller - .rebalance_shards(&mut self.model, ctx.mailbox(), ctx.progress()) + .rebalance_shards( + &mut self.model, + ctx.mailbox(), + ctx.progress(), + is_maintenance, + ) .await { return convert_metastore_error::<()>(metastore_error).map(|_| ()); } - self.indexing_scheduler.control_running_plan(&self.model); + self.indexing_scheduler + .control_running_plan(&self.model, is_maintenance); ctx.schedule_self_msg(CONTROL_PLAN_LOOP_INTERVAL, ControlPlanLoop); Ok(()) } @@ -597,7 +661,8 @@ impl DeferableReplyHandler for ControlPlane { // Now, create index can also add sources to support creating indexes automatically from // index and source config templates. - let should_rebuild_plan = !index_metadata.sources.is_empty(); + let should_rebuild_plan = + !index_metadata.sources.is_empty() && !self.maintenance.is_active(); self.model.add_index(index_metadata); if should_rebuild_plan { @@ -647,6 +712,7 @@ impl Handler for ControlPlane { if self .model .update_index_config(&index_uid, index_metadata.index_config)? + && !self.maintenance.is_active() { let _rebuild_plan_notifier = self.rebuild_plan_debounced(ctx); } @@ -689,7 +755,9 @@ impl Handler for ControlPlane { // TODO: Refine the event. Notify index will have the effect to reload the entire state from // the metastore. We should update the state of the control plane. - let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + if !self.maintenance.is_active() { + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + } info!(%index_uid, "deleted index"); let response = EmptyResponse {}; @@ -731,7 +799,9 @@ impl Handler for ControlPlane { // TODO: Refine the event. Notify index will have the effect to reload the entire state from // the metastore. We should update the state of the control plane. - let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + if !self.maintenance.is_active() { + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + } let response = EmptyResponse {}; Ok(Ok(response)) @@ -771,7 +841,9 @@ impl Handler for ControlPlane { // TODO: Refine the event. Notify index will have the effect to reload the entire state from // the metastore. We should update the state of the control plane. - let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + if !self.maintenance.is_active() { + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + } info!(%index_uid, source_id, "updated source"); let response = EmptyResponse {}; @@ -807,7 +879,7 @@ impl Handler for ControlPlane { .toggle_source(&index_uid, &source_id, enable) .context("failed to toggle source")?; - if mutation_occurred { + if mutation_occurred && !self.maintenance.is_active() { let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); } info!(%index_uid, source_id, enabled=enable, "toggled source"); @@ -862,7 +934,9 @@ impl Handler for ControlPlane { .sync_with_ingesters(&ingesters_needing_resync, &self.model); self.model.delete_source(&source_uid); - let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + if !self.maintenance.is_active() { + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + } info!( index_uid=%source_uid.index_uid, @@ -917,9 +991,12 @@ impl Handler for ControlPlane { request: GetOrCreateOpenShardsRequest, ctx: &ActorContext, ) -> Result { - if let Err(metastore_error) = self - .auto_create_indexes(&request.subrequests, ctx.progress()) - .await + // In maintenance mode, block auto-create indexes but still allow shard routing + // for existing sources (ingest must continue). + if !self.maintenance.is_active() + && let Err(metastore_error) = self + .auto_create_indexes(&request.subrequests, ctx.progress()) + .await { return convert_metastore_error(metastore_error); } @@ -977,6 +1054,11 @@ impl Handler for ControlPlane { local_shards_update: LocalShardsUpdate, ctx: &ActorContext, ) -> Result { + if self.maintenance.is_active() { + // In maintenance mode: skip shard scaling to avoid changing the plan. + debug!("maintenance mode: ignoring local shards update (scaling frozen)"); + return Ok(Ok(())); + } if let Err(metastore_error) = self .ingest_controller .handle_local_shards_update(local_shards_update, &mut self.model, ctx.progress()) @@ -1068,19 +1150,34 @@ impl Handler for ControlPlane { message: IndexerJoined, ctx: &ActorContext, ) -> Result { - info!( - "indexer `{}` joined the cluster: rebalancing shards and rebuilding indexing plan", - message.0.node_id() - ); + let is_maintenance = self.maintenance.is_active(); + if is_maintenance { + info!( + "indexer `{}` joined the cluster during maintenance mode", + message.0.node_id() + ); + } else { + info!( + "indexer `{}` joined the cluster: rebalancing shards and rebuilding indexing plan", + message.0.node_id() + ); + } + // TODO: Update shard table. if let Err(metastore_error) = self .ingest_controller - .rebalance_shards(&mut self.model, ctx.mailbox(), ctx.progress()) + .rebalance_shards( + &mut self.model, + ctx.mailbox(), + ctx.progress(), + is_maintenance, + ) .await { return convert_metastore_error::<()>(metastore_error).map(|_| ()); } - self.indexing_scheduler.rebuild_plan(&self.model); + self.indexing_scheduler + .rebuild_plan(&self.model, is_maintenance); Ok(()) } } @@ -1098,19 +1195,34 @@ impl Handler for ControlPlane { message: IndexerLeft, ctx: &ActorContext, ) -> Result { - info!( - "indexer `{}` left the cluster: rebalancing shards and rebuilding indexing plan", - message.0.node_id() - ); + let is_maintenance = self.maintenance.is_active(); + if is_maintenance { + info!( + "indexer `{}` left the cluster during maintenance mode", + message.0.node_id() + ); + return Ok(()); + } else { + info!( + "indexer `{}` left the cluster: rebalancing shards and rebuilding indexing plan", + message.0.node_id() + ); + } // TODO: Update shard table. if let Err(metastore_error) = self .ingest_controller - .rebalance_shards(&mut self.model, ctx.mailbox(), ctx.progress()) + .rebalance_shards( + &mut self.model, + ctx.mailbox(), + ctx.progress(), + is_maintenance, + ) .await { return convert_metastore_error::<()>(metastore_error).map(|_| ()); } - self.indexing_scheduler.rebuild_plan(&self.model); + self.indexing_scheduler + .rebuild_plan(&self.model, is_maintenance); Ok(()) } } @@ -1142,6 +1254,148 @@ impl Handler for ControlPlane { } } +// -- Maintenance Mode Handlers -- + +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + request: EnableMaintenanceModeRequest, + _ctx: &ActorContext, + ) -> Result { + self.handle_enable_maintenance(request).await + } +} + +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + _request: DisableMaintenanceModeRequest, + _ctx: &ActorContext, + ) -> Result { + self.handle_disable_maintenance().await + } +} + +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + _request: GetMaintenanceModeRequest, + _ctx: &ActorContext, + ) -> Result { + self.handle_get_maintenance() + } +} + +impl ControlPlane { + async fn handle_enable_maintenance( + &mut self, + _request: EnableMaintenanceModeRequest, + ) -> Result, ActorExitStatus> { + if self.maintenance.is_active() { + return Ok(Err(ControlPlaneError::Internal( + "maintenance mode is already enabled".to_string(), + ))); + } + + // Freeze the current plan. + let frozen_plan = self + .indexing_scheduler + .observable_state() + .current_targeted_physical_plan + .unwrap_or_else(|| crate::indexing_plan::PhysicalIndexingPlan::with_indexer_ids(&[])); + + let frozen_plan_json = match serialize_frozen_plan(&frozen_plan) { + Ok(json) => json, + Err(err) => { + return Ok(Err(ControlPlaneError::Internal(format!( + "failed to serialize frozen plan: {err}" + )))); + } + }; + + // Build the metadata (with RFC 3339 datetime). + let metadata = crate::maintenance::MaintenanceModeMetadata::new_now(); + + // Persist to durable storage BEFORE enabling in-memory state. + // This ensures that on restart, the control plane will find the persisted state + // even if it crashes right after this point. + if let Err(err) = self + .maintenance_persistence + .save(&metadata, &frozen_plan) + .await + { + return Ok(Err(ControlPlaneError::Internal(format!( + "failed to persist maintenance state: {err}" + )))); + } + + // Only now enable in-memory state (persistence succeeded). + self.maintenance.load_from_metadata(metadata); + crate::metrics::CONTROL_PLANE_METRICS + .maintenance_mode + .set(1); + + info!( + num_indexers = frozen_plan.num_indexers(), + "maintenance mode enabled: indexing plan frozen" + ); + + Ok(Ok(EnableMaintenanceModeResponse { frozen_plan_json })) + } + + async fn handle_disable_maintenance( + &mut self, + ) -> Result, ActorExitStatus> { + if !self.maintenance.is_active() { + return Ok(Err(ControlPlaneError::Internal( + "maintenance mode is not currently enabled".to_string(), + ))); + } + + // Clear persisted state BEFORE disabling in-memory. + // This ensures that on restart, the control plane will NOT reload maintenance mode + // even if it crashes right after this point. + if let Err(err) = self.maintenance_persistence.clear().await { + return Ok(Err(ControlPlaneError::Internal(format!( + "failed to clear persisted maintenance state: {err}" + )))); + } + + // Only now disable in-memory state (persistence clear succeeded). + self.maintenance.disable(); + crate::metrics::CONTROL_PLANE_METRICS + .maintenance_mode + .set(0); + + // Trigger a full plan rebuild to reconcile the cluster. + info!("maintenance mode disabled: triggering full indexing plan rebuild"); + self.indexing_scheduler.rebuild_plan(&self.model, false); + + Ok(Ok(DisableMaintenanceModeResponse {})) + } + + fn handle_get_maintenance( + &self, + ) -> Result, ActorExitStatus> { + let is_maintenance_mode = self.maintenance.is_active(); + let enabled_at = self.maintenance.enabled_at(); + + Ok(Ok(GetMaintenanceModeResponse { + is_maintenance_mode, + enabled_at, + })) + } +} + fn spawn_watch_indexers_task( weak_mailbox: WeakMailbox, cluster_change_stream: ClusterChangeStream, @@ -1184,9 +1438,11 @@ mod tests { use std::num::NonZero; use std::sync::Arc; + use futures::FutureExt; use mockall::Sequence; use quickwit_actors::{AskError, Observe, SupervisorMetrics}; use quickwit_cluster::ClusterChangeStreamFactoryForTest; + use quickwit_common::test_utils::wait_until_predicate; use quickwit_config::{ CLI_SOURCE_ID, INGEST_V2_SOURCE_ID, IndexConfig, KafkaSourceParams, SourceParams, }; @@ -1208,8 +1464,8 @@ mod tests { }; use quickwit_proto::ingest::{Shard, ShardPKey, ShardState}; use quickwit_proto::metastore::{ - DeleteShardsResponse, EntityKind, FindIndexTemplateMatchesResponse, - ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, + DeleteShardsResponse, EmptyResponse, EntityKind, FindIndexTemplateMatchesResponse, + GetKvResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListShardsSubresponse, MetastoreError, MockMetastoreService, OpenShardSubresponse, OpenShardsResponse, SourceType, }; @@ -1218,6 +1474,538 @@ mod tests { use super::*; use crate::IndexerNodeInfo; + use crate::indexing_plan::PhysicalIndexingPlan; + use crate::maintenance::MetastoreKvPersistence; + + fn setup_disabled_maintenance(mock_metastore: &mut MockMetastoreService) { + mock_metastore + .expect_get_kv() + .returning(|_| Ok(GetKvResponse { value: None })); + } + + fn setup_maintenance_enable(mock_metastore: &mut MockMetastoreService) { + mock_metastore + .expect_get_kv() + .return_once(|_| Ok(GetKvResponse { value: None })); + mock_metastore + .expect_set_kv() + .return_once(|_| Ok(EmptyResponse {})); + } + + async fn observe_current_plan( + control_plane_handle: &ActorHandle>, + ) -> Option { + control_plane_handle + .observe() + .await + .state_opt + .as_ref()? + .indexing_scheduler + .current_targeted_physical_plan + .clone() + } + + #[must_use] + fn add_test_indexer_with_mailbox( + universe: &Universe, + indexer_pool: &IndexerPool, + node_id: NodeId, + ) -> quickwit_actors::Inbox { + let (client_mailbox, client_inbox) = universe.create_test_mailbox(); + let client = IndexingServiceClient::from_mailbox::(client_mailbox); + let indexer_info = IndexerNodeInfo { + node_id: node_id.clone(), + generation_id: 0, + client, + indexing_tasks: Vec::new(), + indexing_capacity: CpuCapacity::from_cpu_millis(4_000), + }; + indexer_pool.insert(node_id, indexer_info); + client_inbox + } + + #[tokio::test] + async fn test_maintenance_mode_allows_create_index_without_rebuild() { + let universe = Universe::with_accelerated_time(); + + let indexer_pool = IndexerPool::default(); + + // Add one indexer to the pool + let node_1: NodeId = "test-node-1".into(); + let _indexing_inbox_1 = + add_test_indexer_with_mailbox(&universe, &indexer_pool, node_1.clone()); + + let ingester_pool = IngesterPool::default(); + + let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + let index_uid_clone = index_uid.clone(); + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); + mock_metastore + .expect_create_index() + .return_once(move |req| { + // re-serialize the received requested config + let index_config = req.deserialize_index_config().unwrap(); + let source_configs = req.deserialize_source_configs().unwrap(); + let mut index_metadata = IndexMetadata::new(index_config); + index_metadata.index_uid = index_uid_clone.clone(); + for source_config in source_configs { + index_metadata.add_source(source_config).unwrap(); + } + let index_metadata_json = serde_json::to_string(&index_metadata).unwrap(); + Ok(CreateIndexResponse { + index_uid: Some(index_uid_clone), + index_metadata_json, + }) + }); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + node_1.clone(), + cluster_change_stream_factory, + indexer_pool.clone(), + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Wait for a first (empty) plan to be calculated. + wait_until_predicate( + || observe_current_plan(&control_plane_handle).map(|plan| plan.is_some()), + Duration::from_secs(5), + Duration::from_millis(100), + ) + .await + .unwrap(); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + let original_physical_plan = observe_current_plan(&control_plane_handle).await; + + // Create index in maintenance mode + let index_config = IndexConfig::for_test("test-index", "ram:///test-index"); + let kafka_source = SourceConfig::for_test( + "kafka-source", + SourceParams::Kafka(KafkaSourceParams { + topic: "test-topic".to_string(), + client_log_level: None, + enable_backfill_mode: false, + client_params: json!({}), + }), + ); + let create_index_request = + CreateIndexRequest::try_from_index_and_source_configs(&index_config, &[kafka_source]) + .unwrap(); + let create_result = control_plane_mailbox + .ask_for_res(create_index_request) + .await; + assert!(create_result.is_ok()); + assert_eq!(create_result.unwrap().index_uid(), &index_uid); + // Check that plan rebuild is skipped + universe.sleep(Duration::from_secs(60)).await; + assert_eq!( + original_physical_plan, + observe_current_plan(&control_plane_handle).await, + "physical plan should not change after creating index in maintenance mode" + ); + + // Add another node + let node_2: NodeId = "test-node-2".into(); + let _indexing_inbox_2 = + add_test_indexer_with_mailbox(&universe, &indexer_pool, node_2.clone()); + // Check that the rebuild is still skipped + universe.sleep(Duration::from_secs(60)).await; + assert_eq!( + original_physical_plan, + observe_current_plan(&control_plane_handle).await, + "physical plan should not change after adding new node in maintenance mode" + ); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_allows_delete_index() { + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); + mock_metastore + .expect_delete_index() + .return_once(|_| Ok(EmptyResponse {})); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + // Delete index in maintenance mode — should succeed, but plan rebuild is skipped. + let index_uid = IndexUid::for_test("test-index", 0); + let delete_index_request = DeleteIndexRequest { + index_uid: Some(index_uid), + }; + let delete_result = control_plane_mailbox + .ask(delete_index_request) + .await + .unwrap(); + assert!(delete_result.is_ok()); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_allows_add_source() { + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + // Pre-load an index with an enabled ingest_v2 source so that + // `create_or_enable_ingest_v2_sources_if_necessary` does not call `add_source` on + // startup and consume the mock expectation meant for the test's own call. + let mut index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); + let mut ingest_v2_source = SourceConfig::ingest_v2(); + ingest_v2_source.enabled = true; + index_metadata.add_source(ingest_v2_source).unwrap(); + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .return_once(move |_| Ok(ListIndexesMetadataResponse::for_test(vec![index_metadata]))); + mock_metastore + .expect_list_shards() + .return_once(|_| Ok(ListShardsResponse::default())); + mock_metastore + .expect_add_source() + .return_once(|_| Ok(EmptyResponse {})); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + // Add source in maintenance mode — should succeed, but plan rebuild is skipped. + let index_uid = IndexUid::for_test("test-index", 0); + let source_config = SourceConfig::for_test("test-source", SourceParams::void()); + let add_source_request = AddSourceRequest { + index_uid: Some(index_uid), + source_config_json: serde_json::to_string(&source_config).unwrap(), + }; + let result = control_plane_mailbox.ask(add_source_request).await.unwrap(); + assert!(result.is_ok()); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_enable_disable_cycle() { + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_get_kv() + .returning(|_| Ok(GetKvResponse { value: None })); + mock_metastore + .expect_set_kv() + .returning(|_| Ok(EmptyResponse {})); + mock_metastore + .expect_delete_kv() + .returning(|_| Ok(EmptyResponse {})); + mock_metastore + .expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Initially not in maintenance mode. + let status = control_plane_mailbox + .ask(GetMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + assert!(!status.is_maintenance_mode); + + // Enable. + let enable_resp = control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + assert!(!enable_resp.frozen_plan_json.is_empty()); + + // Check status. + let status = control_plane_mailbox + .ask(GetMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + assert!(status.is_maintenance_mode); + assert!(status.enabled_at.is_some()); + + // Enable again — should fail. + let double_enable = control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap(); + assert!(double_enable.is_err()); + + // Disable. + let disable_resp = control_plane_mailbox + .ask(DisableMaintenanceModeRequest {}) + .await + .unwrap(); + assert!(disable_resp.is_ok()); + + // Check status again. + let status = control_plane_mailbox + .ask(GetMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + assert!(!status.is_maintenance_mode); + + // Disable again — should fail. + let double_disable = control_plane_mailbox + .ask(DisableMaintenanceModeRequest {}) + .await + .unwrap(); + assert!(double_disable.is_err()); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_observable_state() { + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Observe initial state. + let obs = control_plane_handle.process_pending_and_observe().await; + let state = obs.state_opt.as_ref().unwrap(); + assert!(!state.maintenance_mode); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + // Give the supervisor time to observe the inner actor's updated state. + universe.sleep(Duration::from_secs(1)).await; + + let obs = control_plane_handle.process_pending_and_observe().await; + let state = obs.state_opt.as_ref().unwrap(); + assert!(state.maintenance_mode); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_allows_toggle_source() { + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + // Pre-load an index with the test source and an enabled ingest_v2 source so that + // `create_or_enable_ingest_v2_sources_if_necessary` does not call `add_source` on + // startup and trigger unexpected mock calls. + let mut index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); + let test_source_config = SourceConfig::for_test("test-source", SourceParams::void()); + index_metadata.add_source(test_source_config).unwrap(); + let mut ingest_v2_source = SourceConfig::ingest_v2(); + ingest_v2_source.enabled = true; + index_metadata.add_source(ingest_v2_source).unwrap(); + + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .return_once(move |_| Ok(ListIndexesMetadataResponse::for_test(vec![index_metadata]))); + mock_metastore + .expect_list_shards() + .return_once(|_| Ok(ListShardsResponse::default())); + mock_metastore + .expect_toggle_source() + .return_once(|_| Ok(EmptyResponse {})); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + // Toggle source in maintenance mode — should succeed, but plan rebuild is skipped. + let index_uid = IndexUid::for_test("test-index", 0); + let toggle_request = ToggleSourceRequest { + index_uid: Some(index_uid), + source_id: "test-source".to_string(), + enable: false, + }; + let result = control_plane_mailbox.ask(toggle_request).await.unwrap(); + assert!(result.is_ok()); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_maintenance_mode_allows_get_or_create_open_shards() { + // In maintenance mode, GetOrCreateOpenShards should still work for existing sources + // (ingest must continue), but auto_create_indexes is skipped. + let universe = Universe::with_accelerated_time(); + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let ingester_pool = IngesterPool::default(); + + let mut mock_metastore = MockMetastoreService::new(); + setup_maintenance_enable(&mut mock_metastore); + mock_metastore + .expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); + // Note: no expect_find_index_template_matches — if auto_create was NOT skipped, + // this would panic due to unexpected call. + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + + // Enable maintenance mode. + control_plane_mailbox + .ask(EnableMaintenanceModeRequest {}) + .await + .unwrap() + .unwrap(); + + // Send a GetOrCreateOpenShards with a nonexistent index. + // In maintenance, auto_create is skipped, so the index won't be found. + // The ingest controller will report a failure for unknown indexes, which is expected. + let request = GetOrCreateOpenShardsRequest { + subrequests: vec![GetOrCreateOpenShardsSubrequest { + subrequest_id: 0, + index_id: "nonexistent-index".to_string(), + source_id: "source".to_string(), + }], + closed_shards: Vec::new(), + unavailable_leaders: Vec::new(), + }; + let result = control_plane_mailbox.ask(request).await.unwrap(); + // The request should succeed at the handler level. + // It may fail internally because the index doesn't exist, but that's expected. + match result { + Ok(response) => { + // The response should contain a failure for the unknown index. + assert!(!response.failures.is_empty()); + assert_eq!( + response.failures[0].reason(), + GetOrCreateOpenShardsFailureReason::IndexNotFound + ); + } + Err(_err) => { + // Any internal error is acceptable here (index not found, etc.). + } + } + + universe.assert_quit().await; + } #[tokio::test] async fn test_control_plane_create_index() { @@ -1227,6 +2015,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid_clone = index_uid.clone(); mock_metastore @@ -1284,6 +2073,7 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let index_uid_clone = index_uid.clone(); mock_metastore .expect_delete_index() @@ -1330,6 +2120,7 @@ mod tests { .unwrap(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_add_source() .withf(|add_source_request| { @@ -1427,6 +2218,7 @@ mod tests { .unwrap(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_update_source() .withf(move |update_source_request| { @@ -1494,6 +2286,7 @@ mod tests { index_metadata.add_source(test_source_config).unwrap(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .return_once(|_| Ok(ListIndexesMetadataResponse::for_test(vec![index_metadata]))); @@ -1564,6 +2357,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid_clone = index_uid.clone(); mock_metastore @@ -1612,6 +2406,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); mock_metastore .expect_list_indexes_metadata() @@ -1690,6 +2485,7 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); let source = SourceConfig::ingest_v2(); @@ -1824,18 +2620,14 @@ mod tests { let universe = Universe::with_accelerated_time(); let node_id = NodeId::new("test-control-plane".to_string()); let indexer_pool = IndexerPool::default(); - let (client_mailbox, client_inbox) = universe.create_test_mailbox(); - let client = IndexingServiceClient::from_mailbox::(client_mailbox); - let indexer_node_info = IndexerNodeInfo { - node_id: NodeId::new("test-indexer".to_string()), - generation_id: 0, - client, - indexing_tasks: Vec::new(), - indexing_capacity: CpuCapacity::from_cpu_millis(4_000), - }; - indexer_pool.insert(indexer_node_info.node_id.clone(), indexer_node_info); + let client_inbox = add_test_indexer_with_mailbox( + &universe, + &indexer_pool, + NodeId::new("test-indexer".to_string()), + ); let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); let mut source = SourceConfig::ingest_v2(); @@ -1973,18 +2765,14 @@ mod tests { let universe = Universe::with_accelerated_time(); let node_id = NodeId::new("test-control-plane".to_string()); let indexer_pool = IndexerPool::default(); - let (client_mailbox, _client_inbox) = universe.create_test_mailbox(); - let client = IndexingServiceClient::from_mailbox::(client_mailbox); - let indexer_node_info = IndexerNodeInfo { - node_id: NodeId::new("test-indexer".to_string()), - generation_id: 0, - client, - indexing_tasks: Vec::new(), - indexing_capacity: CpuCapacity::from_cpu_millis(4_000), - }; - indexer_pool.insert(indexer_node_info.node_id.clone(), indexer_node_info); + let _indexing_inbox = add_test_indexer_with_mailbox( + &universe, + &indexer_pool, + NodeId::new("test-indexer".to_string()), + ); let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let mut index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let mut source_config = SourceConfig::ingest_v2(); @@ -2051,18 +2839,14 @@ mod tests { let universe = Universe::default(); let node_id = NodeId::new("test-control-plane".to_string()); let indexer_pool = IndexerPool::default(); - let (client_mailbox, _client_inbox) = universe.create_test_mailbox(); - let client = IndexingServiceClient::from_mailbox::(client_mailbox); - let indexer_node_info = IndexerNodeInfo { - node_id: NodeId::new("test-indexer".to_string()), - generation_id: 0, - client, - indexing_tasks: Vec::new(), - indexing_capacity: CpuCapacity::from_cpu_millis(4_000), - }; - indexer_pool.insert(indexer_node_info.node_id.clone(), indexer_node_info); + let _indexing_inbox = add_test_indexer_with_mailbox( + &universe, + &indexer_pool, + NodeId::new("test-indexer".to_string()), + ); let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); let mut source = SourceConfig::ingest_v2(); @@ -2158,6 +2942,7 @@ mod tests { let index_0_clone = index_0.clone(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .times(1) @@ -2279,6 +3064,7 @@ mod tests { let index_uid_clone = index_0.index_uid.clone(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore.expect_delete_source().return_once( move |delete_source_request: DeleteSourceRequest| { assert_eq!(delete_source_request.index_uid(), &index_uid_clone); @@ -2362,6 +3148,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() @@ -2492,10 +3279,27 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .return_once(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); let metastore = MetastoreServiceClient::from_mock(mock_metastore); + + // Create mock maintenance persistence metastore + let mut mock_persistence_metastore = MockMetastoreService::new(); + mock_persistence_metastore + .expect_get_kv() + .returning(|_| Ok(GetKvResponse { value: None })); + mock_persistence_metastore + .expect_set_kv() + .returning(|_| Ok(EmptyResponse {})); + mock_persistence_metastore + .expect_delete_kv() + .returning(|_| Ok(EmptyResponse {})); + let maintenance_persistence = MetastoreKvPersistence::new( + MetastoreServiceClient::from_mock(mock_persistence_metastore), + ); + let disable_control_loop = true; let (_control_plane_mailbox, control_plane_handle, _readiness_rx) = ControlPlane::spawn_inner( @@ -2507,6 +3311,7 @@ mod tests { ingester_pool, metastore, disable_control_loop, + maintenance_persistence, ); let cluster_change_stream_tx = cluster_change_stream_factory.change_stream_tx(); let indexer_node = @@ -2572,6 +3377,7 @@ mod tests { ingester_pool.insert(ingester_id, ingester); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .return_once(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); @@ -2750,6 +3556,7 @@ mod tests { index_b.add_source(SourceConfig::ingest_v2()).unwrap(); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .return_once(move |_| { @@ -2761,6 +3568,21 @@ mod tests { .expect_list_shards() .return_once(|_| Ok(ListShardsResponse::default())); + // Create mock maintenance persistence metastore + let mut mock_persistence_metastore = MockMetastoreService::new(); + mock_persistence_metastore + .expect_get_kv() + .returning(|_| Ok(GetKvResponse { value: None })); + mock_persistence_metastore + .expect_set_kv() + .returning(|_| Ok(EmptyResponse {})); + mock_persistence_metastore + .expect_delete_kv() + .returning(|_| Ok(EmptyResponse {})); + let maintenance_persistence = MetastoreKvPersistence::new( + MetastoreServiceClient::from_mock(mock_persistence_metastore), + ); + let cluster_config = ClusterConfig::for_test(); let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn_inner( @@ -2772,6 +3594,7 @@ mod tests { ingester_pool, MetastoreServiceClient::from_mock(mock_metastore), false, // keep the control loop enabled + maintenance_persistence, ); // ── Wait for the initial plan to be built ────────────────────────── @@ -2951,6 +3774,7 @@ mod tests { ingester_pool.insert(ingester_id, ingester); let mut mock_metastore = MockMetastoreService::new(); + setup_disabled_maintenance(&mut mock_metastore); mock_metastore .expect_list_indexes_metadata() .return_once(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); diff --git a/quickwit/quickwit-control-plane/src/indexing_plan.rs b/quickwit/quickwit-control-plane/src/indexing_plan.rs index befeef18232..31cce996dfa 100644 --- a/quickwit/quickwit-control-plane/src/indexing_plan.rs +++ b/quickwit/quickwit-control-plane/src/indexing_plan.rs @@ -14,13 +14,13 @@ use fnv::FnvHashMap; use quickwit_proto::indexing::IndexingTask; -use serde::Serialize; +use serde::{Deserialize, Serialize}; /// A [`PhysicalIndexingPlan`] defines the list of indexing tasks /// each indexer, identified by its node ID, should run. /// TODO(fmassot): a metastore version number will be attached to the plan /// to identify if the plan is up to date with the metastore. -#[derive(Debug, PartialEq, Clone, Serialize)] +#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)] pub struct PhysicalIndexingPlan { indexing_tasks_per_indexer_id: FnvHashMap>, } diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs index 049d2379182..9d1a1b9dee3 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs @@ -312,12 +312,30 @@ impl IndexingScheduler { self.state.clone() } - // Should be called whenever a change in the list of index/shard - // has happened. - // - // Prefer not calling this method directly, and instead call - // `ControlPlane::rebuild_indexing_plan_debounced`. - pub(crate) fn rebuild_plan(&mut self, model: &ControlPlaneModel) { + /// Loads a frozen indexing plan without applying it + /// to indexers or triggering any scheduling logic. + /// + /// This is used during control plane initialization when maintenance mode is active: + /// the frozen plan is restored as the `current_targeted_physical_plan` so that the + /// `control_running_plan` loop can re-apply it to indexers that restart during the + /// maintenance window. + pub(crate) fn load_frozen_plan(&mut self, plan: crate::indexing_plan::PhysicalIndexingPlan) { + self.state.current_targeted_physical_plan = Some(plan); + } + + /// Should be called whenever a change in the list of index/shard has + /// happened. + /// + /// When in maintenance mode (`is_maintenance` is true), this function exits + /// early to keep the indexing plan frozen. This design provides a simple + /// safeguard to prevent unintended plan modifications during maintenance. + /// + /// Prefer not calling this method directly, and instead call + /// `ControlPlane::rebuild_indexing_plan_debounced`. + pub(crate) fn rebuild_plan(&mut self, model: &ControlPlaneModel, is_maintenance: bool) { + if is_maintenance { + return; + } crate::metrics::CONTROL_PLANE_METRICS.schedule_total.inc(); let notify_on_drop = self.next_rebuild_tracker.start_rebuild(); @@ -372,7 +390,7 @@ impl IndexingScheduler { /// chitchat cluster state. If true, do nothing. /// - If node IDs differ, schedule a new indexing plan. /// - If indexing tasks differ, apply again the last plan. - pub(crate) fn control_running_plan(&mut self, model: &ControlPlaneModel) { + pub(crate) fn control_running_plan(&mut self, model: &ControlPlaneModel, is_maintenance: bool) { let current_targeted_plan = if let Some(current_targeted) = &self.state.current_targeted_physical_plan { current_targeted @@ -380,7 +398,7 @@ impl IndexingScheduler { // If there is no plan, the node is probably starting and the scheduler did not find // indexers yet. In this case, we want to schedule as soon as possible to find new // indexers. - self.rebuild_plan(model); + self.rebuild_plan(model, is_maintenance); return; }; if let Some(last_applied_plan_timestamp) = self.state.last_applied_plan_timestamp @@ -401,7 +419,7 @@ impl IndexingScheduler { ); if !indexing_plans_diff.has_same_nodes() { info!(plans_diff=?indexing_plans_diff, "running plan and last applied plan node IDs differ: schedule an indexing plan"); - self.rebuild_plan(model); + self.rebuild_plan(model, is_maintenance); } else if !indexing_plans_diff.has_same_tasks() { // Some nodes may have not received their tasks, apply it again. info!(plans_diff=?indexing_plans_diff, "running tasks and last applied tasks differ: reapply last plan"); diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 63295d61eca..ed256f1fb43 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -1006,6 +1006,10 @@ impl IngestController { /// Moving a shard consists of closing the shard on the source ingester and opening a new /// one on the target ingester. /// + /// When in maintenance mode (`is_maintenance` is true), this function exits early to keep + /// the indexing plan frozen. This design provides a simple safeguard to prevent unintended + /// plan modifications during maintenance. + /// /// This method is guarded by a lock to ensure that only one rebalance operation is performed at /// a time. pub(crate) async fn rebalance_shards( @@ -1013,7 +1017,11 @@ impl IngestController { model: &mut ControlPlaneModel, mailbox: &Mailbox, progress: &Progress, + is_maintenance: bool, ) -> MetastoreResult>> { + if is_maintenance { + return Ok(None); + } let Ok(rebalance_guard) = self.rebalance_lock.clone().try_lock_owned() else { debug!("skipping rebalance: another rebalance is already in progress"); return Ok(None); @@ -3262,7 +3270,7 @@ mod tests { let progress = Progress::default(); let close_shards_task_opt = controller - .rebalance_shards(&mut model, &control_plane_mailbox, &progress) + .rebalance_shards(&mut model, &control_plane_mailbox, &progress, false) .await .unwrap(); assert!(close_shards_task_opt.is_none()); @@ -3386,7 +3394,7 @@ mod tests { ingester_pool.insert(ingester_id_1.clone(), ingester_1); let close_shards_task = controller - .rebalance_shards(&mut model, &control_plane_mailbox, &progress) + .rebalance_shards(&mut model, &control_plane_mailbox, &progress, false) .await .unwrap() .unwrap(); diff --git a/quickwit/quickwit-control-plane/src/lib.rs b/quickwit/quickwit-control-plane/src/lib.rs index 01072f7de16..0f1bc8275b8 100644 --- a/quickwit/quickwit-control-plane/src/lib.rs +++ b/quickwit/quickwit-control-plane/src/lib.rs @@ -16,6 +16,7 @@ pub mod control_plane; pub mod indexing_plan; pub mod indexing_scheduler; pub mod ingest; +pub mod maintenance; pub(crate) mod metrics; pub(crate) mod model; diff --git a/quickwit/quickwit-control-plane/src/maintenance.rs b/quickwit/quickwit-control-plane/src/maintenance.rs new file mode 100644 index 00000000000..0149dd453df --- /dev/null +++ b/quickwit/quickwit-control-plane/src/maintenance.rs @@ -0,0 +1,512 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Maintenance mode management for the Quickwit control plane. +//! +//! When maintenance mode is enabled: +//! - Metadata mutations (index/source CRUD) are allowed but the indexing plan is not rebuilt. +//! - The indexing plan is frozen: it is not rebuilt when indexers join or leave. +//! - Shard scaling (up/down) and rebalancing are paused. +//! - The frozen plan and maintenance metadata are persisted to the metastore `kv` table so they +//! survive control plane restarts. +//! +//! # Persistence +//! +//! The state is persisted in the metastore `kv` table under the +//! [`KV_KEY_MAINTENANCE_STATE`] key. The value is a JSON envelope with the +//! with some basic metadata and the binary encoded plan. + +use base64::Engine as _; +use prost::Message; +use quickwit_proto::control_plane::{MaintenanceFrozenPlan, MaintenanceFrozenPlanForNode}; +use quickwit_proto::metastore::{ + DeleteKvRequest, GetKvRequest, MetastoreService, MetastoreServiceClient, SetKvRequest, +}; +use serde::{Deserialize, Serialize}; +use time::OffsetDateTime; +use time::format_description::well_known::Rfc3339; +use tracing::info; + +use crate::indexing_plan::PhysicalIndexingPlan; + +/// Key in the metastore `kv` table for the combined maintenance state. +pub const KV_KEY_MAINTENANCE_STATE: &str = "control_plane_maintenance_state"; + +pub const LATEST_MAINTENANCE_FROZEN_PLAN_VERSION: MaintenanceFrozenPlanVersion = + MaintenanceFrozenPlanVersion::V1; + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +pub enum MaintenanceFrozenPlanVersion { + /// The frozen plan is encoded as protobuf and stored under the + /// "frozen_plan" key as a base64 string. + V1 = 1, +} + +/// Metadata persisted alongside the maintenance mode flag. +/// +/// The `enabled_at` field stores a human-readable RFC 3339 datetime string +/// (e.g., `"2024-06-15T14:30:00Z"`), making it easy to inspect directly in the database. +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +pub struct MaintenanceModeMetadata { + /// RFC 3339 formatted UTC datetime when maintenance mode was enabled. + enabled_at: String, + /// The version of the maintenance state schema. + version: MaintenanceFrozenPlanVersion, +} + +impl MaintenanceModeMetadata { + /// Creates a new metadata instance with `enabled_at` set to the current UTC time. + pub fn new_now() -> Self { + Self { + enabled_at: now_rfc3339(), + version: LATEST_MAINTENANCE_FROZEN_PLAN_VERSION, + } + } +} + +/// In-memory maintenance mode state for the control plane. +#[derive(Debug, Clone, Default)] +pub struct MaintenanceState { + /// If `Some`, maintenance mode is active with the given metadata. + metadata: Option, +} + +impl MaintenanceState { + /// Returns `true` if maintenance mode is currently active. + pub fn is_active(&self) -> bool { + self.metadata.is_some() + } + + /// Returns the metadata if maintenance mode is active. + pub fn metadata(&self) -> Option<&MaintenanceModeMetadata> { + self.metadata.as_ref() + } + + /// Returns the metadata if maintenance mode is active. + pub fn enabled_at(&self) -> Option { + self.metadata + .as_ref() + .map(|metadata| metadata.enabled_at.clone()) + } + + /// Enables maintenance mode. + /// Returns the metadata that was set. + pub fn enable(&mut self) -> MaintenanceModeMetadata { + let metadata = MaintenanceModeMetadata { + enabled_at: now_rfc3339(), + version: LATEST_MAINTENANCE_FROZEN_PLAN_VERSION, + }; + self.metadata = Some(metadata.clone()); + info!( + enabled_at = %metadata.enabled_at, + version = ?metadata.version, + "maintenance mode enabled" + ); + metadata + } + + /// Disables maintenance mode. + /// Returns `true` if it was previously active. + pub fn disable(&mut self) -> bool { + let was_active = self.metadata.is_some(); + self.metadata = None; + if was_active { + info!("maintenance mode disabled"); + } + was_active + } + + /// Loads maintenance state from persisted metadata. + pub fn load_from_metadata(&mut self, metadata: MaintenanceModeMetadata) { + info!( + enabled_at = %metadata.enabled_at, + "loaded maintenance mode from persisted state" + ); + self.metadata = Some(metadata); + } +} + +// -- Persistence Trait -- + +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +pub struct MaintenancePersistedState { + pub metadata: MaintenanceModeMetadata, + pub frozen_plan: PhysicalIndexingPlan, +} + +impl MaintenancePersistedState { + pub fn serialize(&self) -> anyhow::Result { + match self.metadata.version { + MaintenanceFrozenPlanVersion::V1 => self.serialize_v1(), + } + } + + pub fn deserialize(encoded: &str) -> anyhow::Result { + let envelope: serde_json::Value = serde_json::from_str(encoded)?; + let metadata: MaintenanceModeMetadata = + serde_json::from_value(envelope["metadata"].clone())?; + let frozen_plan = match metadata.version { + MaintenanceFrozenPlanVersion::V1 => { + Self::deserialize_v1_frozen_plan(envelope["frozen_plan"].as_str().ok_or_else( + || anyhow::anyhow!("missing frozen_plan field in maintenance state"), + )?)? + } + }; + Ok(Self { + metadata, + frozen_plan, + }) + } + + fn deserialize_v1_frozen_plan(encoded: &str) -> anyhow::Result { + let decoded = base64::engine::general_purpose::STANDARD + .decode(encoded) + .map_err(|err| anyhow::anyhow!("failed to base64 decode frozen plan: {err}"))?; + let proto_state = MaintenanceFrozenPlan::decode(&decoded[..]) + .map_err(|err| anyhow::anyhow!("failed to decode protobuf frozen plan: {err}"))?; + + // Collect all indexer node IDs to initialize the plan + let indexer_ids: Vec = proto_state + .state_per_node + .iter() + .map(|node_state| node_state.index_id.clone()) + .collect(); + + let mut plan = PhysicalIndexingPlan::with_indexer_ids(&indexer_ids); + + for node_state in proto_state.state_per_node { + for task in node_state.indexing_tasks { + plan.add_indexing_task(&node_state.index_id, task); + } + } + Ok(plan) + } + + fn serialize_v1(&self) -> anyhow::Result { + let proto_state = self.frozen_plan_to_proto(); + + // Encode the protobuf message to binary + let mut buf = Vec::new(); + prost::Message::encode(&proto_state, &mut buf) + .map_err(|err| anyhow::anyhow!("failed to encode protobuf: {err}"))?; + + // Base64 encode the binary data + let base64_encoded = base64::engine::general_purpose::STANDARD.encode(&buf); + + let json_value = serde_json::json!({ + "frozen_plan": base64_encoded, + "metadata": serde_json::to_value(&self.metadata)?, + }); + Ok(serde_json::to_string(&json_value)?) + } + + /// Converts the frozen plan to the protobuf representation. + fn frozen_plan_to_proto(&self) -> MaintenanceFrozenPlan { + let state_per_node: Vec = self + .frozen_plan + .indexing_tasks_per_indexer() + .iter() + .map(|(node_id, tasks)| MaintenanceFrozenPlanForNode { + index_id: node_id.clone(), + indexing_tasks: tasks.clone(), + }) + .collect(); + + MaintenanceFrozenPlan { state_per_node } + } +} + +/// Persists maintenance state using the metastore's `GetKv`/`SetKv`/`DeleteKv` +/// RPCs to the PostgreSQL `kv` table. +#[derive(Debug, Clone)] +pub struct MetastoreKvPersistence { + metastore: MetastoreServiceClient, +} + +impl MetastoreKvPersistence { + pub fn new(metastore: MetastoreServiceClient) -> Self { + Self { metastore } + } + + /// Loads the maintenance state from persistent storage. Returns `None` if + /// no maintenance state is persisted. + /// + /// Panics if the state can't be fetched or deserialized. + pub async fn load(&self) -> Option { + let response = self + .metastore + .clone() + .get_kv(GetKvRequest { + key: KV_KEY_MAINTENANCE_STATE.to_string(), + }) + .await + .expect("failed to get maintenance state from metastore"); + let encoded = response.value?; // return None if no value is set + let persisted = MaintenancePersistedState::deserialize(&encoded) + .expect("failed to deserialize maintenance state from metastore"); + Some(persisted) + } + + /// Persists the maintenance metadata and frozen plan atomically. + pub async fn save( + &self, + metadata: &MaintenanceModeMetadata, + frozen_plan: &PhysicalIndexingPlan, + ) -> anyhow::Result<()> { + let persisted = MaintenancePersistedState { + metadata: metadata.clone(), + frozen_plan: frozen_plan.clone(), + }; + let serialized = persisted.serialize()?; + self.metastore + .clone() + .set_kv(SetKvRequest { + key: KV_KEY_MAINTENANCE_STATE.to_string(), + value: serialized, + }) + .await?; + Ok(()) + } + + /// Clears all persisted maintenance state. + pub async fn clear(&self) -> anyhow::Result<()> { + self.metastore + .clone() + .delete_kv(DeleteKvRequest { + key: KV_KEY_MAINTENANCE_STATE.to_string(), + }) + .await?; + Ok(()) + } +} + +// -- Helper functions -- + +/// Serializes a `PhysicalIndexingPlan` to a JSON string for use in API responses. +pub fn serialize_frozen_plan(plan: &PhysicalIndexingPlan) -> serde_json::Result { + serde_json::to_string(plan) +} + +/// Returns the current UTC time formatted as an RFC 3339 string. +fn now_rfc3339() -> String { + OffsetDateTime::now_utc() + .format(&Rfc3339) + .expect("formatting OffsetDateTime as RFC 3339 should never fail") +} + +#[cfg(test)] +mod tests { + use quickwit_proto::metastore::{ + EmptyResponse, GetKvResponse, MetastoreServiceClient, MockMetastoreService, + }; + + use super::*; + + #[test] + fn test_maintenance_state_default_is_inactive() { + let state = MaintenanceState::default(); + assert!(!state.is_active()); + assert!(state.metadata().is_none()); + } + + #[test] + fn test_maintenance_state_enable_disable() { + let mut state = MaintenanceState::default(); + + // Enable + let metadata = state.enable(); + assert!(state.is_active()); + assert!(!metadata.enabled_at.is_empty()); + // Should be a valid RFC 3339 datetime + assert!( + OffsetDateTime::parse(&metadata.enabled_at, &Rfc3339).is_ok(), + "enabled_at should be valid RFC 3339: {}", + metadata.enabled_at + ); + + // Disable + let was_active = state.disable(); + assert!(was_active); + assert!(!state.is_active()); + + // Disable again is a no-op + let was_active = state.disable(); + assert!(!was_active); + } + + #[test] + fn test_current_persisted_state_version_roundtrip() { + let metadata = MaintenanceModeMetadata { + enabled_at: "2024-06-15T14:30:00Z".to_string(), + version: LATEST_MAINTENANCE_FROZEN_PLAN_VERSION, + }; + let plan = PhysicalIndexingPlan::with_indexer_ids(&[ + "indexer-1".to_string(), + "indexer-2".to_string(), + ]); + let state = MaintenancePersistedState { + metadata: metadata.clone(), + frozen_plan: plan.clone(), + }; + let serialized = state + .serialize() + .expect("failed to serialize maintenance state"); + let deserialized: MaintenancePersistedState = + MaintenancePersistedState::deserialize(&serialized).unwrap(); + assert_eq!(deserialized, state); + } + + /// Validates that an existing V1 serialization can still be deserialized. + #[test] + fn test_postcard_v1_deserialization_stability() { + let metadata = MaintenanceModeMetadata { + enabled_at: "2024-06-15T14:30:00Z".to_string(), + version: MaintenanceFrozenPlanVersion::V1, + }; + let plan = PhysicalIndexingPlan::with_indexer_ids(&["indexer-1".to_string()]); + let expected_state = MaintenancePersistedState { + metadata: metadata.clone(), + frozen_plan: plan.clone(), + }; + // // this was used to generate the `encoded` string + // println!( + // "{}", + // expected_state + // .serialize() + // .expect("failed to serialize expected state") + // ); + let encoded = r#"{"frozen_plan":"EgsKCWluZGV4ZXItMQ==","metadata":{"enabled_at":"2024-06-15T14:30:00Z","version":"V1"}}"#; + let deserialized = MaintenancePersistedState::deserialize(encoded).unwrap(); + assert_eq!(deserialized, expected_state); + } + + #[tokio::test] + async fn test_metastore_persistence_save_and_load() { + let mut mock_metastore = MockMetastoreService::new(); + + // Initially empty + mock_metastore + .expect_get_kv() + .times(1) + .returning(|_| Ok(GetKvResponse { value: None })); + + // Save + mock_metastore + .expect_set_kv() + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + // Load + let metadata = MaintenanceModeMetadata { + enabled_at: "2024-01-15T10:00:00Z".to_string(), + version: MaintenanceFrozenPlanVersion::V1, + }; + let plan = PhysicalIndexingPlan::with_indexer_ids(&["indexer-1".to_string()]); + let expected_state = MaintenancePersistedState { + metadata: metadata.clone(), + frozen_plan: plan.clone(), + }; + let expected_encoded = expected_state.serialize().unwrap(); + + mock_metastore.expect_get_kv().times(1).returning(move |_| { + Ok(GetKvResponse { + value: Some(expected_encoded.clone()), + }) + }); + + // Clear + mock_metastore + .expect_delete_kv() + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + // One final load to verify cleared + mock_metastore + .expect_get_kv() + .times(1) + .returning(|_| Ok(GetKvResponse { value: None })); + + let metastore_client = MetastoreServiceClient::from_mock(mock_metastore); + let persistence = MetastoreKvPersistence::new(metastore_client); + + // Initially empty + let loaded = persistence.load().await; + assert!(loaded.is_none()); + + // Save + persistence.save(&metadata, &plan).await.unwrap(); + + // Load + let loaded = persistence.load().await.unwrap(); + assert_eq!(loaded.metadata, metadata); + assert_eq!(loaded.frozen_plan, plan); + + // Clear + persistence.clear().await.unwrap(); + let loaded = persistence.load().await; + assert!(loaded.is_none()); + } + + #[tokio::test] + async fn test_metastore_persistence_overwrite() { + let mut mock_metastore = MockMetastoreService::new(); + + let metadata1 = MaintenanceModeMetadata { + enabled_at: "2024-01-01T00:00:00Z".to_string(), + version: MaintenanceFrozenPlanVersion::V1, + }; + let plan1 = PhysicalIndexingPlan::with_indexer_ids(&["a".to_string()]); + + let metadata2 = MaintenanceModeMetadata { + enabled_at: "2024-06-01T12:00:00Z".to_string(), + version: MaintenanceFrozenPlanVersion::V1, + }; + let plan2 = PhysicalIndexingPlan::with_indexer_ids(&["b".to_string()]); + + // First save + mock_metastore + .expect_set_kv() + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + // Second save (overwrite) + mock_metastore + .expect_set_kv() + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + // Load - return the second state + let expected_state2 = MaintenancePersistedState { + metadata: metadata2.clone(), + frozen_plan: plan2.clone(), + }; + let expected_encoded2 = expected_state2.serialize().unwrap(); + + mock_metastore.expect_get_kv().times(1).returning(move |_| { + Ok(GetKvResponse { + value: Some(expected_encoded2.clone()), + }) + }); + + let metastore_client = MetastoreServiceClient::from_mock(mock_metastore); + let persistence = MetastoreKvPersistence::new(metastore_client); + + persistence.save(&metadata1, &plan1).await.unwrap(); + persistence.save(&metadata2, &plan2).await.unwrap(); + + let loaded = persistence.load().await.unwrap(); + assert_eq!(loaded.metadata, metadata2); + assert_eq!(loaded.frozen_plan, plan2); + } +} diff --git a/quickwit/quickwit-control-plane/src/metrics.rs b/quickwit/quickwit-control-plane/src/metrics.rs index 5e534c4f176..c3370d9b3a7 100644 --- a/quickwit/quickwit-control-plane/src/metrics.rs +++ b/quickwit/quickwit-control-plane/src/metrics.rs @@ -42,6 +42,9 @@ pub struct ControlPlaneMetrics { // Indexing plan metrics. pub local_shards: IntGauge, pub remote_shards: IntGauge, + + // Maintenance mode. + pub maintenance_mode: IntGauge, } impl ControlPlaneMetrics { @@ -128,6 +131,12 @@ impl Default for ControlPlaneMetrics { ), local_shards, remote_shards, + maintenance_mode: new_gauge( + "maintenance_mode", + "Whether the control plane is in maintenance mode (1 = enabled, 0 = disabled).", + "control_plane", + &[], + ), } } } diff --git a/quickwit/quickwit-control-plane/src/tests.rs b/quickwit/quickwit-control-plane/src/tests.rs index 08cbdf9f230..617ee9abcd1 100644 --- a/quickwit/quickwit-control-plane/src/tests.rs +++ b/quickwit/quickwit-control-plane/src/tests.rs @@ -29,7 +29,8 @@ use quickwit_indexing::IndexingService; use quickwit_metastore::{IndexMetadata, ListIndexesMetadataResponseExt}; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, CpuCapacity, IndexingServiceClient}; use quickwit_proto::metastore::{ - ListIndexesMetadataResponse, ListShardsResponse, MetastoreServiceClient, MockMetastoreService, + GetKvResponse, ListIndexesMetadataResponse, ListShardsResponse, MetastoreServiceClient, + MockMetastoreService, }; use quickwit_proto::types::NodeId; use serde_json::json; @@ -121,6 +122,9 @@ async fn start_control_plane( subresponses: Vec::new(), }) }); + mock_metastore + .expect_get_kv() + .returning(|_| Ok(GetKvResponse { value: None })); let mut indexer_inboxes = Vec::new(); let indexer_pool = Pool::default(); diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index cddbcaa002e..0c05f982e31 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -20,19 +20,19 @@ use quickwit_proto::control_plane::{ControlPlaneService, ControlPlaneServiceClie use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, - DeleteIndexTemplatesRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, - DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, + DeleteIndexTemplatesRequest, DeleteKvRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, - GetClusterIdentityResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, - IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataRequest, IndexesMetadataResponse, - LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, + GetClusterIdentityResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, GetKvRequest, + GetKvResponse, IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataRequest, + IndexesMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, + ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, - SoftDeleteDocumentsRequest, SoftDeleteDocumentsResponse, StageSplitsRequest, + SetKvRequest, SoftDeleteDocumentsRequest, SoftDeleteDocumentsResponse, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; @@ -297,4 +297,16 @@ impl MetastoreService for ControlPlaneMetastore { ) -> MetastoreResult { self.metastore.get_cluster_identity(request).await } + + async fn get_kv(&self, request: GetKvRequest) -> MetastoreResult { + self.metastore.get_kv(request).await + } + + async fn set_kv(&self, request: SetKvRequest) -> MetastoreResult { + self.metastore.set_kv(request).await + } + + async fn delete_kv(&self, request: DeleteKvRequest) -> MetastoreResult { + self.metastore.delete_kv(request).await + } } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/manifest.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/manifest.rs index cec811bd3e4..4e9bf42c972 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/manifest.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/manifest.rs @@ -42,6 +42,7 @@ impl LegacyManifest { indexes: self.indexes, templates: HashMap::new(), identity: Uuid::nil(), + kv_store: HashMap::new(), } } } @@ -67,6 +68,7 @@ pub(crate) struct Manifest { // unnecessary here and we can pass the hash map as is to the `MetastoreState` pub templates: HashMap, pub identity: Uuid, + pub kv_store: HashMap, } #[derive(Clone, Debug, Serialize, Deserialize)] @@ -108,6 +110,8 @@ struct ManifestV0_8 { templates: Vec, #[serde(default, skip_serializing_if = "Uuid::is_nil")] identity: Uuid, + #[serde(default, skip_serializing_if = "HashMap::is_empty")] + kv_store: HashMap, } impl From for ManifestV0_8 { @@ -121,6 +125,7 @@ impl From for ManifestV0_8 { indexes: manifest.indexes, templates, identity: manifest.identity, + kv_store: manifest.kv_store, } } } @@ -137,6 +142,7 @@ impl From for Manifest { indexes, templates, identity: manifest.identity, + kv_store: manifest.kv_store, } } } @@ -158,12 +164,14 @@ impl quickwit_config::TestableForRegression for Manifest { indexes, templates, identity: Uuid::nil(), + kv_store: HashMap::new(), } } fn assert_equality(&self, other: &Self) { assert_eq!(self.indexes, other.indexes); assert_eq!(self.templates, other.templates); + assert_eq!(self.kv_store, other.kv_store); } } @@ -338,6 +346,7 @@ mod tests { indexes, templates, identity: Uuid::nil(), + kv_store: HashMap::new(), }; let manifest_json = serde_json::to_string_pretty(&manifest).unwrap(); let manifest_deserialized: Manifest = serde_json::from_str(&manifest_json).unwrap(); diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index af3df2a363d..51dcbc424a8 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -42,12 +42,13 @@ use quickwit_config::IndexTemplate; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, - DeleteIndexTemplatesRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, - DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, EntityKind, - FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, - GetClusterIdentityResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, - IndexMetadataFailure, IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, - IndexTemplateMatch, IndexesMetadataRequest, IndexesMetadataResponse, LastDeleteOpstampRequest, + DeleteIndexTemplatesRequest, DeleteKvRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, + EntityKind, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, + GetClusterIdentityRequest, GetClusterIdentityResponse, GetIndexTemplateRequest, + GetIndexTemplateResponse, GetKvRequest, GetKvResponse, IndexMetadataFailure, + IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, IndexTemplateMatch, + IndexesMetadataRequest, IndexesMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, @@ -55,7 +56,7 @@ use quickwit_proto::metastore::{ ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, - SoftDeleteDocumentsRequest, SoftDeleteDocumentsResponse, StageSplitsRequest, + SetKvRequest, SoftDeleteDocumentsRequest, SoftDeleteDocumentsResponse, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, serde_utils, }; @@ -1293,6 +1294,43 @@ impl MetastoreService for FileBackedMetastore { uuid: state_wlock_guard.identity.hyphenated().to_string(), }) } + + // KV store API + + async fn get_kv(&self, request: GetKvRequest) -> MetastoreResult { + let state = self.state.read().await; + let value = state.kv_store.get(&request.key).cloned(); + Ok(GetKvResponse { value }) + } + + async fn set_kv(&self, request: SetKvRequest) -> MetastoreResult { + let mut state = self.state.write().await; + let previous_value = state.kv_store.insert(request.key.clone(), request.value); + let manifest = state.as_manifest(); + if let Err(error) = save_manifest(&*self.storage, &manifest).await { + // Rollback + match previous_value { + Some(value) => state.kv_store.insert(request.key, value), + None => state.kv_store.remove(&request.key), + }; + return Err(error); + } + Ok(EmptyResponse {}) + } + + async fn delete_kv(&self, request: DeleteKvRequest) -> MetastoreResult { + let mut state = self.state.write().await; + let previous_value = state.kv_store.remove(&request.key); + let manifest = state.as_manifest(); + if let Err(error) = save_manifest(&*self.storage, &manifest).await { + // Rollback + if let Some(value) = previous_value { + state.kv_store.insert(request.key, value); + } + return Err(error); + } + Ok(EmptyResponse {}) + } } impl MetastoreServiceExt for FileBackedMetastore {} diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/state.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/state.rs index 0d42408f430..04235e861cf 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/state.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/state.rs @@ -33,6 +33,7 @@ pub(super) struct MetastoreState { pub templates: HashMap, pub template_matcher: IndexTemplateMatcher, pub identity: Uuid, + pub kv_store: HashMap, } impl MetastoreState { @@ -67,6 +68,7 @@ impl MetastoreState { templates: manifest.templates, template_matcher, identity: manifest.identity, + kv_store: manifest.kv_store, }; Ok(state) } @@ -89,6 +91,7 @@ impl MetastoreState { indexes, templates, identity: self.identity, + kv_store: self.kv_store.clone(), } } } diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index 08c6e378254..a2129b8623a 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -30,20 +30,21 @@ use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, - DeleteIndexTemplatesRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, - DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, EntityKind, - FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, - GetClusterIdentityResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, - IndexMetadataFailure, IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, - IndexStats, IndexTemplateMatch, IndexesMetadataRequest, IndexesMetadataResponse, - LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, - ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, - ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListShardsSubresponse, - ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, - MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, + DeleteIndexTemplatesRequest, DeleteKvRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, + EntityKind, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, + GetClusterIdentityRequest, GetClusterIdentityResponse, GetIndexTemplateRequest, + GetIndexTemplateResponse, GetKvRequest, GetKvResponse, IndexMetadataFailure, + IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, IndexStats, + IndexTemplateMatch, IndexesMetadataRequest, IndexesMetadataResponse, LastDeleteOpstampRequest, + LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, + ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, + ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListShardsRequest, ListShardsResponse, ListShardsSubresponse, ListSplitsRequest, + ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, + MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, SoftDeleteDocumentsRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, SetKvRequest, SoftDeleteDocumentsRequest, SoftDeleteDocumentsResponse, SplitStats, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, serde_utils, @@ -1885,6 +1886,39 @@ impl MetastoreService for PostgresqlMetastore { Ok(EmptyResponse {}) } + async fn get_kv(&self, request: GetKvRequest) -> MetastoreResult { + let value: Option<(String,)> = sqlx::query_as("SELECT value FROM kv WHERE key = $1") + .bind(&request.key) + .fetch_optional(&self.connection_pool) + .await?; + Ok(GetKvResponse { + value: value.map(|(v,)| v), + }) + } + + async fn set_kv(&self, request: SetKvRequest) -> MetastoreResult { + sqlx::query( + r" + INSERT INTO kv (key, value) + VALUES ($1, $2) + ON CONFLICT (key) DO UPDATE SET value = EXCLUDED.value + ", + ) + .bind(&request.key) + .bind(&request.value) + .execute(&self.connection_pool) + .await?; + Ok(EmptyResponse {}) + } + + async fn delete_kv(&self, request: DeleteKvRequest) -> MetastoreResult { + sqlx::query("DELETE FROM kv WHERE key = $1") + .bind(&request.key) + .execute(&self.connection_pool) + .await?; + Ok(EmptyResponse {}) + } + async fn get_cluster_identity( &self, _: GetClusterIdentityRequest, diff --git a/quickwit/quickwit-metastore/src/tests/kv.rs b/quickwit/quickwit-metastore/src/tests/kv.rs new file mode 100644 index 00000000000..32f0834b6f6 --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/kv.rs @@ -0,0 +1,232 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use quickwit_proto::metastore::{DeleteKvRequest, GetKvRequest, MetastoreService, SetKvRequest}; + +use super::DefaultForTest; + +pub async fn test_metastore_kv_set_get() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set a key-value pair + let set_request = SetKvRequest { + key: "test-key".to_string(), + value: "test-value".to_string(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Get the key-value pair + let get_request = GetKvRequest { + key: "test-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, Some("test-value".to_string())); +} + +pub async fn test_metastore_kv_get_non_existent< + MetastoreToTest: MetastoreService + DefaultForTest, +>() { + let metastore = MetastoreToTest::default_for_test().await; + + // Try to get a non-existent key + let get_request = GetKvRequest { + key: "non-existent-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, None); +} + +pub async fn test_metastore_kv_set_overwrite() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set a key-value pair + let set_request = SetKvRequest { + key: "test-key".to_string(), + value: "original-value".to_string(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Overwrite with new value + let set_request = SetKvRequest { + key: "test-key".to_string(), + value: "updated-value".to_string(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Verify the value was updated + let get_request = GetKvRequest { + key: "test-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, Some("updated-value".to_string())); +} + +pub async fn test_metastore_kv_delete() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set a key-value pair + let set_request = SetKvRequest { + key: "test-key".to_string(), + value: "test-value".to_string(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Verify it exists + let get_request = GetKvRequest { + key: "test-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, Some("test-value".to_string())); + + // Delete the key + let delete_request = DeleteKvRequest { + key: "test-key".to_string(), + }; + metastore.delete_kv(delete_request).await.unwrap(); + + // Verify it no longer exists + let get_request = GetKvRequest { + key: "test-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, None); +} + +pub async fn test_metastore_kv_delete_non_existent< + MetastoreToTest: MetastoreService + DefaultForTest, +>() { + let metastore = MetastoreToTest::default_for_test().await; + + // Delete a non-existent key (should succeed without error) + let delete_request = DeleteKvRequest { + key: "non-existent-key".to_string(), + }; + metastore.delete_kv(delete_request).await.unwrap(); +} + +pub async fn test_metastore_kv_multiple_keys() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set multiple key-value pairs + let set_request_1 = SetKvRequest { + key: "key-1".to_string(), + value: "value-1".to_string(), + }; + metastore.set_kv(set_request_1).await.unwrap(); + + let set_request_2 = SetKvRequest { + key: "key-2".to_string(), + value: "value-2".to_string(), + }; + metastore.set_kv(set_request_2).await.unwrap(); + + let set_request_3 = SetKvRequest { + key: "key-3".to_string(), + value: "value-3".to_string(), + }; + metastore.set_kv(set_request_3).await.unwrap(); + + // Verify all keys exist + let get_request_1 = GetKvRequest { + key: "key-1".to_string(), + }; + let response_1 = metastore.get_kv(get_request_1).await.unwrap(); + assert_eq!(response_1.value, Some("value-1".to_string())); + + let get_request_2 = GetKvRequest { + key: "key-2".to_string(), + }; + let response_2 = metastore.get_kv(get_request_2).await.unwrap(); + assert_eq!(response_2.value, Some("value-2".to_string())); + + let get_request_3 = GetKvRequest { + key: "key-3".to_string(), + }; + let response_3 = metastore.get_kv(get_request_3).await.unwrap(); + assert_eq!(response_3.value, Some("value-3".to_string())); + + // Delete one key + let delete_request = DeleteKvRequest { + key: "key-2".to_string(), + }; + metastore.delete_kv(delete_request).await.unwrap(); + + // Verify key-2 is deleted but others remain + let get_request_1 = GetKvRequest { + key: "key-1".to_string(), + }; + let response_1 = metastore.get_kv(get_request_1).await.unwrap(); + assert_eq!(response_1.value, Some("value-1".to_string())); + + let get_request_2 = GetKvRequest { + key: "key-2".to_string(), + }; + let response_2 = metastore.get_kv(get_request_2).await.unwrap(); + assert_eq!(response_2.value, None); + + let get_request_3 = GetKvRequest { + key: "key-3".to_string(), + }; + let response_3 = metastore.get_kv(get_request_3).await.unwrap(); + assert_eq!(response_3.value, Some("value-3".to_string())); +} + +pub async fn test_metastore_kv_empty_key() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set a key-value pair with an empty key + let set_request = SetKvRequest { + key: "".to_string(), + value: "empty-key-value".to_string(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Get the empty key + let get_request = GetKvRequest { + key: "".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, Some("empty-key-value".to_string())); + + // Delete the empty key + let delete_request = DeleteKvRequest { + key: "".to_string(), + }; + metastore.delete_kv(delete_request).await.unwrap(); + + // Verify it's deleted + let get_request = GetKvRequest { + key: "".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, None); +} + +pub async fn test_metastore_kv_empty_value() { + let metastore = MetastoreToTest::default_for_test().await; + + // Set a key-value pair with an empty value + let set_request = SetKvRequest { + key: "test-key".to_string(), + value: String::new(), + }; + metastore.set_kv(set_request).await.unwrap(); + + // Get the key with empty value + let get_request = GetKvRequest { + key: "test-key".to_string(), + }; + let response = metastore.get_kv(get_request).await.unwrap(); + assert_eq!(response.value, Some(String::new())); +} diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index 0c598966a6a..c6177da1b43 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -28,6 +28,7 @@ use quickwit_proto::types::IndexUid; pub(crate) mod delete_task; pub(crate) mod get_identity; pub(crate) mod index; +pub(crate) mod kv; pub(crate) mod list_splits; pub(crate) mod shard; pub(crate) mod source; @@ -576,6 +577,64 @@ macro_rules! metastore_test_suite { $crate::tests::get_identity::test_metastore_get_identity::<$metastore_type>().await; } + /// KV API tests + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_set_get() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_set_get::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_get_non_existent() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_get_non_existent::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_set_overwrite() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_set_overwrite::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_delete() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_delete::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_delete_non_existent() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_delete_non_existent::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_multiple_keys() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_multiple_keys::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_empty_key() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_empty_key::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_kv_empty_value() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::kv::test_metastore_kv_empty_value::<$metastore_type>().await; + } + /// Soft-delete documents API tests #[tokio::test] diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 3474fdd6c2e..6aa7d28d7da 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -72,6 +72,18 @@ service ControlPlaneService { // Swaps indexing pipelines of different indexes between different indexers. rpc SwapIndexingPipelines(SwapIndexingPipelinesRequest) returns (SwapIndexingPipelinesResponse); + + // Maintenance Mode API + + // Enables maintenance mode on the cluster. When active, the indexing plan is frozen, + // metadata mutations (index/source CRUD) are accepted but the plan is not rebuilt, and shard scaling/rebalancing is paused. + rpc EnableMaintenanceMode(EnableMaintenanceModeRequest) returns (EnableMaintenanceModeResponse); + + // Disables maintenance mode. Triggers a full indexing plan rebuild to reconcile the cluster. + rpc DisableMaintenanceMode(DisableMaintenanceModeRequest) returns (DisableMaintenanceModeResponse); + + // Returns the current maintenance mode status. + rpc GetMaintenanceMode(GetMaintenanceModeRequest) returns (GetMaintenanceModeResponse); } // Shard API @@ -150,3 +162,37 @@ message SwapIndexingPipelinesResult { // Human-readable reason when success is false. string reason = 3; } + +// Maintenance Mode API + +message EnableMaintenanceModeRequest { +} + +message EnableMaintenanceModeResponse { + // The frozen physical indexing plan serialized as JSON. + string frozen_plan_json = 1; +} + +message DisableMaintenanceModeRequest { +} + +message DisableMaintenanceModeResponse { +} + +message GetMaintenanceModeRequest { +} + +message GetMaintenanceModeResponse { + bool is_maintenance_mode = 1; + // If maintenance mode is active, the RFC 3339 datetime string when it was enabled. + optional string enabled_at = 2; +} + +message MaintenanceFrozenPlanForNode { + string index_id = 1; + repeated quickwit.indexing.IndexingTask indexing_tasks = 2; +} + +message MaintenanceFrozenPlan { + repeated MaintenanceFrozenPlanForNode state_per_node = 2; +} \ No newline at end of file diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 712538f193a..97a1287068c 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -205,6 +205,17 @@ service MetastoreService { // Deletes index templates. rpc DeleteIndexTemplates(DeleteIndexTemplatesRequest) returns (EmptyResponse); + // Key-Value API (for cluster-wide settings like maintenance mode) + + // Gets a value by key from the cluster-wide key-value store. + rpc GetKv(GetKvRequest) returns (GetKvResponse); + + // Sets a key-value pair in the cluster-wide key-value store. + rpc SetKv(SetKvRequest) returns (EmptyResponse); + + // Deletes a key from the cluster-wide key-value store. + rpc DeleteKv(DeleteKvRequest) returns (EmptyResponse); + // Get cluster identity rpc GetClusterIdentity(GetClusterIdentityRequest) returns (GetClusterIdentityResponse); } @@ -578,6 +589,26 @@ message DeleteIndexTemplatesRequest { repeated string template_ids = 1; } +// Key-Value API + +message GetKvRequest { + string key = 1; +} + +message GetKvResponse { + // Empty if the key does not exist. + optional string value = 1; +} + +message SetKvRequest { + string key = 1; + string value = 2; +} + +message DeleteKvRequest { + string key = 1; +} + message GetClusterIdentityRequest { } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index 15f85d6c0f1..6c1a3e98600 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -108,6 +108,48 @@ pub struct SwapIndexingPipelinesResult { pub reason: ::prost::alloc::string::String, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct EnableMaintenanceModeRequest {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct EnableMaintenanceModeResponse { + /// The frozen physical indexing plan serialized as JSON. + #[prost(string, tag = "1")] + pub frozen_plan_json: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct DisableMaintenanceModeRequest {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct DisableMaintenanceModeResponse {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct GetMaintenanceModeRequest {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct GetMaintenanceModeResponse { + #[prost(bool, tag = "1")] + pub is_maintenance_mode: bool, + /// If maintenance mode is active, the RFC 3339 datetime string when it was enabled. + #[prost(string, optional, tag = "2")] + pub enabled_at: ::core::option::Option<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MaintenanceFrozenPlanForNode { + #[prost(string, tag = "1")] + pub index_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "2")] + pub indexing_tasks: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MaintenanceFrozenPlan { + #[prost(message, repeated, tag = "2")] + pub state_per_node: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -220,6 +262,22 @@ pub trait ControlPlaneService: std::fmt::Debug + Send + Sync + 'static { &self, request: SwapIndexingPipelinesRequest, ) -> crate::control_plane::ControlPlaneResult; + ///Enables maintenance mode on the cluster. When active, the indexing plan is frozen, + ///metadata mutations (index/source CRUD) are accepted but the plan is not rebuilt, and shard scaling/rebalancing is paused. + async fn enable_maintenance_mode( + &self, + request: EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult; + ///Disables maintenance mode. Triggers a full indexing plan rebuild to reconcile the cluster. + async fn disable_maintenance_mode( + &self, + request: DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult; + ///Returns the current maintenance mode status. + async fn get_maintenance_mode( + &self, + request: GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult; } #[derive(Debug, Clone)] pub struct ControlPlaneServiceClient { @@ -398,6 +456,24 @@ impl ControlPlaneService for ControlPlaneServiceClient { ) -> crate::control_plane::ControlPlaneResult { self.inner.0.swap_indexing_pipelines(request).await } + async fn enable_maintenance_mode( + &self, + request: EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.0.enable_maintenance_mode(request).await + } + async fn disable_maintenance_mode( + &self, + request: DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.0.disable_maintenance_mode(request).await + } + async fn get_maintenance_mode( + &self, + request: GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.0.get_maintenance_mode(request).await + } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_control_plane_service { @@ -494,6 +570,30 @@ pub mod mock_control_plane_service { > { self.inner.lock().await.swap_indexing_pipelines(request).await } + async fn enable_maintenance_mode( + &self, + request: super::EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::EnableMaintenanceModeResponse, + > { + self.inner.lock().await.enable_maintenance_mode(request).await + } + async fn disable_maintenance_mode( + &self, + request: super::DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::DisableMaintenanceModeResponse, + > { + self.inner.lock().await.disable_maintenance_mode(request).await + } + async fn get_maintenance_mode( + &self, + request: super::GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::GetMaintenanceModeResponse, + > { + self.inner.lock().await.get_maintenance_mode(request).await + } } } pub type BoxFuture = std::pin::Pin< @@ -683,6 +783,54 @@ impl tower::Service for InnerControlPlaneServiceCl Box::pin(fut) } } +impl tower::Service for InnerControlPlaneServiceClient { + type Response = EnableMaintenanceModeResponse; + type Error = crate::control_plane::ControlPlaneError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: EnableMaintenanceModeRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.enable_maintenance_mode(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerControlPlaneServiceClient { + type Response = DisableMaintenanceModeResponse; + type Error = crate::control_plane::ControlPlaneError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DisableMaintenanceModeRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.disable_maintenance_mode(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerControlPlaneServiceClient { + type Response = GetMaintenanceModeResponse; + type Error = crate::control_plane::ControlPlaneError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: GetMaintenanceModeRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.get_maintenance_mode(request).await }; + Box::pin(fut) + } +} /// A tower service stack is a set of tower services. #[derive(Debug)] struct ControlPlaneServiceTowerServiceStack { @@ -743,6 +891,21 @@ struct ControlPlaneServiceTowerServiceStack { SwapIndexingPipelinesResponse, crate::control_plane::ControlPlaneError, >, + enable_maintenance_mode_svc: quickwit_common::tower::BoxService< + EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + disable_maintenance_mode_svc: quickwit_common::tower::BoxService< + DisableMaintenanceModeRequest, + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + get_maintenance_mode_svc: quickwit_common::tower::BoxService< + GetMaintenanceModeRequest, + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, } #[async_trait::async_trait] impl ControlPlaneService for ControlPlaneServiceTowerServiceStack { @@ -816,6 +979,24 @@ impl ControlPlaneService for ControlPlaneServiceTowerServiceStack { ) -> crate::control_plane::ControlPlaneResult { self.swap_indexing_pipelines_svc.clone().ready().await?.call(request).await } + async fn enable_maintenance_mode( + &self, + request: EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.enable_maintenance_mode_svc.clone().ready().await?.call(request).await + } + async fn disable_maintenance_mode( + &self, + request: DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.disable_maintenance_mode_svc.clone().ready().await?.call(request).await + } + async fn get_maintenance_mode( + &self, + request: GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.get_maintenance_mode_svc.clone().ready().await?.call(request).await + } } type CreateIndexLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< @@ -927,6 +1108,36 @@ type SwapIndexingPipelinesLayer = quickwit_common::tower::BoxLayer< SwapIndexingPipelinesResponse, crate::control_plane::ControlPlaneError, >; +type EnableMaintenanceModeLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, +>; +type DisableMaintenanceModeLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + DisableMaintenanceModeRequest, + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + DisableMaintenanceModeRequest, + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, +>; +type GetMaintenanceModeLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + GetMaintenanceModeRequest, + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + GetMaintenanceModeRequest, + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, +>; #[derive(Debug, Default)] pub struct ControlPlaneServiceTowerLayerStack { create_index_layers: Vec, @@ -940,6 +1151,9 @@ pub struct ControlPlaneServiceTowerLayerStack { advise_reset_shards_layers: Vec, prune_shards_layers: Vec, swap_indexing_pipelines_layers: Vec, + enable_maintenance_mode_layers: Vec, + disable_maintenance_mode_layers: Vec, + get_maintenance_mode_layers: Vec, } impl ControlPlaneServiceTowerLayerStack { pub fn stack_layer(mut self, layer: L) -> Self @@ -1239,6 +1453,87 @@ impl ControlPlaneServiceTowerLayerStack { >>::Service as tower::Service< SwapIndexingPipelinesRequest, >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + EnableMaintenanceModeRequest, + Response = EnableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + EnableMaintenanceModeRequest, + >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + DisableMaintenanceModeRequest, + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + DisableMaintenanceModeRequest, + Response = DisableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + DisableMaintenanceModeRequest, + >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + GetMaintenanceModeRequest, + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + GetMaintenanceModeRequest, + Response = GetMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + GetMaintenanceModeRequest, + >>::Future: Send + 'static, { self.create_index_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); @@ -1262,6 +1557,12 @@ impl ControlPlaneServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.swap_indexing_pipelines_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.enable_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.disable_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.get_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self } pub fn stack_create_index_layer(mut self, layer: L) -> Self @@ -1496,6 +1797,72 @@ impl ControlPlaneServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_enable_maintenance_mode_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + EnableMaintenanceModeRequest, + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + EnableMaintenanceModeRequest, + Response = EnableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.enable_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_disable_maintenance_mode_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + DisableMaintenanceModeRequest, + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + DisableMaintenanceModeRequest, + Response = DisableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.disable_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_get_maintenance_mode_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + GetMaintenanceModeRequest, + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + GetMaintenanceModeRequest, + Response = GetMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.get_maintenance_mode_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn build(self, instance: T) -> ControlPlaneServiceClient where T: ControlPlaneService, @@ -1647,6 +2014,30 @@ impl ControlPlaneServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let enable_maintenance_mode_svc = self + .enable_maintenance_mode_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let disable_maintenance_mode_svc = self + .disable_maintenance_mode_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let get_maintenance_mode_svc = self + .get_maintenance_mode_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let tower_svc_stack = ControlPlaneServiceTowerServiceStack { inner: inner_client, create_index_svc, @@ -1660,6 +2051,9 @@ impl ControlPlaneServiceTowerLayerStack { advise_reset_shards_svc, prune_shards_svc, swap_indexing_pipelines_svc, + enable_maintenance_mode_svc, + disable_maintenance_mode_svc, + get_maintenance_mode_svc, }; ControlPlaneServiceClient::new(tower_svc_stack) } @@ -1834,6 +2228,33 @@ where SwapIndexingPipelinesResponse, crate::control_plane::ControlPlaneError, >, + > + + tower::Service< + EnableMaintenanceModeRequest, + Response = EnableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + EnableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + + tower::Service< + DisableMaintenanceModeRequest, + Response = DisableMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + DisableMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, + > + + tower::Service< + GetMaintenanceModeRequest, + Response = GetMaintenanceModeResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + GetMaintenanceModeResponse, + crate::control_plane::ControlPlaneError, + >, >, { async fn create_index( @@ -1906,6 +2327,24 @@ where ) -> crate::control_plane::ControlPlaneResult { self.clone().call(request).await } + async fn enable_maintenance_mode( + &self, + request: EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.clone().call(request).await + } + async fn disable_maintenance_mode( + &self, + request: DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.clone().call(request).await + } + async fn get_maintenance_mode( + &self, + request: GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.clone().call(request).await + } } #[derive(Debug, Clone)] pub struct ControlPlaneServiceGrpcClientAdapter { @@ -2099,6 +2538,48 @@ where SwapIndexingPipelinesRequest::rpc_name(), )) } + async fn enable_maintenance_mode( + &self, + request: EnableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .clone() + .enable_maintenance_mode(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + EnableMaintenanceModeRequest::rpc_name(), + )) + } + async fn disable_maintenance_mode( + &self, + request: DisableMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .clone() + .disable_maintenance_mode(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + DisableMaintenanceModeRequest::rpc_name(), + )) + } + async fn get_maintenance_mode( + &self, + request: GetMaintenanceModeRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .clone() + .get_maintenance_mode(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + GetMaintenanceModeRequest::rpc_name(), + )) + } } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServerAdapter { @@ -2241,6 +2722,39 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn enable_maintenance_mode( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .enable_maintenance_mode(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn disable_maintenance_mode( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .disable_maintenance_mode(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn get_maintenance_mode( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .get_maintenance_mode(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } } /// Generated client implementations. pub mod control_plane_service_grpc_client { @@ -2672,6 +3186,97 @@ pub mod control_plane_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Enables maintenance mode on the cluster. When active, the indexing plan is frozen, + /// metadata mutations (index/source CRUD) are accepted but the plan is not rebuilt, and shard scaling/rebalancing is paused. + pub async fn enable_maintenance_mode( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.control_plane.ControlPlaneService/EnableMaintenanceMode", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "EnableMaintenanceMode", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Disables maintenance mode. Triggers a full indexing plan rebuild to reconcile the cluster. + pub async fn disable_maintenance_mode( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.control_plane.ControlPlaneService/DisableMaintenanceMode", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "DisableMaintenanceMode", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Returns the current maintenance mode status. + pub async fn get_maintenance_mode( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.control_plane.ControlPlaneService/GetMaintenanceMode", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "GetMaintenanceMode", + ), + ); + self.inner.unary(req, path, codec).await + } } } /// Generated server implementations. @@ -2776,6 +3381,31 @@ pub mod control_plane_service_grpc_server { tonic::Response, tonic::Status, >; + /// Enables maintenance mode on the cluster. When active, the indexing plan is frozen, + /// metadata mutations (index/source CRUD) are accepted but the plan is not rebuilt, and shard scaling/rebalancing is paused. + async fn enable_maintenance_mode( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// Disables maintenance mode. Triggers a full indexing plan rebuild to reconcile the cluster. + async fn disable_maintenance_mode( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// Returns the current maintenance mode status. + async fn get_maintenance_mode( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServer { @@ -3418,6 +4048,157 @@ pub mod control_plane_service_grpc_server { }; Box::pin(fut) } + "/quickwit.control_plane.ControlPlaneService/EnableMaintenanceMode" => { + #[allow(non_camel_case_types)] + struct EnableMaintenanceModeSvc( + pub Arc, + ); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService + for EnableMaintenanceModeSvc { + type Response = super::EnableMaintenanceModeResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::enable_maintenance_mode( + &inner, + request, + ) + .await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = EnableMaintenanceModeSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.control_plane.ControlPlaneService/DisableMaintenanceMode" => { + #[allow(non_camel_case_types)] + struct DisableMaintenanceModeSvc( + pub Arc, + ); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService + for DisableMaintenanceModeSvc { + type Response = super::DisableMaintenanceModeResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::disable_maintenance_mode( + &inner, + request, + ) + .await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = DisableMaintenanceModeSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.control_plane.ControlPlaneService/GetMaintenanceMode" => { + #[allow(non_camel_case_types)] + struct GetMaintenanceModeSvc(pub Arc); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService + for GetMaintenanceModeSvc { + type Response = super::GetMaintenanceModeResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::get_maintenance_mode( + &inner, + request, + ) + .await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = GetMaintenanceModeSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } _ => { Box::pin(async move { let mut response = http::Response::new( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index d503a940f44..267cc8aac31 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -546,6 +546,33 @@ pub struct DeleteIndexTemplatesRequest { pub template_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct GetKvRequest { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct GetKvResponse { + /// Empty if the key does not exist. + #[prost(string, optional, tag = "1")] + pub value: ::core::option::Option<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct SetKvRequest { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub value: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct DeleteKvRequest { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct GetClusterIdentityRequest {} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -820,6 +847,21 @@ impl RpcName for DeleteIndexTemplatesRequest { "delete_index_templates" } } +impl RpcName for GetKvRequest { + fn rpc_name() -> &'static str { + "get_kv" + } +} +impl RpcName for SetKvRequest { + fn rpc_name() -> &'static str { + "set_kv" + } +} +impl RpcName for DeleteKvRequest { + fn rpc_name() -> &'static str { + "delete_kv" + } +} impl RpcName for GetClusterIdentityRequest { fn rpc_name() -> &'static str { "get_cluster_identity" @@ -1012,6 +1054,21 @@ pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult; + ///Gets a value by key from the cluster-wide key-value store. + async fn get_kv( + &self, + request: GetKvRequest, + ) -> crate::metastore::MetastoreResult; + ///Sets a key-value pair in the cluster-wide key-value store. + async fn set_kv( + &self, + request: SetKvRequest, + ) -> crate::metastore::MetastoreResult; + ///Deletes a key from the cluster-wide key-value store. + async fn delete_kv( + &self, + request: DeleteKvRequest, + ) -> crate::metastore::MetastoreResult; ///Get cluster identity async fn get_cluster_identity( &self, @@ -1325,6 +1382,24 @@ impl MetastoreService for MetastoreServiceClient { ) -> crate::metastore::MetastoreResult { self.inner.0.delete_index_templates(request).await } + async fn get_kv( + &self, + request: GetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.get_kv(request).await + } + async fn set_kv( + &self, + request: SetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.set_kv(request).await + } + async fn delete_kv( + &self, + request: DeleteKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.delete_kv(request).await + } async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, @@ -1549,6 +1624,24 @@ pub mod mock_metastore_service { ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_index_templates(request).await } + async fn get_kv( + &self, + request: super::GetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.get_kv(request).await + } + async fn set_kv( + &self, + request: super::SetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.set_kv(request).await + } + async fn delete_kv( + &self, + request: super::DeleteKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_kv(request).await + } async fn get_cluster_identity( &self, request: super::GetClusterIdentityRequest, @@ -2094,6 +2187,54 @@ impl tower::Service for InnerMetastoreServiceClient Box::pin(fut) } } +impl tower::Service for InnerMetastoreServiceClient { + type Response = GetKvResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: GetKvRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.get_kv(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: SetKvRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.set_kv(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteKvRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.delete_kv(request).await }; + Box::pin(fut) + } +} impl tower::Service for InnerMetastoreServiceClient { type Response = GetClusterIdentityResponse; type Error = crate::metastore::MetastoreError; @@ -2280,6 +2421,21 @@ struct MetastoreServiceTowerServiceStack { EmptyResponse, crate::metastore::MetastoreError, >, + get_kv_svc: quickwit_common::tower::BoxService< + GetKvRequest, + GetKvResponse, + crate::metastore::MetastoreError, + >, + set_kv_svc: quickwit_common::tower::BoxService< + SetKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + delete_kv_svc: quickwit_common::tower::BoxService< + DeleteKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, get_cluster_identity_svc: quickwit_common::tower::BoxService< GetClusterIdentityRequest, GetClusterIdentityResponse, @@ -2486,6 +2642,24 @@ impl MetastoreService for MetastoreServiceTowerServiceStack { ) -> crate::metastore::MetastoreResult { self.delete_index_templates_svc.clone().ready().await?.call(request).await } + async fn get_kv( + &self, + request: GetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.get_kv_svc.clone().ready().await?.call(request).await + } + async fn set_kv( + &self, + request: SetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.set_kv_svc.clone().ready().await?.call(request).await + } + async fn delete_kv( + &self, + request: DeleteKvRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_kv_svc.clone().ready().await?.call(request).await + } async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, @@ -2829,6 +3003,36 @@ type DeleteIndexTemplatesLayer = quickwit_common::tower::BoxLayer< EmptyResponse, crate::metastore::MetastoreError, >; +type GetKvLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + GetKvRequest, + GetKvResponse, + crate::metastore::MetastoreError, + >, + GetKvRequest, + GetKvResponse, + crate::metastore::MetastoreError, +>; +type SetKvLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + SetKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + SetKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; +type DeleteKvLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + DeleteKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + DeleteKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; type GetClusterIdentityLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< GetClusterIdentityRequest, @@ -2874,6 +3078,9 @@ pub struct MetastoreServiceTowerLayerStack { find_index_template_matches_layers: Vec, list_index_templates_layers: Vec, delete_index_templates_layers: Vec, + get_kv_layers: Vec, + set_kv_layers: Vec, + delete_kv_layers: Vec, get_cluster_identity_layers: Vec, } impl MetastoreServiceTowerLayerStack { @@ -3722,6 +3929,81 @@ impl MetastoreServiceTowerLayerStack { >>::Service as tower::Service< DeleteIndexTemplatesRequest, >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + GetKvRequest, + GetKvResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + GetKvRequest, + Response = GetKvResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + SetKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + SetKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + DeleteKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + DeleteKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, L: tower::Layer< quickwit_common::tower::BoxService< GetClusterIdentityRequest, @@ -3816,6 +4098,9 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.delete_index_templates_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.get_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.set_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.delete_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.get_cluster_identity_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self @@ -4477,6 +4762,63 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_get_kv_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + GetKvRequest, + GetKvResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + GetKvRequest, + Response = GetKvResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.get_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_set_kv_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + SetKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + SetKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.set_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_delete_kv_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + DeleteKvRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + DeleteKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_kv_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn stack_get_cluster_identity_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -4823,6 +5165,30 @@ impl MetastoreServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let get_kv_svc = self + .get_kv_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let set_kv_svc = self + .set_kv_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let delete_kv_svc = self + .delete_kv_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let get_cluster_identity_svc = self .get_cluster_identity_layers .into_iter() @@ -4866,6 +5232,9 @@ impl MetastoreServiceTowerLayerStack { find_index_template_matches_svc, list_index_templates_svc, delete_index_templates_svc, + get_kv_svc, + set_kv_svc, + delete_kv_svc, get_cluster_identity_svc, }; MetastoreServiceClient::new(tower_svc_stack) @@ -5165,6 +5534,24 @@ where Error = crate::metastore::MetastoreError, Future = BoxFuture, > + + tower::Service< + GetKvRequest, + Response = GetKvResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + SetKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + DeleteKvRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + tower::Service< GetClusterIdentityRequest, Response = GetClusterIdentityResponse, @@ -5373,6 +5760,24 @@ where ) -> crate::metastore::MetastoreResult { self.clone().call(request).await } + async fn get_kv( + &self, + request: GetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn set_kv( + &self, + request: SetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn delete_kv( + &self, + request: DeleteKvRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, @@ -5896,6 +6301,48 @@ where DeleteIndexTemplatesRequest::rpc_name(), )) } + async fn get_kv( + &self, + request: GetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .get_kv(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + GetKvRequest::rpc_name(), + )) + } + async fn set_kv( + &self, + request: SetKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .set_kv(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + SetKvRequest::rpc_name(), + )) + } + async fn delete_kv( + &self, + request: DeleteKvRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .delete_kv(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + DeleteKvRequest::rpc_name(), + )) + } async fn get_cluster_identity( &self, request: GetClusterIdentityRequest, @@ -6311,6 +6758,39 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn get_kv( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .get_kv(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn set_kv( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .set_kv(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn delete_kv( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .delete_kv(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } async fn get_cluster_identity( &self, request: tonic::Request, @@ -7405,6 +7885,74 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Gets a value by key from the cluster-wide key-value store. + pub async fn get_kv( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.metastore.MetastoreService/GetKv", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("quickwit.metastore.MetastoreService", "GetKv")); + self.inner.unary(req, path, codec).await + } + /// Sets a key-value pair in the cluster-wide key-value store. + pub async fn set_kv( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.metastore.MetastoreService/SetKv", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("quickwit.metastore.MetastoreService", "SetKv")); + self.inner.unary(req, path, codec).await + } + /// Deletes a key from the cluster-wide key-value store. + pub async fn delete_kv( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.metastore.MetastoreService/DeleteKv", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new("quickwit.metastore.MetastoreService", "DeleteKv"), + ); + self.inner.unary(req, path, codec).await + } /// Get cluster identity pub async fn get_cluster_identity( &mut self, @@ -7691,6 +8239,21 @@ pub mod metastore_service_grpc_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; + /// Gets a value by key from the cluster-wide key-value store. + async fn get_kv( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Sets a key-value pair in the cluster-wide key-value store. + async fn set_kv( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Deletes a key from the cluster-wide key-value store. + async fn delete_kv( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; /// Get cluster identity async fn get_cluster_identity( &self, @@ -9407,6 +9970,140 @@ pub mod metastore_service_grpc_server { }; Box::pin(fut) } + "/quickwit.metastore.MetastoreService/GetKv" => { + #[allow(non_camel_case_types)] + struct GetKvSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService for GetKvSvc { + type Response = super::GetKvResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::get_kv(&inner, request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = GetKvSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.metastore.MetastoreService/SetKv" => { + #[allow(non_camel_case_types)] + struct SetKvSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService for SetKvSvc { + type Response = super::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::set_kv(&inner, request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = SetKvSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.metastore.MetastoreService/DeleteKv" => { + #[allow(non_camel_case_types)] + struct DeleteKvSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for DeleteKvSvc { + type Response = super::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::delete_kv(&inner, request) + .await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = DeleteKvSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/quickwit.metastore.MetastoreService/GetClusterIdentity" => { #[allow(non_camel_case_types)] struct GetClusterIdentitySvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/control_plane/mod.rs b/quickwit/quickwit-proto/src/control_plane/mod.rs index 32df01062b7..fd278bc6199 100644 --- a/quickwit/quickwit-proto/src/control_plane/mod.rs +++ b/quickwit/quickwit-proto/src/control_plane/mod.rs @@ -160,6 +160,24 @@ impl GetOrCreateOpenShardsFailureReason { } } +impl RpcName for EnableMaintenanceModeRequest { + fn rpc_name() -> &'static str { + "enable_maintenance_mode" + } +} + +impl RpcName for DisableMaintenanceModeRequest { + fn rpc_name() -> &'static str { + "disable_maintenance_mode" + } +} + +impl RpcName for GetMaintenanceModeRequest { + fn rpc_name() -> &'static str { + "get_maintenance_mode" + } +} + impl From for GetOrCreateOpenShardsSubrequest { fn from(metastore_open_shard_subrequest: OpenShardSubrequest) -> Self { let index_id = metastore_open_shard_subrequest.index_uid().index_id.clone(); diff --git a/quickwit/quickwit-rest-client/src/rest_client.rs b/quickwit/quickwit-rest-client/src/rest_client.rs index de7e05106f3..ab51a3fd1dd 100644 --- a/quickwit/quickwit-rest-client/src/rest_client.rs +++ b/quickwit/quickwit-rest-client/src/rest_client.rs @@ -293,6 +293,10 @@ impl QuickwitClient { ClusterClient::new(&self.transport, self.timeout) } + pub fn maintenance(&self) -> MaintenanceClient<'_> { + MaintenanceClient::new(&self.transport, self.timeout) + } + pub fn node_stats(&self) -> NodeStatsClient<'_> { NodeStatsClient::new(&self.transport, self.timeout) } @@ -781,6 +785,79 @@ impl<'a> NodeHealthClient<'a> { } } +/// Response from the maintenance status endpoint. +#[derive(Debug, serde::Deserialize)] +pub struct MaintenanceStatusResponse { + pub is_maintenance_mode: bool, + pub enabled_at: Option, +} + +/// Response from the enable maintenance endpoint. +#[derive(Debug, serde::Deserialize)] +pub struct EnableMaintenanceResponse { + pub frozen_plan_json: String, +} + +/// Client for maintenance mode APIs. +pub struct MaintenanceClient<'a> { + transport: &'a Transport, + timeout: Timeout, +} + +impl<'a> MaintenanceClient<'a> { + fn new(transport: &'a Transport, timeout: Timeout) -> Self { + Self { transport, timeout } + } + + pub async fn status(&self) -> Result { + let response = self + .transport + .send::<()>( + Method::GET, + "cluster/maintenance", + None, + None, + None, + self.timeout, + ) + .await?; + let status = response.deserialize().await?; + Ok(status) + } + + pub async fn enable(&self) -> Result { + let response = self + .transport + .send::<()>( + Method::PUT, + "cluster/maintenance", + None, + None, + None, + self.timeout, + ) + .await?; + let result = response.deserialize().await?; + Ok(result) + } + + pub async fn disable(&self) -> Result<(), Error> { + let response = self + .transport + .send::<()>( + Method::DELETE, + "cluster/maintenance", + None, + None, + None, + self.timeout, + ) + .await?; + response.check().await?; + Ok(()) + } +} + fn header_from_config_format(config_format: ConfigFormat) -> HeaderMap { let mut header_map = HeaderMap::new(); let content_type_value = format!("application/{}", config_format.as_str()); diff --git a/quickwit/quickwit-serve/src/cluster_api/rest_handler.rs b/quickwit/quickwit-serve/src/cluster_api/rest_handler.rs index f38ddac1627..071122f799d 100644 --- a/quickwit/quickwit-serve/src/cluster_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/cluster_api/rest_handler.rs @@ -15,6 +15,11 @@ use std::convert::Infallible; use quickwit_cluster::{Cluster, ClusterSnapshot, NodeIdSchema}; +use quickwit_proto::control_plane::{ + ControlPlaneError, ControlPlaneService, ControlPlaneServiceClient, + DisableMaintenanceModeRequest, EnableMaintenanceModeRequest, EnableMaintenanceModeResponse, + GetMaintenanceModeRequest, GetMaintenanceModeResponse, +}; use warp::{Filter, Rejection}; use crate::format::extract_format_from_qs; @@ -23,22 +28,39 @@ use crate::rest_api_response::into_rest_api_response; #[derive(utoipa::OpenApi)] #[openapi( - paths(get_cluster), - components(schemas(ClusterSnapshot, NodeIdSchema,)) + paths( + get_cluster, + get_maintenance_endpoint, + enable_maintenance_endpoint, + disable_maintenance_endpoint + ), + components(schemas( + ClusterSnapshot, + NodeIdSchema, + GetMaintenanceModeResponse, + EnableMaintenanceModeResponse + )) )] pub struct ClusterApi; /// Cluster handler. pub fn cluster_handler( cluster: Cluster, + control_plane_client: ControlPlaneServiceClient, ) -> impl Filter + Clone { - warp::path!("cluster") + let cluster_info_handler = warp::path!("cluster") .and(warp::path::end()) .and(warp::get()) .and(warp::path::end().map(move || cluster.clone())) .then(get_cluster) .and(extract_format_from_qs()) .map(into_rest_api_response) + .boxed(); + + let maintenance_routes = maintenance_handler(control_plane_client); + + cluster_info_handler + .or(maintenance_routes) .recover(recover_fn) .boxed() } @@ -57,3 +79,100 @@ async fn get_cluster(cluster: Cluster) -> Result { let snapshot = cluster.snapshot().await; Ok(snapshot) } + +#[utoipa::path( + get, + tag = "Cluster Info", + path = "/cluster/maintenance", + responses( + (status = 200, description = "Successfully fetched maintenance mode status.", body = GetMaintenanceModeResponse) + ) +)] +async fn get_maintenance_endpoint( + control_plane_client: ControlPlaneServiceClient, +) -> Result { + control_plane_client + .get_maintenance_mode(GetMaintenanceModeRequest {}) + .await +} + +#[utoipa::path( + put, + tag = "Cluster Info", + path = "/cluster/maintenance", + responses( + (status = 200, description = "Successfully enabled maintenance mode.", body = EnableMaintenanceModeResponse) + ) +)] +async fn enable_maintenance_endpoint( + control_plane_client: ControlPlaneServiceClient, +) -> Result { + control_plane_client + .enable_maintenance_mode(EnableMaintenanceModeRequest {}) + .await +} + +#[utoipa::path( + delete, + tag = "Cluster Info", + path = "/cluster/maintenance", + responses( + (status = 200, description = "Successfully disabled maintenance mode.") + ) +)] +async fn disable_maintenance_endpoint( + control_plane_client: ControlPlaneServiceClient, +) -> Result<(), ControlPlaneError> { + control_plane_client + .disable_maintenance_mode(DisableMaintenanceModeRequest {}) + .await?; + Ok(()) +} + +fn maintenance_get_filter() -> impl Filter + Clone { + warp::path!("cluster" / "maintenance").and(warp::get()) +} + +fn maintenance_put_filter() -> impl Filter + Clone { + warp::path!("cluster" / "maintenance").and(warp::put()) +} + +fn maintenance_delete_filter() -> impl Filter + Clone { + warp::path!("cluster" / "maintenance").and(warp::delete()) +} + +/// Maintenance mode endpoints handler. +/// +/// - `GET /api/v1/cluster/maintenance` — get maintenance status +/// - `PUT /api/v1/cluster/maintenance` — enable maintenance mode +/// - `DELETE /api/v1/cluster/maintenance` — disable maintenance mode +fn maintenance_handler( + control_plane_client: ControlPlaneServiceClient, +) -> impl Filter + Clone { + let get_client = control_plane_client.clone(); + let put_client = control_plane_client.clone(); + let delete_client = control_plane_client; + + let get_handler = maintenance_get_filter() + .and(warp::any().map(move || get_client.clone())) + .then(get_maintenance_endpoint) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed(); + + let put_handler = maintenance_put_filter() + .and(warp::any().map(move || put_client.clone())) + .then(enable_maintenance_endpoint) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed(); + + let delete_handler = maintenance_delete_filter() + .and(warp::any().map(move || delete_client.clone())) + .then(disable_maintenance_endpoint) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed(); + + get_handler.or(put_handler).or(delete_handler).boxed() +} diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 71fc69b0ad9..eaeaa8d76af 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -28,6 +28,7 @@ mod indexing_api; mod ingest_api; mod jaeger_api; mod load_shield; + mod metrics; mod metrics_api; mod node_info_handler; diff --git a/quickwit/quickwit-serve/src/rest.rs b/quickwit/quickwit-serve/src/rest.rs index 5c00fda8c47..96c0f03fefd 100644 --- a/quickwit/quickwit-serve/src/rest.rs +++ b/quickwit/quickwit-serve/src/rest.rs @@ -400,7 +400,10 @@ fn api_v1_routes( !disable_ingest_v1(), enable_ingest_v2(), ) - .or(cluster_handler(quickwit_services.cluster.clone())) + .or(cluster_handler( + quickwit_services.cluster.clone(), + quickwit_services.control_plane_client.clone(), + )) .boxed() .or(node_info_handler( BuildInfo::get(),