diff --git a/.env.example b/.env.example index ac61ea6af64..72668701919 100644 --- a/.env.example +++ b/.env.example @@ -3,8 +3,14 @@ POSTGRES_USER=quickwit-dev POSTGRES_PASSWORD=quickwit-dev POSTGRES_DB=quickwit-metastore-dev +# Change the user name and password of the mysql database +MYSQL_USER=quickwit-dev +MYSQL_PASSWORD=quickwit-dev +MYSQL_DB=quickwit-metastore-dev + # Update all services to the latest versions LOCALSTACK_VERSION=latest +MYSQL_VERSION=latest POSTGRES_VERSION=latest PULSAR_VERSION=latest CP_VERSION=latest @@ -23,4 +29,5 @@ MAP_HOST_AZURITE=0.0.0.0 MAP_HOST_GRAFANA=0.0.0.0 MAP_HOST_JAEGER=0.0.0.0 MAP_HOST_OTEL=0.0.0.0 +MAP_HOST_MYSQL=0.0.0.0 MAP_HOST_PROMETHEUS=0.0.0.0 diff --git a/Makefile b/Makefile index 4feb8035234..b607ab5d6ea 100644 --- a/Makefile +++ b/Makefile @@ -38,8 +38,11 @@ docker-compose-monitoring: docker-rm-postgres-volume: docker volume rm quickwit_postgres_data +docker-rm-mysql-volume: + docker volume rm quickwit_mysql_data + docker-rm-volumes: - docker volume rm quickwit_azurite_data quickwit_fake_gcs_server_data quickwit_grafana_conf quickwit_grafana_data quickwit_localstack_data quickwit_postgres_data + docker volume rm quickwit_azurite_data quickwit_fake_gcs_server_data quickwit_grafana_conf quickwit_grafana_data quickwit_localstack_data quickwit_mysql_data quickwit_postgres_data doc: @$(MAKE) -C $(QUICKWIT_SRC) doc diff --git a/docker-compose.yml b/docker-compose.yml index 24f5ed29b47..144fc1a8acd 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -69,6 +69,33 @@ services: timeout: 5s retries: 100 + mysql: + # MySQL 8.0 is the minimum supported version. + image: mysql:${MYSQL_VERSION:-8.0} + container_name: mysql + ports: + - "${MAP_HOST_MYSQL:-127.0.0.1}:3307:3306" + profiles: + - all + - mysql + environment: + MYSQL_DATABASE: ${MYSQL_DB:-quickwit-metastore-dev} + MYSQL_USER: ${MYSQL_USER:-quickwit-dev} + MYSQL_PASSWORD: ${MYSQL_PASSWORD:-quickwit-dev} + MYSQL_ROOT_PASSWORD: root + TZ: UTC + command: > + --default-authentication-plugin=mysql_native_password + --default-time-zone=+00:00 + --sql-mode=STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION + volumes: + - mysql_data:/var/lib/mysql + healthcheck: + test: ["CMD", "mysqladmin", "ping", "-h", "localhost", "-u", "quickwit-dev", "-pquickwit-dev"] + interval: 1s + timeout: 5s + retries: 100 + pulsar-broker: # The oldest version with arm64 docker images. EOL May 2 2025 image: apachepulsar/pulsar:${PULSAR_VERSION:-3.0.0} @@ -230,4 +257,5 @@ volumes: grafana_conf: grafana_data: localstack_data: + mysql_data: postgres_data: diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index fae390095d9..74ae2afff8d 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7172,6 +7172,9 @@ version = "0.8.0" dependencies = [ "anyhow", "async-trait", + "aws-config", + "aws-credential-types", + "aws-sigv4", "bytes", "bytesize", "dotenvy", @@ -7208,6 +7211,7 @@ dependencies = [ "tracing", "tracing-subscriber", "ulid", + "url", "utoipa", "uuid", ] diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 2b61b579fbf..1dfad07f753 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -217,6 +217,7 @@ rustls-pemfile = "2.2" sea-query = { version = "0.32" } sea-query-binder = { version = "0.7", features = [ "runtime-tokio-rustls", + "sqlx-mysql", "sqlx-postgres", ] } # ^1.0.184 due to serde-rs/serde#2538 @@ -232,6 +233,7 @@ siphasher = "1.0" smallvec = "1" sqlx = { version = "0.8", features = [ "migrate", + "mysql", "postgres", "runtime-tokio-rustls", "time", @@ -290,6 +292,7 @@ tracing-subscriber = { version = "0.3", features = [ ttl_cache = "0.5" typetag = "0.2" ulid = "1.2" +url = "2.5" ureq = "3" username = "0.2" # We cannot upgrade to utoipa 5.0+ due to significant breaking changes: @@ -308,6 +311,7 @@ wiremock = "0.6" zstd = { version = "0.13", default-features = false } aws-config = "1.8" +aws-sigv4 = "1.3" aws-credential-types = { version = "1.2", features = ["hardcoded-credentials"] } aws-runtime = "1.5" aws-sdk-kinesis = "1.97" diff --git a/quickwit/Makefile b/quickwit/Makefile index f06d320dbaa..cca28ca6a5b 100644 --- a/quickwit/Makefile +++ b/quickwit/Makefile @@ -39,6 +39,7 @@ test-all: QW_S3_ENDPOINT=http://localhost:4566 \ QW_S3_FORCE_PATH_STYLE_ACCESS=1 \ QW_TEST_DATABASE_URL=postgres://quickwit-dev:quickwit-dev@localhost:5432/quickwit-metastore-dev \ + QW_TEST_MYSQL_DATABASE_URL=mysql://quickwit-dev:quickwit-dev@localhost:3307/quickwit-metastore-dev \ RUST_MIN_STACK=67108864 \ cargo nextest run --all-features --retries 5 cargo nextest run --test failpoints --features fail/failpoints diff --git a/quickwit/quickwit-cli/Cargo.toml b/quickwit/quickwit-cli/Cargo.toml index 5d9dc955107..da5441d1dd1 100644 --- a/quickwit/quickwit-cli/Cargo.toml +++ b/quickwit/quickwit-cli/Cargo.toml @@ -105,6 +105,7 @@ release-feature-set = [ "quickwit-serve/lambda", "quickwit-storage/azure", "quickwit-storage/gcs", + "quickwit-metastore/mysql", "quickwit-metastore/postgres", ] release-feature-vendored-set = [ @@ -119,6 +120,7 @@ release-feature-vendored-set = [ "quickwit-serve/lambda", "quickwit-storage/azure", "quickwit-storage/gcs", + "quickwit-metastore/mysql", "quickwit-metastore/postgres", ] release-macos-feature-vendored-set = [ @@ -132,6 +134,7 @@ release-macos-feature-vendored-set = [ "quickwit-serve/lambda", "quickwit-storage/azure", "quickwit-storage/gcs", + "quickwit-metastore/mysql", "quickwit-metastore/postgres", ] release-jemalloc-profiled = [ diff --git a/quickwit/quickwit-common/src/uri.rs b/quickwit/quickwit-common/src/uri.rs index a094ae34ee8..b9775b87500 100644 --- a/quickwit/quickwit-common/src/uri.rs +++ b/quickwit/quickwit-common/src/uri.rs @@ -37,6 +37,7 @@ pub enum Protocol { Ram = 6, S3 = 7, Google = 8, + MySQL = 9, } impl Protocol { @@ -50,6 +51,7 @@ impl Protocol { Protocol::Ram => "ram", Protocol::S3 => "s3", Protocol::Google => "gs", + Protocol::MySQL => "mysql", } } @@ -66,7 +68,7 @@ impl Protocol { } pub fn is_database(&self) -> bool { - matches!(&self, Protocol::PostgreSQL) + matches!(&self, Protocol::PostgreSQL | Protocol::MySQL) } } @@ -86,6 +88,7 @@ impl FromStr for Protocol { "grpc" => Ok(Protocol::Grpc), "actor" => Ok(Protocol::Actor), "pg" | "postgres" | "postgresql" => Ok(Protocol::PostgreSQL), + "mysql" => Ok(Protocol::MySQL), "ram" => Ok(Protocol::Ram), "s3" => Ok(Protocol::S3), "gs" => Ok(Protocol::Google), @@ -199,7 +202,7 @@ impl Uri { /// Returns the last component of the URI. pub fn file_name(&self) -> Option<&Path> { - if self.protocol() == Protocol::PostgreSQL { + if self.protocol().is_database() { return None; } let path = self.path(); @@ -236,8 +239,8 @@ impl Uri { .join(path) .to_string_lossy() .to_string(), - Protocol::PostgreSQL => bail!( - "cannot join PostgreSQL URI `{}` with path `{:?}`", + Protocol::PostgreSQL | Protocol::MySQL => bail!( + "cannot join database URI `{}` with path `{:?}`", self.uri, path ), @@ -573,6 +576,9 @@ mod tests { Uri::for_test("postgres://username:password@localhost:5432/metastore") .join("table") .unwrap_err(); + Uri::for_test("mysql://username:password@localhost:3306/metastore") + .join("table") + .unwrap_err(); } #[test] @@ -589,6 +595,11 @@ mod tests { .parent() .is_none() ); + assert!( + Uri::for_test("mysql://localhost:3306/db") + .parent() + .is_none() + ); assert!(Uri::for_test("ram:///").parent().is_none()); assert_eq!(Uri::for_test("ram:///foo").parent().unwrap(), "ram:///"); @@ -681,6 +692,11 @@ mod tests { .file_name() .is_none() ); + assert!( + Uri::for_test("mysql://localhost:3306/db") + .file_name() + .is_none() + ); assert!(Uri::for_test("ram:///").file_name().is_none()); assert_eq!( @@ -804,6 +820,17 @@ mod tests { ); } } + assert_eq!( + Uri::for_test("mysql://localhost:3306/metastore").as_redacted_str(), + "mysql://localhost:3306/metastore" + ); + { + let uri = Uri::from_str("mysql://username:password@localhost:3306/metastore").unwrap(); + let expected_uri = + "mysql://username:***redacted***@localhost:3306/metastore".to_string(); + assert_eq!(uri.as_redacted_str(), expected_uri); + assert_eq!(format!("{uri}"), expected_uri); + } } #[test] diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 22cdb2538b4..1dc7efe9ca9 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -70,7 +70,8 @@ use crate::merge_policy_config::{ ConstWriteAmplificationMergePolicyConfig, MergePolicyConfig, StableLogMergePolicyConfig, }; pub use crate::metastore_config::{ - MetastoreBackend, MetastoreConfig, MetastoreConfigs, PostgresMetastoreConfig, + MetastoreBackend, MetastoreConfig, MetastoreConfigs, MysqlAuthMode, MysqlMetastoreConfig, + PostgresMetastoreConfig, }; pub use crate::node_config::{ CacheConfig, CachePolicy, DEFAULT_QW_CONFIG_PATH, GrpcConfig, IndexerConfig, IngestApiConfig, diff --git a/quickwit/quickwit-config/src/metastore_config.rs b/quickwit/quickwit-config/src/metastore_config.rs index 47a5ef5f917..fb629cffded 100644 --- a/quickwit/quickwit-config/src/metastore_config.rs +++ b/quickwit/quickwit-config/src/metastore_config.rs @@ -19,15 +19,26 @@ use std::time::Duration; use anyhow::{Context, ensure}; use humantime::parse_duration; use itertools::Itertools; +use quickwit_common::uri::Protocol; use serde::{Deserialize, Serialize}; use serde_with::{EnumMap, serde_as}; +#[derive(Debug, Clone, Default, Eq, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "snake_case")] +pub enum MysqlAuthMode { + #[default] + Password, + AwsIam, +} + #[derive(Debug, Clone, Copy, Eq, PartialEq, Ord, PartialOrd, Hash, Serialize, Deserialize)] #[serde(rename_all = "snake_case")] pub enum MetastoreBackend { File, #[serde(alias = "pg", alias = "postgres")] PostgreSQL, + #[serde(alias = "mysql")] + MySQL, } /// Holds the metastore configurations defined in the `metastore` section of node config files. @@ -90,6 +101,25 @@ impl MetastoreConfigs { _ => None, }) } + + pub fn find_mysql(&self) -> Option<&MysqlMetastoreConfig> { + self.0 + .iter() + .find_map(|metastore_config| match metastore_config { + MetastoreConfig::MySQL(mysql_metastore_config) => Some(mysql_metastore_config), + _ => None, + }) + } + + /// Returns the `max_connections` for the database backend matching the given protocol, + /// or `None` if the protocol is not a database or no matching config is found. + pub fn find_database_max_connections(&self, protocol: Protocol) -> Option { + match protocol { + Protocol::PostgreSQL => self.find_postgres().map(|config| config.max_connections), + Protocol::MySQL => self.find_mysql().map(|config| config.max_connections), + _ => None, + } + } } impl Deref for MetastoreConfigs { @@ -106,6 +136,8 @@ pub enum MetastoreConfig { File(FileMetastoreConfig), #[serde(alias = "pg", alias = "postgres")] PostgreSQL(PostgresMetastoreConfig), + #[serde(alias = "mysql")] + MySQL(MysqlMetastoreConfig), } impl MetastoreConfig { @@ -113,6 +145,7 @@ impl MetastoreConfig { match self { Self::File(_) => MetastoreBackend::File, Self::PostgreSQL(_) => MetastoreBackend::PostgreSQL, + Self::MySQL(_) => MetastoreBackend::MySQL, } } @@ -130,6 +163,13 @@ impl MetastoreConfig { } } + pub fn as_mysql(&self) -> Option<&MysqlMetastoreConfig> { + match self { + Self::MySQL(mysql_metastore_config) => Some(mysql_metastore_config), + _ => None, + } + } + pub fn redact(&mut self) { // TODO: Implement this method when we end up storing secrets in the // metastore config. @@ -139,6 +179,7 @@ impl MetastoreConfig { match self { Self::File(file_metastore_config) => file_metastore_config.validate()?, Self::PostgreSQL(postgres_metastore_config) => postgres_metastore_config.validate()?, + Self::MySQL(mysql_metastore_config) => mysql_metastore_config.validate()?, } Ok(()) } @@ -156,6 +197,12 @@ impl From for MetastoreConfig { } } +impl From for MetastoreConfig { + fn from(mysql_metastore_config: MysqlMetastoreConfig) -> Self { + Self::MySQL(mysql_metastore_config) + } +} + #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)] #[serde(deny_unknown_fields)] pub struct PostgresMetastoreConfig { @@ -274,6 +321,121 @@ impl FileMetastoreConfig { } } +#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)] +#[serde(deny_unknown_fields)] +pub struct MysqlMetastoreConfig { + #[serde(default = "MysqlMetastoreConfig::default_min_connections")] + pub min_connections: usize, + #[serde( + alias = "max_num_connections", + default = "MysqlMetastoreConfig::default_max_connections" + )] + pub max_connections: NonZeroUsize, + #[serde(default = "MysqlMetastoreConfig::default_acquire_connection_timeout")] + pub acquire_connection_timeout: String, + #[serde(default = "MysqlMetastoreConfig::default_idle_connection_timeout")] + pub idle_connection_timeout: String, + #[serde(default = "MysqlMetastoreConfig::default_max_connection_lifetime")] + pub max_connection_lifetime: String, + #[serde(default)] + pub auth_mode: MysqlAuthMode, + /// AWS region for IAM token signing. Auto-detected from the AWS config chain if omitted. + #[serde(default)] + pub aws_region: Option, +} + +impl Default for MysqlMetastoreConfig { + fn default() -> Self { + Self { + min_connections: Self::default_min_connections(), + max_connections: Self::default_max_connections(), + acquire_connection_timeout: Self::default_acquire_connection_timeout(), + idle_connection_timeout: Self::default_idle_connection_timeout(), + max_connection_lifetime: Self::default_max_connection_lifetime(), + auth_mode: MysqlAuthMode::default(), + aws_region: None, + } + } +} + +impl MysqlMetastoreConfig { + pub fn default_min_connections() -> usize { + 0 + } + + pub fn default_max_connections() -> NonZeroUsize { + NonZeroUsize::new(10).unwrap() + } + + pub fn default_acquire_connection_timeout() -> String { + "10s".to_string() + } + + pub fn default_idle_connection_timeout() -> String { + "10min".to_string() + } + + pub fn default_max_connection_lifetime() -> String { + "30min".to_string() + } + + pub fn acquire_connection_timeout(&self) -> anyhow::Result { + parse_duration(&self.acquire_connection_timeout).with_context(|| { + format!( + "failed to parse `acquire_connection_timeout` value `{}`", + self.acquire_connection_timeout + ) + }) + } + + pub fn idle_connection_timeout_opt(&self) -> anyhow::Result> { + if self.idle_connection_timeout.is_empty() || self.idle_connection_timeout == "0" { + return Ok(None); + } + let idle_connection_timeout = + parse_duration(&self.idle_connection_timeout).with_context(|| { + format!( + "failed to parse `idle_connection_timeout` value `{}`", + self.idle_connection_timeout + ) + })?; + if idle_connection_timeout.is_zero() { + Ok(None) + } else { + Ok(Some(idle_connection_timeout)) + } + } + + pub fn max_connection_lifetime_opt(&self) -> anyhow::Result> { + if self.max_connection_lifetime.is_empty() || self.max_connection_lifetime == "0" { + return Ok(None); + } + let max_connection_lifetime = + parse_duration(&self.max_connection_lifetime).with_context(|| { + format!( + "failed to parse `max_connection_lifetime` value `{}`", + self.max_connection_lifetime + ) + })?; + if max_connection_lifetime.is_zero() { + Ok(None) + } else { + Ok(Some(max_connection_lifetime)) + } + } + + pub fn validate(&self) -> anyhow::Result<()> { + ensure!( + self.min_connections <= self.max_connections.get(), + "`min_connections` must be less than or equal to `max_connections`" + ); + self.acquire_connection_timeout()?; + self.idle_connection_timeout_opt()?; + self.max_connection_lifetime_opt()?; + Ok(()) + } +} + #[cfg(test)] mod tests { use super::*; @@ -420,4 +582,149 @@ mod tests { ); } } + + #[test] + fn test_mysql_metastore_config_default_serde() { + let mysql_config_yaml = ""; + let mysql_config: MysqlMetastoreConfig = serde_yaml::from_str(mysql_config_yaml).unwrap(); + assert_eq!(mysql_config, MysqlMetastoreConfig::default()); + assert_eq!(mysql_config.auth_mode, MysqlAuthMode::Password); + assert!(mysql_config.aws_region.is_none()); + } + + #[test] + fn test_mysql_metastore_config_with_auth_mode_password() { + let mysql_config_yaml = r#" + auth_mode: password + max_connections: 5 + "#; + let mysql_config: MysqlMetastoreConfig = serde_yaml::from_str(mysql_config_yaml).unwrap(); + assert_eq!(mysql_config.auth_mode, MysqlAuthMode::Password); + assert_eq!(mysql_config.max_connections, NonZeroUsize::new(5).unwrap()); + assert!(mysql_config.aws_region.is_none()); + } + + #[test] + fn test_mysql_metastore_config_with_auth_mode_aws_iam() { + let mysql_config_yaml = r#" + auth_mode: aws_iam + aws_region: us-east-1 + "#; + let mysql_config: MysqlMetastoreConfig = serde_yaml::from_str(mysql_config_yaml).unwrap(); + assert_eq!(mysql_config.auth_mode, MysqlAuthMode::AwsIam); + assert_eq!(mysql_config.aws_region.as_deref(), Some("us-east-1")); + } + + #[test] + fn test_mysql_metastore_config_aws_iam_without_region() { + let mysql_config_yaml = r#" + auth_mode: aws_iam + "#; + let mysql_config: MysqlMetastoreConfig = serde_yaml::from_str(mysql_config_yaml).unwrap(); + assert_eq!(mysql_config.auth_mode, MysqlAuthMode::AwsIam); + assert!(mysql_config.aws_region.is_none()); + } + + #[test] + fn test_mysql_metastore_config_unknown_field_rejected() { + let mysql_config_yaml = r#" + auth_mode: password + unknown_field: true + "#; + let result: Result = serde_yaml::from_str(mysql_config_yaml); + assert!(result.is_err()); + } + + #[test] + fn test_mysql_metastore_config_validate() { + let valid_config = MysqlMetastoreConfig::default(); + valid_config.validate().unwrap(); + + let invalid_config = MysqlMetastoreConfig { + min_connections: 20, + max_connections: NonZeroUsize::new(5).unwrap(), + ..Default::default() + }; + let error = invalid_config.validate().unwrap_err(); + assert!(error.to_string().contains("min_connections")); + } + + #[test] + fn test_mysql_metastore_config_in_metastore_configs() { + let metastore_configs_yaml = r#" + mysql: + auth_mode: aws_iam + aws_region: eu-west-1 + max_connections: 8 + "#; + let metastore_configs: MetastoreConfigs = + serde_yaml::from_str(metastore_configs_yaml).unwrap(); + let mysql_config = metastore_configs.find_mysql().unwrap(); + assert_eq!(mysql_config.auth_mode, MysqlAuthMode::AwsIam); + assert_eq!(mysql_config.aws_region.as_deref(), Some("eu-west-1")); + assert_eq!(mysql_config.max_connections, NonZeroUsize::new(8).unwrap()); + } + + #[test] + fn test_mysql_auth_mode_serde_roundtrip() { + let password_yaml = serde_yaml::to_string(&MysqlAuthMode::Password).unwrap(); + assert!(password_yaml.contains("password")); + + let aws_iam_yaml = serde_yaml::to_string(&MysqlAuthMode::AwsIam).unwrap(); + assert!(aws_iam_yaml.contains("aws_iam")); + + let roundtripped: MysqlAuthMode = serde_yaml::from_str(&aws_iam_yaml).unwrap(); + assert_eq!(roundtripped, MysqlAuthMode::AwsIam); + } + + #[test] + fn test_find_database_max_connections_postgres() { + let configs_yaml = r#" + postgres: + max_connections: 20 + "#; + let configs: MetastoreConfigs = serde_yaml::from_str(configs_yaml).unwrap(); + let max_conn = configs + .find_database_max_connections(Protocol::PostgreSQL) + .unwrap(); + assert_eq!(max_conn.get(), 20); + } + + #[test] + fn test_find_database_max_connections_mysql() { + let configs_yaml = r#" + mysql: + max_connections: 15 + "#; + let configs: MetastoreConfigs = serde_yaml::from_str(configs_yaml).unwrap(); + let max_conn = configs + .find_database_max_connections(Protocol::MySQL) + .unwrap(); + assert_eq!(max_conn.get(), 15); + } + + #[test] + fn test_find_database_max_connections_file_returns_none() { + let configs: MetastoreConfigs = serde_yaml::from_str("").unwrap(); + assert!( + configs + .find_database_max_connections(Protocol::File) + .is_none() + ); + } + + #[test] + fn test_find_database_max_connections_missing_config_returns_none() { + let configs: MetastoreConfigs = serde_yaml::from_str("").unwrap(); + assert!( + configs + .find_database_max_connections(Protocol::PostgreSQL) + .is_none() + ); + assert!( + configs + .find_database_max_connections(Protocol::MySQL) + .is_none() + ); + } } diff --git a/quickwit/quickwit-metastore/Cargo.toml b/quickwit/quickwit-metastore/Cargo.toml index 877d18bc6ea..d715cc82d89 100644 --- a/quickwit/quickwit-metastore/Cargo.toml +++ b/quickwit/quickwit-metastore/Cargo.toml @@ -12,6 +12,9 @@ license.workspace = true [dependencies] anyhow = { workspace = true } async-trait = { workspace = true } +aws-config = { workspace = true, optional = true } +aws-credential-types = { workspace = true, optional = true } +aws-sigv4 = { workspace = true, optional = true } bytes = { workspace = true } bytesize = { workspace = true } futures = { workspace = true } @@ -37,6 +40,7 @@ tokio-stream = { workspace = true } tower = { workspace = true } tracing = { workspace = true } ulid = { workspace = true, features = ["serde"] } +url = { workspace = true, optional = true } uuid = { workspace = true } utoipa = { workspace = true } @@ -66,5 +70,6 @@ quickwit-storage = { workspace = true, features = ["testsuite"] } [features] ci-test = [] +mysql = ["quickwit-proto/mysql", "sea-query", "sea-query-binder", "sqlx", "aws-config", "aws-credential-types", "aws-sigv4", "url"] postgres = ["quickwit-proto/postgres", "sea-query", "sea-query-binder", "sqlx"] testsuite = ["mockall", "tempfile", "quickwit-config/testsuite"] diff --git a/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.down.sql b/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.down.sql new file mode 100644 index 00000000000..28e071b4a3b --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.down.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS kv; +DROP TABLE IF EXISTS index_templates; +DROP TABLE IF EXISTS shards; +DROP TABLE IF EXISTS delete_tasks; +DROP TABLE IF EXISTS splits; +DROP TABLE IF EXISTS indexes; diff --git a/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.up.sql b/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.up.sql new file mode 100644 index 00000000000..bfc58716d51 --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/mysql/1_initial-schema.up.sql @@ -0,0 +1,81 @@ +-- indexes table +CREATE TABLE indexes ( + index_uid VARCHAR(282) NOT NULL, + index_id VARCHAR(255) NOT NULL, + index_metadata_json TEXT NOT NULL, + create_timestamp DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY (index_uid) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; + +CREATE UNIQUE INDEX indexes_index_id_unique ON indexes (index_id); + +-- splits table +CREATE TABLE splits ( + index_uid VARCHAR(282) NOT NULL, + split_id VARCHAR(50) NOT NULL, + split_state VARCHAR(30) NOT NULL, + time_range_start BIGINT NULL, + time_range_end BIGINT NULL, + tags JSON NOT NULL, + split_metadata_json TEXT NOT NULL, + create_timestamp DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + update_timestamp DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + publish_timestamp DATETIME NULL, + delete_opstamp BIGINT NOT NULL DEFAULT 0 CHECK (delete_opstamp >= 0), + maturity_timestamp DATETIME NOT NULL DEFAULT '1970-01-01 00:00:00', + node_id VARCHAR(253) NOT NULL, + split_size_bytes BIGINT GENERATED ALWAYS AS ( + CAST(JSON_UNQUOTE(JSON_EXTRACT(split_metadata_json, '$.footer_offsets.end')) AS SIGNED) + ) STORED, + PRIMARY KEY (index_uid, split_id), + FOREIGN KEY (index_uid) REFERENCES indexes (index_uid) ON DELETE CASCADE +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; + +CREATE INDEX splits_time_range_start_idx ON splits (time_range_start); +CREATE INDEX splits_time_range_end_idx ON splits (time_range_end); +CREATE INDEX splits_node_id_idx ON splits (node_id); +CREATE INDEX idx_splits_stats ON splits (index_uid, split_state, split_size_bytes); +CREATE INDEX idx_splits_tags ON splits ((CAST(tags->'$' AS CHAR(128) ARRAY))); + +-- delete_tasks table +CREATE TABLE delete_tasks ( + create_timestamp DATETIME NOT NULL DEFAULT CURRENT_TIMESTAMP, + opstamp BIGINT NOT NULL AUTO_INCREMENT, + index_uid VARCHAR(282) NOT NULL, + delete_query_json TEXT NOT NULL, + PRIMARY KEY (opstamp), + FOREIGN KEY (index_uid) REFERENCES indexes (index_uid) ON DELETE CASCADE +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; + +-- shards table +CREATE TABLE shards ( + index_uid VARCHAR(282) NOT NULL, + source_id VARCHAR(128) NOT NULL, + shard_id VARCHAR(128) NOT NULL, + leader_id VARCHAR(255) NOT NULL, + follower_id VARCHAR(255) NULL, + shard_state ENUM('unspecified', 'open', 'unavailable', 'closed') NOT NULL DEFAULT 'open', + publish_position_inclusive VARCHAR(255) NOT NULL DEFAULT '', + publish_token VARCHAR(255) NULL, + doc_mapping_uid VARCHAR(26) NOT NULL, + update_timestamp DATETIME NOT NULL DEFAULT '2024-01-01 00:00:00', + PRIMARY KEY (index_uid, source_id, shard_id), + FOREIGN KEY (index_uid) REFERENCES indexes (index_uid) ON DELETE CASCADE +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; + +-- index_templates table +CREATE TABLE index_templates ( + template_id VARCHAR(255) NOT NULL, + positive_index_id_patterns JSON NOT NULL, + negative_index_id_patterns JSON NOT NULL, + priority INTEGER NOT NULL DEFAULT 0, + index_template_json TEXT NOT NULL, + PRIMARY KEY (template_id) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; + +-- kv table +CREATE TABLE kv ( + `key` VARCHAR(50) NOT NULL, + value TEXT NOT NULL, + PRIMARY KEY (`key`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index 9a29391e3db..97cd8504ceb 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -40,6 +40,8 @@ pub use error::MetastoreResolverError; pub use metastore::control_plane_metastore::ControlPlaneMetastore; pub use metastore::file_backed::FileBackedMetastore; pub(crate) use metastore::index_metadata::serialize::{IndexMetadataV0_8, VersionedIndexMetadata}; +#[cfg(feature = "mysql")] +pub use metastore::mysql::MysqlMetastore; #[cfg(feature = "postgres")] pub use metastore::postgres::PostgresqlMetastore; pub use metastore::{ diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index db50d097ce7..a8c3da665a4 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -14,6 +14,8 @@ pub mod file_backed; pub(crate) mod index_metadata; +#[cfg(feature = "mysql")] +pub mod mysql; #[cfg(feature = "postgres")] pub mod postgres; diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/auth.rs b/quickwit/quickwit-metastore/src/metastore/mysql/auth.rs new file mode 100644 index 00000000000..e0d47c824d3 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/auth.rs @@ -0,0 +1,214 @@ +// 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 std::time::{Duration, SystemTime, UNIX_EPOCH}; + +use aws_credential_types::provider::ProvideCredentials; +use aws_sigv4::http_request::{SignableBody, SignableRequest, SignatureLocation, SigningSettings}; +use aws_sigv4::sign::v4; +use quickwit_proto::metastore::{MetastoreError, MetastoreResult}; +use sqlx::MySql; +use sqlx::mysql::MySqlConnectOptions; + +use super::metrics::MYSQL_METRICS; +use super::pool::TrackedPool; + +/// Spawns a background task that refreshes the IAM auth token every 10 minutes. +/// RDS IAM tokens are valid for 15 minutes, so this provides a 5-minute buffer. +pub(super) fn spawn_token_refresh_task( + pool: TrackedPool, + base_connect_options: MySqlConnectOptions, + host: String, + port: u16, + user: String, + region: String, + aws_config: aws_config::SdkConfig, +) { + tokio::spawn(async move { + let mut interval = tokio::time::interval(Duration::from_secs(600)); + interval.tick().await; // skip immediate first tick + + loop { + interval.tick().await; + + match generate_rds_iam_token(&host, port, &user, ®ion, &aws_config).await { + Ok(token) => { + let new_options = base_connect_options + .clone() + .password(&token) + .enable_cleartext_plugin(true); + pool.set_connect_options(new_options); + MYSQL_METRICS.iam_token_refresh_success.inc(); + let now_secs = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap_or_default() + .as_secs() as i64; + MYSQL_METRICS + .iam_token_last_refresh_timestamp_secs + .set(now_secs); + tracing::debug!("refreshed RDS IAM auth token"); + } + Err(error) => { + MYSQL_METRICS.iam_token_refresh_failure.inc(); + quickwit_common::rate_limited_error!( + limit_per_min = 6, + error = ?error, + "failed to refresh RDS IAM auth token" + ); + } + } + } + }); +} + +/// Generates a SigV4-signed IAM authentication token for Aurora MySQL. +/// +/// The token is a presigned URL (with `https://` stripped) that Aurora +/// accepts as a password via the `mysql_clear_password` auth plugin. +/// Tokens are valid for 15 minutes. +pub(super) async fn generate_rds_iam_token( + host: &str, + port: u16, + user: &str, + region: &str, + aws_config: &aws_config::SdkConfig, +) -> MetastoreResult { + let credentials_provider = + aws_config + .credentials_provider() + .ok_or_else(|| MetastoreError::Internal { + message: "failed to generate RDS IAM token".to_string(), + cause: "no AWS credentials provider configured; ensure AWS credentials are \ + available via environment variables, instance profile, or IRSA" + .to_string(), + })?; + + let credentials = credentials_provider + .provide_credentials() + .await + .map_err(|err| MetastoreError::Internal { + message: "failed to resolve AWS credentials for RDS IAM token".to_string(), + cause: err.to_string(), + })?; + + let identity = credentials.into(); + + let mut settings = SigningSettings::default(); + settings.expires_in = Some(Duration::from_secs(900)); + settings.signature_location = SignatureLocation::QueryParams; + + let signing_params = v4::SigningParams::builder() + .identity(&identity) + .region(region) + .name("rds-db") + .time(SystemTime::now()) + .settings(settings) + .build() + .map_err(|err| MetastoreError::Internal { + message: "failed to build SigV4 signing params".to_string(), + cause: err.to_string(), + })?; + + let url = format!("https://{host}:{port}/?Action=connect&DBUser={user}"); + let signable = SignableRequest::new("GET", &url, std::iter::empty(), SignableBody::Bytes(&[])) + .map_err(|err| MetastoreError::Internal { + message: "failed to create signable request for RDS IAM token".to_string(), + cause: err.to_string(), + })?; + + let (instructions, _signature) = + aws_sigv4::http_request::sign(signable, &signing_params.into()) + .map_err(|err| MetastoreError::Internal { + message: "failed to sign RDS IAM token".to_string(), + cause: err.to_string(), + })? + .into_parts(); + + let mut signed_url = url::Url::parse(&url).expect("URL was already validated during signing"); + for (name, value) in instructions.params() { + signed_url.query_pairs_mut().append_pair(name, value); + } + + // Strip the "https://" prefix — the token is the host:port/?query… portion. + let token = signed_url + .as_str() + .strip_prefix("https://") + .unwrap_or(signed_url.as_str()); + Ok(token.to_string()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[tokio::test] + async fn test_generate_rds_iam_token_missing_credentials_provider() { + let aws_config = aws_config::SdkConfig::builder().build(); + let result = generate_rds_iam_token( + "my-db.us-east-1.rds.amazonaws.com", + 3306, + "admin", + "us-east-1", + &aws_config, + ) + .await; + let error = result.unwrap_err(); + assert!( + matches!(error, MetastoreError::Internal { .. }), + "expected MetastoreError::Internal, got: {error:?}" + ); + let MetastoreError::Internal { cause, .. } = &error else { + panic!("expected Internal error"); + }; + assert!( + cause.contains("no AWS credentials provider"), + "unexpected cause: {cause}" + ); + } + + // End-to-end IAM token generation requires real AWS credentials and a reachable + // RDS/Aurora endpoint. To test manually: + // + // DB_HOST=my-db.us-east-1.rds.amazonaws.com \ + // DB_USER=admin \ + // AWS_REGION=us-east-1 \ + // cargo test -p quickwit-metastore --features mysql test_generate_rds_iam_token_e2e -- + // --ignored + #[tokio::test] + #[ignore] + async fn test_generate_rds_iam_token_e2e() { + let host = std::env::var("DB_HOST").expect("DB_HOST must be set"); + let user = std::env::var("DB_USER").expect("DB_USER must be set"); + let region = std::env::var("AWS_REGION").expect("AWS_REGION must be set"); + + let aws_config = aws_config::defaults(aws_config::BehaviorVersion::latest()) + .region(aws_config::Region::new(region.clone())) + .load() + .await; + + let token = generate_rds_iam_token(&host, 3306, &user, ®ion, &aws_config) + .await + .expect("token generation should succeed with valid AWS credentials"); + + assert!(!token.is_empty(), "token should not be empty"); + assert!( + token.contains(&host), + "token should contain the host: {token}" + ); + assert!( + token.contains("Action=connect"), + "token should contain Action=connect: {token}" + ); + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/error.rs b/quickwit/quickwit-metastore/src/metastore/mysql/error.rs new file mode 100644 index 00000000000..a72405da9e1 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/error.rs @@ -0,0 +1,74 @@ +// 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::{EntityKind, MetastoreError}; +use sqlx::mysql::MySqlDatabaseError; +use tracing::error; + +// https://dev.mysql.com/doc/mysql-errors/8.0/en/server-error-reference.html +mod mysql_error_codes { + pub const DUPLICATE_ENTRY: u16 = 1062; + pub const FOREIGN_KEY_VIOLATION: u16 = 1452; +} + +pub(super) fn convert_sqlx_err(index_id: &str, sqlx_error: sqlx::Error) -> MetastoreError { + match &sqlx_error { + sqlx::Error::Database(boxed_db_error) => { + let Some(mysql_db_error) = boxed_db_error.try_downcast_ref::() + else { + error!(error=?boxed_db_error, "mysql-error"); + return MetastoreError::Db { + message: boxed_db_error.to_string(), + }; + }; + let mysql_error_code = mysql_db_error.number(); + + match mysql_error_code { + mysql_error_codes::FOREIGN_KEY_VIOLATION => { + MetastoreError::NotFound(EntityKind::Index { + index_id: index_id.to_string(), + }) + } + mysql_error_codes::DUPLICATE_ENTRY => { + let message = boxed_db_error.message(); + if message.contains("indexes_index_id_unique") + || message.contains("PRIMARY") && message.contains("indexes") + { + MetastoreError::AlreadyExists(EntityKind::Index { + index_id: index_id.to_string(), + }) + } else { + error!(error=?boxed_db_error, "mysql-error"); + MetastoreError::Internal { + message: "unique key violation".to_string(), + cause: format!("DB error {boxed_db_error:?}"), + } + } + } + _ => { + error!(error=?boxed_db_error, "mysql-error"); + MetastoreError::Db { + message: boxed_db_error.to_string(), + } + } + } + } + _ => { + error!(error=?sqlx_error, "an error has occurred in the database operation"); + MetastoreError::Db { + message: sqlx_error.to_string(), + } + } + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/factory.rs b/quickwit/quickwit-metastore/src/metastore/mysql/factory.rs new file mode 100644 index 00000000000..b5ca82fb860 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/factory.rs @@ -0,0 +1,92 @@ +// 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 std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use quickwit_common::uri::Uri; +use quickwit_config::{MetastoreBackend, MetastoreConfig}; +use quickwit_proto::metastore::MetastoreServiceClient; +use tokio::sync::Mutex; +use tracing::debug; + +use crate::{MetastoreFactory, MetastoreResolverError}; + +#[derive(Clone, Default)] +pub struct MysqlMetastoreFactory { + // Under normal conditions of use, this cache will contain a single `Metastore`. + // + // In contrast to the file-backed metastore, we use a strong pointer here, so that the + // `Metastore` doesn't get dropped. This is done in order to keep the underlying connection + // pool to MySQL alive. + cache: Arc>>, +} + +impl MysqlMetastoreFactory { + async fn get_from_cache(&self, uri: &Uri) -> Option { + let cache_lock = self.cache.lock().await; + cache_lock.get(uri).cloned() + } + + /// If there is a valid entry in the cache to begin with, we trash the new + /// one and return the old one. + /// + /// This way we make sure that we keep only one instance associated + /// to the key `uri` outside of this struct. + async fn cache_metastore( + &self, + uri: Uri, + metastore: MetastoreServiceClient, + ) -> MetastoreServiceClient { + let mut cache_lock = self.cache.lock().await; + if let Some(metastore) = cache_lock.get(&uri) { + return metastore.clone(); + } + cache_lock.insert(uri, metastore.clone()); + metastore + } +} + +#[async_trait] +impl MetastoreFactory for MysqlMetastoreFactory { + fn backend(&self) -> MetastoreBackend { + MetastoreBackend::MySQL + } + + async fn resolve( + &self, + metastore_config: &MetastoreConfig, + uri: &Uri, + ) -> Result { + if let Some(metastore) = self.get_from_cache(uri).await { + debug!("using metastore from cache"); + return Ok(metastore); + } + debug!("metastore not found in cache"); + let mysql_metastore_config = metastore_config.as_mysql().ok_or_else(|| { + let message = format!( + "expected MySQL metastore config, got `{:?}`", + metastore_config.backend() + ); + MetastoreResolverError::InvalidConfig(message) + })?; + let mysql_metastore = super::metastore::MysqlMetastore::new(mysql_metastore_config, uri) + .await + .map_err(MetastoreResolverError::Initialization)?; + let metastore = MetastoreServiceClient::new(mysql_metastore); + let metastore = self.cache_metastore(uri.clone(), metastore).await; + Ok(metastore) + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/metastore.rs b/quickwit/quickwit-metastore/src/metastore/mysql/metastore.rs new file mode 100644 index 00000000000..d2783612543 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/metastore.rs @@ -0,0 +1,2158 @@ +// 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 std::collections::HashMap; +use std::fmt::{self, Write}; +use std::str::FromStr; +use std::time::Duration; + +use async_trait::async_trait; +use futures::StreamExt; +use itertools::Itertools; +use quickwit_common::pretty::PrettySample; +use quickwit_common::uri::Uri; +use quickwit_common::{ServiceStream, get_bool_from_env, rate_limited_error}; +use quickwit_config::{ + IndexTemplate, IndexTemplateId, MysqlMetastoreConfig, validate_index_id_pattern, +}; +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, + OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, SplitStats, StageSplitsRequest, + ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, serde_utils, +}; +use quickwit_proto::types::{IndexId, IndexUid, Position, PublishToken, ShardId, SourceId}; +use sea_query::{Asterisk, Expr, MysqlQueryBuilder, Query, UnionType}; +use sea_query_binder::SqlxBinder; +use sqlx::{Acquire, Executor, MySql, Transaction}; +use time::OffsetDateTime; +use tracing::{debug, info, instrument, warn}; +use uuid::Uuid; + +use super::error::convert_sqlx_err; +use super::migrator::run_migrations; +use super::model::{ + MysqlDeleteTask, MysqlIndex, MysqlIndexTemplate, MysqlShard, MysqlSplit, Splits, +}; +use super::pool::TrackedPool; +use super::split_stream::SplitStream; +use super::utils::{append_query_filters_and_order_by, establish_connection}; +use super::{ + QW_MYSQL_READ_ONLY_ENV_KEY, QW_MYSQL_SKIP_MIGRATION_LOCKING_ENV_KEY, + QW_MYSQL_SKIP_MIGRATIONS_ENV_KEY, +}; +use crate::checkpoint::{ + IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, +}; +use crate::file_backed::MutationOccurred; +use crate::metastore::mysql::model::Shards; +use crate::metastore::mysql::utils::split_maturity_timestamp; +use crate::metastore::{ + IndexesMetadataResponseExt, PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, + UpdateSourceRequestExt, use_shard_api, +}; +use crate::{ + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, + ListIndexesMetadataResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, + MetastoreServiceExt, Split, SplitState, StageSplitsRequestExt, UpdateIndexRequestExt, +}; + +/// MySQL metastore implementation. +#[derive(Clone)] +pub struct MysqlMetastore { + uri: Uri, + connection_pool: TrackedPool, +} + +impl fmt::Debug for MysqlMetastore { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("MysqlMetastore") + .field("uri", &self.uri) + .finish() + } +} + +impl MysqlMetastore { + /// Creates a metastore given a database URI. + pub async fn new( + mysql_metastore_config: &MysqlMetastoreConfig, + connection_uri: &Uri, + ) -> MetastoreResult { + let min_connections = mysql_metastore_config.min_connections; + let max_connections = mysql_metastore_config.max_connections.get(); + let acquire_timeout = mysql_metastore_config + .acquire_connection_timeout() + .expect("MySQL metastore config should have been validated"); + let idle_timeout_opt = mysql_metastore_config + .idle_connection_timeout_opt() + .expect("MySQL metastore config should have been validated"); + let max_lifetime_opt = mysql_metastore_config + .max_connection_lifetime_opt() + .expect("MySQL metastore config should have been validated"); + + let read_only = get_bool_from_env(QW_MYSQL_READ_ONLY_ENV_KEY, false); + let skip_migrations = get_bool_from_env(QW_MYSQL_SKIP_MIGRATIONS_ENV_KEY, false); + let skip_locking = get_bool_from_env(QW_MYSQL_SKIP_MIGRATION_LOCKING_ENV_KEY, false); + + let connection_pool = establish_connection( + connection_uri, + min_connections, + max_connections, + acquire_timeout, + idle_timeout_opt, + max_lifetime_opt, + read_only, + &mysql_metastore_config.auth_mode, + mysql_metastore_config.aws_region.as_deref(), + ) + .await?; + + run_migrations(&connection_pool, skip_migrations, skip_locking).await?; + + let metastore = MysqlMetastore { + uri: connection_uri.clone(), + connection_pool, + }; + Ok(metastore) + } +} + +/// Returns an Index object given an index_id or None if it does not exist. +async fn index_opt<'a, E>( + executor: E, + index_id: &str, + lock: bool, +) -> MetastoreResult> +where + E: sqlx::Executor<'a, Database = MySql>, +{ + let index_opt: Option = sqlx::query_as::<_, MysqlIndex>(&format!( + r#" + SELECT * + FROM indexes + WHERE index_id = ? + {} + "#, + if lock { "FOR UPDATE" } else { "" } + )) + .bind(index_id) + .fetch_optional(executor) + .await?; + Ok(index_opt) +} + +/// Returns an Index object given an index_uid or None if it does not exist. +async fn index_opt_for_uid<'a, E>( + executor: E, + index_uid: IndexUid, + lock: bool, +) -> MetastoreResult> +where + E: sqlx::Executor<'a, Database = MySql>, +{ + let index_opt: Option = sqlx::query_as::<_, MysqlIndex>(&format!( + r#" + SELECT * + FROM indexes + WHERE index_uid = ? + {} + "#, + if lock { "FOR UPDATE" } else { "" } + )) + .bind(&index_uid) + .fetch_optional(executor) + .await?; + Ok(index_opt) +} + +async fn index_metadata( + tx: &mut Transaction<'_, MySql>, + index_id: &str, + lock: bool, +) -> MetastoreResult { + index_opt(tx.as_mut(), index_id, lock) + .await? + .ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Index { + index_id: index_id.to_string(), + }) + })? + .index_metadata() +} + +async fn try_apply_delta_v2( + tx: &mut Transaction<'_, MySql>, + index_uid: &IndexUid, + source_id: &SourceId, + checkpoint_delta: SourceCheckpointDelta, + publish_token: PublishToken, +) -> MetastoreResult<()> { + let num_partitions = checkpoint_delta.num_partitions(); + let shard_ids: Vec = checkpoint_delta + .partitions() + .map(|partition_id| partition_id.to_string()) + .collect(); + + if shard_ids.is_empty() { + return Ok(()); + } + + let placeholders = vec!["?"; shard_ids.len()].join(", "); + let sql = format!( + r#" + SELECT + shard_id, publish_position_inclusive, publish_token + FROM + shards + WHERE + index_uid = ? + AND source_id = ? + AND shard_id IN ({placeholders}) + FOR UPDATE + "#, + ); + let mut query = sqlx::query_as::<_, (String, String, Option)>(&sql) + .bind(index_uid) + .bind(source_id); + for shard_id in &shard_ids { + query = query.bind(shard_id); + } + let shards: Vec<(String, String, Option)> = query.fetch_all(tx.as_mut()).await?; + + if shards.len() != num_partitions { + let queue_id = format!("{index_uid}/{source_id}"); + let entity_kind = EntityKind::Shard { queue_id }; + return Err(MetastoreError::NotFound(entity_kind)); + } + let mut current_checkpoint = SourceCheckpoint::default(); + + for (shard_id, current_position, current_publish_token_opt) in shards { + if current_publish_token_opt.is_none() + || current_publish_token_opt.unwrap() != publish_token + { + let message = "failed to apply checkpoint delta: invalid publish token".to_string(); + return Err(MetastoreError::InvalidArgument { message }); + } + let partition_id = PartitionId::from(shard_id); + let current_position = Position::from(current_position); + current_checkpoint.add_partition(partition_id, current_position); + } + current_checkpoint + .try_apply_delta(checkpoint_delta) + .map_err(|error| MetastoreError::InvalidArgument { + message: error.to_string(), + })?; + + let now = OffsetDateTime::now_utc(); + + for (partition_id, new_position) in current_checkpoint.iter() { + let shard_id = partition_id.to_string(); + let new_position = new_position.to_string(); + sqlx::query( + r#" + UPDATE shards + SET + publish_position_inclusive = ?, + shard_state = CASE WHEN ? LIKE '~%' THEN 'closed' ELSE shard_state END, + update_timestamp = ? + WHERE + index_uid = ? + AND source_id = ? + AND shard_id = ? + "#, + ) + .bind(&new_position) + .bind(&new_position) + .bind(now) + .bind(index_uid) + .bind(source_id) + .bind(&shard_id) + .execute(tx.as_mut()) + .await?; + } + Ok(()) +} + +/// This macro is used to systematically wrap the metastore +/// into transaction, commit them on Result::Ok and rollback on Error. +/// +/// Note this is suboptimal. +/// Some of the methods actually did not require a transaction. +/// +/// We still use this macro for them in order to make the code +/// "trivially correct". +macro_rules! run_with_tx { + ($connection_pool:expr, $tx_refmut:ident, $label:literal, $x:block) => {{ + let mut tx: Transaction<'_, MySql> = $connection_pool.begin().await?; + let $tx_refmut = &mut tx; + let op_fut = move || async move { $x }; + let op_result: MetastoreResult<_> = op_fut().await; + match &op_result { + Ok(_) => { + debug!("committing transaction"); + tx.commit().await?; + } + Err(error) => { + rate_limited_error!(limit_per_min = 60, error=%error, "failed to {}, rolling transaction back", $label); + tx.rollback().await?; + } + } + op_result + }}; +} + +async fn mutate_index_metadata( + tx: &mut Transaction<'_, MySql>, + index_uid: IndexUid, + mutate_fn: M, +) -> MetastoreResult +where + MetastoreError: From, + M: FnOnce(&mut IndexMetadata) -> Result, E>, +{ + let index_id = &index_uid.index_id; + let mut index_metadata = index_metadata(tx, index_id, true).await?; + + if index_metadata.index_uid != index_uid { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_id.to_string(), + })); + } + if let MutationOccurred::No(()) = mutate_fn(&mut index_metadata)? { + return Ok(index_metadata); + } + let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; + + let update_index_res = sqlx::query( + r#" + UPDATE indexes + SET index_metadata_json = ? + WHERE index_uid = ? + "#, + ) + .bind(index_metadata_json) + .bind(&index_uid) + .execute(tx.as_mut()) + .await?; + if update_index_res.rows_affected() == 0 { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_id.to_string(), + })); + } + Ok(index_metadata) +} + +#[async_trait] +impl MetastoreService for MysqlMetastore { + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.connection_pool.acquire().await?; + Ok(()) + } + + fn endpoints(&self) -> Vec { + vec![self.uri.clone()] + } + + // Index API + + #[instrument(skip(self))] + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_config = request.deserialize_index_config()?; + let mut index_metadata = IndexMetadata::new(index_config); + + let source_configs = request.deserialize_source_configs()?; + + for source_config in source_configs { + index_metadata.add_source(source_config)?; + } + let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; + + sqlx::query( + "INSERT INTO indexes (index_uid, index_id, index_metadata_json) VALUES (?, ?, ?)", + ) + .bind(index_metadata.index_uid.to_string()) + .bind(&index_metadata.index_uid.index_id) + .bind(&index_metadata_json) + .execute(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(index_metadata.index_id(), sqlx_error))?; + + let response = CreateIndexResponse { + index_uid: index_metadata.index_uid.into(), + index_metadata_json, + }; + Ok(response) + } + + async fn update_index( + &self, + request: UpdateIndexRequest, + ) -> MetastoreResult { + let doc_mapping = request.deserialize_doc_mapping()?; + let indexing_settings = request.deserialize_indexing_settings()?; + let ingest_settings = request.deserialize_ingest_settings()?; + let search_settings = request.deserialize_search_settings()?; + let retention_policy_opt = request.deserialize_retention_policy()?; + + let index_uid: IndexUid = request.index_uid().clone(); + let updated_index_metadata = run_with_tx!(self.connection_pool, tx, "update index", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + let mutation_occurred = index_metadata.update_index_config( + doc_mapping, + indexing_settings, + ingest_settings, + search_settings, + retention_policy_opt, + )?; + Ok(MutationOccurred::from(mutation_occurred)) + }) + .await + })?; + IndexMetadataResponse::try_from_index_metadata(&updated_index_metadata) + } + + #[instrument(skip(self))] + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let mysql_index_opt = if let Some(index_uid) = &request.index_uid { + index_opt_for_uid(&self.connection_pool, index_uid.clone(), false).await? + } else if let Some(index_id) = &request.index_id { + index_opt(&self.connection_pool, index_id, false).await? + } else { + let message = "invalid request: neither `index_id` nor `index_uid` is set".to_string(); + return Err(MetastoreError::Internal { + message, + cause: "".to_string(), + }); + }; + let index_metadata = mysql_index_opt + .ok_or(MetastoreError::NotFound(EntityKind::Index { + index_id: request + .into_index_id() + .expect("`index_id` or `index_uid` should be set"), + }))? + .index_metadata()?; + let response = IndexMetadataResponse::try_from_index_metadata(&index_metadata)?; + Ok(response) + } + + #[instrument(skip(self))] + async fn indexes_metadata( + &self, + request: IndexesMetadataRequest, + ) -> MetastoreResult { + let num_subrequests = request.subrequests.len(); + + if num_subrequests == 0 { + return Ok(Default::default()); + } + let mut index_ids: Vec = Vec::new(); + let mut index_uids: Vec = Vec::with_capacity(num_subrequests); + let mut failures: Vec = Vec::new(); + + for subrequest in request.subrequests { + if let Some(index_id) = subrequest.index_id { + index_ids.push(index_id); + } else if let Some(index_uid) = subrequest.index_uid { + index_uids.push(index_uid); + } else { + let failure = IndexMetadataFailure { + index_id: subrequest.index_id, + index_uid: subrequest.index_uid, + reason: IndexMetadataFailureReason::Internal as i32, + }; + failures.push(failure); + } + } + + // Build dynamic query with IN (?, ?, ...) for both index_ids and index_uids + let mut conditions = Vec::new(); + let mut bind_values: Vec = Vec::new(); + + if !index_ids.is_empty() { + let placeholders = vec!["?"; index_ids.len()].join(", "); + conditions.push(format!("index_id IN ({placeholders})")); + bind_values.extend(index_ids.iter().cloned()); + } + if !index_uids.is_empty() { + let placeholders = vec!["?"; index_uids.len()].join(", "); + conditions.push(format!("index_uid IN ({placeholders})")); + bind_values.extend(index_uids.iter().map(|uid| uid.to_string())); + } + + let sql = format!("SELECT * FROM indexes WHERE {}", conditions.join(" OR ")); + let mut query = sqlx::query_as::<_, MysqlIndex>(&sql); + for val in &bind_values { + query = query.bind(val); + } + let mysql_indexes: Vec = query.fetch_all(&self.connection_pool).await?; + + let indexes_metadata: Vec = mysql_indexes + .iter() + .map(|mysql_index| mysql_index.index_metadata()) + .collect::>()?; + + if mysql_indexes.len() + failures.len() < num_subrequests { + for index_id in index_ids { + if mysql_indexes + .iter() + .all(|mysql_index| mysql_index.index_id != index_id) + { + let failure = IndexMetadataFailure { + index_id: Some(index_id), + index_uid: None, + reason: IndexMetadataFailureReason::NotFound as i32, + }; + failures.push(failure); + } + } + for index_uid in index_uids { + if mysql_indexes + .iter() + .all(|mysql_index| mysql_index.index_uid != index_uid) + { + let failure = IndexMetadataFailure { + index_id: None, + index_uid: Some(index_uid), + reason: IndexMetadataFailureReason::NotFound as i32, + }; + failures.push(failure); + } + } + } + let response = + IndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata, failures).await?; + Ok(response) + } + + #[instrument(skip(self))] + async fn list_indexes_metadata( + &self, + request: ListIndexesMetadataRequest, + ) -> MetastoreResult { + let sql = + build_index_id_patterns_sql_query(&request.index_id_patterns).map_err(|error| { + MetastoreError::Internal { + message: "failed to build `list_indexes_metadata` SQL query".to_string(), + cause: error.to_string(), + } + })?; + let mysql_indexes = sqlx::query_as::<_, MysqlIndex>(&sql) + .fetch_all(&self.connection_pool) + .await?; + let indexes_metadata: Vec = mysql_indexes + .into_iter() + .map(|mysql_index| mysql_index.index_metadata()) + .collect::>()?; + let response = + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata).await?; + Ok(response) + } + + #[instrument(skip_all, fields(index_id=%request.index_uid()))] + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let delete_result = sqlx::query("DELETE FROM indexes WHERE index_uid = ?") + .bind(&index_uid) + .execute(&self.connection_pool) + .await?; + if delete_result.rows_affected() == 0 { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + info!(index_id = index_uid.index_id, "deleted index successfully"); + Ok(EmptyResponse {}) + } + + // Split Lifecycle + + #[instrument(skip_all, fields(split_ids))] + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let splits_metadata = request.deserialize_splits_metadata()?; + + if splits_metadata.is_empty() { + return Ok(Default::default()); + } + + let mut split_ids = Vec::with_capacity(splits_metadata.len()); + let mut time_range_start_list = Vec::with_capacity(splits_metadata.len()); + let mut time_range_end_list = Vec::with_capacity(splits_metadata.len()); + let mut tags_list = Vec::with_capacity(splits_metadata.len()); + let mut splits_metadata_json = Vec::with_capacity(splits_metadata.len()); + let mut delete_opstamps = Vec::with_capacity(splits_metadata.len()); + let mut maturity_timestamps = Vec::with_capacity(splits_metadata.len()); + let mut node_ids = Vec::with_capacity(splits_metadata.len()); + + for split_metadata in splits_metadata { + let split_metadata_json = serde_utils::to_json_str(&split_metadata)?; + splits_metadata_json.push(split_metadata_json); + + let time_range_start = split_metadata + .time_range + .as_ref() + .map(|range| *range.start()); + time_range_start_list.push(time_range_start); + maturity_timestamps.push(split_maturity_timestamp(&split_metadata)); + + let time_range_end = split_metadata.time_range.map(|range| *range.end()); + time_range_end_list.push(time_range_end); + + let tags: Vec = split_metadata.tags.into_iter().collect(); + tags_list.push(tags); + split_ids.push(split_metadata.split_id); + delete_opstamps.push(split_metadata.delete_opstamp as i64); + node_ids.push(split_metadata.node_id); + } + tracing::Span::current().record("split_ids", format!("{split_ids:?}")); + + run_with_tx!(self.connection_pool, tx, "stage splits", { + let split_state_str = SplitState::Staged.as_str(); + + // Lock the index row first to ensure consistent lock ordering + // (indexes → splits) with publish_splits, preventing deadlocks. + let _ = index_opt_for_uid(tx.as_mut(), index_uid.clone(), true) + .await? + .ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id.to_string(), + }) + })?; + + // Check if any of the splits exist in a non-Staged state. + let placeholders = vec!["?"; split_ids.len()].join(", "); + let check_sql = format!( + "SELECT split_id FROM splits WHERE index_uid = ? AND split_id IN ({placeholders}) \ + AND split_state != ? FOR UPDATE" + ); + let mut check_query = sqlx::query_as::<_, (String,)>(&check_sql).bind(&index_uid); + for split_id in &split_ids { + check_query = check_query.bind(split_id); + } + check_query = check_query.bind(split_state_str); + let non_staged: Vec<(String,)> = check_query.fetch_all(tx.as_mut()).await?; + + if !non_staged.is_empty() { + let failed_split_ids: Vec = + non_staged.into_iter().map(|(id,)| id).collect(); + let entity = EntityKind::Splits { + split_ids: failed_split_ids, + }; + let message = "splits are not staged".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + + for idx in 0..split_ids.len() { + let tags_json = serde_json::to_string(&tags_list[idx]).map_err(|error| { + MetastoreError::Internal { + message: "failed to serialize tags".to_string(), + cause: error.to_string(), + } + })?; + + sqlx::query( + r#" + INSERT INTO splits + (split_id, time_range_start, time_range_end, tags, + split_metadata_json, delete_opstamp, maturity_timestamp, + split_state, index_uid, node_id) + VALUES (?, ?, ?, ?, ?, ?, FROM_UNIXTIME(?), ?, ?, ?) + AS new + ON DUPLICATE KEY UPDATE + time_range_start = new.time_range_start, + time_range_end = new.time_range_end, + tags = new.tags, + split_metadata_json = new.split_metadata_json, + delete_opstamp = new.delete_opstamp, + maturity_timestamp = new.maturity_timestamp, + node_id = new.node_id, + update_timestamp = UTC_TIMESTAMP(), + create_timestamp = UTC_TIMESTAMP() + "#, + ) + .bind(&split_ids[idx]) + .bind(time_range_start_list[idx]) + .bind(time_range_end_list[idx]) + .bind(&tags_json) + .bind(&splits_metadata_json[idx]) + .bind(delete_opstamps[idx]) + .bind(maturity_timestamps[idx]) + .bind(split_state_str) + .bind(&index_uid) + .bind(&node_ids[idx]) + .execute(tx.as_mut()) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + } + + info!( + %index_uid, + "staged `{}` splits successfully", split_ids.len() + ); + Ok(EmptyResponse {}) + }) + } + + #[instrument(skip(self))] + async fn publish_splits( + &self, + request: PublishSplitsRequest, + ) -> MetastoreResult { + let checkpoint_delta_opt: Option = + request.deserialize_index_checkpoint()?; + let index_uid: IndexUid = request.index_uid().clone(); + let staged_split_ids = request.staged_split_ids; + let replaced_split_ids = request.replaced_split_ids; + + run_with_tx!(self.connection_pool, tx, "publish splits", { + let mut index_metadata = index_metadata(tx, &index_uid.index_id, true).await?; + if index_metadata.index_uid != index_uid { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + if let Some(checkpoint_delta) = checkpoint_delta_opt { + let source_id = checkpoint_delta.source_id.clone(); + let source = index_metadata.sources.get(&source_id).ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Source { + index_id: index_uid.index_id.to_string(), + source_id: source_id.to_string(), + }) + })?; + + if use_shard_api(&source.source_params) { + let publish_token = request.publish_token_opt.ok_or_else(|| { + let message = format!( + "publish token is required for publishing splits for source \ + `{source_id}`" + ); + MetastoreError::InvalidArgument { message } + })?; + try_apply_delta_v2( + tx, + &index_uid, + &source_id, + checkpoint_delta.source_delta, + publish_token, + ) + .await?; + } else { + index_metadata + .checkpoint + .try_apply_delta(checkpoint_delta) + .map_err(|error| { + let entity = EntityKind::CheckpointDelta { + index_id: index_uid.index_id.to_string(), + source_id, + }; + let message = error.to_string(); + MetastoreError::FailedPrecondition { entity, message } + })?; + } + } + let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; + + // Collect all split_ids we need to validate, lock them. + let all_split_ids: Vec<&str> = staged_split_ids + .iter() + .chain(replaced_split_ids.iter()) + .map(|s| s.as_str()) + .collect(); + + if !all_split_ids.is_empty() { + let placeholders = vec!["?"; all_split_ids.len()].join(", "); + let select_sql = format!( + "SELECT split_id, split_state FROM splits WHERE index_uid = ? AND split_id IN \ + ({placeholders}) FOR UPDATE" + ); + let mut select_query = + sqlx::query_as::<_, (String, String)>(&select_sql).bind(&index_uid); + for split_id in &all_split_ids { + select_query = select_query.bind(split_id); + } + let found_splits: Vec<(String, String)> = + select_query.fetch_all(tx.as_mut()).await?; + + let found_map: HashMap<&str, &str> = found_splits + .iter() + .map(|(id, state)| (id.as_str(), state.as_str())) + .collect(); + + // Check for missing splits. + let not_found_split_ids: Vec = all_split_ids + .iter() + .filter(|id| !found_map.contains_key(**id)) + .map(|id| id.to_string()) + .unique() + .collect(); + if !not_found_split_ids.is_empty() { + return Err(MetastoreError::NotFound(EntityKind::Splits { + split_ids: not_found_split_ids, + })); + } + + // Validate staged splits are in 'Staged' state. + let not_staged_split_ids: Vec = staged_split_ids + .iter() + .filter(|id| found_map.get(id.as_str()) != Some(&"Staged")) + .cloned() + .unique() + .collect(); + if !not_staged_split_ids.is_empty() { + let entity = EntityKind::Splits { + split_ids: not_staged_split_ids, + }; + let message = "splits are not staged".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + + // Validate replaced splits are in 'Published' state. + let not_published_split_ids: Vec = replaced_split_ids + .iter() + .filter(|id| found_map.get(id.as_str()) != Some(&"Published")) + .cloned() + .unique() + .collect(); + if !not_published_split_ids.is_empty() { + let entity = EntityKind::Splits { + split_ids: not_published_split_ids, + }; + let message = "splits are not marked for deletion".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + } + + // Update index metadata (checkpoint). + sqlx::query("UPDATE indexes SET index_metadata_json = ? WHERE index_uid = ?") + .bind(&index_metadata_json) + .bind(&index_uid) + .execute(tx.as_mut()) + .await?; + + // Publish staged splits. + if !staged_split_ids.is_empty() { + let placeholders = vec!["?"; staged_split_ids.len()].join(", "); + let update_sql = format!( + "UPDATE splits SET split_state = 'Published', update_timestamp = \ + UTC_TIMESTAMP(), publish_timestamp = UTC_TIMESTAMP() WHERE index_uid = ? AND \ + split_id IN ({placeholders}) AND split_state = 'Staged'" + ); + let mut update_query = sqlx::query(&update_sql).bind(&index_uid); + for split_id in &staged_split_ids { + update_query = update_query.bind(split_id); + } + let num_published_splits = update_query.execute(tx.as_mut()).await?.rows_affected(); + + // Mark replaced splits for deletion. + if !replaced_split_ids.is_empty() { + let placeholders = vec!["?"; replaced_split_ids.len()].join(", "); + let mark_sql = format!( + "UPDATE splits SET split_state = 'MarkedForDeletion', update_timestamp = \ + UTC_TIMESTAMP() WHERE index_uid = ? AND split_id IN ({placeholders}) AND \ + split_state = 'Published'" + ); + let mut mark_query = sqlx::query(&mark_sql).bind(&index_uid); + for split_id in &replaced_split_ids { + mark_query = mark_query.bind(split_id); + } + let num_marked_splits = mark_query.execute(tx.as_mut()).await?.rows_affected(); + info!( + %index_uid, + "published {num_published_splits} splits and marked {num_marked_splits} for deletion successfully" + ); + } else { + info!( + %index_uid, + "published {num_published_splits} splits successfully" + ); + } + } else if !replaced_split_ids.is_empty() { + let placeholders = vec!["?"; replaced_split_ids.len()].join(", "); + let mark_sql = format!( + "UPDATE splits SET split_state = 'MarkedForDeletion', update_timestamp = \ + UTC_TIMESTAMP() WHERE index_uid = ? AND split_id IN ({placeholders}) AND \ + split_state = 'Published'" + ); + let mut mark_query = sqlx::query(&mark_sql).bind(&index_uid); + for split_id in &replaced_split_ids { + mark_query = mark_query.bind(split_id); + } + let num_marked_splits = mark_query.execute(tx.as_mut()).await?.rows_affected(); + info!( + %index_uid, + "published 0 splits and marked {num_marked_splits} for deletion successfully" + ); + } + Ok(EmptyResponse {}) + }) + } + + #[instrument(skip(self))] + async fn list_splits( + &self, + request: ListSplitsRequest, + ) -> MetastoreResult> { + let list_splits_query = request.deserialize_list_splits_query()?; + let mut sql_query_builder = Query::select(); + sql_query_builder.column(Asterisk).from(Splits::Table); + append_query_filters_and_order_by(&mut sql_query_builder, &list_splits_query); + + let (sql_query, values) = sql_query_builder.build_sqlx(MysqlQueryBuilder); + let mysql_split_stream = SplitStream::new( + self.connection_pool.clone(), + sql_query, + |connection_pool: &TrackedPool, sql_query: &String| { + sqlx::query_as_with::<_, MysqlSplit, _>(sql_query, values).fetch(connection_pool) + }, + ); + let split_stream = + mysql_split_stream + .chunks(STREAM_SPLITS_CHUNK_SIZE) + .map(|mysql_splits_results| { + let mut splits = Vec::with_capacity(mysql_splits_results.len()); + for mysql_split_result in mysql_splits_results { + let mysql_split = match mysql_split_result { + Ok(mysql_split) => mysql_split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to fetch splits".to_string(), + cause: error.to_string(), + }); + } + }; + let split: Split = match mysql_split.try_into() { + Ok(split) => split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to convert `MysqlSplit` to `Split`" + .to_string(), + cause: error.to_string(), + }); + } + }; + splits.push(split); + } + ListSplitsResponse::try_from_splits(splits) + }); + let service_stream = ServiceStream::new(Box::pin(split_stream)); + Ok(service_stream) + } + + async fn list_index_stats( + &self, + request: ListIndexStatsRequest, + ) -> MetastoreResult { + let index_pattern_sql = build_index_id_patterns_sql_query(&request.index_id_patterns) + .map_err(|error| MetastoreError::Internal { + message: "failed to build `list_index_stats` SQL query".to_string(), + cause: error.to_string(), + })?; + let sql = format!( + "SELECT + i.index_uid, + s.split_state, + COUNT(s.split_state) AS num_splits, + COALESCE(CAST(SUM(s.split_size_bytes) AS SIGNED), 0) AS total_size_bytes + FROM ({index_pattern_sql}) i + LEFT JOIN splits s ON s.index_uid = i.index_uid + GROUP BY i.index_uid, s.split_state" + ); + + let rows: Vec<(String, Option, i64, i64)> = sqlx::query_as(&sql) + .fetch_all(&self.connection_pool) + .await?; + + let mut index_stats = HashMap::new(); + for (index_uid_str, split_state, num_splits, total_size_bytes) in rows { + let Ok(index_uid) = IndexUid::from_str(&index_uid_str) else { + return Err(MetastoreError::Internal { + message: "failed to parse index_uid".to_string(), + cause: index_uid_str.to_string(), + }); + }; + let stats = index_stats + .entry(index_uid_str) + .or_insert_with(|| IndexStats { + index_uid: Some(index_uid), + staged: Some(SplitStats::default()), + published: Some(SplitStats::default()), + marked_for_deletion: Some(SplitStats::default()), + }); + let num_splits = num_splits as u64; + let total_size_bytes = total_size_bytes as u64; + match split_state.as_deref() { + Some("Staged") => { + stats.staged = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + Some("Published") => { + stats.published = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + Some("MarkedForDeletion") => { + stats.marked_for_deletion = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + None => {} + Some(split_state) => { + return Err(MetastoreError::Internal { + message: "invalid split state".to_string(), + cause: split_state.to_string(), + }); + } + } + } + + Ok(ListIndexStatsResponse { + index_stats: index_stats.into_values().collect(), + }) + } + + #[instrument(skip(self))] + async fn mark_splits_for_deletion( + &self, + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + + if split_ids.is_empty() { + if index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + return Ok(EmptyResponse {}); + } + + run_with_tx!(self.connection_pool, tx, "mark splits for deletion", { + // Lock the candidate splits. + let placeholders = vec!["?"; split_ids.len()].join(", "); + let select_sql = format!( + "SELECT split_id, split_state FROM splits WHERE index_uid = ? AND split_id IN \ + ({placeholders}) FOR UPDATE" + ); + let mut select_query = + sqlx::query_as::<_, (String, String)>(&select_sql).bind(&index_uid); + for split_id in &split_ids { + select_query = select_query.bind(split_id); + } + let found_splits: Vec<(String, String)> = select_query.fetch_all(tx.as_mut()).await?; + + let num_found_splits = found_splits.len() as i64; + let not_found_split_ids: Vec = split_ids + .iter() + .filter(|id| found_splits.iter().all(|(fid, _)| fid != *id)) + .cloned() + .collect(); + + // Mark the staged and published splits for deletion. + let update_sql = format!( + "UPDATE splits SET split_state = 'MarkedForDeletion', update_timestamp = \ + UTC_TIMESTAMP() WHERE index_uid = ? AND split_id IN ({placeholders}) AND \ + split_state IN ('Staged', 'Published')" + ); + let mut update_query = sqlx::query(&update_sql).bind(&index_uid); + for split_id in &split_ids { + update_query = update_query.bind(split_id); + } + let num_marked_splits = update_query.execute(tx.as_mut()).await?.rows_affected(); + + if num_found_splits == 0 + && index_opt(tx.as_mut(), &index_uid.index_id, false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id.clone(), + })); + } + info!( + %index_uid, + "marked {} splits for deletion, among which {} were newly marked", + split_ids.len() - not_found_split_ids.len(), + num_marked_splits + ); + if !not_found_split_ids.is_empty() { + warn!( + %index_uid, + split_ids=?PrettySample::new(¬_found_split_ids, 5), + "{} splits were not found and could not be marked for deletion", + not_found_split_ids.len() + ); + } + Ok(EmptyResponse {}) + }) + } + + #[instrument(skip(self))] + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + + if split_ids.is_empty() { + if index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + return Ok(EmptyResponse {}); + } + + run_with_tx!(self.connection_pool, tx, "delete splits", { + let placeholders = vec!["?"; split_ids.len()].join(", "); + // Lock and read all candidate splits. + let select_sql = format!( + "SELECT split_id, split_state FROM splits WHERE index_uid = ? AND split_id IN \ + ({placeholders}) FOR UPDATE" + ); + let mut select_query = + sqlx::query_as::<_, (String, String)>(&select_sql).bind(&index_uid); + for split_id in &split_ids { + select_query = select_query.bind(split_id); + } + let found_splits: Vec<(String, String)> = select_query.fetch_all(tx.as_mut()).await?; + + let num_found_splits = found_splits.len() as i64; + let not_found_split_ids: Vec = split_ids + .iter() + .filter(|id| found_splits.iter().all(|(fid, _)| fid != *id)) + .cloned() + .collect(); + + let not_deletable_split_ids: Vec = found_splits + .iter() + .filter(|(_, state)| state == "Staged" || state == "Published") + .map(|(id, _)| id.clone()) + .collect(); + + if num_found_splits == 0 + && index_opt_for_uid(tx.as_mut(), index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id.clone(), + })); + } + if !not_deletable_split_ids.is_empty() { + let message = format!( + "splits `{}` are not deletable", + not_deletable_split_ids.join(", ") + ); + let entity = EntityKind::Splits { + split_ids: not_deletable_split_ids, + }; + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + + // Delete only MarkedForDeletion splits. + if !split_ids.is_empty() { + let delete_sql = format!( + "DELETE FROM splits WHERE index_uid = ? AND split_id IN ({placeholders}) AND \ + split_state = 'MarkedForDeletion'" + ); + let mut delete_query = sqlx::query(&delete_sql).bind(&index_uid); + for split_id in &split_ids { + delete_query = delete_query.bind(split_id); + } + let num_deleted_splits = delete_query.execute(tx.as_mut()).await?.rows_affected(); + info!(%index_uid, "deleted {} splits from index", num_deleted_splits); + } + + if !not_found_split_ids.is_empty() { + warn!( + %index_uid, + split_ids=?PrettySample::new(¬_found_split_ids, 5), + "{} splits were not found and could not be deleted", + not_found_split_ids.len() + ); + } + Ok(EmptyResponse {}) + }) + } + + // Source API + + #[instrument(skip(self))] + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "add source", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + index_metadata.add_source(source_config)?; + Ok(MutationOccurred::Yes(())) + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + + #[instrument(skip(self))] + async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "update source", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + let mutation_occurred = index_metadata.update_source(source_config)?; + Ok(MutationOccurred::from(mutation_occurred)) + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + + #[instrument(skip(self))] + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "toggle source", { + mutate_index_metadata(tx, index_uid, |index_metadata| { + if index_metadata.toggle_source(&request.source_id, request.enable)? { + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + } else { + Ok::<_, MetastoreError>(MutationOccurred::No(())) + } + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + + #[instrument(skip(self))] + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let source_id = request.source_id.clone(); + run_with_tx!(self.connection_pool, tx, "delete source", { + mutate_index_metadata(tx, index_uid.clone(), |index_metadata| { + index_metadata.delete_source(&source_id)?; + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + }) + .await?; + sqlx::query( + r#" + DELETE FROM shards + WHERE + index_uid = ? + AND source_id = ? + "#, + ) + .bind(&index_uid) + .bind(source_id) + .execute(tx.as_mut()) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + + #[instrument(skip(self))] + async fn reset_source_checkpoint( + &self, + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "reset source checkpoint", { + mutate_index_metadata(tx, index_uid, |index_metadata| { + if index_metadata.checkpoint.reset_source(&request.source_id) { + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + } else { + Ok::<_, MetastoreError>(MutationOccurred::No(())) + } + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + + // Delete Tasks API + + #[instrument(skip(self))] + async fn last_delete_opstamp( + &self, + request: LastDeleteOpstampRequest, + ) -> MetastoreResult { + let max_opstamp: i64 = sqlx::query_scalar( + r#" + SELECT COALESCE(MAX(opstamp), 0) + FROM delete_tasks + WHERE index_uid = ? + "#, + ) + .bind(request.index_uid()) + .fetch_one(&self.connection_pool) + .await + .map_err(|error| MetastoreError::Db { + message: error.to_string(), + })?; + + Ok(LastDeleteOpstampResponse::new(max_opstamp as u64)) + } + + /// Creates a delete task from a delete query. + /// CRITICAL: INSERT + SELECT must use same connection (LAST_INSERT_ID is connection-scoped). + #[instrument(skip(self))] + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { + let delete_query_json = serde_utils::to_json_str(&delete_query)?; + + let mut conn = self.connection_pool.acquire().await?; + + sqlx::query("INSERT INTO delete_tasks (index_uid, delete_query_json) VALUES (?, ?)") + .bind(delete_query.index_uid().to_string()) + .bind(&delete_query_json) + .execute(&mut *conn) + .await + .map_err(|error| convert_sqlx_err(&delete_query.index_uid().index_id, error))?; + + let (create_timestamp, opstamp): (sqlx::types::time::PrimitiveDateTime, i64) = + sqlx::query_as( + "SELECT create_timestamp, opstamp FROM delete_tasks WHERE opstamp = \ + LAST_INSERT_ID()", + ) + .fetch_one(&mut *conn) + .await + .map_err(|error| convert_sqlx_err(&delete_query.index_uid().index_id, error))?; + + Ok(DeleteTask { + create_timestamp: create_timestamp.assume_utc().unix_timestamp(), + opstamp: opstamp as u64, + delete_query: Some(delete_query), + }) + } + + #[instrument(skip(self))] + async fn update_splits_delete_opstamp( + &self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + if split_ids.is_empty() { + return Ok(UpdateSplitsDeleteOpstampResponse {}); + } + let placeholders = vec!["?"; split_ids.len()].join(", "); + let sql = format!( + r#" + UPDATE splits + SET + delete_opstamp = ?, + update_timestamp = CASE + WHEN delete_opstamp != ? THEN UTC_TIMESTAMP() + ELSE update_timestamp + END + WHERE + index_uid = ? + AND split_id IN ({placeholders}) + "#, + ); + let mut query = sqlx::query(&sql) + .bind(request.delete_opstamp as i64) + .bind(request.delete_opstamp as i64) + .bind(&index_uid); + for split_id in &split_ids { + query = query.bind(split_id); + } + let update_result = query.execute(&self.connection_pool).await?; + + if update_result.rows_affected() == 0 + && index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + Ok(UpdateSplitsDeleteOpstampResponse {}) + } + + #[instrument(skip(self))] + async fn list_delete_tasks( + &self, + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let mysql_delete_tasks: Vec = sqlx::query_as::<_, MysqlDeleteTask>( + r#" + SELECT * FROM delete_tasks + WHERE + index_uid = ? + AND opstamp > ? + "#, + ) + .bind(&index_uid) + .bind(request.opstamp_start as i64) + .fetch_all(&self.connection_pool) + .await?; + let delete_tasks: Vec = mysql_delete_tasks + .into_iter() + .map(|mysql_delete_task| mysql_delete_task.try_into()) + .collect::>()?; + Ok(ListDeleteTasksResponse { delete_tasks }) + } + + #[instrument(skip(self))] + async fn list_stale_splits( + &self, + request: ListStaleSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let stale_mysql_splits: Vec = sqlx::query_as::<_, MysqlSplit>( + r#" + SELECT * + FROM splits + WHERE + index_uid = ? + AND delete_opstamp < ? + AND split_state = ? + AND (maturity_timestamp = FROM_UNIXTIME(0) OR UTC_TIMESTAMP() >= maturity_timestamp) + ORDER BY delete_opstamp ASC, publish_timestamp ASC + LIMIT ? + "#, + ) + .bind(&index_uid) + .bind(request.delete_opstamp as i64) + .bind(SplitState::Published.as_str()) + .bind(request.num_splits as i64) + .fetch_all(&self.connection_pool) + .await?; + + let stale_splits: Vec = stale_mysql_splits + .into_iter() + .map(|mysql_split| mysql_split.try_into()) + .collect::>()?; + let response = ListSplitsResponse::try_from_splits(stale_splits)?; + Ok(response) + } + + // Shard API + + async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { + let mut subresponses = Vec::with_capacity(request.subrequests.len()); + + for subrequest in request.subrequests { + let open_shard: Shard = open_or_fetch_shard(&self.connection_pool, &subrequest).await?; + let subresponse = OpenShardSubresponse { + subrequest_id: subrequest.subrequest_id, + open_shard: Some(open_shard), + }; + subresponses.push(subresponse); + } + Ok(OpenShardsResponse { subresponses }) + } + + async fn acquire_shards( + &self, + request: AcquireShardsRequest, + ) -> MetastoreResult { + if request.shard_ids.is_empty() { + return Ok(Default::default()); + } + + // MySQL doesn't support RETURNING, so we UPDATE then SELECT. + let placeholders = vec!["?"; request.shard_ids.len()].join(", "); + + let update_sql = format!( + "UPDATE shards SET publish_token = ? WHERE index_uid = ? AND source_id = ? AND \ + shard_id IN ({placeholders})" + ); + let mut update_query = sqlx::query(&update_sql) + .bind(&request.publish_token) + .bind(request.index_uid()) + .bind(&request.source_id); + for shard_id in &request.shard_ids { + update_query = update_query.bind(shard_id); + } + update_query.execute(&self.connection_pool).await?; + + let select_sql = format!( + "SELECT * FROM shards WHERE index_uid = ? AND source_id = ? AND shard_id IN \ + ({placeholders})" + ); + let mut select_query = sqlx::query_as::<_, MysqlShard>(&select_sql) + .bind(request.index_uid()) + .bind(&request.source_id); + for shard_id in &request.shard_ids { + select_query = select_query.bind(shard_id); + } + let mysql_shards: Vec = select_query.fetch_all(&self.connection_pool).await?; + + let acquired_shards = mysql_shards + .into_iter() + .map(|mysql_shard| mysql_shard.into()) + .collect(); + let response = AcquireShardsResponse { acquired_shards }; + Ok(response) + } + + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { + if request.subrequests.is_empty() { + return Ok(Default::default()); + } + let mut sql_query_builder = Query::select(); + + for (idx, subrequest) in request.subrequests.iter().enumerate() { + let mut sql_subquery_builder = Query::select(); + + sql_subquery_builder + .column(Asterisk) + .from(Shards::Table) + .and_where(Expr::col(Shards::IndexUid).eq(subrequest.index_uid())) + .and_where(Expr::col(Shards::SourceId).eq(&subrequest.source_id)); + + let shard_state = subrequest.shard_state(); + + if shard_state != ShardState::Unspecified { + let shard_state_str = shard_state.as_json_str_name(); + // MySQL ENUM accepts string values directly — no CAST needed. + sql_subquery_builder.and_where(Expr::col(Shards::ShardState).eq(shard_state_str)); + } + if idx == 0 { + sql_query_builder = sql_subquery_builder; + } else { + sql_query_builder.union(UnionType::All, sql_subquery_builder); + } + } + let (sql_query, values) = sql_query_builder.build_sqlx(MysqlQueryBuilder); + + let mysql_shards: Vec = + sqlx::query_as_with::<_, MysqlShard, _>(&sql_query, values) + .fetch_all(&self.connection_pool) + .await?; + + let mut per_source_subresponses: HashMap<(IndexUid, SourceId), ListShardsSubresponse> = + request + .subrequests + .into_iter() + .map(|subrequest| { + let index_uid = subrequest.index_uid().clone(); + let source_id = subrequest.source_id.clone(); + ( + (index_uid, source_id), + ListShardsSubresponse { + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shards: Vec::new(), + }, + ) + }) + .collect(); + + for mysql_shard in mysql_shards { + let shard: Shard = mysql_shard.into(); + let source_key = (shard.index_uid().clone(), shard.source_id.clone()); + + let Some(subresponse) = per_source_subresponses.get_mut(&source_key) else { + warn!( + index_uid=%shard.index_uid(), + source_id=%shard.source_id, + "could not find source in subresponses: this should never happen, please report" + ); + continue; + }; + subresponse.shards.push(shard); + } + let subresponses = per_source_subresponses.into_values().collect(); + let response = ListShardsResponse { subresponses }; + Ok(response) + } + + async fn delete_shards( + &self, + request: DeleteShardsRequest, + ) -> MetastoreResult { + if request.shard_ids.is_empty() { + return Ok(Default::default()); + } + + let placeholders = vec!["?"; request.shard_ids.len()].join(", "); + + // Delete shards that are fully indexed (position starts with '~') or forced. + let delete_sql = format!( + "DELETE FROM shards WHERE index_uid = ? AND source_id = ? AND shard_id IN \ + ({placeholders}) AND (? OR publish_position_inclusive LIKE '~%')" + ); + let mut delete_query = sqlx::query(&delete_sql) + .bind(request.index_uid()) + .bind(&request.source_id); + for shard_id in &request.shard_ids { + delete_query = delete_query.bind(shard_id); + } + delete_query = delete_query.bind(request.force); + let query_result = delete_query.execute(&self.connection_pool).await?; + + // Happy path: all shards were deleted. + if request.force || query_result.rows_affected() == request.shard_ids.len() as u64 { + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes: request.shard_ids, + failures: Vec::new(), + }; + return Ok(response); + } + + // Unhappy path: find shards that are not deletable. + let find_sql = format!( + "SELECT * FROM shards WHERE index_uid = ? AND source_id = ? AND shard_id IN \ + ({placeholders}) AND publish_position_inclusive NOT LIKE '~%'" + ); + let mut find_query = sqlx::query_as::<_, MysqlShard>(&find_sql) + .bind(request.index_uid()) + .bind(&request.source_id); + for shard_id in &request.shard_ids { + find_query = find_query.bind(shard_id); + } + let not_deletable_mysql_shards: Vec = + find_query.fetch_all(&self.connection_pool).await?; + + if not_deletable_mysql_shards.is_empty() { + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes: request.shard_ids, + failures: Vec::new(), + }; + return Ok(response); + } + let failures: Vec = not_deletable_mysql_shards + .into_iter() + .map(|mysql_shard| mysql_shard.shard_id) + .collect(); + warn!( + index_uid=%request.index_uid(), + source_id=%request.source_id, + "failed to delete shards `{}`: shards are not fully indexed", + failures.iter().join(", ") + ); + let successes: Vec = request + .shard_ids + .into_iter() + .filter(|shard_id| !failures.contains(shard_id)) + .collect(); + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes, + failures, + }; + Ok(response) + } + + async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { + const PRUNE_AGE_SHARDS_QUERY: &str = include_str!("queries/shards/prune_age.sql"); + const PRUNE_COUNT_SHARDS_QUERY: &str = include_str!("queries/shards/prune_count.sql"); + + if let Some(max_age_secs) = request.max_age_secs { + let limit_datetime = + OffsetDateTime::now_utc() - Duration::from_secs(max_age_secs as u64); + sqlx::query(PRUNE_AGE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(limit_datetime) + .execute(&self.connection_pool) + .await?; + } + + if let Some(max_count) = request.max_count { + sqlx::query(PRUNE_COUNT_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(max_count as i64) + .execute(&self.connection_pool) + .await?; + } + Ok(EmptyResponse {}) + } + + // Index Template API + + async fn create_index_template( + &self, + request: CreateIndexTemplateRequest, + ) -> MetastoreResult { + const INSERT_INDEX_TEMPLATE_QUERY: &str = + include_str!("queries/index_templates/insert.sql"); + const UPSERT_INDEX_TEMPLATE_QUERY: &str = + include_str!("queries/index_templates/upsert.sql"); + + let index_template: IndexTemplate = + serde_utils::from_json_str(&request.index_template_json)?; + + index_template + .validate() + .map_err(|error| MetastoreError::InvalidArgument { + message: format!( + "invalid index template `{}`: `{error}`", + index_template.template_id + ), + })?; + + let mut positive_patterns = Vec::new(); + let mut negative_patterns = Vec::new(); + + for pattern in &index_template.index_id_patterns { + if let Some(negative_pattern) = pattern.strip_prefix('-') { + negative_patterns.push(negative_pattern.replace('*', "%")); + } else { + positive_patterns.push(pattern.replace('*', "%")); + } + } + + let positive_json = serde_json::to_string(&positive_patterns).map_err(|error| { + MetastoreError::Internal { + message: "failed to serialize positive patterns".to_string(), + cause: error.to_string(), + } + })?; + let negative_json = serde_json::to_string(&negative_patterns).map_err(|error| { + MetastoreError::Internal { + message: "failed to serialize negative patterns".to_string(), + cause: error.to_string(), + } + })?; + + if request.overwrite { + sqlx::query(UPSERT_INDEX_TEMPLATE_QUERY) + .bind(&index_template.template_id) + .bind(&positive_json) + .bind(&negative_json) + .bind(index_template.priority as i32) + .bind(&request.index_template_json) + .execute(&self.connection_pool) + .await?; + + return Ok(EmptyResponse {}); + } + let insert_result = sqlx::query(INSERT_INDEX_TEMPLATE_QUERY) + .bind(&index_template.template_id) + .bind(&positive_json) + .bind(&negative_json) + .bind(index_template.priority as i32) + .bind(&request.index_template_json) + .execute(&self.connection_pool) + .await; + + match insert_result { + Ok(_) => Ok(EmptyResponse {}), + Err(sqlx::Error::Database(ref db_error)) + if db_error + .try_downcast_ref::() + .map(|err| err.number() == 1062) + .unwrap_or(false) => + { + Err(MetastoreError::AlreadyExists(EntityKind::IndexTemplate { + template_id: index_template.template_id, + })) + } + Err(error) => Err(error.into()), + } + } + + async fn get_index_template( + &self, + request: GetIndexTemplateRequest, + ) -> MetastoreResult { + let mysql_index_template: MysqlIndexTemplate = + sqlx::query_as("SELECT * FROM index_templates WHERE template_id = ?") + .bind(&request.template_id) + .fetch_optional(&self.connection_pool) + .await? + .ok_or({ + MetastoreError::NotFound(EntityKind::IndexTemplate { + template_id: request.template_id, + }) + })?; + let response = GetIndexTemplateResponse { + index_template_json: mysql_index_template.index_template_json, + }; + Ok(response) + } + + async fn find_index_template_matches( + &self, + request: FindIndexTemplateMatchesRequest, + ) -> MetastoreResult { + if request.index_ids.is_empty() { + return Ok(Default::default()); + } + const FIND_INDEX_TEMPLATE_MATCHES_QUERY: &str = + include_str!("queries/index_templates/find.sql"); + + let index_ids_json = serde_json::to_string(&request.index_ids).map_err(|error| { + MetastoreError::Internal { + message: "failed to serialize index_ids".to_string(), + cause: error.to_string(), + } + })?; + + let sql_matches: Vec<(IndexId, IndexTemplateId, String)> = + sqlx::query_as(FIND_INDEX_TEMPLATE_MATCHES_QUERY) + .bind(&index_ids_json) + .fetch_all(&self.connection_pool) + .await?; + + let matches = sql_matches + .into_iter() + .map( + |(index_id, template_id, index_template_json)| IndexTemplateMatch { + index_id, + template_id, + index_template_json, + }, + ) + .collect(); + let response = FindIndexTemplateMatchesResponse { matches }; + Ok(response) + } + + async fn list_index_templates( + &self, + _request: ListIndexTemplatesRequest, + ) -> MetastoreResult { + let mysql_index_templates_json: Vec<(String,)> = sqlx::query_as( + "SELECT index_template_json FROM index_templates ORDER BY template_id ASC", + ) + .fetch_all(&self.connection_pool) + .await?; + let index_templates_json: Vec = mysql_index_templates_json + .into_iter() + .map(|(index_template_json,)| index_template_json) + .collect(); + let response = ListIndexTemplatesResponse { + index_templates_json, + }; + Ok(response) + } + + async fn delete_index_templates( + &self, + request: DeleteIndexTemplatesRequest, + ) -> MetastoreResult { + if request.template_ids.is_empty() { + return Ok(EmptyResponse {}); + } + let placeholders = vec!["?"; request.template_ids.len()].join(", "); + let sql = format!("DELETE FROM index_templates WHERE template_id IN ({placeholders})"); + let mut query = sqlx::query(&sql); + for template_id in &request.template_ids { + query = query.bind(template_id); + } + query.execute(&self.connection_pool).await?; + Ok(EmptyResponse {}) + } + + // Cluster Identity + + async fn get_cluster_identity( + &self, + _: GetClusterIdentityRequest, + ) -> MetastoreResult { + // MySQL: INSERT + SELECT on same connection (no RETURNING). + let mut conn = self.connection_pool.acquire().await?; + let new_uuid = Uuid::new_v4().hyphenated().to_string(); + + sqlx::query( + r#" + INSERT INTO kv (`key`, value) + VALUES ('cluster_identity', ?) + ON DUPLICATE KEY UPDATE `key` = `key` + "#, + ) + .bind(&new_uuid) + .execute(&mut *conn) + .await?; + + let (uuid,): (String,) = + sqlx::query_as("SELECT value FROM kv WHERE `key` = 'cluster_identity'") + .fetch_one(&mut *conn) + .await?; + + Ok(GetClusterIdentityResponse { uuid }) + } +} + +async fn open_or_fetch_shard<'e>( + executor: impl Executor<'e, Database = MySql> + Clone, + subrequest: &OpenShardSubrequest, +) -> MetastoreResult { + // MySQL: INSERT ... ON DUPLICATE KEY UPDATE pk = pk + separate SELECT + // instead of INSERT ... ON CONFLICT DO NOTHING RETURNING * + let insert_result = sqlx::query( + r#" + INSERT INTO shards(index_uid, source_id, shard_id, leader_id, follower_id, + doc_mapping_uid, publish_token, update_timestamp) + VALUES (?, ?, ?, ?, ?, ?, ?, ?) + ON DUPLICATE KEY UPDATE index_uid = index_uid + "#, + ) + .bind(subrequest.index_uid()) + .bind(&subrequest.source_id) + .bind(subrequest.shard_id().as_str()) + .bind(&subrequest.leader_id) + .bind(&subrequest.follower_id) + .bind(subrequest.doc_mapping_uid) + .bind(&subrequest.publish_token) + .bind(OffsetDateTime::now_utc()) + .execute(executor.clone()) + .await; + + match insert_result { + Ok(result) => { + // Fetch the shard (whether newly inserted or already existing). + const FETCH_SHARD_QUERY: &str = include_str!("queries/shards/fetch.sql"); + let mysql_shard_opt: Option = sqlx::query_as(FETCH_SHARD_QUERY) + .bind(subrequest.index_uid()) + .bind(&subrequest.source_id) + .bind(subrequest.shard_id().as_str()) + .fetch_optional(executor.clone()) + .await?; + + if let Some(mysql_shard) = mysql_shard_opt { + let shard: Shard = mysql_shard.into(); + if result.rows_affected() == 1 { + info!( + index_uid=%shard.index_uid(), + source_id=%shard.source_id, + shard_id=%shard.shard_id(), + leader_id=%shard.leader_id, + follower_id=?shard.follower_id, + "opened shard" + ); + } + return Ok(shard); + } + } + Err(sqlx_error) => { + // FK violation means the source/index doesn't exist. + if let sqlx::Error::Database(ref db_error) = sqlx_error + && let Some(mysql_error) = + db_error.try_downcast_ref::() + && mysql_error.number() == 1452 + { + return Err(MetastoreError::NotFound(EntityKind::Source { + index_id: subrequest.index_uid().to_string(), + source_id: subrequest.source_id.clone(), + })); + } + return Err(sqlx_error.into()); + } + } + + Err(MetastoreError::NotFound(EntityKind::Source { + index_id: subrequest.index_uid().to_string(), + source_id: subrequest.source_id.clone(), + })) +} + +impl MetastoreServiceExt for MysqlMetastore {} + +/// Builds the SQL query that returns indexes matching at least one pattern in +/// `index_id_patterns`, and none of the patterns starting with '-' +fn build_index_id_patterns_sql_query(index_id_patterns: &[String]) -> anyhow::Result { + let mut positive_patterns = Vec::new(); + let mut negative_patterns = Vec::new(); + for pattern in index_id_patterns { + if let Some(negative_pattern) = pattern.strip_prefix('-') { + negative_patterns.push(negative_pattern.to_string()); + } else { + positive_patterns.push(pattern); + } + } + + if positive_patterns.is_empty() { + anyhow::bail!("The list of index id patterns may not be empty."); + } + + if index_id_patterns.iter().any(|pattern| pattern == "*") && negative_patterns.is_empty() { + return Ok("SELECT * FROM indexes".to_string()); + } + + let mut where_like_query = String::new(); + for (index_id_pattern_idx, index_id_pattern) in positive_patterns.iter().enumerate() { + validate_index_id_pattern(index_id_pattern, false).map_err(|error| { + MetastoreError::Internal { + message: "failed to build list indexes query".to_string(), + cause: error.to_string(), + } + })?; + if index_id_pattern_idx != 0 { + where_like_query.push_str(" OR "); + } + if index_id_pattern.contains('*') { + let sql_pattern = index_id_pattern.replace('*', "%"); + let _ = write!(where_like_query, "index_id LIKE '{sql_pattern}'"); + } else { + let _ = write!(where_like_query, "index_id = '{index_id_pattern}'"); + } + } + let mut negative_like_query = String::new(); + for index_id_pattern in negative_patterns.iter() { + validate_index_id_pattern(index_id_pattern, false).map_err(|error| { + MetastoreError::Internal { + message: "failed to build list indexes query".to_string(), + cause: error.to_string(), + } + })?; + negative_like_query.push_str(" AND "); + if index_id_pattern.contains('*') { + let sql_pattern = index_id_pattern.replace('*', "%"); + let _ = write!(negative_like_query, "index_id NOT LIKE '{sql_pattern}'"); + } else { + let _ = write!(negative_like_query, "index_id <> '{index_id_pattern}'"); + } + } + + Ok(format!( + "SELECT * FROM indexes WHERE ({where_like_query}){negative_like_query}" + )) +} + +#[cfg(test)] +#[async_trait] +impl crate::tests::DefaultForTest for MysqlMetastore { + async fn default_for_test() -> Self { + dotenvy::dotenv().ok(); + let uri: Uri = std::env::var("QW_TEST_MYSQL_DATABASE_URL") + .expect("environment variable `QW_TEST_MYSQL_DATABASE_URL` should be set") + .parse() + .expect("environment variable `QW_TEST_MYSQL_DATABASE_URL` should be a valid URI"); + MysqlMetastore::new(&MysqlMetastoreConfig::default(), &uri) + .await + .expect("failed to initialize MySQL metastore test") + } +} + +#[cfg(test)] +mod tests { + use async_trait::async_trait; + use quickwit_proto::ingest::{Shard, ShardState}; + use quickwit_proto::types::{IndexUid, SourceId}; + use time::OffsetDateTime; + + use super::*; + use crate::metastore::mysql::model::MysqlShard; + use crate::metastore_test_suite; + use crate::tests::shard::ReadWriteShardsForTest; + + #[async_trait] + impl ReadWriteShardsForTest for MysqlMetastore { + async fn insert_shards( + &self, + index_uid: &IndexUid, + source_id: &SourceId, + shards: Vec, + ) { + const INSERT_SHARD_QUERY: &str = include_str!("queries/shards/insert.sql"); + + for shard in shards { + assert_eq!(&shard.source_id, source_id); + assert_eq!(shard.index_uid(), index_uid); + let Shard { + doc_mapping_uid, + follower_id, + index_uid, + leader_id, + publish_position_inclusive, + publish_token, + shard_id, + shard_state, + source_id, + update_timestamp, + } = shard; + let shard_state_name = ShardState::try_from(shard_state) + .unwrap() + .as_json_str_name(); + let update_timestamp = OffsetDateTime::from_unix_timestamp(update_timestamp) + .expect("bad timestamp format"); + sqlx::query(INSERT_SHARD_QUERY) + .bind(index_uid) + .bind(source_id) + .bind(shard_id.unwrap()) + .bind(shard_state_name) + .bind(leader_id) + .bind(follower_id) + .bind(doc_mapping_uid) + .bind(publish_position_inclusive.unwrap().to_string()) + .bind(publish_token) + .bind(update_timestamp) + .execute(&self.connection_pool) + .await + .unwrap(); + } + } + + async fn list_all_shards(&self, index_uid: &IndexUid, source_id: &SourceId) -> Vec { + let mysql_shards: Vec = sqlx::query_as( + r#" + SELECT * + FROM shards + WHERE + index_uid = ? + AND source_id = ? + "#, + ) + .bind(index_uid) + .bind(source_id) + .fetch_all(&self.connection_pool) + .await + .unwrap(); + + mysql_shards + .into_iter() + .map(|mysql_shard| mysql_shard.into()) + .collect() + } + } + + metastore_test_suite!(crate::MysqlMetastore); +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/metrics.rs b/quickwit/quickwit-metastore/src/metastore/mysql/metrics.rs new file mode 100644 index 00000000000..d36d8d76248 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/metrics.rs @@ -0,0 +1,71 @@ +// 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 once_cell::sync::Lazy; +use quickwit_common::metrics::{IntCounter, IntGauge, new_counter, new_gauge}; + +#[derive(Clone)] +pub(super) struct MysqlMetrics { + pub acquire_connections: IntGauge, + pub active_connections: IntGauge, + pub idle_connections: IntGauge, + pub iam_token_refresh_success: IntCounter, + pub iam_token_refresh_failure: IntCounter, + pub iam_token_last_refresh_timestamp_secs: IntGauge, +} + +impl Default for MysqlMetrics { + fn default() -> Self { + Self { + acquire_connections: new_gauge( + "mysql_acquire_connections", + "Number of MySQL connections being acquired.", + "metastore", + &[], + ), + active_connections: new_gauge( + "mysql_active_connections", + "Number of active (used + idle) MySQL connections.", + "metastore", + &[], + ), + idle_connections: new_gauge( + "mysql_idle_connections", + "Number of idle MySQL connections.", + "metastore", + &[], + ), + iam_token_refresh_success: new_counter( + "mysql_iam_token_refresh_success_total", + "Number of successful IAM token refreshes.", + "metastore", + &[], + ), + iam_token_refresh_failure: new_counter( + "mysql_iam_token_refresh_failure_total", + "Number of failed IAM token refreshes.", + "metastore", + &[], + ), + iam_token_last_refresh_timestamp_secs: new_gauge( + "mysql_iam_token_last_refresh_timestamp_secs", + "Unix timestamp of the last successful IAM token refresh.", + "metastore", + &[], + ), + } + } +} + +pub(super) static MYSQL_METRICS: Lazy = Lazy::new(MysqlMetrics::default); diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/migrator.rs b/quickwit/quickwit-metastore/src/metastore/mysql/migrator.rs new file mode 100644 index 00000000000..e3a8d9ac579 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/migrator.rs @@ -0,0 +1,146 @@ +// 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 std::collections::BTreeMap; + +use quickwit_proto::metastore::{MetastoreError, MetastoreResult}; +use sqlx::migrate::{Migrate, Migrator}; +use sqlx::{Acquire, MySql, MySqlConnection}; +use tracing::{error, instrument}; + +use super::pool::TrackedPool; + +fn get_migrations() -> Migrator { + sqlx::migrate!("migrations/mysql") +} + +/// Initializes the database and runs the SQL migrations stored in the +/// `quickwit-metastore/migrations/mysql` directory. +#[instrument(skip_all)] +pub(super) async fn run_migrations( + pool: &TrackedPool, + skip_migrations: bool, + skip_locking: bool, +) -> MetastoreResult<()> { + let mut migrator = get_migrations(); + + if skip_locking { + migrator.set_locking(false); + } + + if !skip_migrations { + // MySQL DDL statements (CREATE TABLE, ALTER TABLE, etc.) cause implicit + // commits, which breaks transaction-wrapped migrations. We must use a plain + // connection instead of a transaction. + let mut conn = pool.acquire().await?; + let migrate_result = migrator.run_direct(&mut *conn).await; + + if let Err(migrate_error) = migrate_result { + error!(error=%migrate_error, "failed to run MySQL migrations"); + return Err(MetastoreError::Internal { + message: "failed to run MySQL migrations".to_string(), + cause: migrate_error.to_string(), + }); + } + Ok(()) + } else { + let mut conn = pool.acquire().await?; + check_migrations(migrator, &mut conn).await + } +} + +async fn check_migrations(migrator: Migrator, conn: &mut MySqlConnection) -> MetastoreResult<()> { + let dirty = match conn.dirty_version().await { + Ok(dirty) => dirty, + Err(migrate_error) => { + error!(error=%migrate_error, "failed to validate MySQL migrations"); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: migrate_error.to_string(), + }); + } + }; + if let Some(dirty) = dirty { + error!("migration {dirty} is dirty"); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: format!("migration {dirty} is dirty"), + }); + }; + let applied_migrations = match conn.list_applied_migrations().await { + Ok(applied_migrations) => applied_migrations, + Err(migrate_error) => { + error!(error=%migrate_error, "failed to validate MySQL migrations"); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: migrate_error.to_string(), + }); + } + }; + let expected_migrations: BTreeMap<_, _> = migrator + .iter() + .filter(|migration| migration.migration_type.is_up_migration()) + .map(|migration| (migration.version, migration)) + .collect(); + if applied_migrations.len() < expected_migrations.len() { + error!( + "missing migrations, expected {} migrations, only {} present in database", + expected_migrations.len(), + applied_migrations.len() + ); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: format!( + "missing migrations, expected {} migrations, only {} present in database", + expected_migrations.len(), + applied_migrations.len() + ), + }); + } + for applied_migration in applied_migrations { + let Some(migration) = expected_migrations.get(&applied_migration.version) else { + error!( + "found unknown migration {} in database", + applied_migration.version + ); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: format!( + "found unknown migration {} in database", + applied_migration.version + ), + }); + }; + if migration.checksum != applied_migration.checksum { + error!( + "migration {} differ between database and expected value", + applied_migration.version + ); + + return Err(MetastoreError::Internal { + message: "failed to validate MySQL migrations".to_string(), + cause: format!( + "migration {} differ between database and expected value", + applied_migration.version + ), + }); + } + } + Ok(()) +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/mod.rs b/quickwit/quickwit-metastore/src/metastore/mysql/mod.rs new file mode 100644 index 00000000000..eb952487014 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/mod.rs @@ -0,0 +1,32 @@ +// 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. + +mod auth; +mod error; +mod factory; +mod metastore; +mod metrics; +mod migrator; +mod model; +mod pool; +mod split_stream; +mod tags; +mod utils; + +pub use factory::MysqlMetastoreFactory; +pub use metastore::MysqlMetastore; + +const QW_MYSQL_SKIP_MIGRATIONS_ENV_KEY: &str = "QW_MYSQL_SKIP_MIGRATIONS"; +const QW_MYSQL_SKIP_MIGRATION_LOCKING_ENV_KEY: &str = "QW_MYSQL_SKIP_MIGRATION_LOCKING"; +const QW_MYSQL_READ_ONLY_ENV_KEY: &str = "QW_MYSQL_READ_ONLY"; diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/model.rs b/quickwit/quickwit-metastore/src/metastore/mysql/model.rs new file mode 100644 index 00000000000..3103153c5f3 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/model.rs @@ -0,0 +1,259 @@ +// 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. + +#![allow(dead_code)] + +use std::convert::TryInto; +use std::str::FromStr; + +use quickwit_proto::ingest::{Shard, ShardState}; +use quickwit_proto::metastore::{DeleteQuery, DeleteTask, MetastoreError, MetastoreResult}; +use quickwit_proto::types::{DocMappingUid, IndexId, IndexUid, ShardId, SourceId, SplitId}; +use sea_query::{Iden, Write}; +use tracing::error; + +use crate::{IndexMetadata, Split, SplitMetadata, SplitState}; + +#[derive(Iden, Clone, Copy)] +#[allow(dead_code)] +pub enum Indexes { + Table, + IndexUid, + IndexId, + IndexMetadataJson, + CreateTimestamp, +} + +/// A model structure for handling index metadata in a database. +#[derive(sqlx::FromRow)] +pub(super) struct MysqlIndex { + #[sqlx(try_from = "String")] + pub index_uid: IndexUid, + pub index_id: IndexId, + pub index_metadata_json: String, + pub create_timestamp: sqlx::types::time::PrimitiveDateTime, +} + +impl MysqlIndex { + pub fn index_metadata(&self) -> MetastoreResult { + let mut index_metadata = serde_json::from_str::(&self.index_metadata_json) + .map_err(|error| { + error!(index_id=%self.index_id, error=?error, "failed to deserialize index metadata"); + + MetastoreError::JsonDeserializeError { + struct_name: "IndexMetadata".to_string(), + message: error.to_string(), + } + })?; + index_metadata.create_timestamp = self.create_timestamp.assume_utc().unix_timestamp(); + Ok(index_metadata) + } +} + +#[derive(Iden, Clone, Copy)] +#[allow(dead_code)] +pub enum Splits { + Table, + SplitId, + SplitState, + TimeRangeStart, + TimeRangeEnd, + CreateTimestamp, + UpdateTimestamp, + PublishTimestamp, + MaturityTimestamp, + Tags, + SplitMetadataJson, + IndexUid, + NodeId, + DeleteOpstamp, +} + +pub(super) struct FromUnixTimeFunc; + +impl Iden for FromUnixTimeFunc { + fn unquoted(&self, s: &mut dyn Write) { + write!(s, "FROM_UNIXTIME").unwrap() + } +} + +/// A model structure for handling split metadata in a database. +#[derive(sqlx::FromRow)] +pub(super) struct MysqlSplit { + pub split_id: SplitId, + pub split_state: String, + pub time_range_start: Option, + pub time_range_end: Option, + pub create_timestamp: sqlx::types::time::PrimitiveDateTime, + pub update_timestamp: sqlx::types::time::PrimitiveDateTime, + pub publish_timestamp: Option, + pub maturity_timestamp: sqlx::types::time::PrimitiveDateTime, + pub tags: sqlx::types::Json>, + pub split_metadata_json: String, + #[sqlx(try_from = "String")] + pub index_uid: IndexUid, + pub delete_opstamp: i64, +} + +impl MysqlSplit { + fn split_metadata(&self) -> MetastoreResult { + serde_json::from_str::(&self.split_metadata_json).map_err(|error| { + error!(index_id=%self.index_uid.index_id, split_id=%self.split_id, error=?error, "failed to deserialize split metadata"); + + MetastoreError::JsonDeserializeError { + struct_name: "SplitMetadata".to_string(), + message: error.to_string(), + } + }) + } + + fn split_state(&self) -> MetastoreResult { + SplitState::from_str(&self.split_state).map_err(|error| { + error!(index_id=%self.index_uid.index_id, split_id=%self.split_id, split_state=?self.split_state, error=?error, "failed to deserialize split state"); + MetastoreError::JsonDeserializeError { + struct_name: "SplitState".to_string(), + message: error, + } + }) + } +} + +impl TryInto for MysqlSplit { + type Error = MetastoreError; + + fn try_into(self) -> Result { + let mut split_metadata = self.split_metadata()?; + split_metadata.create_timestamp = self.create_timestamp.assume_utc().unix_timestamp(); + let split_state = self.split_state()?; + let update_timestamp = self.update_timestamp.assume_utc().unix_timestamp(); + let publish_timestamp = self + .publish_timestamp + .map(|publish_timestamp| publish_timestamp.assume_utc().unix_timestamp()); + split_metadata.index_uid = self.index_uid; + split_metadata.delete_opstamp = self.delete_opstamp as u64; + Ok(Split { + split_metadata, + split_state, + update_timestamp, + publish_timestamp, + }) + } +} + +/// A model structure for handling delete tasks in a database. +#[derive(sqlx::FromRow)] +pub(super) struct MysqlDeleteTask { + pub create_timestamp: sqlx::types::time::PrimitiveDateTime, + pub opstamp: i64, + #[sqlx(try_from = "String")] + pub index_uid: IndexUid, + pub delete_query_json: String, +} + +impl MysqlDeleteTask { + fn delete_query(&self) -> MetastoreResult { + serde_json::from_str::(&self.delete_query_json).map_err(|error| { + error!(index_id=%self.index_uid.index_id, opstamp=%self.opstamp, error=?error, "failed to deserialize delete query"); + + MetastoreError::JsonDeserializeError { + struct_name: "DeleteQuery".to_string(), + message: error.to_string(), + } + }) + } +} + +impl TryInto for MysqlDeleteTask { + type Error = MetastoreError; + + fn try_into(self) -> Result { + let delete_query = self.delete_query()?; + Ok(DeleteTask { + create_timestamp: self.create_timestamp.assume_utc().unix_timestamp(), + opstamp: self.opstamp as u64, + delete_query: Some(delete_query), + }) + } +} + +#[derive(Iden, Clone, Copy)] +pub(super) enum Shards { + Table, + IndexUid, + SourceId, + ShardId, + ShardState, + LeaderId, + FollowerId, + PublishPositionInclusive, + PublishToken, +} + +pub(super) fn shard_state_to_str(shard_state: i32) -> &'static str { + match ShardState::try_from(shard_state).unwrap_or(ShardState::Unspecified) { + ShardState::Unspecified => "unspecified", + ShardState::Open => "open", + ShardState::Unavailable => "unavailable", + ShardState::Closed => "closed", + } +} + +pub(super) fn str_to_shard_state(s: &str) -> ShardState { + match s { + "open" => ShardState::Open, + "unavailable" => ShardState::Unavailable, + "closed" => ShardState::Closed, + _ => ShardState::Unspecified, + } +} + +#[derive(sqlx::FromRow, Debug)] +pub(super) struct MysqlShard { + #[sqlx(try_from = "String")] + pub index_uid: IndexUid, + #[sqlx(try_from = "String")] + pub source_id: SourceId, + #[sqlx(try_from = "String")] + pub shard_id: ShardId, + pub leader_id: String, + pub follower_id: Option, + pub shard_state: String, + #[sqlx(try_from = "String")] + pub doc_mapping_uid: DocMappingUid, + pub publish_position_inclusive: String, + pub publish_token: Option, + pub update_timestamp: sqlx::types::time::PrimitiveDateTime, +} + +impl From for Shard { + fn from(mysql_shard: MysqlShard) -> Self { + Shard { + index_uid: Some(mysql_shard.index_uid), + source_id: mysql_shard.source_id, + shard_id: Some(mysql_shard.shard_id), + shard_state: str_to_shard_state(&mysql_shard.shard_state) as i32, + leader_id: mysql_shard.leader_id, + follower_id: mysql_shard.follower_id, + doc_mapping_uid: Some(mysql_shard.doc_mapping_uid), + publish_position_inclusive: Some(mysql_shard.publish_position_inclusive.into()), + publish_token: mysql_shard.publish_token, + update_timestamp: mysql_shard.update_timestamp.assume_utc().unix_timestamp(), + } + } +} + +#[derive(sqlx::FromRow, Debug)] +pub(super) struct MysqlIndexTemplate { + pub index_template_json: String, +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/pool.rs b/quickwit/quickwit-metastore/src/metastore/mysql/pool.rs new file mode 100644 index 00000000000..aeec0373b52 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/pool.rs @@ -0,0 +1,126 @@ +// 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 futures::future::BoxFuture; +use futures::stream::BoxStream; +use quickwit_common::metrics::GaugeGuard; +use sqlx::pool::PoolConnection; +use sqlx::pool::maybe::MaybePoolConnection; +use sqlx::{ + Acquire, Database, Describe, Either, Error, Execute, Executor, MySql, Pool, Transaction, +}; + +use super::metrics::MYSQL_METRICS; + +#[derive(Debug)] +pub(super) struct TrackedPool { + inner_pool: Pool, +} + +impl TrackedPool { + pub fn new(inner_pool: Pool) -> Self { + Self { inner_pool } + } + + pub fn set_connect_options(&self, options: sqlx::mysql::MySqlConnectOptions) { + self.inner_pool.set_connect_options(options); + } +} + +impl Clone for TrackedPool { + fn clone(&self) -> Self { + Self { + inner_pool: self.inner_pool.clone(), + } + } +} + +impl<'a, DB: Database> Acquire<'a> for &TrackedPool { + type Database = DB; + + type Connection = PoolConnection; + + fn acquire(self) -> BoxFuture<'static, Result> { + let acquire_conn_fut = self.inner_pool.acquire(); + + MYSQL_METRICS + .active_connections + .set(self.inner_pool.size() as i64); + MYSQL_METRICS + .idle_connections + .set(self.inner_pool.num_idle() as i64); + + Box::pin(async move { + let mut gauge_guard = GaugeGuard::from_gauge(&MYSQL_METRICS.acquire_connections); + gauge_guard.add(1); + + let conn = acquire_conn_fut.await?; + Ok(conn) + }) + } + + fn begin(self) -> BoxFuture<'static, Result, Error>> { + let acquire_conn_fut = self.acquire(); + + Box::pin(async move { + Transaction::begin( + MaybePoolConnection::PoolConnection(acquire_conn_fut.await?), + None, + ) + .await + }) + } +} + +impl Executor<'_> for &TrackedPool +where for<'c> &'c mut DB::Connection: Executor<'c, Database = DB> +{ + type Database = DB; + + fn fetch_many<'e, 'q: 'e, E>( + self, + query: E, + ) -> BoxStream<'e, Result, Error>> + where + E: Execute<'q, Self::Database> + 'q, + { + self.inner_pool.fetch_many(query) + } + + fn fetch_optional<'e, 'q: 'e, E>( + self, + query: E, + ) -> BoxFuture<'e, Result, Error>> + where + E: Execute<'q, Self::Database> + 'q, + { + self.inner_pool.fetch_optional(query) + } + + fn prepare_with<'e, 'q: 'e>( + self, + sql: &'q str, + parameters: &'e [::TypeInfo], + ) -> BoxFuture<'e, Result<::Statement<'q>, Error>> { + self.inner_pool.prepare_with(sql, parameters) + } + + #[doc(hidden)] + fn describe<'e, 'q: 'e>( + self, + sql: &'q str, + ) -> BoxFuture<'e, Result, Error>> { + self.inner_pool.describe(sql) + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/find.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/find.sql new file mode 100644 index 00000000000..452cf3871ba --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/find.sql @@ -0,0 +1,19 @@ +SELECT index_id, template_id, index_template_json +FROM ( + SELECT + ids.index_id, + t.template_id, + t.index_template_json, + ROW_NUMBER() OVER (PARTITION BY ids.index_id ORDER BY t.priority DESC, t.template_id ASC) AS rn + FROM JSON_TABLE(CAST(? AS JSON), '$[*]' COLUMNS(index_id VARCHAR(255) PATH '$')) AS ids + CROSS JOIN index_templates t + WHERE ( + SELECT COUNT(*) FROM JSON_TABLE(t.positive_index_id_patterns, '$[*]' COLUMNS(pat VARCHAR(255) PATH '$')) AS pp + WHERE ids.index_id LIKE pp.pat + ) > 0 + AND ( + SELECT COUNT(*) FROM JSON_TABLE(t.negative_index_id_patterns, '$[*]' COLUMNS(pat VARCHAR(255) PATH '$')) AS np + WHERE ids.index_id LIKE np.pat + ) = 0 +) ranked +WHERE rn = 1 diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/insert.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/insert.sql new file mode 100644 index 00000000000..514186534a9 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/insert.sql @@ -0,0 +1,2 @@ +INSERT INTO index_templates(template_id, positive_index_id_patterns, negative_index_id_patterns, priority, index_template_json) + VALUES (?, ?, ?, ?, ?) diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/upsert.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/upsert.sql new file mode 100644 index 00000000000..ce759b8b150 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/index_templates/upsert.sql @@ -0,0 +1,8 @@ +INSERT INTO index_templates(template_id, positive_index_id_patterns, negative_index_id_patterns, priority, index_template_json) + VALUES (?, ?, ?, ?, ?) + AS new +ON DUPLICATE KEY UPDATE + positive_index_id_patterns = new.positive_index_id_patterns, + negative_index_id_patterns = new.negative_index_id_patterns, + priority = new.priority, + index_template_json = new.index_template_json diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/fetch.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/fetch.sql new file mode 100644 index 00000000000..22ea479fead --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/fetch.sql @@ -0,0 +1 @@ +SELECT * FROM shards WHERE index_uid = ? AND source_id = ? AND shard_id = ? diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/insert.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/insert.sql new file mode 100644 index 00000000000..f1e522df45f --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/insert.sql @@ -0,0 +1,2 @@ +INSERT INTO shards(index_uid, source_id, shard_id, shard_state, leader_id, follower_id, doc_mapping_uid, publish_position_inclusive, publish_token, update_timestamp) + VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?) diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_age.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_age.sql new file mode 100644 index 00000000000..346c49fa60c --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_age.sql @@ -0,0 +1,4 @@ +DELETE FROM shards +WHERE index_uid = ? + AND source_id = ? + AND update_timestamp < ? diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_count.sql b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_count.sql new file mode 100644 index 00000000000..992246beed6 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/queries/shards/prune_count.sql @@ -0,0 +1,13 @@ +DELETE FROM shards +WHERE index_uid = ? + AND source_id = ? + AND shard_id NOT IN ( + SELECT shard_id FROM ( + SELECT shard_id + FROM shards + WHERE index_uid = ? + AND source_id = ? + ORDER BY update_timestamp DESC, shard_id DESC + LIMIT ? + ) AS recent + ) diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/split_stream.rs b/quickwit/quickwit-metastore/src/metastore/mysql/split_stream.rs new file mode 100644 index 00000000000..9bc838935d7 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/split_stream.rs @@ -0,0 +1,40 @@ +// 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 std::pin::Pin; +use std::task::{Context, Poll}; + +use futures::stream::BoxStream; +use ouroboros::self_referencing; +use sqlx::MySql; +use tokio_stream::Stream; + +use super::pool::TrackedPool; + +#[self_referencing(pub_extras)] +pub struct SplitStream { + connection_pool: TrackedPool, + sql: String, + #[borrows(connection_pool, sql)] + #[covariant] + inner: BoxStream<'this, Result>, +} + +impl Stream for SplitStream { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + SplitStream::with_inner_mut(&mut self, |this| Pin::new(&mut this.as_mut()).poll_next(cx)) + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/tags.rs b/quickwit/quickwit-metastore/src/metastore/mysql/tags.rs new file mode 100644 index 00000000000..2bae6992bce --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/tags.rs @@ -0,0 +1,203 @@ +// 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_doc_mapper::tag_pruning::TagFilterAst; +use sea_query::{Cond, Expr, all}; + +/// Takes a tag filter AST and returns a SQL condition using MySQL's `MEMBER OF()` +/// for JSON array membership testing. Tag values are always bound as parameters +/// via `Expr::cust_with_values` to prevent SQL injection. +pub(super) fn generate_sql_condition(tag_ast: &TagFilterAst) -> Cond { + match tag_ast { + TagFilterAst::And(child_asts) => { + if child_asts.is_empty() { + return all![Expr::cust("TRUE")]; + } + child_asts + .iter() + .map(generate_sql_condition) + .fold(Cond::all(), |cond, child_cond| cond.add(child_cond)) + } + TagFilterAst::Or(child_asts) => { + if child_asts.is_empty() { + return all![Expr::cust("TRUE")]; + } + child_asts + .iter() + .map(generate_sql_condition) + .fold(Cond::any(), |cond, child_cond| cond.add(child_cond)) + } + TagFilterAst::Tag { tag, is_present } => { + let expr = Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new(tag.to_string())))], + ); + if *is_present { + all![expr] + } else { + all![expr.not()] + } + } + } +} + +#[cfg(test)] +mod tests { + use quickwit_doc_mapper::tag_pruning::{no_tag, tag}; + use sea_query::any; + + use super::*; + + fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: Cond) { + assert_eq!(generate_sql_condition(&tags_ast), expected); + } + + #[test] + fn test_tags_filter_expression_single_tag() { + let tags_ast = tag("my_field:titi"); + + let expected = all![Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "my_field:titi".to_string() + )))], + )]; + + test_tags_filter_expression_helper(tags_ast, expected); + } + + #[test] + fn test_tags_filter_expression_not_tag() { + let expected = all![ + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "my_field:titi".to_string() + )))], + ) + .not() + ]; + + test_tags_filter_expression_helper(no_tag("my_field:titi"), expected); + } + + #[test] + fn test_tags_filter_expression_ands() { + let tags_ast = TagFilterAst::And(vec![tag("tag:val1"), tag("tag:val2"), tag("tag:val3")]); + + let expected = all![ + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val1".to_string() + )))], + ), + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val2".to_string() + )))], + ), + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val3".to_string() + )))], + ), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); + } + + #[test] + fn test_tags_filter_expression_and_or() { + let tags_ast = TagFilterAst::Or(vec![ + TagFilterAst::And(vec![tag("tag:val1"), tag("tag:val2")]), + tag("tag:val3"), + ]); + + let expected = any![ + all![ + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val1".to_string() + )))], + ), + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val2".to_string() + )))], + ), + ], + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val3".to_string() + )))], + ), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); + } + + #[test] + fn test_tags_filter_expression_and_or_correct_parenthesis() { + let tags_ast = TagFilterAst::And(vec![ + TagFilterAst::Or(vec![tag("tag:val1"), tag("tag:val2")]), + tag("tag:val3"), + ]); + + let expected = all![ + any![ + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val1".to_string() + )))], + ), + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val2".to_string() + )))], + ), + ], + Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:val3".to_string() + )))], + ), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); + } + + #[test] + fn test_tags_special_characters() { + // Verify that special characters are handled safely via parameter binding + let tags_ast = tag("tag:value'with\"quotes$and\\backslash"); + + let expected = all![Expr::cust_with_values( + "? MEMBER OF(tags)", + [sea_query::Value::String(Some(Box::new( + "tag:value'with\"quotes$and\\backslash".to_string() + )))], + )]; + + test_tags_filter_expression_helper(tags_ast, expected); + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/mysql/utils.rs b/quickwit/quickwit-metastore/src/metastore/mysql/utils.rs new file mode 100644 index 00000000000..a74a03d97b7 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/mysql/utils.rs @@ -0,0 +1,294 @@ +// 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 std::fmt::Display; +use std::ops::Bound; +use std::str::FromStr; +use std::time::Duration; + +use quickwit_common::uri::Uri; +use quickwit_proto::metastore::{MetastoreError, MetastoreResult}; +use sea_query::{Expr, Func, Order, SelectStatement, any}; +use sqlx::mysql::{MySqlConnectOptions, MySqlPoolOptions}; +use sqlx::{ConnectOptions, Executor, MySql}; +use tracing::log::LevelFilter; +use tracing::{error, info}; + +use super::auth::{generate_rds_iam_token, spawn_token_refresh_task}; +use super::model::{FromUnixTimeFunc, Splits}; +use super::pool::TrackedPool; +use super::tags::generate_sql_condition; +use crate::metastore::{FilterRange, SortBy}; +use crate::{ListSplitsQuery, SplitMaturity, SplitMetadata}; + +/// Establishes a connection to the given database URI. +#[allow(clippy::too_many_arguments)] +pub(super) async fn establish_connection( + connection_uri: &Uri, + min_connections: usize, + max_connections: usize, + acquire_timeout: Duration, + idle_timeout_opt: Option, + max_lifetime_opt: Option, + read_only: bool, + auth_mode: &quickwit_config::MysqlAuthMode, + aws_region: Option<&str>, +) -> MetastoreResult> { + let mut connect_options: MySqlConnectOptions = + MySqlConnectOptions::from_str(connection_uri.as_str())?.log_statements(LevelFilter::Info); + + let aws_config_opt = match auth_mode { + quickwit_config::MysqlAuthMode::AwsIam => { + let mut aws_config_loader = aws_config::defaults(aws_config::BehaviorVersion::latest()); + if let Some(region) = aws_region { + aws_config_loader = + aws_config_loader.region(aws_config::Region::new(region.to_string())); + } + let aws_config = aws_config_loader.load().await; + + let region = aws_config + .region() + .map(|region| region.as_ref().to_string()) + .ok_or_else(|| MetastoreError::Internal { + message: "failed to determine AWS region for RDS IAM auth".to_string(), + cause: "no region configured; set AWS_REGION, AWS_DEFAULT_REGION, or \ + metastore.mysql.aws_region" + .to_string(), + })?; + + let host = connect_options.get_host().to_string(); + let port = connect_options.get_port(); + let user = connect_options.get_username().to_string(); + + let token = generate_rds_iam_token(&host, port, &user, ®ion, &aws_config).await?; + + connect_options = connect_options + .password(&token) + .enable_cleartext_plugin(true); + + info!(host=%host, port=%port, user=%user, region=%region, "connecting to Aurora MySQL with IAM auth"); + Some((aws_config, host, port, user, region)) + } + quickwit_config::MysqlAuthMode::Password => None, + }; + + let pool_options = MySqlPoolOptions::new() + .min_connections(min_connections as u32) + .max_connections(max_connections as u32) + .acquire_timeout(acquire_timeout) + .idle_timeout(idle_timeout_opt) + .max_lifetime(max_lifetime_opt) + .after_connect(move |conn, _meta| { + Box::pin(async move { + // Ensure all sessions use UTC for timestamp consistency. + conn.execute("SET time_zone = '+00:00'").await?; + if read_only { + conn.execute("SET SESSION TRANSACTION READ ONLY").await?; + } + Ok(()) + }) + }); + + let sqlx_pool = pool_options + .connect_with(connect_options.clone()) + .await + .map_err(|error| { + error!(connection_uri=%connection_uri, error=?error, "failed to establish connection to database"); + MetastoreError::Connection { + message: error.to_string(), + } + })?; + let tracked_pool = TrackedPool::new(sqlx_pool); + + if let Some((aws_config, host, port, user, region)) = aws_config_opt { + spawn_token_refresh_task( + tracked_pool.clone(), + connect_options, + host, + port, + user, + region, + aws_config, + ); + } + + Ok(tracked_pool) +} + +/// Extends an existing SQL string with the generated filter range appended to the query. +/// +/// This method is **not** SQL injection proof and should not be used with user-defined values. +pub(super) fn append_range_filters( + sql: &mut SelectStatement, + field_name: Splits, + filter_range: &FilterRange, + value_formatter: impl Fn(&V) -> Expr, +) { + if let Bound::Included(value) = &filter_range.start { + sql.cond_where(Expr::col(field_name).gte((value_formatter)(value))); + }; + + if let Bound::Excluded(value) = &filter_range.start { + sql.cond_where(Expr::col(field_name).gt((value_formatter)(value))); + }; + + if let Bound::Included(value) = &filter_range.end { + sql.cond_where(Expr::col(field_name).lte((value_formatter)(value))); + }; + + if let Bound::Excluded(value) = &filter_range.end { + sql.cond_where(Expr::col(field_name).lt((value_formatter)(value))); + }; +} + +pub(super) fn append_query_filters_and_order_by( + sql: &mut SelectStatement, + query: &ListSplitsQuery, +) { + if let Some(index_uids) = &query.index_uids { + sql.cond_where(Expr::col(Splits::IndexUid).is_in(index_uids)); + } + + if let Some(node_id) = &query.node_id { + sql.cond_where(Expr::col(Splits::NodeId).eq(node_id)); + }; + + if !query.split_states.is_empty() { + sql.cond_where( + Expr::col(Splits::SplitState) + .is_in(query.split_states.iter().map(|val| val.to_string())), + ); + }; + + if let Some(tags) = &query.tags { + sql.cond_where(generate_sql_condition(tags)); + }; + + if let Some(v) = query.max_time_range_end { + sql.cond_where(Expr::col(Splits::TimeRangeEnd).lte(v)); + } + + match query.time_range.start { + Bound::Included(v) => { + sql.cond_where(any![ + Expr::col(Splits::TimeRangeEnd).gte(v), + Expr::col(Splits::TimeRangeEnd).is_null() + ]); + } + Bound::Excluded(v) => { + sql.cond_where(any![ + Expr::col(Splits::TimeRangeEnd).gt(v), + Expr::col(Splits::TimeRangeEnd).is_null() + ]); + } + Bound::Unbounded => {} + }; + + match query.time_range.end { + Bound::Included(v) => { + sql.cond_where(any![ + Expr::col(Splits::TimeRangeStart).lte(v), + Expr::col(Splits::TimeRangeStart).is_null() + ]); + } + Bound::Excluded(v) => { + sql.cond_where(any![ + Expr::col(Splits::TimeRangeStart).lt(v), + Expr::col(Splits::TimeRangeStart).is_null() + ]); + } + Bound::Unbounded => {} + }; + + match &query.mature { + Bound::Included(evaluation_datetime) => { + sql.cond_where(any![ + Expr::col(Splits::MaturityTimestamp) + .eq(Func::cust(FromUnixTimeFunc).arg(Expr::val(0))), + Expr::col(Splits::MaturityTimestamp).lte( + Func::cust(FromUnixTimeFunc) + .arg(Expr::val(evaluation_datetime.unix_timestamp())) + ) + ]); + } + Bound::Excluded(evaluation_datetime) => { + sql.cond_where(Expr::col(Splits::MaturityTimestamp).gt( + Func::cust(FromUnixTimeFunc).arg(Expr::val(evaluation_datetime.unix_timestamp())), + )); + } + Bound::Unbounded => {} + }; + append_range_filters( + sql, + Splits::UpdateTimestamp, + &query.update_timestamp, + |&val| Expr::expr(Func::cust(FromUnixTimeFunc).arg(Expr::val(val))), + ); + append_range_filters( + sql, + Splits::CreateTimestamp, + &query.create_timestamp, + |&val| Expr::expr(Func::cust(FromUnixTimeFunc).arg(Expr::val(val))), + ); + append_range_filters(sql, Splits::DeleteOpstamp, &query.delete_opstamp, |&val| { + Expr::expr(val) + }); + + if let Some((index_uid, split_id)) = &query.after_split { + sql.cond_where( + Expr::tuple([ + Expr::col(Splits::IndexUid).into(), + Expr::col(Splits::SplitId).into(), + ]) + .gt(Expr::tuple([Expr::value(index_uid), Expr::value(split_id)])), + ); + } + + match query.sort_by { + SortBy::Staleness => { + sql.order_by(Splits::DeleteOpstamp, Order::Asc) + .order_by(Splits::PublishTimestamp, Order::Asc); + } + SortBy::IndexUid => { + sql.order_by(Splits::IndexUid, Order::Asc) + .order_by(Splits::SplitId, Order::Asc); + } + SortBy::None => (), + } + + if let Some(limit) = query.limit { + sql.limit(limit as u64); + } + + if let Some(offset) = query.offset { + // MySQL requires LIMIT before OFFSET; use a large limit if none was set. + if query.limit.is_none() { + sql.limit(u64::MAX); + } + sql.order_by(Splits::SplitId, Order::Asc) + .offset(offset as u64); + } +} + +/// Returns the unix timestamp at which the split becomes mature. +/// If the split is mature (`SplitMaturity::Mature`), we return 0 +/// as we don't want the maturity to depend on datetime. +pub(super) fn split_maturity_timestamp(split_metadata: &SplitMetadata) -> i64 { + match split_metadata.maturity { + SplitMaturity::Mature => 0, + SplitMaturity::Immature { maturation_period } => { + split_metadata.create_timestamp + maturation_period.as_secs() as i64 + } + } +} diff --git a/quickwit/quickwit-metastore/src/metastore_resolver.rs b/quickwit/quickwit-metastore/src/metastore_resolver.rs index 7793fdbbf45..938ebef5259 100644 --- a/quickwit/quickwit-metastore/src/metastore_resolver.rs +++ b/quickwit/quickwit-metastore/src/metastore_resolver.rs @@ -24,6 +24,8 @@ use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_storage::StorageResolver; use crate::metastore::file_backed::FileBackedMetastoreFactory; +#[cfg(feature = "mysql")] +use crate::metastore::mysql::MysqlMetastoreFactory; #[cfg(feature = "postgres")] use crate::metastore::postgres::PostgresqlMetastoreFactory; use crate::{MetastoreFactory, MetastoreResolverError}; @@ -62,6 +64,7 @@ impl MetastoreResolver { Protocol::Ram => MetastoreBackend::File, Protocol::S3 => MetastoreBackend::File, Protocol::PostgreSQL => MetastoreBackend::PostgreSQL, + Protocol::MySQL => MetastoreBackend::MySQL, _ => { return Err(MetastoreResolverError::UnsupportedBackend( "no implementation exists for this backend".to_string(), @@ -130,6 +133,31 @@ impl MetastoreResolver { PostgresMetastoreConfig::default().into(), ); } + #[cfg(feature = "mysql")] + { + builder = builder.register( + MysqlMetastoreFactory::default(), + metastore_configs + .find_mysql() + .cloned() + .unwrap_or_default() + .into(), + ); + } + #[cfg(not(feature = "mysql"))] + { + use quickwit_config::MysqlMetastoreConfig; + + use crate::UnsupportedMetastore; + + builder = builder.register( + UnsupportedMetastore::new( + MetastoreBackend::MySQL, + "Quickwit was compiled without the `mysql` feature", + ), + MysqlMetastoreConfig::default().into(), + ); + } builder .build() .expect("metastore factory and config backends should match") diff --git a/quickwit/quickwit-proto/Cargo.toml b/quickwit/quickwit-proto/Cargo.toml index fa52a1b3fea..531a4c00bc9 100644 --- a/quickwit/quickwit-proto/Cargo.toml +++ b/quickwit/quickwit-proto/Cargo.toml @@ -57,5 +57,6 @@ tonic-prost-build = { workspace = true } quickwit-codegen = { workspace = true } [features] +mysql = ["sea-query", "sqlx"] postgres = ["sea-query", "sqlx"] testsuite = ["mockall", "futures"] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs index ae434b45529..6ed35eacdb0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -470,9 +470,8 @@ impl ReplicateFailureReason { #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum IngesterStatus { - /// For nodes without an ingester. Unspecified = 0, - /// The ingester is live but not ready yet to accept requests (i.e. the Write-Ahead Log is not loaded yet). + /// The ingester is live but not ready yet to accept requests. Initializing = 1, /// The ingester is ready and accepts read and write requests. Ready = 2, diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index ba371c13d4a..7bf4071b43e 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -178,7 +178,7 @@ impl MetastoreError { } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl From for MetastoreError { fn from(error: sqlx::Error) -> Self { MetastoreError::Db { diff --git a/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs b/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs index d7657e60704..bee0c13714e 100644 --- a/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs +++ b/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs @@ -133,7 +133,7 @@ impl FromStr for DocMappingUid { } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl TryFrom for DocMappingUid { type Error = anyhow::Error; @@ -159,6 +159,23 @@ impl sqlx::Encode<'_, sqlx::Postgres> for DocMappingUid { } } +#[cfg(feature = "mysql")] +impl sqlx::Type for DocMappingUid { + fn type_info() -> sqlx::mysql::MySqlTypeInfo { + >::type_info() + } +} + +#[cfg(feature = "mysql")] +impl sqlx::Encode<'_, sqlx::MySql> for DocMappingUid { + fn encode_by_ref( + &self, + buf: &mut ::ArgumentBuffer<'_>, + ) -> Result { + >::encode_by_ref(&self.0.to_string(), buf) + } +} + #[cfg(test)] mod tests { use bytes::Bytes; diff --git a/quickwit/quickwit-proto/src/types/index_uid.rs b/quickwit/quickwit-proto/src/types/index_uid.rs index b653da7cb4b..c22dbffd38c 100644 --- a/quickwit/quickwit-proto/src/types/index_uid.rs +++ b/quickwit/quickwit-proto/src/types/index_uid.rs @@ -169,7 +169,7 @@ impl prost::Message for IndexUid { } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl TryFrom for IndexUid { type Error = InvalidIndexUid; @@ -204,20 +204,37 @@ impl sqlx::postgres::PgHasArrayType for IndexUid { } } +#[cfg(feature = "mysql")] +impl sqlx::Type for IndexUid { + fn type_info() -> sqlx::mysql::MySqlTypeInfo { + >::type_info() + } +} + +#[cfg(feature = "mysql")] +impl sqlx::Encode<'_, sqlx::MySql> for IndexUid { + fn encode_by_ref( + &self, + buf: &mut ::ArgumentBuffer<'_>, + ) -> Result { + >::encode_by_ref(&self.to_string(), buf) + } +} + impl PartialEq<(&'static str, u128)> for IndexUid { fn eq(&self, (index_id, incarnation_id): &(&str, u128)) -> bool { self.index_id == *index_id && self.incarnation_id == Ulid::from(*incarnation_id) } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl From for sea_query::Value { fn from(index_uid: IndexUid) -> Self { index_uid.to_string().into() } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl From<&IndexUid> for sea_query::Value { fn from(index_uid: &IndexUid) -> Self { index_uid.to_string().into() diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index d7e5d1ebf5f..44b63b0a644 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -276,7 +276,7 @@ impl ToOwned for NodeIdRef { } } -#[cfg(feature = "postgres")] +#[cfg(any(feature = "postgres", feature = "mysql"))] impl From<&NodeId> for sea_query::Value { fn from(node_id: &NodeId) -> Self { node_id.to_string().into() diff --git a/quickwit/quickwit-proto/src/types/shard_id.rs b/quickwit/quickwit-proto/src/types/shard_id.rs index 7a73feb6baa..92563cc28b8 100644 --- a/quickwit/quickwit-proto/src/types/shard_id.rs +++ b/quickwit/quickwit-proto/src/types/shard_id.rs @@ -154,6 +154,23 @@ impl sqlx::postgres::PgHasArrayType for ShardId { } } +#[cfg(feature = "mysql")] +impl sqlx::Type for ShardId { + fn type_info() -> sqlx::mysql::MySqlTypeInfo { + >::type_info() + } +} + +#[cfg(feature = "mysql")] +impl sqlx::Encode<'_, sqlx::MySql> for ShardId { + fn encode_by_ref( + &self, + buf: &mut ::ArgumentBuffer<'_>, + ) -> Result { + >::encode_by_ref(&self.as_str().to_string(), buf) + } +} + #[cfg(test)] mod tests { diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 1137decf466..426a021a326 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -458,16 +458,12 @@ pub async fn serve_quickwit( node_config.metastore_uri ) })?; - let max_in_flight_requests = if node_config.metastore_uri.protocol().is_database() { - node_config - .metastore_configs - .find_postgres() - .map(|config| config.max_connections.get() * 2) - .unwrap_or_default() - .max(100) - } else { - 100 - }; + let max_in_flight_requests = node_config + .metastore_configs + .find_database_max_connections(node_config.metastore_uri.protocol()) + .map(|max_conn| max_conn.get() * 2) + .unwrap_or(100) + .max(100); // These layers apply to all the RPCs of the metastore. let shared_layer = ServiceBuilder::new() .layer(METASTORE_GRPC_SERVER_METRICS_LAYER.clone())