From 29df46975fce8af89082efc78d5de91f755680ed Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 11 Mar 2022 09:38:09 -0500 Subject: [PATCH 01/22] refactor: extract out querier creation from config (#4017) * refactor: extract out querier creation from config * fix: clippy --- influxdb_iox/src/commands/run/querier.rs | 19 ++++++------ influxdb_ioxd/src/server_type/querier/mod.rs | 32 +++++++++++++++++++- 2 files changed, 40 insertions(+), 11 deletions(-) diff --git a/influxdb_iox/src/commands/run/querier.rs b/influxdb_iox/src/commands/run/querier.rs index 1f097fda71..ca06edc395 100644 --- a/influxdb_iox/src/commands/run/querier.rs +++ b/influxdb_iox/src/commands/run/querier.rs @@ -2,7 +2,6 @@ use object_store::ObjectStore; use observability_deps::tracing::*; -use querier::{database::QuerierDatabase, handler::QuerierHandlerImpl, server::QuerierServer}; use query::exec::Executor; use std::sync::Arc; use thiserror::Error; @@ -11,8 +10,10 @@ use time::SystemProvider; use clap_blocks::{catalog_dsn::CatalogDsnConfig, run_config::RunConfig}; use influxdb_ioxd::{ self, - server_type::common_state::{CommonServerState, CommonServerStateError}, - server_type::querier::QuerierServerType, + server_type::{ + common_state::{CommonServerState, CommonServerStateError}, + querier::create_querier_server_type, + }, }; #[derive(Debug, Error)] @@ -82,17 +83,15 @@ pub async fn command(config: Config) -> Result<(), Error> { info!(%num_threads, "using specified number of threads per thread pool"); let exec = Arc::new(Executor::new(num_threads)); - let database = Arc::new(QuerierDatabase::new( + let server_type = create_querier_server_type( + &common_state, + metric_registry, catalog, - Arc::clone(&metric_registry), object_store, time_provider, exec, - )); - let querier_handler = Arc::new(QuerierHandlerImpl::new(Arc::clone(&database))); - - let querier = QuerierServer::new(metric_registry, querier_handler); - let server_type = Arc::new(QuerierServerType::new(querier, database, &common_state)); + ) + .await; info!("starting querier"); diff --git a/influxdb_ioxd/src/server_type/querier/mod.rs b/influxdb_ioxd/src/server_type/querier/mod.rs index 4b74e0c886..2f7d7ef5eb 100644 --- a/influxdb_ioxd/src/server_type/querier/mod.rs +++ b/influxdb_ioxd/src/server_type/querier/mod.rs @@ -5,8 +5,16 @@ use std::{ use async_trait::async_trait; use hyper::{Body, Request, Response}; +use iox_catalog::interface::Catalog; use metric::Registry; -use querier::{database::QuerierDatabase, handler::QuerierHandler, server::QuerierServer}; +use object_store::ObjectStore; +use querier::{ + database::QuerierDatabase, + handler::{QuerierHandler, QuerierHandlerImpl}, + server::QuerierServer, +}; +use query::exec::Executor; +use time::TimeProvider; use tokio_util::sync::CancellationToken; use trace::TraceCollector; @@ -113,3 +121,25 @@ impl HttpApiErrorSource for IoxHttpError { HttpApiError::new(self.status_code(), self.to_string()) } } + +/// Instantiate a querier server +pub async fn create_querier_server_type( + common_state: &CommonServerState, + metric_registry: Arc, + catalog: Arc, + object_store: Arc, + time_provider: Arc, + exec: Arc, +) -> Arc { + let database = Arc::new(QuerierDatabase::new( + catalog, + Arc::clone(&metric_registry), + object_store, + time_provider, + exec, + )); + let querier_handler = Arc::new(QuerierHandlerImpl::new(Arc::clone(&database))); + + let querier = QuerierServer::new(metric_registry, querier_handler); + Arc::new(QuerierServerType::new(querier, database, common_state)) +} From b24ae7d23b478d0ff23bb0569bd86b7a0ea68fa0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 11 Mar 2022 09:46:34 -0500 Subject: [PATCH 02/22] refactor: extract out compactor creation from config (#4018) * refactor: extract out compactor creation from config * fix: fmt Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- compactor/src/compact.rs | 4 +-- compactor/src/handler.rs | 2 +- influxdb_iox/src/commands/run/compactor.rs | 24 +++++++++-------- influxdb_ioxd/src/server_type/compactor.rs | 30 +++++++++++++++++++++- 4 files changed, 45 insertions(+), 15 deletions(-) diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index 2159c3ab10..e95f905ad9 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -42,7 +42,7 @@ pub struct Compactor { catalog: Arc, /// Executor for running queries and compacting and persisting - exec: Executor, + exec: Arc, /// Backoff config backoff_config: BackoffConfig, @@ -54,7 +54,7 @@ impl Compactor { sequencers: Vec, catalog: Arc, object_store: Arc, - exec: Executor, + exec: Arc, backoff_config: BackoffConfig, ) -> Self { Self { diff --git a/compactor/src/handler.rs b/compactor/src/handler.rs index 8cccbf5c11..340172948a 100644 --- a/compactor/src/handler.rs +++ b/compactor/src/handler.rs @@ -62,7 +62,7 @@ impl CompactorHandlerImpl { sequencers: Vec, catalog: Arc, object_store: Arc, - exec: Executor, + exec: Arc, _registry: &metric::Registry, ) -> Self { let shutdown = CancellationToken::new(); diff --git a/influxdb_iox/src/commands/run/compactor.rs b/influxdb_iox/src/commands/run/compactor.rs index 11c58a549d..9df4ddc5ad 100644 --- a/influxdb_iox/src/commands/run/compactor.rs +++ b/influxdb_iox/src/commands/run/compactor.rs @@ -1,6 +1,5 @@ //! Implementation of command line option for running the compactor -use compactor::{handler::CompactorHandlerImpl, server::CompactorServer}; use data_types2::SequencerId; use object_store::ObjectStore; use observability_deps::tracing::*; @@ -11,8 +10,10 @@ use thiserror::Error; use clap_blocks::{catalog_dsn::CatalogDsnConfig, run_config::RunConfig}; use influxdb_ioxd::{ self, - server_type::common_state::{CommonServerState, CommonServerStateError}, - server_type::compactor::CompactorServerType, + server_type::{ + common_state::{CommonServerState, CommonServerStateError}, + compactor::create_compactor_server_type, + }, }; #[derive(Debug, Error)] @@ -85,19 +86,20 @@ pub async fn command(config: Config) -> Result<(), Error> { .map_err(Error::ObjectStoreParsing)?, ); + let exec = Arc::new(Executor::new(config.query_exect_thread_count)); + // TODO: modify config to let us get assigned sequence numbers let sequencers: Vec = vec![]; - let compactor_handler = Arc::new(CompactorHandlerImpl::new( - sequencers, + let server_type = create_compactor_server_type( + &common_state, + metric_registry, catalog, object_store, - Executor::new(config.query_exect_thread_count), - &metric_registry, - )); - - let compactor = CompactorServer::new(metric_registry, compactor_handler); - let server_type = Arc::new(CompactorServerType::new(compactor, &common_state)); + exec, + sequencers, + ) + .await; info!("starting compactor"); diff --git a/influxdb_ioxd/src/server_type/compactor.rs b/influxdb_ioxd/src/server_type/compactor.rs index d9dda0f160..8c64272cf6 100644 --- a/influxdb_ioxd/src/server_type/compactor.rs +++ b/influxdb_ioxd/src/server_type/compactor.rs @@ -4,9 +4,16 @@ use std::{ }; use async_trait::async_trait; -use compactor::{handler::CompactorHandler, server::CompactorServer}; +use compactor::{ + handler::{CompactorHandler, CompactorHandlerImpl}, + server::CompactorServer, +}; +use data_types2::SequencerId; use hyper::{Body, Request, Response}; +use iox_catalog::interface::Catalog; use metric::Registry; +use object_store::ObjectStore; +use query::exec::Executor; use tokio_util::sync::CancellationToken; use trace::TraceCollector; @@ -98,3 +105,24 @@ impl HttpApiErrorSource for IoxHttpError { HttpApiError::new(self.status_code(), self.to_string()) } } + +/// Instantiate a compactor server +pub async fn create_compactor_server_type( + common_state: &CommonServerState, + metric_registry: Arc, + catalog: Arc, + object_store: Arc, + exec: Arc, + sequencers: Vec, +) -> Arc { + let compactor_handler = Arc::new(CompactorHandlerImpl::new( + sequencers, + catalog, + object_store, + exec, + &metric_registry, + )); + + let compactor = CompactorServer::new(metric_registry, compactor_handler); + Arc::new(CompactorServerType::new(compactor, common_state)) +} From cc4875cca0d3bc2e997a61b37478569be5ddf81f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 11 Mar 2022 14:25:50 -0500 Subject: [PATCH 03/22] refactor: decouple ingester setup and creation logic from the config structs (#4020) * refactor: decouple ingester setup and creation logic from the config structs * fix: clippy * refactor: remove comments --- clap_blocks/src/ingester.rs | 65 +++++++++ clap_blocks/src/lib.rs | 1 + influxdb_iox/src/commands/run/ingester.rs | 161 +++------------------- influxdb_ioxd/src/server_type/ingester.rs | 104 +++++++++++++- ingester/src/data.rs | 7 +- ingester/src/handler.rs | 6 +- ingester/src/test_util.rs | 4 +- 7 files changed, 196 insertions(+), 152 deletions(-) create mode 100644 clap_blocks/src/ingester.rs diff --git a/clap_blocks/src/ingester.rs b/clap_blocks/src/ingester.rs new file mode 100644 index 0000000000..a3f06801c7 --- /dev/null +++ b/clap_blocks/src/ingester.rs @@ -0,0 +1,65 @@ +/// CLI config for catalog ingest lifecycle +#[derive(Debug, Clone, clap::Parser)] +pub struct IngesterConfig { + /// Write buffer partition number to start (inclusive) range with + #[clap( + long = "--write-buffer-partition-range-start", + env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_START" + )] + pub write_buffer_partition_range_start: i32, + + /// Write buffer partition number to end (inclusive) range with + #[clap( + long = "--write-buffer-partition-range-end", + env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_END" + )] + pub write_buffer_partition_range_end: i32, + + /// The ingester will continue to pull data and buffer it from Kafka + /// as long as it is below this size. If it hits this size it will pause + /// ingest from Kafka until persistence goes below this threshold. + #[clap( + long = "--pause-ingest-size-bytes", + env = "INFLUXDB_IOX_PAUSE_INGEST_SIZE_BYTES" + )] + pub pause_ingest_size_bytes: usize, + + /// Once the ingester crosses this threshold of data buffered across + /// all sequencers, it will pick the largest partitions and persist + /// them until it falls below this threshold. An ingester running in + /// a steady state is expected to take up this much memory. + #[clap( + long = "--persist-memory-threshold-bytes", + env = "INFLUXDB_IOX_PERSIST_MEMORY_THRESHOLD_BYTES" + )] + pub persist_memory_threshold_bytes: usize, + + /// If an individual partition crosses this size threshold, it will be persisted. + /// The default value is 300MB (in bytes). + #[clap( + long = "--persist-partition-size-threshold-bytes", + env = "INFLUXDB_IOX_PERSIST_PARTITION_SIZE_THRESHOLD_BYTES", + default_value = "314572800" + )] + pub persist_partition_size_threshold_bytes: usize, + + /// If a partition has had data buffered for longer than this period of time + /// it will be persisted. This puts an upper bound on how far back the + /// ingester may need to read in Kafka on restart or recovery. The default value + /// is 30 minutes (in seconds). + #[clap( + long = "--persist-partition-age-threshold-seconds", + env = "INFLUXDB_IOX_PERSIST_PARTITION_AGE_THRESHOLD_SECONDS", + default_value = "1800" + )] + pub persist_partition_age_threshold_seconds: u64, + + /// If a partition has had data buffered and hasn't received a write for this + /// period of time, it will be persisted. The default value is 300 seconds (5 minutes). + #[clap( + long = "--persist-partition-cold-threshold-seconds", + env = "INFLUXDB_IOX_PERSIST_PARTITION_COLD_THRESHOLD_SECONDS", + default_value = "300" + )] + pub persist_partition_cold_threshold_seconds: u64, +} diff --git a/clap_blocks/src/lib.rs b/clap_blocks/src/lib.rs index d0652163b9..9ef11a1b2c 100644 --- a/clap_blocks/src/lib.rs +++ b/clap_blocks/src/lib.rs @@ -2,6 +2,7 @@ //! //! They can easily be re-used using `#[clap(flatten)]`. pub mod catalog_dsn; +pub mod ingester; pub mod object_store; pub mod run_config; pub mod server_id; diff --git a/influxdb_iox/src/commands/run/ingester.rs b/influxdb_iox/src/commands/run/ingester.rs index 3929562351..d4655a6550 100644 --- a/influxdb_iox/src/commands/run/ingester.rs +++ b/influxdb_iox/src/commands/run/ingester.rs @@ -1,25 +1,20 @@ //! Implementation of command line option for running ingester use clap_blocks::{ - catalog_dsn::CatalogDsnConfig, run_config::RunConfig, write_buffer::WriteBufferConfig, + catalog_dsn::CatalogDsnConfig, ingester::IngesterConfig, run_config::RunConfig, + write_buffer::WriteBufferConfig, }; -use data_types2::KafkaPartition; use influxdb_ioxd::{ self, server_type::{ common_state::{CommonServerState, CommonServerStateError}, - ingester::IngesterServerType, + ingester::create_ingester_server_type, }, }; -use ingester::{ - handler::IngestHandlerImpl, - lifecycle::LifecycleConfig, - server::{grpc::GrpcDelegate, http::HttpDelegate, IngesterServer}, -}; use object_store::ObjectStore; use observability_deps::tracing::*; use query::exec::Executor; -use std::{collections::BTreeMap, convert::TryFrom, sync::Arc, time::Duration}; +use std::{convert::TryFrom, sync::Arc}; use thiserror::Error; #[derive(Debug, Error)] @@ -30,26 +25,11 @@ pub enum Error { #[error("Invalid config: {0}")] InvalidConfig(#[from] CommonServerStateError), - #[error("Catalog error: {0}")] - Catalog(#[from] iox_catalog::interface::Error), - - #[error("Kafka topic {0} not found in the catalog")] - KafkaTopicNotFound(String), - #[error("Cannot parse object store config: {0}")] ObjectStoreParsing(#[from] clap_blocks::object_store::ParseError), - #[error("kafka_partition_range_start must be <= kafka_partition_range_end")] - KafkaRange, - #[error("error initializing ingester: {0}")] - Ingester(#[from] ingester::handler::Error), - - #[error("error initializing write buffer {0}")] - WriteBuffer(#[from] write_buffer::core::WriteBufferError), - - #[error("Invalid number of sequencers: {0}")] - NumSequencers(#[from] std::num::TryFromIntError), + Ingester(#[from] influxdb_ioxd::server_type::ingester::Error), #[error("Catalog DSN error: {0}")] CatalogDsn(#[from] clap_blocks::catalog_dsn::Error), @@ -81,67 +61,8 @@ pub struct Config { #[clap(flatten)] pub(crate) write_buffer_config: WriteBufferConfig, - /// Write buffer partition number to start (inclusive) range with - #[clap( - long = "--write-buffer-partition-range-start", - env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_START" - )] - pub write_buffer_partition_range_start: i32, - - /// Write buffer partition number to end (inclusive) range with - #[clap( - long = "--write-buffer-partition-range-end", - env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_END" - )] - pub write_buffer_partition_range_end: i32, - - /// The ingester will continue to pull data and buffer it from Kafka - /// as long as it is below this size. If it hits this size it will pause - /// ingest from Kafka until persistence goes below this threshold. - #[clap( - long = "--pause-ingest-size-bytes", - env = "INFLUXDB_IOX_PAUSE_INGEST_SIZE_BYTES" - )] - pub pause_ingest_size_bytes: usize, - - /// Once the ingester crosses this threshold of data buffered across - /// all sequencers, it will pick the largest partitions and persist - /// them until it falls below this threshold. An ingester running in - /// a steady state is expected to take up this much memory. - #[clap( - long = "--persist-memory-threshold-bytes", - env = "INFLUXDB_IOX_PERSIST_MEMORY_THRESHOLD_BYTES" - )] - pub persist_memory_threshold_bytes: usize, - - /// If an individual partition crosses this size threshold, it will be persisted. - /// The default value is 300MB (in bytes). - #[clap( - long = "--persist-partition-size-threshold-bytes", - env = "INFLUXDB_IOX_PERSIST_PARTITION_SIZE_THRESHOLD_BYTES", - default_value = "314572800" - )] - pub persist_partition_size_threshold_bytes: usize, - - /// If a partition has had data buffered for longer than this period of time - /// it will be persisted. This puts an upper bound on how far back the - /// ingester may need to read in Kafka on restart or recovery. The default value - /// is 30 minutes (in seconds). - #[clap( - long = "--persist-partition-age-threshold-seconds", - env = "INFLUXDB_IOX_PERSIST_PARTITION_AGE_THRESHOLD_SECONDS", - default_value = "1800" - )] - pub persist_partition_age_threshold_seconds: u64, - - /// If a partition has had data buffered and hasn't received a write for this - /// period of time, it will be persisted. The default value is 300 seconds (5 minutes). - #[clap( - long = "--persist-partition-cold-threshold-seconds", - env = "INFLUXDB_IOX_PERSIST_PARTITION_COLD_THRESHOLD_SECONDS", - default_value = "300" - )] - pub persist_partition_cold_threshold_seconds: u64, + #[clap(flatten)] + pub(crate) ingester_config: IngesterConfig, /// Number of threads to use for the ingester query execution, compaction and persistence. #[clap( @@ -149,7 +70,7 @@ pub struct Config { env = "INFLUXDB_IOX_QUERY_EXEC_THREAD_COUNT", default_value = "4" )] - pub query_exect_thread_count: usize, + pub query_exec_thread_count: usize, } pub async fn command(config: Config) -> Result<()> { @@ -161,66 +82,22 @@ pub async fn command(config: Config) -> Result<()> { .get_catalog("ingester", Arc::clone(&metric_registry)) .await?; - let mut txn = catalog.start_transaction().await?; - let kafka_topic = txn - .kafka_topics() - .get_by_name(config.write_buffer_config.topic()) - .await? - .ok_or_else(|| Error::KafkaTopicNotFound(config.write_buffer_config.topic().to_string()))?; - - if config.write_buffer_partition_range_start > config.write_buffer_partition_range_end { - return Err(Error::KafkaRange); - } - - let kafka_partitions: Vec<_> = (config.write_buffer_partition_range_start - ..=config.write_buffer_partition_range_end) - .map(KafkaPartition::new) - .collect(); - let object_store = Arc::new( ObjectStore::try_from(config.run_config.object_store_config()) .map_err(Error::ObjectStoreParsing)?, ); - let mut sequencers = BTreeMap::new(); - for k in kafka_partitions { - let s = txn.sequencers().create_or_get(&kafka_topic, k).await?; - sequencers.insert(k, s); - } - txn.commit().await?; - - let trace_collector = common_state.trace_collector(); - - let write_buffer = config - .write_buffer_config - .reading(Arc::clone(&metric_registry), trace_collector.clone()) - .await?; - - let lifecycle_config = LifecycleConfig::new( - config.pause_ingest_size_bytes, - config.persist_memory_threshold_bytes, - config.persist_partition_size_threshold_bytes, - Duration::from_secs(config.persist_partition_age_threshold_seconds), - Duration::from_secs(config.persist_partition_cold_threshold_seconds), - ); - let ingest_handler = Arc::new( - IngestHandlerImpl::new( - lifecycle_config, - kafka_topic, - sequencers, - catalog, - object_store, - write_buffer, - Executor::new(config.query_exect_thread_count), - Arc::clone(&metric_registry), - ) - .await?, - ); - let http = HttpDelegate::new(Arc::clone(&ingest_handler)); - let grpc = GrpcDelegate::new(Arc::clone(&ingest_handler)); - - let ingester = IngesterServer::new(metric_registry, http, grpc, ingest_handler); - let server_type = Arc::new(IngesterServerType::new(ingester, &common_state)); + let exec = Arc::new(Executor::new(config.query_exec_thread_count)); + let server_type = create_ingester_server_type( + &common_state, + metric_registry, + catalog, + object_store, + exec, + &config.write_buffer_config, + config.ingester_config, + ) + .await?; info!("starting ingester"); diff --git a/influxdb_ioxd/src/server_type/ingester.rs b/influxdb_ioxd/src/server_type/ingester.rs index bc4ee2f0bb..e973e0f4b5 100644 --- a/influxdb_ioxd/src/server_type/ingester.rs +++ b/influxdb_ioxd/src/server_type/ingester.rs @@ -1,12 +1,23 @@ use std::{ + collections::BTreeMap, fmt::{Debug, Display}, sync::Arc, + time::Duration, }; use async_trait::async_trait; +use clap_blocks::{ingester::IngesterConfig, write_buffer::WriteBufferConfig}; +use data_types2::KafkaPartition; use hyper::{Body, Request, Response}; -use ingester::server::IngesterServer; +use ingester::{ + handler::IngestHandlerImpl, + lifecycle::LifecycleConfig, + server::{grpc::GrpcDelegate, http::HttpDelegate, IngesterServer}, +}; +use iox_catalog::interface::Catalog; use metric::Registry; +use object_store::ObjectStore; +use query::exec::Executor; use trace::TraceCollector; use crate::{ @@ -15,6 +26,27 @@ use crate::{ server_type::{common_state::CommonServerState, RpcError, ServerType}, }; use ingester::handler::IngestHandler; +use thiserror::Error; + +#[derive(Debug, Error)] +pub enum Error { + #[error("Catalog error: {0}")] + Catalog(#[from] iox_catalog::interface::Error), + + #[error("Kafka topic {0} not found in the catalog")] + KafkaTopicNotFound(String), + + #[error("kafka_partition_range_start must be <= kafka_partition_range_end")] + KafkaRange, + + #[error("error initializing ingester: {0}")] + Ingester(#[from] ingester::handler::Error), + + #[error("error initializing write buffer {0}")] + WriteBuffer(#[from] write_buffer::core::WriteBufferError), +} + +pub type Result = std::result::Result; #[derive(Debug)] pub struct IngesterServerType { @@ -96,3 +128,73 @@ impl HttpApiErrorSource for IoxHttpError { HttpApiError::new(self.status_code(), self.to_string()) } } + +/// Instantiate an ingester server type +pub async fn create_ingester_server_type( + common_state: &CommonServerState, + metric_registry: Arc, + catalog: Arc, + object_store: Arc, + exec: Arc, + write_buffer_config: &WriteBufferConfig, + ingester_config: IngesterConfig, +) -> Result> { + let mut txn = catalog.start_transaction().await?; + let kafka_topic = txn + .kafka_topics() + .get_by_name(write_buffer_config.topic()) + .await? + .ok_or_else(|| Error::KafkaTopicNotFound(write_buffer_config.topic().to_string()))?; + + if ingester_config.write_buffer_partition_range_start + > ingester_config.write_buffer_partition_range_end + { + return Err(Error::KafkaRange); + } + + let kafka_partitions: Vec<_> = (ingester_config.write_buffer_partition_range_start + ..=ingester_config.write_buffer_partition_range_end) + .map(KafkaPartition::new) + .collect(); + + let mut sequencers = BTreeMap::new(); + for k in kafka_partitions { + let s = txn.sequencers().create_or_get(&kafka_topic, k).await?; + sequencers.insert(k, s); + } + txn.commit().await?; + + let trace_collector = common_state.trace_collector(); + + let write_buffer = write_buffer_config + .reading(Arc::clone(&metric_registry), trace_collector.clone()) + .await?; + + let lifecycle_config = LifecycleConfig::new( + ingester_config.pause_ingest_size_bytes, + ingester_config.persist_memory_threshold_bytes, + ingester_config.persist_partition_size_threshold_bytes, + Duration::from_secs(ingester_config.persist_partition_age_threshold_seconds), + Duration::from_secs(ingester_config.persist_partition_cold_threshold_seconds), + ); + let ingest_handler = Arc::new( + IngestHandlerImpl::new( + lifecycle_config, + kafka_topic, + sequencers, + catalog, + object_store, + write_buffer, + exec, + Arc::clone(&metric_registry), + ) + .await?, + ); + let http = HttpDelegate::new(Arc::clone(&ingest_handler)); + let grpc = GrpcDelegate::new(Arc::clone(&ingest_handler)); + + let ingester = IngesterServer::new(metric_registry, http, grpc, ingest_handler); + let server_type = Arc::new(IngesterServerType::new(ingester, common_state)); + + Ok(server_type) +} diff --git a/ingester/src/data.rs b/ingester/src/data.rs index a9ce2aa641..ec500e191f 100644 --- a/ingester/src/data.rs +++ b/ingester/src/data.rs @@ -99,8 +99,7 @@ pub struct IngesterData { /// get ingested. pub(crate) sequencers: BTreeMap, /// Executor for running queries and compacting and persisting - pub(crate) exec: Executor, - + pub(crate) exec: Arc, /// Backoff config pub(crate) backoff_config: BackoffConfig, } @@ -1427,7 +1426,7 @@ mod tests { object_store: Arc::clone(&object_store), catalog: Arc::clone(&catalog), sequencers, - exec: Executor::new(1), + exec: Arc::new(Executor::new(1)), backoff_config: BackoffConfig::default(), }); @@ -1508,7 +1507,7 @@ mod tests { object_store: Arc::clone(&object_store), catalog: Arc::clone(&catalog), sequencers, - exec: Executor::new(1), + exec: Arc::new(Executor::new(1)), backoff_config: BackoffConfig::default(), }); diff --git a/ingester/src/handler.rs b/ingester/src/handler.rs index cc13e36a79..daf9e925b9 100644 --- a/ingester/src/handler.rs +++ b/ingester/src/handler.rs @@ -116,7 +116,7 @@ impl IngestHandlerImpl { catalog: Arc, object_store: Arc, write_buffer: Arc, - exec: Executor, + exec: Arc, metric_registry: Arc, ) -> Result { // build the initial ingester data state @@ -729,7 +729,7 @@ mod tests { Arc::clone(&catalog), object_store, reading, - Executor::new(1), + Arc::new(Executor::new(1)), Arc::clone(&metrics), ) .await @@ -829,7 +829,7 @@ mod tests { Arc::clone(&catalog), object_store, reading, - Executor::new(1), + Arc::new(Executor::new(1)), Arc::clone(&metrics), ) .await diff --git a/ingester/src/test_util.rs b/ingester/src/test_util.rs index c11e9812f8..107ae6ba94 100644 --- a/ingester/src/test_util.rs +++ b/ingester/src/test_util.rs @@ -608,7 +608,7 @@ pub fn make_ingester_data(two_partitions: bool, loc: DataLocation) -> IngesterDa let metrics: Arc = Default::default(); let catalog: Arc = Arc::new(MemCatalog::new(metrics)); let object_store = Arc::new(object_store::ObjectStore::new_in_memory()); - let exec = query::exec::Executor::new(1); + let exec = Arc::new(query::exec::Executor::new(1)); // Make data for one sequencer/shard and two tables let seq_id = SequencerId::new(1); @@ -661,7 +661,7 @@ pub async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterDa let metrics: Arc = Default::default(); let catalog: Arc = Arc::new(MemCatalog::new(metrics)); let object_store = Arc::new(object_store::ObjectStore::new_in_memory()); - let exec = query::exec::Executor::new(1); + let exec = Arc::new(query::exec::Executor::new(1)); // Make data for one sequencer/shard and two tables let seq_id = SequencerId::new(1); From 5a29d070ea6f5d8e3e961bf4deea15ef29be463c Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Fri, 11 Mar 2022 15:25:19 -0500 Subject: [PATCH 04/22] feat: Implement the compact function for NG Compactor (#4001) * feat: initial implementation of compact a given list of overlapped parquet files * feat: Add QueryableParquetChunk and some refactoring * feat: build queryable parquet chunks for parquet files with tombstones * feat: second half the implementation for Compactor's compact. Tests will be next * fix: comments for trait funnctions fof QueryChunkMeta * test: add tests for compactor's compact function * fix: typos * refactor: address Jake's review comments * refactor: address Andrew's comments and add one more test for files in different order in the vector Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 8 + compactor/Cargo.toml | 8 + compactor/src/compact.rs | 514 +++++++++++++++++++-- compactor/src/handler.rs | 3 + compactor/src/lib.rs | 2 + compactor/src/query.rs | 216 +++++++++ compactor/src/utils.rs | 117 +++++ data_types2/src/lib.rs | 24 +- influxdb_iox/src/commands/run/compactor.rs | 3 + influxdb_ioxd/src/server_type/compactor.rs | 3 + ingester/src/compact.rs | 70 +-- ingester/src/query.rs | 31 +- iox_tests/src/util.rs | 19 +- query/Cargo.toml | 1 + query/src/util.rs | 76 ++- 15 files changed, 963 insertions(+), 132 deletions(-) create mode 100644 compactor/src/query.rs create mode 100644 compactor/src/utils.rs diff --git a/Cargo.lock b/Cargo.lock index 0923ce8989..205eeb46f7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -738,23 +738,31 @@ dependencies = [ name = "compactor" version = "0.1.0" dependencies = [ + "arrow", + "arrow_util", "async-trait", "backoff 0.1.0", "data_types2", + "datafusion 0.1.0", "futures", "iox_catalog", + "iox_object_store", + "iox_tests", "metric", "object_store", "observability_deps", "parking_lot 0.12.0", + "parquet_file", "predicate", "query", + "schema", "snafu", "test_helpers", "thiserror", "time 0.1.0", "tokio", "tokio-util 0.7.0", + "uuid", "workspace-hack", ] diff --git a/compactor/Cargo.toml b/compactor/Cargo.toml index 18a2c076c9..9b008bdfcc 100644 --- a/compactor/Cargo.toml +++ b/compactor/Cargo.toml @@ -5,23 +5,31 @@ authors = ["Luke Bond "] edition = "2021" [dependencies] +arrow = { version = "10.0.0", features = ["prettyprint"] } async-trait = "0.1.42" backoff = { path = "../backoff" } data_types2 = { path = "../data_types2" } +datafusion = { path = "../datafusion" } futures = "0.3" iox_catalog = { path = "../iox_catalog" } +iox_object_store = { path = "../iox_object_store" } metric = { path = "../metric" } object_store = { path = "../object_store" } observability_deps = { path = "../observability_deps" } parking_lot = "0.12" +parquet_file = { path = "../parquet_file" } predicate = { path = "../predicate" } query = { path = "../query" } +schema = { path = "../schema" } snafu = "0.7" thiserror = "1.0" time = { path = "../time" } tokio = { version = "1.17", features = ["macros", "parking_lot", "rt-multi-thread", "sync", "time"] } tokio-util = { version = "0.7.0" } +uuid = { version = "0.8", features = ["v4"] } workspace-hack = { path = "../workspace-hack"} [dev-dependencies] +arrow_util = { path = "../arrow_util" } +iox_tests = { path = "../iox_tests" } test_helpers = { path = "../test_helpers" } diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index e95f905ad9..0882dedd5c 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -1,18 +1,31 @@ //! Data Points for the lifecycle of the Compactor use std::{ + cmp::{max, min}, collections::{BTreeMap, HashSet}, sync::Arc, }; use backoff::BackoffConfig; -use data_types2::{ - ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, Tombstone, TombstoneId, -}; +use data_types2::{ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, TombstoneId}; +use datafusion::error::DataFusionError; use iox_catalog::interface::Catalog; use object_store::ObjectStore; -use query::exec::Executor; -use snafu::Snafu; +use parquet_file::metadata::IoxMetadata; +use query::{ + compute_sort_key_for_chunks, + exec::{Executor, ExecutorType}, + frontend::reorg::ReorgPlanner, + util::compute_timenanosecond_min_max, +}; +use snafu::{ensure, ResultExt, Snafu}; +use time::{Time, TimeProvider}; +use uuid::Uuid; + +use crate::{ + query::QueryableParquetChunk, + utils::{CompactedData, ParquetFileWithTombstone, TablePartition}, +}; /// 24 hours in nanoseconds // TODO: make this a config parameter @@ -26,6 +39,60 @@ pub enum Error { sequencer_id ))] SequencerNotFound { sequencer_id: SequencerId }, + + #[snafu(display( + "The given parquet files are not in the same partition ({}, {}, {}), ({}, {}, {})", + sequencer_id_1, + table_id_1, + partition_id_1, + sequencer_id_2, + table_id_2, + partition_id_2 + ))] + NotSamePartition { + sequencer_id_1: SequencerId, + table_id_1: TableId, + partition_id_1: PartitionId, + sequencer_id_2: SequencerId, + table_id_2: TableId, + partition_id_2: PartitionId, + }, + + #[snafu(display( + "Cannot compact parquet files for table ID {} due to an internal error: {}", + table_id, + source + ))] + TableNotFound { + source: iox_catalog::interface::Error, + table_id: TableId, + }, + + #[snafu(display( + "Cannot compact parquet files for an non-existing table ID {}", + table_id + ))] + TableNotExist { table_id: TableId }, + + #[snafu(display("Error building compact logical plan {}", source))] + CompactLogicalPlan { + source: query::frontend::reorg::Error, + }, + + #[snafu(display("Error building compact physical plan {}", source))] + CompactPhysicalPlan { source: DataFusionError }, + + #[snafu(display("Error executing compact plan {}", source))] + ExecuteCompactPlan { source: DataFusionError }, + + #[snafu(display("Error collecting stream yto record batches {}", source))] + CollectStream { source: DataFusionError }, + + #[snafu(display("Could not convert row count to i64"))] + RowCountTypeConversion { source: std::num::TryFromIntError }, + + #[snafu(display("Error computing min and max for record batches: {}", source))] + MinMax { source: query::util::Error }, } /// A specialized `Error` for Compactor Data errors @@ -43,6 +110,8 @@ pub struct Compactor { /// Executor for running queries and compacting and persisting exec: Arc, + /// Time provider for all activities in this compactor + pub time_provider: Arc, /// Backoff config backoff_config: BackoffConfig, @@ -55,6 +124,7 @@ impl Compactor { catalog: Arc, object_store: Arc, exec: Arc, + time_provider: Arc, backoff_config: BackoffConfig, ) -> Self { Self { @@ -62,6 +132,7 @@ impl Compactor { catalog, object_store, exec, + time_provider, backoff_config, } } @@ -107,7 +178,7 @@ impl Compactor { // keep tombstone ids tombstones = Self::union_tombstone_ids(tombstones, &overlapped_files); - // Only one file, no need to compact + // Only one file without tombstones, no need to compact if overlapped_files.len() == 1 && overlapped_files[0].no_tombstones() { // If the file is old enough, it would not have any overlaps. Add it // to the list to be upgraded to level 1 @@ -118,7 +189,10 @@ impl Compactor { } // compact - let output_parquet_files: Vec = vec![]; // TODO: #3907 + let _compacted_data = self.compact(overlapped_files).await; + + // split the compacted data into 2 files 90/10 + let output_parquet_files: Vec = vec![]; // TODO: #3999 for _file in output_parquet_files { // persist the file @@ -167,47 +241,411 @@ impl Compactor { } tombstones } -} -#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)] -struct TablePartition { - sequencer_id: SequencerId, - table_id: TableId, - partition_id: PartitionId, -} - -impl TablePartition { - fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self { - Self { - sequencer_id, - table_id, - partition_id, + // Compact given files. Assume the given files are overlaped in time. + // If the assumption does not meet, we will spend time not to compact anything but put data together + async fn compact( + &self, + overlapped_files: Vec, + ) -> Result> { + // Nothing to compact + if overlapped_files.is_empty() { + return Ok(None); } + + // One file without tombstone, no need to compact + if overlapped_files.len() == 1 && overlapped_files[0].tombstones.is_empty() { + return Ok(None); + } + + // Keep the fist IoxMetadata to reuse same IDs and names + let iox_metadata = overlapped_files[0].iox_metadata(); + + // Verify if the given files belong to the same partition + // Note: we can ignore this verification if we assume this is a must-have condition + if let Some((head, tail)) = overlapped_files.split_first() { + for file in tail { + let is_same = file.data.sequencer_id == head.data.sequencer_id + && file.data.table_id == head.data.table_id + && file.data.partition_id == head.data.partition_id; + + ensure!( + is_same, + NotSamePartitionSnafu { + sequencer_id_1: head.data.sequencer_id, + table_id_1: head.data.table_id, + partition_id_1: head.data.partition_id, + sequencer_id_2: file.data.sequencer_id, + table_id_2: file.data.table_id, + partition_id_2: file.data.partition_id + } + ) + } + } + + // Convert the input files into QueryableParquetChunk for making query plan + let query_chunks: Vec<_> = overlapped_files + .iter() + .map(|f| { + f.to_queryable_parquet_chunk( + Arc::clone(&self.object_store), + iox_metadata.table_name.to_string(), + iox_metadata.partition_key.to_string(), + ) + }) + .collect(); + + // Compute min & max sequence numbers + // unwrap here will work becasue the len of the query_chunks already >= 1 + let (head, tail) = query_chunks.split_first().unwrap(); + let mut min_sequence_number = head.min_sequence_number(); + let mut max_sequence_number = head.max_sequence_number(); + for c in tail { + min_sequence_number = min(min_sequence_number, c.min_sequence_number()); + max_sequence_number = max(max_sequence_number, c.max_sequence_number()); + } + + // Merge schema of the compacting chunks + let merged_schema = QueryableParquetChunk::merge_schemas(&query_chunks); + + // Compute the sorted output of the compacting result + let sort_key = compute_sort_key_for_chunks(&merged_schema, &query_chunks); + + // Build compact query plan + let plan = ReorgPlanner::new() + .compact_plan( + Arc::clone(&merged_schema), + query_chunks.into_iter().map(Arc::new), + sort_key.clone(), + ) + .context(CompactLogicalPlanSnafu)?; + let ctx = self.exec.new_context(ExecutorType::Reorg); + let physical_plan = ctx + .prepare_plan(&plan) + .await + .context(CompactPhysicalPlanSnafu)?; + + // Run the plan + let stream = ctx + .execute_stream(physical_plan) + .await + .context(ExecuteCompactPlanSnafu)?; + + // Collect compacted data into record batches for computing statistics + let output_batches = datafusion::physical_plan::common::collect(stream) + .await + .context(CollectStreamSnafu)?; + + // Filter empty record batches + let output_batches: Vec<_> = output_batches + .into_iter() + .filter(|b| b.num_rows() != 0) + .collect(); + + let row_count: usize = output_batches.iter().map(|b| b.num_rows()).sum(); + let row_count = row_count.try_into().context(RowCountTypeConversionSnafu)?; + + if row_count == 0 { + return Ok(None); + } + + // Compute min and max of the `time` column + let (min_time, max_time) = + compute_timenanosecond_min_max(&output_batches).context(MinMaxSnafu)?; + + let meta = IoxMetadata { + object_store_id: Uuid::new_v4(), + creation_timestamp: self.time_provider.now(), + sequencer_id: iox_metadata.sequencer_id, + namespace_id: iox_metadata.namespace_id, + namespace_name: iox_metadata.namespace_name, + table_id: iox_metadata.table_id, + table_name: iox_metadata.table_name, + partition_id: iox_metadata.partition_id, + partition_key: iox_metadata.partition_key, + time_of_first_write: Time::from_timestamp_nanos(min_time), + time_of_last_write: Time::from_timestamp_nanos(max_time), + min_sequence_number, + max_sequence_number, + row_count, + }; + + let compacted_data = CompactedData::new(output_batches, meta); + + Ok(Some(compacted_data)) } } -struct ParquetFileWithTombstone { - data: Arc, - tombstones: Vec, -} +#[cfg(test)] +mod tests { -impl ParquetFileWithTombstone { - fn tombstone_ids(&self) -> HashSet { - self.tombstones.iter().map(|t| t.id).collect::>() + use super::*; + use arrow_util::assert_batches_sorted_eq; + use iox_tests::util::TestCatalog; + use time::SystemProvider; + + #[tokio::test] + async fn test_compact_one_file() { + let catalog = TestCatalog::new(); + + let lp = vec![ + "table,tag1=WA field_int=1000 8000", + "table,tag1=VT field_int=10 10000", + "table,tag1=UT field_int=70 20000", + ] + .join("\n"); + let ns = catalog.create_namespace("ns").await; + let sequencer = ns.create_sequencer(1).await; + let table = ns.create_table("table").await; + let parquet_file = table + .with_sequencer(&sequencer) + .create_partition("part") + .await + .create_parquet_file(&lp) + .await + .parquet_file + .clone(); + + let compactor = Compactor { + sequencers: vec![sequencer.sequencer.id], + object_store: Arc::clone(&catalog.object_store), + catalog: Arc::clone(&catalog.catalog), + exec: Arc::new(Executor::new(1)), + time_provider: Arc::new(SystemProvider::new()), + backoff_config: BackoffConfig::default(), + }; + + // ------------------------------------------------ + // no files provided + let result = compactor.compact(vec![]).await.unwrap(); + assert!(result.is_none()); + + // ------------------------------------------------ + // File without tombstones + let mut pf = ParquetFileWithTombstone { + data: Arc::new(parquet_file), + tombstones: vec![], + }; + // Nothing compacted for one file without tombstones + let result = compactor.compact(vec![pf.clone()]).await.unwrap(); + assert!(result.is_none()); + + // ------------------------------------------------ + // Let add a tombstone + let tombstone = table + .with_sequencer(&sequencer) + .create_tombstone(20, 6000, 12000, "tag1=VT") + .await; + pf.add_tombstones(vec![tombstone.tombstone.clone()]); + // should have compacted data + let batches = compactor.compact(vec![pf]).await.unwrap().unwrap().data; + // one row tag1=VT was removed + assert_batches_sorted_eq!( + &[ + "+-----------+------+-----------------------------+", + "| field_int | tag1 | time |", + "+-----------+------+-----------------------------+", + "| 70 | UT | 1970-01-01T00:00:00.000020Z |", + "| 1000 | WA | 1970-01-01T00:00:00.000008Z |", + "+-----------+------+-----------------------------+", + ], + &batches + ); } - fn no_tombstones(&self) -> bool { - self.tombstones.is_empty() + #[tokio::test] + async fn test_compact_two_files() { + let catalog = TestCatalog::new(); + + let lp1 = vec![ + "table,tag1=WA field_int=1000 8000", // will be eliminated due to duplicate + "table,tag1=VT field_int=10 10000", // will be deleted + "table,tag1=UT field_int=70 20000", + ] + .join("\n"); + + let lp2 = vec![ + "table,tag1=WA field_int=1500 8000", // latest duplicate and kept + "table,tag1=VT field_int=10 6000", + "table,tag1=UT field_int=270 25000", + ] + .join("\n"); + + let ns = catalog.create_namespace("ns").await; + let sequencer = ns.create_sequencer(1).await; + let table = ns.create_table("table").await; + let partition = table + .with_sequencer(&sequencer) + .create_partition("part") + .await; + let parquet_file1 = partition + .create_parquet_file_with_sequence_numbers(&lp1, 1, 5) + .await + .parquet_file + .clone(); + let parquet_file2 = partition + .create_parquet_file_with_sequence_numbers(&lp2, 10, 15) + .await + .parquet_file + .clone(); + + let compactor = Compactor { + sequencers: vec![sequencer.sequencer.id], + object_store: Arc::clone(&catalog.object_store), + catalog: Arc::clone(&catalog.catalog), + exec: Arc::new(Executor::new(1)), + time_provider: Arc::new(SystemProvider::new()), + backoff_config: BackoffConfig::default(), + }; + + // File 1 with tombstone + let tombstone = table + .with_sequencer(&sequencer) + .create_tombstone(6, 6000, 12000, "tag1=VT") + .await; + let pf1 = ParquetFileWithTombstone { + data: Arc::new(parquet_file1), + tombstones: vec![tombstone.tombstone.clone()], + }; + // File 2 without tombstones + let pf2 = ParquetFileWithTombstone { + data: Arc::new(parquet_file2), + tombstones: vec![], + }; + + // Compact them + let batches = compactor + .compact(vec![pf1, pf2]) + .await + .unwrap() + .unwrap() + .data; + // Should have 4 rows left + assert_batches_sorted_eq!( + &[ + "+-----------+------+-----------------------------+", + "| field_int | tag1 | time |", + "+-----------+------+-----------------------------+", + "| 10 | VT | 1970-01-01T00:00:00.000006Z |", + "| 1500 | WA | 1970-01-01T00:00:00.000008Z |", + "| 270 | UT | 1970-01-01T00:00:00.000025Z |", + "| 70 | UT | 1970-01-01T00:00:00.000020Z |", + "+-----------+------+-----------------------------+", + ], + &batches + ); } - // Check if the parquet file is old enough to upgarde its level - fn level_upgradable(&self) -> bool { - // TODO: need to wait for creation_time added - // if time_provider.now() - self.data.creation_time > LEVEL_UPGRADE_THRESHOLD_NANO - true - } + #[tokio::test] + async fn test_compact_three_files_different_cols() { + let catalog = TestCatalog::new(); - fn parquet_file_id(&self) -> ParquetFileId { - self.data.id + let lp1 = vec![ + "table,tag1=WA field_int=1000 8000", // will be eliminated due to duplicate + "table,tag1=VT field_int=10 10000", // will be deleted + "table,tag1=UT field_int=70 20000", + ] + .join("\n"); + + let lp2 = vec![ + "table,tag1=WA field_int=1500 8000", // latest duplicate and kept + "table,tag1=VT field_int=10 6000", + "table,tag1=UT field_int=270 25000", + ] + .join("\n"); + + let lp3 = vec![ + "table,tag2=WA,tag3=10 field_int=1500 8000", + "table,tag2=VT,tag3=20 field_int=10 6000", + ] + .join("\n"); + + let ns = catalog.create_namespace("ns").await; + let sequencer = ns.create_sequencer(1).await; + let table = ns.create_table("table").await; + let partition = table + .with_sequencer(&sequencer) + .create_partition("part") + .await; + // Sequence numbers are important here. + // Time/sequence order from small to large: parquet_file_1, parquet_file_2, parquet_file_3 + let parquet_file1 = partition + .create_parquet_file_with_sequence_numbers(&lp1, 1, 5) + .await + .parquet_file + .clone(); + let parquet_file2 = partition + .create_parquet_file_with_sequence_numbers(&lp2, 10, 15) + .await + .parquet_file + .clone(); + let parquet_file3 = partition + .create_parquet_file_with_sequence_numbers(&lp3, 20, 25) + .await + .parquet_file + .clone(); + + let compactor = Compactor { + sequencers: vec![sequencer.sequencer.id], + object_store: Arc::clone(&catalog.object_store), + catalog: Arc::clone(&catalog.catalog), + exec: Arc::new(Executor::new(1)), + time_provider: Arc::new(SystemProvider::new()), + backoff_config: BackoffConfig::default(), + }; + + // File 1 with tombstone + let tombstone = table + .with_sequencer(&sequencer) + .create_tombstone(6, 6000, 12000, "tag1=VT") + .await; + let pf1 = ParquetFileWithTombstone { + data: Arc::new(parquet_file1), + tombstones: vec![tombstone.tombstone.clone()], + }; + // File 2 without tombstones + let pf2 = ParquetFileWithTombstone { + data: Arc::new(parquet_file2), + tombstones: vec![], + }; + // File 3 without tombstones + let pf3 = ParquetFileWithTombstone { + data: Arc::new(parquet_file3), + tombstones: vec![], + }; + + // Compact them + let batches = compactor + .compact(vec![pf1.clone(), pf2.clone(), pf3.clone()]) + .await + .unwrap() + .unwrap() + .data; + // Should have 6 rows + let expected = vec![ + "+-----------+------+------+------+-----------------------------+", + "| field_int | tag1 | tag2 | tag3 | time |", + "+-----------+------+------+------+-----------------------------+", + "| 10 | | VT | 20 | 1970-01-01T00:00:00.000006Z |", + "| 10 | VT | | | 1970-01-01T00:00:00.000006Z |", + "| 1500 | | WA | 10 | 1970-01-01T00:00:00.000008Z |", + "| 1500 | WA | | | 1970-01-01T00:00:00.000008Z |", + "| 270 | UT | | | 1970-01-01T00:00:00.000025Z |", + "| 70 | UT | | | 1970-01-01T00:00:00.000020Z |", + "+-----------+------+------+------+-----------------------------+", + ]; + assert_batches_sorted_eq!(&expected, &batches); + + // Make a vector of different file order but the result is still the same + // becasue the actual order for deduplication does not rely on their order in the vector + // Compact them + let batches = compactor + .compact(vec![pf2, pf3, pf1]) // different order in the vector + .await + .unwrap() + .unwrap() + .data; + // Should have 6 rows + assert_batches_sorted_eq!(&expected, &batches); } } diff --git a/compactor/src/handler.rs b/compactor/src/handler.rs index 340172948a..588964caae 100644 --- a/compactor/src/handler.rs +++ b/compactor/src/handler.rs @@ -14,6 +14,7 @@ use observability_deps::tracing::warn; use query::exec::Executor; use std::sync::Arc; use thiserror::Error; +use time::TimeProvider; use tokio::task::{JoinError, JoinHandle}; use tokio_util::sync::CancellationToken; @@ -63,6 +64,7 @@ impl CompactorHandlerImpl { catalog: Arc, object_store: Arc, exec: Arc, + time_provider: Arc, _registry: &metric::Registry, ) -> Self { let shutdown = CancellationToken::new(); @@ -74,6 +76,7 @@ impl CompactorHandlerImpl { catalog, object_store, exec, + time_provider, BackoffConfig::default(), )); diff --git a/compactor/src/lib.rs b/compactor/src/lib.rs index 422eef9044..c4c345b981 100644 --- a/compactor/src/lib.rs +++ b/compactor/src/lib.rs @@ -14,4 +14,6 @@ pub mod compact; pub mod handler; +pub mod query; pub mod server; +pub mod utils; diff --git a/compactor/src/query.rs b/compactor/src/query.rs new file mode 100644 index 0000000000..c5aaf82178 --- /dev/null +++ b/compactor/src/query.rs @@ -0,0 +1,216 @@ +//! Queryable Compactor Data + +use std::sync::Arc; + +use data_types2::{ + tombstones_to_delete_predicates, ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, + SequenceNumber, TableSummary, Tombstone, +}; +use datafusion::physical_plan::SendableRecordBatchStream; +use observability_deps::tracing::trace; +use parquet_file::{chunk::ParquetChunk, metadata::IoxMetadata}; +use predicate::{Predicate, PredicateMatch}; +use query::{ + exec::{stringset::StringSet, IOxExecutionContext}, + QueryChunk, QueryChunkMeta, +}; +use schema::{merge::SchemaMerger, selection::Selection, sort::SortKey, Schema}; +use snafu::{ResultExt, Snafu}; + +#[derive(Debug, Snafu)] +#[allow(missing_copy_implementations, missing_docs)] +pub enum Error { + #[snafu(display("Failed to read parquet: {}", source))] + ReadParquet { source: parquet_file::chunk::Error }, + + #[snafu(display( + "Error reading IOx Metadata from Parquet IoxParquetMetadata: {}", + source + ))] + ReadParquetMeta { + source: parquet_file::metadata::Error, + }, +} + +/// A specialized `Error` for Compactor's query errors +pub type Result = std::result::Result; + +/// QueryableParquetChunk that implements QueryChunk and QueryMetaChunk for building query plan +#[derive(Debug, Clone)] +pub struct QueryableParquetChunk { + data: Arc, // data of the parquet file + iox_metadata: Arc, // metadata of the parquet file + delete_predicates: Vec>, // converted from tombstones + table_name: String, // needed to build query plan +} + +impl QueryableParquetChunk { + /// Initialize a QueryableParquetChunk + pub fn new( + table_name: impl Into, + data: Arc, + iox_metadata: Arc, + deletes: &[Tombstone], + ) -> Self { + let delete_predicates = tombstones_to_delete_predicates(deletes); + Self { + data, + iox_metadata, + delete_predicates, + table_name: table_name.into(), + } + } + + /// Merge schema of the given chunks + pub fn merge_schemas(chunks: &[Self]) -> Arc { + let mut merger = SchemaMerger::new(); + for chunk in chunks { + merger = merger + .merge(&chunk.data.schema()) + .expect("schemas compatible"); + } + Arc::new(merger.build()) + } + + /// Return min sequence number + pub fn min_sequence_number(&self) -> SequenceNumber { + self.iox_metadata.min_sequence_number + } + + /// Return max sequence number + pub fn max_sequence_number(&self) -> SequenceNumber { + self.iox_metadata.max_sequence_number + } +} + +impl QueryChunkMeta for QueryableParquetChunk { + fn summary(&self) -> Option<&TableSummary> { + None + } + + fn schema(&self) -> Arc { + self.data.schema() + } + + fn sort_key(&self) -> Option<&SortKey> { + None // TODO: return the sortkey when it is available in the parquet file #3968 + } + + fn delete_predicates(&self) -> &[Arc] { + self.delete_predicates.as_ref() + } +} + +impl QueryChunk for QueryableParquetChunk { + type Error = Error; + + // Todo: This function should not be used in this NG chunk context + // For now, since we also use scan for both OG and NG, the chunk id + // is used as second key in build_deduplicate_plan_for_overlapped_chunks + // to sort the chunk to deduplicate them correctly. + // Since we make the first key, order, always different, it is fine + // to have the second key the sames and always 0 + fn id(&self) -> ChunkId { + // always return id 0 for debugging mode and with reason above + ChunkId::new_test(0) + } + + // This function should not be used in this context + fn addr(&self) -> ChunkAddr { + unimplemented!() + } + + /// Returns the name of the table stored in this chunk + fn table_name(&self) -> &str { + &self.table_name + } + + /// Returns true if the chunk may contain a duplicate "primary + /// key" within itself + fn may_contain_pk_duplicates(&self) -> bool { + // data within this parquet chunk was deduplicated + false + } + + /// Returns the result of applying the `predicate` to the chunk + /// using an efficient, but inexact method, based on metadata. + /// + /// NOTE: This method is suitable for calling during planning, and + /// may return PredicateMatch::Unknown for certain types of + /// predicates. + fn apply_predicate_to_metadata( + &self, + _predicate: &Predicate, + ) -> Result { + Ok(PredicateMatch::Unknown) + } + + /// Returns a set of Strings with column names from the specified + /// table that have at least one row that matches `predicate`, if + /// the predicate can be evaluated entirely on the metadata of + /// this Chunk. Returns `None` otherwise + fn column_names( + &self, + _ctx: IOxExecutionContext, + _predicate: &Predicate, + _columns: Selection<'_>, + ) -> Result, Self::Error> { + Ok(None) + } + + /// Return a set of Strings containing the distinct values in the + /// specified columns. If the predicate can be evaluated entirely + /// on the metadata of this Chunk. Returns `None` otherwise + /// + /// The requested columns must all have String type. + fn column_values( + &self, + _ctx: IOxExecutionContext, + _column_name: &str, + _predicate: &Predicate, + ) -> Result, Self::Error> { + Ok(None) + } + + /// Provides access to raw `QueryChunk` data as an + /// asynchronous stream of `RecordBatch`es filtered by a *required* + /// predicate. Note that not all chunks can evaluate all types of + /// predicates and this function will return an error + /// if requested to evaluate with a predicate that is not supported + /// + /// This is the analog of the `TableProvider` in DataFusion + /// + /// The reason we can't simply use the `TableProvider` trait + /// directly is that the data for a particular Table lives in + /// several chunks within a partition, so there needs to be an + /// implementation of `TableProvider` that stitches together the + /// streams from several different `QueryChunk`s. + fn read_filter( + &self, + mut ctx: IOxExecutionContext, + predicate: &Predicate, + selection: Selection<'_>, + ) -> Result { + ctx.set_metadata("storage", "compactor"); + ctx.set_metadata("projection", format!("{}", selection)); + trace!(?selection, "selection"); + + self.data + .read_filter(predicate, selection) + .context(ReadParquetSnafu) + } + + /// Returns chunk type + fn chunk_type(&self) -> &str { + "QueryableParquetChunk" + } + + // Order of the chunk so they can be deduplicate correctly + fn order(&self) -> ChunkOrder { + let seq_num = self.iox_metadata.min_sequence_number.get(); + let seq_num = u32::try_from(seq_num) + .expect("Sequence number should have been converted to chunk order successfully"); + ChunkOrder::new(seq_num) + .expect("Sequence number should have been converted to chunk order successfully") + } +} diff --git a/compactor/src/utils.rs b/compactor/src/utils.rs new file mode 100644 index 0000000000..79a2600690 --- /dev/null +++ b/compactor/src/utils.rs @@ -0,0 +1,117 @@ +//! Helpers of the Compactor + +use std::{collections::HashSet, sync::Arc}; + +use arrow::record_batch::RecordBatch; +use data_types2::{ + ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, Tombstone, TombstoneId, +}; +use iox_object_store::IoxObjectStore; +use object_store::ObjectStore; +use parquet_file::{ + chunk::{new_parquet_chunk, ChunkMetrics, DecodedParquetFile}, + metadata::IoxMetadata, +}; + +use crate::query::QueryableParquetChunk; + +/// Define table partition +#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)] +pub struct TablePartition { + sequencer_id: SequencerId, + table_id: TableId, + partition_id: PartitionId, +} + +impl TablePartition { + /// Return a new table partition + pub fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self { + Self { + sequencer_id, + table_id, + partition_id, + } + } +} + +/// Wrapper of a parquet file and its tombstones +#[allow(missing_docs)] +#[derive(Debug, Clone)] +pub struct ParquetFileWithTombstone { + pub(crate) data: Arc, + pub(crate) tombstones: Vec, +} + +impl ParquetFileWithTombstone { + /// Return all tombstone ids + pub fn tombstone_ids(&self) -> HashSet { + self.tombstones.iter().map(|t| t.id).collect() + } + + /// Return true if there is no tombstone + pub fn no_tombstones(&self) -> bool { + self.tombstones.is_empty() + } + + /// Check if the parquet file is old enough to upgarde its level + pub fn level_upgradable(&self) -> bool { + // TODO: need to wait for creation_time added + // if time_provider.now() - self.data.creation_time > LEVEL_UPGRADE_THRESHOLD_NANO + true + } + + /// Return id of this parquet file + pub fn parquet_file_id(&self) -> ParquetFileId { + self.data.id + } + + /// Add more tombstones + pub fn add_tombstones(&mut self, tombstones: Vec) { + self.tombstones.extend(tombstones); + } + + /// Convert to a QueryableParquetChunk + pub fn to_queryable_parquet_chunk( + &self, + object_store: Arc, + table_name: String, + partition_key: String, + ) -> QueryableParquetChunk { + let decoded_parquet_file = DecodedParquetFile::new((*self.data).clone()); + let root_path = IoxObjectStore::root_path_for(&object_store, self.data.object_store_id); + let iox_object_store = IoxObjectStore::existing(object_store, root_path); + let parquet_chunk = new_parquet_chunk( + &decoded_parquet_file, + Arc::from(table_name.clone()), + Arc::from(partition_key), + ChunkMetrics::new_unregistered(), // TODO: need to add metrics + Arc::new(iox_object_store), + ); + + QueryableParquetChunk::new( + table_name, + Arc::new(parquet_chunk), + Arc::new(decoded_parquet_file.iox_metadata), + &self.tombstones, + ) + } + + /// Return iox metadata of the parquet file + pub fn iox_metadata(&self) -> IoxMetadata { + let decoded_parquet_file = DecodedParquetFile::new((*self.data).clone()); + decoded_parquet_file.iox_metadata + } +} + +/// Struct holding output of a compacted stream +pub struct CompactedData { + pub(crate) data: Vec, + pub(crate) meta: IoxMetadata, +} + +impl CompactedData { + /// Initialize compacted data + pub fn new(data: Vec, meta: IoxMetadata) -> Self { + Self { data, meta } + } +} diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 7093ec0220..548ff41067 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -11,12 +11,13 @@ )] use influxdb_line_protocol::FieldValue; -use predicate::Predicate; +use predicate::{delete_predicate::parse_delete_predicate, Predicate}; use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType, Schema}; use std::{ collections::BTreeMap, convert::TryFrom, fmt::{Debug, Formatter}, + sync::Arc, }; use uuid::Uuid; @@ -619,6 +620,27 @@ pub struct Tombstone { pub serialized_predicate: String, } +/// Convert tombstones to delete predicates +pub fn tombstones_to_delete_predicates(tombstones: &[Tombstone]) -> Vec> { + tombstones_to_delete_predicates_iter(tombstones).collect() +} + +/// Return Iterator of delete predicates +pub fn tombstones_to_delete_predicates_iter( + tombstones: &[Tombstone], +) -> impl Iterator> + '_ { + tombstones.iter().map(|tombstone| { + Arc::new( + parse_delete_predicate( + &tombstone.min_time.get().to_string(), + &tombstone.max_time.get().to_string(), + &tombstone.serialized_predicate, + ) + .expect("Error building delete predicate"), + ) + }) +} + /// Data for a parquet file reference that has been inserted in the catalog. #[derive(Debug, Clone, PartialEq, sqlx::FromRow)] pub struct ParquetFile { diff --git a/influxdb_iox/src/commands/run/compactor.rs b/influxdb_iox/src/commands/run/compactor.rs index 9df4ddc5ad..f51e37d1e9 100644 --- a/influxdb_iox/src/commands/run/compactor.rs +++ b/influxdb_iox/src/commands/run/compactor.rs @@ -6,6 +6,7 @@ use observability_deps::tracing::*; use query::exec::Executor; use std::sync::Arc; use thiserror::Error; +use time::SystemProvider; use clap_blocks::{catalog_dsn::CatalogDsnConfig, run_config::RunConfig}; use influxdb_ioxd::{ @@ -87,6 +88,7 @@ pub async fn command(config: Config) -> Result<(), Error> { ); let exec = Arc::new(Executor::new(config.query_exect_thread_count)); + let time_provider = Arc::new(SystemProvider::new()); // TODO: modify config to let us get assigned sequence numbers let sequencers: Vec = vec![]; @@ -97,6 +99,7 @@ pub async fn command(config: Config) -> Result<(), Error> { catalog, object_store, exec, + time_provider, sequencers, ) .await; diff --git a/influxdb_ioxd/src/server_type/compactor.rs b/influxdb_ioxd/src/server_type/compactor.rs index 8c64272cf6..a3dadc7a22 100644 --- a/influxdb_ioxd/src/server_type/compactor.rs +++ b/influxdb_ioxd/src/server_type/compactor.rs @@ -14,6 +14,7 @@ use iox_catalog::interface::Catalog; use metric::Registry; use object_store::ObjectStore; use query::exec::Executor; +use time::TimeProvider; use tokio_util::sync::CancellationToken; use trace::TraceCollector; @@ -113,6 +114,7 @@ pub async fn create_compactor_server_type( catalog: Arc, object_store: Arc, exec: Arc, + time_provider: Arc, sequencers: Vec, ) -> Arc { let compactor_handler = Arc::new(CompactorHandlerImpl::new( @@ -120,6 +122,7 @@ pub async fn create_compactor_server_type( catalog, object_store, exec, + time_provider, &metric_registry, )); diff --git a/ingester/src/compact.rs b/ingester/src/compact.rs index 855d43e630..740b2edf72 100644 --- a/ingester/src/compact.rs +++ b/ingester/src/compact.rs @@ -1,17 +1,17 @@ //! This module is responsible for compacting Ingester's data use crate::data::{PersistingBatch, QueryableBatch}; -use arrow::{array::TimestampNanosecondArray, record_batch::RecordBatch}; +use arrow::record_batch::RecordBatch; use data_types2::NamespaceId; use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream}; use parquet_file::metadata::IoxMetadata; use query::{ exec::{Executor, ExecutorType}, frontend::reorg::ReorgPlanner, + util::compute_timenanosecond_min_max, QueryChunkMeta, }; -use schema::TIME_COLUMN_NAME; -use snafu::{OptionExt, ResultExt, Snafu}; +use snafu::{ResultExt, Snafu}; use std::sync::Arc; use time::{Time, TimeProvider}; @@ -40,71 +40,16 @@ pub enum Error { predicate: String, }, - #[snafu(display("The Record batch is empty"))] - EmptyBatch, - - #[snafu(display("Error while searching Time column in a Record Batch"))] - TimeColumn { source: arrow::error::ArrowError }, - - #[snafu(display("Error while casting Timenanosecond on Time column"))] - TimeCasting, - #[snafu(display("Could not convert row count to i64"))] RowCountTypeConversion { source: std::num::TryFromIntError }, + + #[snafu(display("Error computing min and max for record batches: {}", source))] + MinMax { source: query::util::Error }, } /// A specialized `Error` for Ingester's Compact errors pub type Result = std::result::Result; -/// Return min and max for column `time` of the given set of record batches -pub fn compute_timenanosecond_min_max(batches: &[RecordBatch]) -> Result<(i64, i64)> { - let mut min_time = i64::MAX; - let mut max_time = i64::MIN; - for batch in batches { - let (min, max) = compute_timenanosecond_min_max_for_one_record_bacth(batch)?; - if min_time > min { - min_time = min; - } - if max_time < max { - max_time = max; - } - } - Ok((min_time, max_time)) -} - -/// Return min and max for column `time` in the given record batch -pub fn compute_timenanosecond_min_max_for_one_record_bacth( - batch: &RecordBatch, -) -> Result<(i64, i64)> { - if batch.num_columns() == 0 { - return Err(Error::EmptyBatch); - } - - let index = batch - .schema() - .index_of(TIME_COLUMN_NAME) - .context(TimeColumnSnafu {})?; - - let time_col = batch - .column(index) - .as_any() - .downcast_ref::() - .context(TimeCastingSnafu {})?; - - let min = time_col - .iter() - .min() - .expect("Time column must have values") - .expect("Time column cannot be NULL"); - let max = time_col - .iter() - .max() - .expect("Time column must have values") - .expect("Time column cannot be NULL"); - - Ok((min, max)) -} - /// Compact a given persisting batch /// Return compacted data with its metadata pub async fn compact_persisting_batch( @@ -138,7 +83,8 @@ pub async fn compact_persisting_batch( let row_count = row_count.try_into().context(RowCountTypeConversionSnafu)?; // Compute min and max of the `time` column - let (min_time, max_time) = compute_timenanosecond_min_max(&output_batches)?; + let (min_time, max_time) = + compute_timenanosecond_min_max(&output_batches).context(MinMaxSnafu)?; // Compute min and max sequence numbers let (min_seq, max_seq) = batch.data.min_max_sequence_numbers(); diff --git a/ingester/src/query.rs b/ingester/src/query.rs index 0f4b8bf893..344388b02a 100644 --- a/ingester/src/query.rs +++ b/ingester/src/query.rs @@ -7,7 +7,8 @@ use arrow::{ }; use arrow_util::util::merge_record_batches; use data_types2::{ - ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, SequenceNumber, TableSummary, Tombstone, + tombstones_to_delete_predicates, tombstones_to_delete_predicates_iter, ChunkAddr, ChunkId, + ChunkOrder, DeletePredicate, SequenceNumber, TableSummary, Tombstone, }; use datafusion::{ error::DataFusionError, @@ -19,7 +20,7 @@ use datafusion::{ }, }; use observability_deps::tracing::{debug, trace}; -use predicate::{delete_predicate::parse_delete_predicate, Predicate, PredicateMatch}; +use predicate::{Predicate, PredicateMatch}; use query::{ exec::{stringset::StringSet, IOxExecutionContext}, util::{df_physical_expr_from_schema_and_expr, MissingColumnsToNull}, @@ -62,7 +63,7 @@ pub type Result = std::result::Result; impl QueryableBatch { /// Initilaize a QueryableBatch pub fn new(table_name: &str, data: Vec>, deletes: Vec) -> Self { - let delete_predicates = Self::tombstones_to_delete_predicates(&deletes); + let delete_predicates = tombstones_to_delete_predicates(&deletes); Self { data, delete_predicates, @@ -72,30 +73,10 @@ impl QueryableBatch { /// Add more tombstones pub fn add_tombstones(&mut self, deletes: &[Tombstone]) { - let delete_predicates = Self::tombstones_to_delete_predicates_iter(deletes); + let delete_predicates = tombstones_to_delete_predicates_iter(deletes); self.delete_predicates.extend(delete_predicates); } - /// Convert tombstones to delete predicates - pub fn tombstones_to_delete_predicates(tombstones: &[Tombstone]) -> Vec> { - Self::tombstones_to_delete_predicates_iter(tombstones).collect() - } - - fn tombstones_to_delete_predicates_iter( - tombstones: &[Tombstone], - ) -> impl Iterator> + '_ { - tombstones.iter().map(|tombstone| { - Arc::new( - parse_delete_predicate( - &tombstone.min_time.get().to_string(), - &tombstone.max_time.get().to_string(), - &tombstone.serialized_predicate, - ) - .expect("Error building delete predicate"), - ) - }) - } - /// return min and max of all the snapshots pub fn min_max_sequence_numbers(&self) -> (SequenceNumber, SequenceNumber) { let min = self @@ -128,7 +109,7 @@ impl QueryChunkMeta for QueryableBatch { fn schema(&self) -> Arc { // TODO: may want store this schema as a field of QueryableBatch and - // only do this schema merge the first time it is call + // only do this schema merge the first time it is called // Merge schema of all RecordBatches of the PerstingBatch let batches: Vec> = diff --git a/iox_tests/src/util.rs b/iox_tests/src/util.rs index 04f4ee21a8..abb7f78470 100644 --- a/iox_tests/src/util.rs +++ b/iox_tests/src/util.rs @@ -158,7 +158,7 @@ pub struct TestTable { } impl TestTable { - /// Attach a sequncer to the table + /// Attach a sequencer to the table pub fn with_sequencer( self: &Arc, sequencer: &Arc, @@ -257,8 +257,19 @@ pub struct TestPartition { } impl TestPartition { - /// CReate a parquet for the partition + /// Create a parquet for the partition pub async fn create_parquet_file(self: &Arc, lp: &str) -> Arc { + self.create_parquet_file_with_sequence_numbers(lp, 1, 100) + .await + } + + /// Create a parquet for the partition + pub async fn create_parquet_file_with_sequence_numbers( + self: &Arc, + lp: &str, + min_seq: i64, + max_seq: i64, + ) -> Arc { let mut repos = self.catalog.catalog.repositories().await; let (table, batch) = lp_to_mutable_batch(lp); @@ -267,8 +278,8 @@ impl TestPartition { let record_batch = batch.to_arrow(Selection::All).unwrap(); let object_store_id = Uuid::new_v4(); - let min_sequence_number = SequenceNumber::new(1); - let max_sequence_number = SequenceNumber::new(100); + let min_sequence_number = SequenceNumber::new(min_seq); + let max_sequence_number = SequenceNumber::new(max_seq); let metadata = IoxMetadata { object_store_id, creation_timestamp: now(), diff --git a/query/Cargo.toml b/query/Cargo.toml index df927da8ca..aac6c9f0e4 100644 --- a/query/Cargo.toml +++ b/query/Cargo.toml @@ -25,6 +25,7 @@ datafusion_util = { path = "../datafusion_util" } executor = { path = "../executor"} futures = "0.3" hashbrown = "0.12" +itertools = "0.10.2" observability_deps = { path = "../observability_deps" } parking_lot = "0.12" pin-project = "1.0" diff --git a/query/src/util.rs b/query/src/util.rs index 886bebc732..3fbda190d5 100644 --- a/query/src/util.rs +++ b/query/src/util.rs @@ -1,8 +1,13 @@ //! This module contains DataFusion utility functions and helpers -use std::{convert::TryInto, sync::Arc}; +use std::{ + cmp::{max, min}, + convert::TryInto, + sync::Arc, +}; use arrow::{ + array::TimestampNanosecondArray, compute::SortOptions, datatypes::{DataType, Schema as ArrowSchema}, record_batch::RecordBatch, @@ -22,9 +27,33 @@ use datafusion::{ scalar::ScalarValue, }; +use itertools::Itertools; use observability_deps::tracing::trace; use predicate::rpc_predicate::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME}; -use schema::{sort::SortKey, Schema}; +use schema::{sort::SortKey, Schema, TIME_COLUMN_NAME}; +use snafu::{ensure, OptionExt, ResultExt, Snafu}; + +#[derive(Debug, Snafu)] +#[allow(missing_copy_implementations, missing_docs)] +pub enum Error { + #[snafu(display("The Record batch is empty"))] + EmptyBatch, + + #[snafu(display("Error while searching Time column in a Record Batch"))] + TimeColumn { source: arrow::error::ArrowError }, + + #[snafu(display("Error while casting Timenanosecond on Time column"))] + TimeCasting, + + #[snafu(display("Time column does not have value"))] + TimeValue, + + #[snafu(display("Time column is null"))] + TimeNull, +} + +/// A specialized `Error` +pub type Result = std::result::Result; /// Create a logical plan that produces the record batch pub fn make_scan_plan(batch: RecordBatch) -> std::result::Result { @@ -198,6 +227,49 @@ impl<'a> ExprRewriter for MissingColumnsToNull<'a> { } } +/// Return min and max for column `time` of the given set of record batches +pub fn compute_timenanosecond_min_max(batches: &[RecordBatch]) -> Result<(i64, i64)> { + let mut min_time = i64::MAX; + let mut max_time = i64::MIN; + for batch in batches { + let (mi, ma) = compute_timenanosecond_min_max_for_one_record_batch(batch)?; + min_time = min(min_time, mi); + max_time = max(max_time, ma); + } + Ok((min_time, max_time)) +} + +/// Return min and max for column `time` in the given record batch +pub fn compute_timenanosecond_min_max_for_one_record_batch( + batch: &RecordBatch, +) -> Result<(i64, i64)> { + ensure!(batch.num_columns() > 0, EmptyBatchSnafu); + + let index = batch + .schema() + .index_of(TIME_COLUMN_NAME) + .context(TimeColumnSnafu {})?; + + let time_col = batch + .column(index) + .as_any() + .downcast_ref::() + .context(TimeCastingSnafu {})?; + + let (min, max) = match time_col.iter().minmax() { + itertools::MinMaxResult::NoElements => return Err(Error::TimeValue), + itertools::MinMaxResult::OneElement(val) => { + let val = val.context(TimeNullSnafu)?; + (val, val) + } + itertools::MinMaxResult::MinMax(min, max) => { + (min.context(TimeNullSnafu)?, max.context(TimeNullSnafu)?) + } + }; + + Ok((min, max)) +} + #[cfg(test)] mod tests { use arrow::datatypes::DataType; From fabd262442a8e4fb16ff9cfa5fb3609ec49213d4 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 9 Mar 2022 16:05:32 -0500 Subject: [PATCH 05/22] feat: Add a function to the catalog to fetch level 0 parquet files Connects to #3946. --- compactor/src/compact.rs | 46 ++++++++----- iox_catalog/src/interface.rs | 129 ++++++++++++++++++++++++++++++++++- iox_catalog/src/mem.rs | 18 ++++- iox_catalog/src/metrics.rs | 1 + iox_catalog/src/postgres.rs | 35 +++++++++- 5 files changed, 207 insertions(+), 22 deletions(-) diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index 0882dedd5c..3e5a541084 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -1,11 +1,9 @@ //! Data Points for the lifecycle of the Compactor -use std::{ - cmp::{max, min}, - collections::{BTreeMap, HashSet}, - sync::Arc, +use crate::{ + query::QueryableParquetChunk, + utils::{CompactedData, ParquetFileWithTombstone, TablePartition}, }; - use backoff::BackoffConfig; use data_types2::{ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, TombstoneId}; use datafusion::error::DataFusionError; @@ -19,14 +17,14 @@ use query::{ util::compute_timenanosecond_min_max, }; use snafu::{ensure, ResultExt, Snafu}; +use std::{ + cmp::{max, min}, + collections::{BTreeMap, HashSet}, + sync::Arc, +}; use time::{Time, TimeProvider}; use uuid::Uuid; -use crate::{ - query::QueryableParquetChunk, - utils::{CompactedData, ParquetFileWithTombstone, TablePartition}, -}; - /// 24 hours in nanoseconds // TODO: make this a config parameter pub const LEVEL_UPGRADE_THRESHOLD_NANO: u64 = 60 * 60 * 24 * 1000000000; @@ -93,6 +91,11 @@ pub enum Error { #[snafu(display("Error computing min and max for record batches: {}", source))] MinMax { source: query::util::Error }, + + #[snafu(display("Error while requesting level 0 parquet files {}", source))] + Level0 { + source: iox_catalog::interface::Error, + }, } /// A specialized `Error` for Compactor Data errors @@ -137,6 +140,15 @@ impl Compactor { } } + async fn level_0_parquet_files(&self, sequencer_id: SequencerId) -> Result> { + let mut repos = self.catalog.repositories().await; + repos + .parquet_files() + .level_0(sequencer_id) + .await + .context(Level0Snafu) + } + // TODO: this function should be invoked in a backround loop /// Find and compact parquet files for a given sequencer pub async fn find_and_compact(&self, sequencer_id: SequencerId) -> Result<()> { @@ -145,14 +157,14 @@ impl Compactor { } // Read level-0 parquet files - let level_0_files: Vec> = vec![]; // TODO: #3946 + let level_0_files = self.level_0_parquet_files(sequencer_id).await?; // Group files into table partition let mut partitions = Self::group_parquet_files_into_partition(level_0_files); // Get level-1 files overlapped with level-0 for (_key, val) in &mut partitions.iter_mut() { - let level_1_files: Vec> = vec![]; // TODO: #3946 + let level_1_files: Vec = vec![]; // TODO: #3946 val.extend(level_1_files); } @@ -160,7 +172,7 @@ impl Compactor { // groups overlapped files in each partition let mut overlapped_file_groups = vec![]; for _val in partitions.values_mut() { - let overlapped_files: Vec>> = vec![]; // TODO: #3949 + let overlapped_files: Vec> = vec![]; // TODO: #3949 overlapped_file_groups.extend(overlapped_files); } @@ -212,11 +224,11 @@ impl Compactor { Ok(()) } - // Group given parquet files into parittion of the same (sequencer_id, table_id, partition_id) + // Group given parquet files into partition of the same (sequencer_id, table_id, partition_id) fn group_parquet_files_into_partition( - parquet_files: Vec>, - ) -> BTreeMap>> { - let mut groups: BTreeMap>> = BTreeMap::default(); + parquet_files: Vec, + ) -> BTreeMap> { + let mut groups: BTreeMap> = BTreeMap::default(); for file in parquet_files { let key = TablePartition::new(file.sequencer_id, file.table_id, file.partition_id); if let Some(val) = groups.get_mut(&key) { diff --git a/iox_catalog/src/interface.rs b/iox_catalog/src/interface.rs index 1994258b4e..2efc6a94f2 100644 --- a/iox_catalog/src/interface.rs +++ b/iox_catalog/src/interface.rs @@ -409,6 +409,9 @@ pub trait TombstoneRepo: Send + Sync { /// The starting compaction level for parquet files is zero. pub const INITIAL_COMPACTION_LEVEL: i16 = 0; +/// The maximum size, in bytes, of a file to be considered for compaction. +pub const MAX_COMPACT_SIZE: i64 = 1024 * 1024; + /// Functions for working with parquet file pointers in the catalog #[async_trait] pub trait ParquetFileRepo: Send + Sync { @@ -434,6 +437,10 @@ pub trait ParquetFileRepo: Send + Sync { namespace_id: NamespaceId, ) -> Result>; + /// List parquet files for a given sequencer with compaction level 0 and other criteria that + /// define a file as a candidate for compaction + async fn level_0(&mut self, sequencer_id: SequencerId) -> Result>; + /// Verify if the parquet file exists by selecting its id async fn exist(&mut self, id: ParquetFileId) -> Result; @@ -536,6 +543,7 @@ pub(crate) mod test_helpers { test_partition(Arc::clone(&catalog)).await; test_tombstone(Arc::clone(&catalog)).await; test_parquet_file(Arc::clone(&catalog)).await; + test_parquet_file_compaction_level_0(Arc::clone(&catalog)).await; test_add_parquet_file_with_tombstones(Arc::clone(&catalog)).await; test_txn_isolation(Arc::clone(&catalog)).await; test_txn_drop(Arc::clone(&catalog)).await; @@ -1358,7 +1366,7 @@ pub(crate) mod test_helpers { .list_by_namespace_not_to_delete(namespace2.id) .await .unwrap(); - assert_eq!(Vec::::new(), files); + assert!(files.is_empty()); let f1_params = ParquetFileParams { table_id: partition2.table_id, @@ -1383,19 +1391,134 @@ pub(crate) mod test_helpers { .await .unwrap(); assert_eq!(vec![f1.clone(), f2.clone()], files); + repos.parquet_files().flag_for_delete(f2.id).await.unwrap(); let files = repos .parquet_files() .list_by_namespace_not_to_delete(namespace2.id) .await .unwrap(); - assert_eq!(vec![f1], files); + assert_eq!(vec![f1.clone()], files); + let files = repos .parquet_files() .list_by_namespace_not_to_delete(NamespaceId::new(i32::MAX)) .await .unwrap(); - assert_eq!(Vec::::new(), files); + assert!(files.is_empty()); + } + + async fn test_parquet_file_compaction_level_0(catalog: Arc) { + let mut repos = catalog.repositories().await; + let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap(); + let pool = repos.query_pools().create_or_get("foo").await.unwrap(); + let namespace = repos + .namespaces() + .create( + "namespace_parquet_file_compaction_level_0_test", + "inf", + kafka.id, + pool.id, + ) + .await + .unwrap(); + let table = repos + .tables() + .create_or_get("test_table", namespace.id) + .await + .unwrap(); + let sequencer = repos + .sequencers() + .create_or_get(&kafka, KafkaPartition::new(100)) + .await + .unwrap(); + let other_sequencer = repos + .sequencers() + .create_or_get(&kafka, KafkaPartition::new(101)) + .await + .unwrap(); + + let partition = repos + .partitions() + .create_or_get("one", sequencer.id, table.id) + .await + .unwrap(); + + let min_time = Timestamp::new(1); + let max_time = Timestamp::new(10); + + let parquet_file_params = ParquetFileParams { + sequencer_id: sequencer.id, + table_id: partition.table_id, + partition_id: partition.id, + object_store_id: Uuid::new_v4(), + min_sequence_number: SequenceNumber::new(10), + max_sequence_number: SequenceNumber::new(140), + min_time, + max_time, + file_size_bytes: 1337, + parquet_metadata: b"md1".to_vec(), + row_count: 0, + created_at: Timestamp::new(1), + }; + + let parquet_file = repos + .parquet_files() + .create(parquet_file_params.clone()) + .await + .unwrap(); + + // Create a compaction level 0 file for some other sequencer + let other_sequencer_params = ParquetFileParams { + sequencer_id: other_sequencer.id, + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + + let _other_sequencer_file = repos + .parquet_files() + .create(other_sequencer_params) + .await + .unwrap(); + + // Create a compaction level 0 file marked to delete + let to_delete_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let to_delete_file = repos + .parquet_files() + .create(to_delete_params) + .await + .unwrap(); + repos + .parquet_files() + .flag_for_delete(to_delete_file.id) + .await + .unwrap(); + + // Create a compaction level 0 file over the MAX_COMPACT_SIZE + let too_big_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + file_size_bytes: MAX_COMPACT_SIZE + 1, + ..parquet_file_params.clone() + }; + let _too_big_file = repos.parquet_files().create(too_big_params).await.unwrap(); + + // Level 0 parquet files for a sequencer should contain only those that match the right + // criteria + let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap(); + let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect(); + level_0_ids.sort(); + let expected = vec![parquet_file]; + let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect(); + expected_ids.sort(); + + assert_eq!( + level_0_ids, expected_ids, + "\nlevel 0: {:#?}\nexpected: {:#?}", + level_0, expected, + ); } async fn test_add_parquet_file_with_tombstones(catalog: Arc) { diff --git a/iox_catalog/src/mem.rs b/iox_catalog/src/mem.rs index a1547962d7..6854fcabd0 100644 --- a/iox_catalog/src/mem.rs +++ b/iox_catalog/src/mem.rs @@ -6,7 +6,7 @@ use crate::{ sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error, KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo, QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo, - TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, + TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, MAX_COMPACT_SIZE, }, metrics::MetricDecorator, }; @@ -843,6 +843,22 @@ impl ParquetFileRepo for MemTxn { Ok(parquet_files) } + async fn level_0(&mut self, sequencer_id: SequencerId) -> Result> { + let stage = self.stage(); + + Ok(stage + .parquet_files + .iter() + .filter(|f| { + f.sequencer_id == sequencer_id + && f.compaction_level == 0 + && !f.to_delete + && f.file_size_bytes <= MAX_COMPACT_SIZE + }) + .cloned() + .collect()) + } + async fn exist(&mut self, id: ParquetFileId) -> Result { let stage = self.stage(); diff --git a/iox_catalog/src/metrics.rs b/iox_catalog/src/metrics.rs index 987130f54b..2b90a3062c 100644 --- a/iox_catalog/src/metrics.rs +++ b/iox_catalog/src/metrics.rs @@ -262,6 +262,7 @@ decorate!( "parquet_flag_for_delete" = flag_for_delete(&mut self, id: ParquetFileId) -> Result<()>; "parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result>; "parquet_list_by_namespace_not_to_delete" = list_by_namespace_not_to_delete(&mut self, namespace_id: NamespaceId) -> Result>; + "parquet_level_0" = level_0(&mut self, sequencer_id: SequencerId) -> Result>; "parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result; "parquet_count" = count(&mut self) -> Result; ] diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index 724ac4af59..03b529d152 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -5,7 +5,7 @@ use crate::{ sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error, KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo, QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo, - TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, + TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, MAX_COMPACT_SIZE, }, metrics::MetricDecorator, }; @@ -1267,6 +1267,39 @@ WHERE table_name.namespace_id = $1 AND parquet_file.to_delete = false; .map_err(|e| Error::SqlxError { source: e }) } + async fn level_0(&mut self, sequencer_id: SequencerId) -> Result> { + sqlx::query_as::<_, ParquetFile>( + r#" +SELECT + parquet_file.id as id, + parquet_file.sequencer_id as sequencer_id, + parquet_file.table_id as table_id, + parquet_file.partition_id as partition_id, + parquet_file.object_store_id as object_store_id, + parquet_file.min_sequence_number as min_sequence_number, + parquet_file.max_sequence_number as max_sequence_number, + parquet_file.min_time as min_time, + parquet_file.max_time as max_time, + parquet_file.to_delete as to_delete, + parquet_file.file_size_bytes as file_size_bytes, + parquet_file.parquet_metadata as parquet_metadata, + parquet_file.row_count as row_count, + parquet_file.compaction_level as compaction_level, + parquet_file.created_at as created_at +FROM parquet_file +WHERE parquet_file.sequencer_id = $1 + AND parquet_file.compaction_level = 0 + AND parquet_file.to_delete = false + AND parquet_file.file_size_bytes <= $2; + "#, + ) + .bind(&sequencer_id) // $1 + .bind(MAX_COMPACT_SIZE) // $2 + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e }) + } + async fn exist(&mut self, id: ParquetFileId) -> Result { let read_result = sqlx::query_as::<_, Count>( r#"SELECT count(*) as count FROM parquet_file WHERE id = $1;"#, From f184b7023c0c84e5055aac168cf0eb1264f29278 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 10 Mar 2022 13:51:03 -0500 Subject: [PATCH 06/22] feat: Update specified parquet file records to compaction level 1 Fixes #3950. --- compactor/src/compact.rs | 32 +++++++- iox_catalog/src/interface.rs | 139 +++++++++++++++++++++++++++++++++++ iox_catalog/src/mem.rs | 20 +++++ iox_catalog/src/metrics.rs | 1 + iox_catalog/src/postgres.rs | 24 ++++++ 5 files changed, 215 insertions(+), 1 deletion(-) diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index 3e5a541084..758d8dc3c7 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -9,6 +9,7 @@ use data_types2::{ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, use datafusion::error::DataFusionError; use iox_catalog::interface::Catalog; use object_store::ObjectStore; +use observability_deps::tracing::warn; use parquet_file::metadata::IoxMetadata; use query::{ compute_sort_key_for_chunks, @@ -96,6 +97,11 @@ pub enum Error { Level0 { source: iox_catalog::interface::Error, }, + + #[snafu(display("Error updating catalog {}", source))] + Update { + source: iox_catalog::interface::Error, + }, } /// A specialized `Error` for Compactor Data errors @@ -142,6 +148,7 @@ impl Compactor { async fn level_0_parquet_files(&self, sequencer_id: SequencerId) -> Result> { let mut repos = self.catalog.repositories().await; + repos .parquet_files() .level_0(sequencer_id) @@ -149,6 +156,29 @@ impl Compactor { .context(Level0Snafu) } + async fn update_to_level_1(&self, parquet_file_ids: &[ParquetFileId]) -> Result<()> { + let mut repos = self.catalog.repositories().await; + + let updated = repos + .parquet_files() + .update_to_level_1(parquet_file_ids) + .await + .context(UpdateSnafu)?; + + if updated.len() < parquet_file_ids.len() { + let parquet_file_ids: HashSet<_> = parquet_file_ids.iter().collect(); + let updated: HashSet<_> = updated.iter().collect(); + let not_updated = parquet_file_ids.difference(&updated); + + warn!( + "Unable to update to level 1 parquet files with IDs: {:?}", + not_updated + ); + } + + Ok(()) + } + // TODO: this function should be invoked in a backround loop /// Find and compact parquet files for a given sequencer pub async fn find_and_compact(&self, sequencer_id: SequencerId) -> Result<()> { @@ -219,7 +249,7 @@ impl Compactor { // TODO: #3953 - remove_fully_processed_tombstones(tombstones) // Upgrade old level-0 to level 1 - // TODO: #3950 - update_to_level_1(upgrade_level_list) + self.update_to_level_1(&upgrade_level_list).await?; Ok(()) } diff --git a/iox_catalog/src/interface.rs b/iox_catalog/src/interface.rs index 2efc6a94f2..7e2fb8eb1b 100644 --- a/iox_catalog/src/interface.rs +++ b/iox_catalog/src/interface.rs @@ -441,6 +441,13 @@ pub trait ParquetFileRepo: Send + Sync { /// define a file as a candidate for compaction async fn level_0(&mut self, sequencer_id: SequencerId) -> Result>; + /// Update the compaction level of the specified parquet files to level 1. Returns the IDs + /// of the files that were successfully updated. + async fn update_to_level_1( + &mut self, + parquet_file_ids: &[ParquetFileId], + ) -> Result>; + /// Verify if the parquet file exists by selecting its id async fn exist(&mut self, id: ParquetFileId) -> Result; @@ -544,6 +551,7 @@ pub(crate) mod test_helpers { test_tombstone(Arc::clone(&catalog)).await; test_parquet_file(Arc::clone(&catalog)).await; test_parquet_file_compaction_level_0(Arc::clone(&catalog)).await; + test_update_to_compaction_level_1(Arc::clone(&catalog)).await; test_add_parquet_file_with_tombstones(Arc::clone(&catalog)).await; test_txn_isolation(Arc::clone(&catalog)).await; test_txn_drop(Arc::clone(&catalog)).await; @@ -1505,6 +1513,18 @@ pub(crate) mod test_helpers { }; let _too_big_file = repos.parquet_files().create(too_big_params).await.unwrap(); + // Create a compaction level 1 file + let level_1_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let level_1_file = repos.parquet_files().create(level_1_params).await.unwrap(); + repos + .parquet_files() + .update_to_level_1(&[level_1_file.id]) + .await + .unwrap(); + // Level 0 parquet files for a sequencer should contain only those that match the right // criteria let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap(); @@ -1521,6 +1541,125 @@ pub(crate) mod test_helpers { ); } + async fn test_update_to_compaction_level_1(catalog: Arc) { + let mut repos = catalog.repositories().await; + let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap(); + let pool = repos.query_pools().create_or_get("foo").await.unwrap(); + let namespace = repos + .namespaces() + .create( + "namespace_update_to_compaction_level_1_test", + "inf", + kafka.id, + pool.id, + ) + .await + .unwrap(); + let table = repos + .tables() + .create_or_get("update_table", namespace.id) + .await + .unwrap(); + let sequencer = repos + .sequencers() + .create_or_get(&kafka, KafkaPartition::new(1000)) + .await + .unwrap(); + let partition = repos + .partitions() + .create_or_get("one", sequencer.id, table.id) + .await + .unwrap(); + + // Set up the window of times we're interested in level 1 files for + let query_min_time = Timestamp::new(5); + let query_max_time = Timestamp::new(10); + + // Create a file with times entirely within the window + let parquet_file_params = ParquetFileParams { + sequencer_id: sequencer.id, + table_id: partition.table_id, + partition_id: partition.id, + object_store_id: Uuid::new_v4(), + min_sequence_number: SequenceNumber::new(10), + max_sequence_number: SequenceNumber::new(140), + min_time: query_min_time + 1, + max_time: query_max_time - 1, + file_size_bytes: 1337, + parquet_metadata: b"md1".to_vec(), + row_count: 0, + created_at: Timestamp::new(1), + }; + let parquet_file = repos + .parquet_files() + .create(parquet_file_params.clone()) + .await + .unwrap(); + + // Create a file that will remain as level 0 + let level_0_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let level_0_file = repos.parquet_files().create(level_0_params).await.unwrap(); + + // Create a ParquetFileId that doesn't actually exist in the catalog + let nonexistent_parquet_file_id = ParquetFileId::new(level_0_file.id.get() + 1); + + // Level 0 parquet files should contain both existing files at this point + let expected = vec![parquet_file.clone(), level_0_file.clone()]; + let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap(); + let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect(); + level_0_ids.sort(); + let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect(); + expected_ids.sort(); + assert_eq!( + level_0_ids, expected_ids, + "\nlevel 0: {:#?}\nexpected: {:#?}", + level_0, expected, + ); + + // Make parquet_file compaction level 1, attempt to mark the nonexistent file; operation + // should succeed + let updated = repos + .parquet_files() + .update_to_level_1(&[parquet_file.id, nonexistent_parquet_file_id]) + .await + .unwrap(); + assert_eq!(updated, vec![parquet_file.id]); + + // Level 0 parquet files should only contain level_0_file + let expected = vec![level_0_file]; + let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap(); + let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect(); + level_0_ids.sort(); + let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect(); + expected_ids.sort(); + assert_eq!( + level_0_ids, expected_ids, + "\nlevel 0: {:#?}\nexpected: {:#?}", + level_0, expected, + ); + + // Level 1 parquet files for a sequencer should only contain parquet_file + let expected = vec![parquet_file]; + let table_partition = TablePartition::new(sequencer.id, table.id, partition.id); + let level_1 = repos + .parquet_files() + .level_1(table_partition, query_min_time, query_max_time) + .await + .unwrap(); + let mut level_1_ids: Vec<_> = level_1.iter().map(|pf| pf.id).collect(); + level_1_ids.sort(); + let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect(); + expected_ids.sort(); + assert_eq!( + level_1_ids, expected_ids, + "\nlevel 1: {:#?}\nexpected: {:#?}", + level_1, expected, + ); + } + async fn test_add_parquet_file_with_tombstones(catalog: Arc) { let mut txn = catalog.start_transaction().await.unwrap(); let kafka = txn.kafka_topics().create_or_get("foo").await.unwrap(); diff --git a/iox_catalog/src/mem.rs b/iox_catalog/src/mem.rs index 6854fcabd0..02930e1b2f 100644 --- a/iox_catalog/src/mem.rs +++ b/iox_catalog/src/mem.rs @@ -859,6 +859,26 @@ impl ParquetFileRepo for MemTxn { .collect()) } + async fn update_to_level_1( + &mut self, + parquet_file_ids: &[ParquetFileId], + ) -> Result> { + let stage = self.stage(); + + let mut updated = Vec::with_capacity(parquet_file_ids.len()); + + for f in stage + .parquet_files + .iter_mut() + .filter(|p| parquet_file_ids.contains(&p.id)) + { + f.compaction_level = 1; + updated.push(f.id); + } + + Ok(updated) + } + async fn exist(&mut self, id: ParquetFileId) -> Result { let stage = self.stage(); diff --git a/iox_catalog/src/metrics.rs b/iox_catalog/src/metrics.rs index 2b90a3062c..fd0c137615 100644 --- a/iox_catalog/src/metrics.rs +++ b/iox_catalog/src/metrics.rs @@ -263,6 +263,7 @@ decorate!( "parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result>; "parquet_list_by_namespace_not_to_delete" = list_by_namespace_not_to_delete(&mut self, namespace_id: NamespaceId) -> Result>; "parquet_level_0" = level_0(&mut self, sequencer_id: SequencerId) -> Result>; + "parquet_update_to_level_1" = update_to_level_1(&mut self, parquet_file_ids: &[ParquetFileId]) -> Result>; "parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result; "parquet_count" = count(&mut self) -> Result; ] diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index 03b529d152..38ae369bb9 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -1300,6 +1300,30 @@ WHERE parquet_file.sequencer_id = $1 .map_err(|e| Error::SqlxError { source: e }) } + async fn update_to_level_1( + &mut self, + parquet_file_ids: &[ParquetFileId], + ) -> Result> { + // If I try to do `.bind(parquet_file_ids)` directly, I get a compile error from sqlx. + // See https://github.com/launchbadge/sqlx/issues/1744 + let ids: Vec<_> = parquet_file_ids.iter().map(|p| p.get()).collect(); + let updated = sqlx::query( + r#" + UPDATE parquet_file + SET compaction_level = 1 + WHERE id = ANY($1) + RETURNING id + ;"#, + ) + .bind(&ids[..]) + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e })?; + + let updated = updated.into_iter().map(|row| row.get("id")).collect(); + Ok(updated) + } + async fn exist(&mut self, id: ParquetFileId) -> Result { let read_result = sqlx::query_as::<_, Count>( r#"SELECT count(*) as count FROM parquet_file WHERE id = $1;"#, From 1dacf567d95c8d5ea27f9488c085408037a4504e Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 10 Mar 2022 16:06:37 -0500 Subject: [PATCH 07/22] feat: Add a function to the catalog to fetch level 1 parquet files Fixes #3946. --- compactor/src/compact.rs | 53 +++++++- compactor/src/utils.rs | 29 +---- data_types2/src/lib.rs | 40 ++++++ iox_catalog/src/interface.rs | 238 ++++++++++++++++++++++++++++++++++- iox_catalog/src/mem.rs | 27 +++- iox_catalog/src/metrics.rs | 3 +- iox_catalog/src/postgres.rs | 48 ++++++- 7 files changed, 402 insertions(+), 36 deletions(-) diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index 758d8dc3c7..161c67a79d 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -2,10 +2,13 @@ use crate::{ query::QueryableParquetChunk, - utils::{CompactedData, ParquetFileWithTombstone, TablePartition}, + utils::{CompactedData, ParquetFileWithTombstone}, }; use backoff::BackoffConfig; -use data_types2::{ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, TombstoneId}; +use data_types2::{ + ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, TablePartition, Timestamp, + TombstoneId, +}; use datafusion::error::DataFusionError; use iox_catalog::interface::Catalog; use object_store::ObjectStore; @@ -98,6 +101,11 @@ pub enum Error { source: iox_catalog::interface::Error, }, + #[snafu(display("Error while requesting level 1 parquet files {}", source))] + Level1 { + source: iox_catalog::interface::Error, + }, + #[snafu(display("Error updating catalog {}", source))] Update { source: iox_catalog::interface::Error, @@ -156,6 +164,21 @@ impl Compactor { .context(Level0Snafu) } + async fn level_1_parquet_files( + &self, + table_partition: TablePartition, + min_time: Timestamp, + max_time: Timestamp, + ) -> Result> { + let mut repos = self.catalog.repositories().await; + + repos + .parquet_files() + .level_1(table_partition, min_time, max_time) + .await + .context(Level1Snafu) + } + async fn update_to_level_1(&self, parquet_file_ids: &[ParquetFileId]) -> Result<()> { let mut repos = self.catalog.repositories().await; @@ -189,12 +212,29 @@ impl Compactor { // Read level-0 parquet files let level_0_files = self.level_0_parquet_files(sequencer_id).await?; + // If there are no level-0 parquet files, return because there's nothing to do + if level_0_files.is_empty() { + return Ok(()); + } + // Group files into table partition let mut partitions = Self::group_parquet_files_into_partition(level_0_files); - // Get level-1 files overlapped with level-0 - for (_key, val) in &mut partitions.iter_mut() { - let level_1_files: Vec = vec![]; // TODO: #3946 + // Get level-1 files overlapped in time with level-0 + for (key, val) in &mut partitions.iter_mut() { + let overall_min_time = val + .iter() + .map(|pf| pf.min_time) + .min() + .expect("The list of files was checked for emptiness above"); + let overall_max_time = val + .iter() + .map(|pf| pf.max_time) + .max() + .expect("The list of files was checked for emptiness above"); + let level_1_files = self + .level_1_parquet_files(*key, overall_min_time, overall_max_time) + .await?; val.extend(level_1_files); } @@ -285,7 +325,8 @@ impl Compactor { } // Compact given files. Assume the given files are overlaped in time. - // If the assumption does not meet, we will spend time not to compact anything but put data together + // If the assumption does not meet, we will spend time not to compact anything but put data + // together async fn compact( &self, overlapped_files: Vec, diff --git a/compactor/src/utils.rs b/compactor/src/utils.rs index 79a2600690..f060c8679d 100644 --- a/compactor/src/utils.rs +++ b/compactor/src/utils.rs @@ -1,38 +1,15 @@ //! Helpers of the Compactor -use std::{collections::HashSet, sync::Arc}; - +use crate::query::QueryableParquetChunk; use arrow::record_batch::RecordBatch; -use data_types2::{ - ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, Tombstone, TombstoneId, -}; +use data_types2::{ParquetFile, ParquetFileId, Tombstone, TombstoneId}; use iox_object_store::IoxObjectStore; use object_store::ObjectStore; use parquet_file::{ chunk::{new_parquet_chunk, ChunkMetrics, DecodedParquetFile}, metadata::IoxMetadata, }; - -use crate::query::QueryableParquetChunk; - -/// Define table partition -#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)] -pub struct TablePartition { - sequencer_id: SequencerId, - table_id: TableId, - partition_id: PartitionId, -} - -impl TablePartition { - /// Return a new table partition - pub fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self { - Self { - sequencer_id, - table_id, - partition_id, - } - } -} +use std::{collections::HashSet, sync::Arc}; /// Wrapper of a parquet file and its tombstones #[allow(missing_docs)] diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 548ff41067..a373205ac8 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -17,6 +17,7 @@ use std::{ collections::BTreeMap, convert::TryFrom, fmt::{Debug, Formatter}, + ops::{Add, Sub}, sync::Arc, }; use uuid::Uuid; @@ -189,6 +190,29 @@ impl std::fmt::Display for PartitionId { } } +/// Combination of Sequencer ID, Table ID, and Partition ID useful for identifying groups of +/// Parquet files to be compacted together. +#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)] +pub struct TablePartition { + /// The sequencer ID + pub sequencer_id: SequencerId, + /// The table ID + pub table_id: TableId, + /// The partition ID + pub partition_id: PartitionId, +} + +impl TablePartition { + /// Combine the relevant parts + pub fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self { + Self { + sequencer_id, + table_id, + partition_id, + } + } +} + /// Unique ID for a `Tombstone` #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)] #[sqlx(transparent)] @@ -234,6 +258,22 @@ impl Timestamp { } } +impl Add for Timestamp { + type Output = Self; + + fn add(self, other: i64) -> Self { + Self(self.0 + other) + } +} + +impl Sub for Timestamp { + type Output = Self; + + fn sub(self, other: i64) -> Self { + Self(self.0 - other) + } +} + /// Unique ID for a `ParquetFile` #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)] #[sqlx(transparent)] diff --git a/iox_catalog/src/interface.rs b/iox_catalog/src/interface.rs index 7e2fb8eb1b..77dfc20877 100644 --- a/iox_catalog/src/interface.rs +++ b/iox_catalog/src/interface.rs @@ -5,7 +5,8 @@ use data_types2::{ Column, ColumnSchema, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId, NamespaceSchema, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, - Sequencer, SequencerId, Table, TableId, TableSchema, Timestamp, Tombstone, TombstoneId, + Sequencer, SequencerId, Table, TableId, TablePartition, TableSchema, Timestamp, Tombstone, + TombstoneId, }; use snafu::{OptionExt, Snafu}; use std::{collections::BTreeMap, convert::TryFrom, fmt::Debug, sync::Arc}; @@ -441,6 +442,16 @@ pub trait ParquetFileRepo: Send + Sync { /// define a file as a candidate for compaction async fn level_0(&mut self, sequencer_id: SequencerId) -> Result>; + /// List parquet files for a given table partition, in a given time range, with compaction + /// level 1, and other criteria that define a file as a candidate for compaction with a level 0 + /// file + async fn level_1( + &mut self, + table_partition: TablePartition, + min_time: Timestamp, + max_time: Timestamp, + ) -> Result>; + /// Update the compaction level of the specified parquet files to level 1. Returns the IDs /// of the files that were successfully updated. async fn update_to_level_1( @@ -551,6 +562,7 @@ pub(crate) mod test_helpers { test_tombstone(Arc::clone(&catalog)).await; test_parquet_file(Arc::clone(&catalog)).await; test_parquet_file_compaction_level_0(Arc::clone(&catalog)).await; + test_parquet_file_compaction_level_1(Arc::clone(&catalog)).await; test_update_to_compaction_level_1(Arc::clone(&catalog)).await; test_add_parquet_file_with_tombstones(Arc::clone(&catalog)).await; test_txn_isolation(Arc::clone(&catalog)).await; @@ -1541,6 +1553,230 @@ pub(crate) mod test_helpers { ); } + async fn test_parquet_file_compaction_level_1(catalog: Arc) { + let mut repos = catalog.repositories().await; + let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap(); + let pool = repos.query_pools().create_or_get("foo").await.unwrap(); + let namespace = repos + .namespaces() + .create( + "namespace_parquet_file_compaction_level_1_test", + "inf", + kafka.id, + pool.id, + ) + .await + .unwrap(); + let table = repos + .tables() + .create_or_get("test_table", namespace.id) + .await + .unwrap(); + let other_table = repos + .tables() + .create_or_get("test_table2", namespace.id) + .await + .unwrap(); + let sequencer = repos + .sequencers() + .create_or_get(&kafka, KafkaPartition::new(100)) + .await + .unwrap(); + let other_sequencer = repos + .sequencers() + .create_or_get(&kafka, KafkaPartition::new(101)) + .await + .unwrap(); + let partition = repos + .partitions() + .create_or_get("one", sequencer.id, table.id) + .await + .unwrap(); + let other_partition = repos + .partitions() + .create_or_get("two", sequencer.id, table.id) + .await + .unwrap(); + + // Set up the window of times we're interested in level 1 files for + let query_min_time = Timestamp::new(5); + let query_max_time = Timestamp::new(10); + + // Create a file with times entirely within the window + let parquet_file_params = ParquetFileParams { + sequencer_id: sequencer.id, + table_id: partition.table_id, + partition_id: partition.id, + object_store_id: Uuid::new_v4(), + min_sequence_number: SequenceNumber::new(10), + max_sequence_number: SequenceNumber::new(140), + min_time: query_min_time + 1, + max_time: query_max_time - 1, + file_size_bytes: 1337, + parquet_metadata: b"md1".to_vec(), + row_count: 0, + created_at: Timestamp::new(1), + }; + let parquet_file = repos + .parquet_files() + .create(parquet_file_params.clone()) + .await + .unwrap(); + + // Create a file that will remain as level 0 + let level_0_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let _level_0_file = repos.parquet_files().create(level_0_params).await.unwrap(); + + // Create a file completely before the window + let too_early_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + min_time: query_min_time - 2, + max_time: query_min_time - 1, + ..parquet_file_params.clone() + }; + let too_early_file = repos + .parquet_files() + .create(too_early_params) + .await + .unwrap(); + + // Create a file overlapping the window on the lower end + let overlap_lower_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + min_time: query_min_time - 1, + max_time: query_min_time + 1, + ..parquet_file_params.clone() + }; + let overlap_lower_file = repos + .parquet_files() + .create(overlap_lower_params) + .await + .unwrap(); + + // Create a file overlapping the window on the upper end + let overlap_upper_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + min_time: query_max_time - 1, + max_time: query_max_time + 1, + ..parquet_file_params.clone() + }; + let overlap_upper_file = repos + .parquet_files() + .create(overlap_upper_params) + .await + .unwrap(); + + // Create a file completely after the window + let too_late_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + min_time: query_max_time + 1, + max_time: query_max_time + 2, + ..parquet_file_params.clone() + }; + let too_late_file = repos.parquet_files().create(too_late_params).await.unwrap(); + + // Create a file for some other sequencer + let other_sequencer_params = ParquetFileParams { + sequencer_id: other_sequencer.id, + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let other_sequencer_file = repos + .parquet_files() + .create(other_sequencer_params) + .await + .unwrap(); + + // Create a file for the same sequencer but a different table + let other_table_params = ParquetFileParams { + table_id: other_table.id, + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let other_table_file = repos + .parquet_files() + .create(other_table_params) + .await + .unwrap(); + + // Create a file for the same sequencer and table but a different partition + let other_partition_params = ParquetFileParams { + partition_id: other_partition.id, + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let other_partition_file = repos + .parquet_files() + .create(other_partition_params) + .await + .unwrap(); + + // Create a file too big to be considered + let too_big_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + file_size_bytes: MAX_COMPACT_SIZE + 1, + ..parquet_file_params.clone() + }; + let too_big_file = repos.parquet_files().create(too_big_params).await.unwrap(); + + // Create a file marked to be deleted + let to_delete_params = ParquetFileParams { + object_store_id: Uuid::new_v4(), + ..parquet_file_params.clone() + }; + let to_delete_file = repos + .parquet_files() + .create(to_delete_params) + .await + .unwrap(); + repos + .parquet_files() + .flag_for_delete(to_delete_file.id) + .await + .unwrap(); + + // Make all but _level_0_file compaction level 1 + repos + .parquet_files() + .update_to_level_1(&[ + parquet_file.id, + too_early_file.id, + too_late_file.id, + overlap_lower_file.id, + overlap_upper_file.id, + other_sequencer_file.id, + other_table_file.id, + other_partition_file.id, + too_big_file.id, + to_delete_file.id, + ]) + .await + .unwrap(); + + // Level 1 parquet files for a sequencer should contain only those that match the right + // criteria + let table_partition = TablePartition::new(sequencer.id, table.id, partition.id); + let level_1 = repos + .parquet_files() + .level_1(table_partition, query_min_time, query_max_time) + .await + .unwrap(); + let mut level_1_ids: Vec<_> = level_1.iter().map(|pf| pf.id).collect(); + level_1_ids.sort(); + let expected = vec![parquet_file, overlap_lower_file, overlap_upper_file]; + let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect(); + expected_ids.sort(); + + assert_eq!( + level_1_ids, expected_ids, + "\nlevel 1: {:#?}\nexpected: {:#?}", + level_1, expected, + ); + } + async fn test_update_to_compaction_level_1(catalog: Arc) { let mut repos = catalog.repositories().await; let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap(); diff --git a/iox_catalog/src/mem.rs b/iox_catalog/src/mem.rs index 02930e1b2f..5db4476d11 100644 --- a/iox_catalog/src/mem.rs +++ b/iox_catalog/src/mem.rs @@ -15,7 +15,7 @@ use data_types2::{ Column, ColumnId, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table, - TableId, Timestamp, Tombstone, TombstoneId, + TableId, TablePartition, Timestamp, Tombstone, TombstoneId, }; use observability_deps::tracing::warn; use std::fmt::Formatter; @@ -859,6 +859,31 @@ impl ParquetFileRepo for MemTxn { .collect()) } + async fn level_1( + &mut self, + table_partition: TablePartition, + min_time: Timestamp, + max_time: Timestamp, + ) -> Result> { + let stage = self.stage(); + + Ok(stage + .parquet_files + .iter() + .filter(|f| { + f.sequencer_id == table_partition.sequencer_id + && f.table_id == table_partition.table_id + && f.partition_id == table_partition.partition_id + && f.compaction_level == 1 + && !f.to_delete + && f.file_size_bytes <= MAX_COMPACT_SIZE + && ((f.min_time <= min_time && f.max_time >= min_time) + || (f.min_time > min_time && f.min_time <= max_time)) + }) + .cloned() + .collect()) + } + async fn update_to_level_1( &mut self, parquet_file_ids: &[ParquetFileId], diff --git a/iox_catalog/src/metrics.rs b/iox_catalog/src/metrics.rs index fd0c137615..123396a86c 100644 --- a/iox_catalog/src/metrics.rs +++ b/iox_catalog/src/metrics.rs @@ -10,7 +10,7 @@ use data_types2::{ Column, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table, - TableId, Timestamp, Tombstone, TombstoneId, + TableId, TablePartition, Timestamp, Tombstone, TombstoneId, }; use metric::{Metric, U64Histogram, U64HistogramOptions}; use std::{fmt::Debug, sync::Arc}; @@ -263,6 +263,7 @@ decorate!( "parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result>; "parquet_list_by_namespace_not_to_delete" = list_by_namespace_not_to_delete(&mut self, namespace_id: NamespaceId) -> Result>; "parquet_level_0" = level_0(&mut self, sequencer_id: SequencerId) -> Result>; + "parquet_level_1" = level_1(&mut self, table_partition: TablePartition, min_time: Timestamp, max_time: Timestamp) -> Result>; "parquet_update_to_level_1" = update_to_level_1(&mut self, parquet_file_ids: &[ParquetFileId]) -> Result>; "parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result; "parquet_count" = count(&mut self) -> Result; diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index 38ae369bb9..cf465d91e7 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -14,7 +14,7 @@ use data_types2::{ Column, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table, - TableId, Timestamp, Tombstone, TombstoneId, + TableId, TablePartition, Timestamp, Tombstone, TombstoneId, }; use observability_deps::tracing::{info, warn}; use sqlx::{migrate::Migrator, postgres::PgPoolOptions, Acquire, Executor, Postgres, Row}; @@ -1300,6 +1300,52 @@ WHERE parquet_file.sequencer_id = $1 .map_err(|e| Error::SqlxError { source: e }) } + async fn level_1( + &mut self, + table_partition: TablePartition, + min_time: Timestamp, + max_time: Timestamp, + ) -> Result> { + sqlx::query_as::<_, ParquetFile>( + r#" +SELECT + parquet_file.id as id, + parquet_file.sequencer_id as sequencer_id, + parquet_file.table_id as table_id, + parquet_file.partition_id as partition_id, + parquet_file.object_store_id as object_store_id, + parquet_file.min_sequence_number as min_sequence_number, + parquet_file.max_sequence_number as max_sequence_number, + parquet_file.min_time as min_time, + parquet_file.max_time as max_time, + parquet_file.to_delete as to_delete, + parquet_file.file_size_bytes as file_size_bytes, + parquet_file.parquet_metadata as parquet_metadata, + parquet_file.row_count as row_count, + parquet_file.compaction_level as compaction_level, + parquet_file.created_at as created_at +FROM parquet_file +WHERE parquet_file.sequencer_id = $1 + AND parquet_file.table_id = $2 + AND parquet_file.partition_id = $3 + AND parquet_file.compaction_level = 1 + AND parquet_file.to_delete = false + AND parquet_file.file_size_bytes <= $4 + AND ((parquet_file.min_time <= $5 AND parquet_file.max_time >= $5) + OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $6)) + ;"#, + ) + .bind(&table_partition.sequencer_id) // $1 + .bind(&table_partition.table_id) // $2 + .bind(&table_partition.partition_id) // $3 + .bind(MAX_COMPACT_SIZE) // $4 + .bind(min_time) // $5 + .bind(max_time) // $6 + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e }) + } + async fn update_to_level_1( &mut self, parquet_file_ids: &[ParquetFileId], From 8888e4c3a276289ba9238afcf56e88719d36acd2 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Sun, 13 Mar 2022 20:09:30 -0400 Subject: [PATCH 08/22] fix: Remove MAX_COMPACT_SIZE from the compaction queries --- iox_catalog/src/interface.rs | 20 -------------------- iox_catalog/src/mem.rs | 10 ++-------- iox_catalog/src/postgres.rs | 16 ++++++---------- 3 files changed, 8 insertions(+), 38 deletions(-) diff --git a/iox_catalog/src/interface.rs b/iox_catalog/src/interface.rs index 77dfc20877..f992c3b94e 100644 --- a/iox_catalog/src/interface.rs +++ b/iox_catalog/src/interface.rs @@ -410,9 +410,6 @@ pub trait TombstoneRepo: Send + Sync { /// The starting compaction level for parquet files is zero. pub const INITIAL_COMPACTION_LEVEL: i16 = 0; -/// The maximum size, in bytes, of a file to be considered for compaction. -pub const MAX_COMPACT_SIZE: i64 = 1024 * 1024; - /// Functions for working with parquet file pointers in the catalog #[async_trait] pub trait ParquetFileRepo: Send + Sync { @@ -1517,14 +1514,6 @@ pub(crate) mod test_helpers { .await .unwrap(); - // Create a compaction level 0 file over the MAX_COMPACT_SIZE - let too_big_params = ParquetFileParams { - object_store_id: Uuid::new_v4(), - file_size_bytes: MAX_COMPACT_SIZE + 1, - ..parquet_file_params.clone() - }; - let _too_big_file = repos.parquet_files().create(too_big_params).await.unwrap(); - // Create a compaction level 1 file let level_1_params = ParquetFileParams { object_store_id: Uuid::new_v4(), @@ -1714,14 +1703,6 @@ pub(crate) mod test_helpers { .await .unwrap(); - // Create a file too big to be considered - let too_big_params = ParquetFileParams { - object_store_id: Uuid::new_v4(), - file_size_bytes: MAX_COMPACT_SIZE + 1, - ..parquet_file_params.clone() - }; - let too_big_file = repos.parquet_files().create(too_big_params).await.unwrap(); - // Create a file marked to be deleted let to_delete_params = ParquetFileParams { object_store_id: Uuid::new_v4(), @@ -1750,7 +1731,6 @@ pub(crate) mod test_helpers { other_sequencer_file.id, other_table_file.id, other_partition_file.id, - too_big_file.id, to_delete_file.id, ]) .await diff --git a/iox_catalog/src/mem.rs b/iox_catalog/src/mem.rs index 5db4476d11..05b92f6c68 100644 --- a/iox_catalog/src/mem.rs +++ b/iox_catalog/src/mem.rs @@ -6,7 +6,7 @@ use crate::{ sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error, KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo, QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo, - TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, MAX_COMPACT_SIZE, + TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, }, metrics::MetricDecorator, }; @@ -849,12 +849,7 @@ impl ParquetFileRepo for MemTxn { Ok(stage .parquet_files .iter() - .filter(|f| { - f.sequencer_id == sequencer_id - && f.compaction_level == 0 - && !f.to_delete - && f.file_size_bytes <= MAX_COMPACT_SIZE - }) + .filter(|f| f.sequencer_id == sequencer_id && f.compaction_level == 0 && !f.to_delete) .cloned() .collect()) } @@ -876,7 +871,6 @@ impl ParquetFileRepo for MemTxn { && f.partition_id == table_partition.partition_id && f.compaction_level == 1 && !f.to_delete - && f.file_size_bytes <= MAX_COMPACT_SIZE && ((f.min_time <= min_time && f.max_time >= min_time) || (f.min_time > min_time && f.min_time <= max_time)) }) diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index cf465d91e7..b9d953148e 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -5,7 +5,7 @@ use crate::{ sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error, KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo, QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo, - TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, MAX_COMPACT_SIZE, + TombstoneRepo, Transaction, INITIAL_COMPACTION_LEVEL, }, metrics::MetricDecorator, }; @@ -1289,12 +1289,10 @@ SELECT FROM parquet_file WHERE parquet_file.sequencer_id = $1 AND parquet_file.compaction_level = 0 - AND parquet_file.to_delete = false - AND parquet_file.file_size_bytes <= $2; + AND parquet_file.to_delete = false; "#, ) .bind(&sequencer_id) // $1 - .bind(MAX_COMPACT_SIZE) // $2 .fetch_all(&mut self.inner) .await .map_err(|e| Error::SqlxError { source: e }) @@ -1330,17 +1328,15 @@ WHERE parquet_file.sequencer_id = $1 AND parquet_file.partition_id = $3 AND parquet_file.compaction_level = 1 AND parquet_file.to_delete = false - AND parquet_file.file_size_bytes <= $4 - AND ((parquet_file.min_time <= $5 AND parquet_file.max_time >= $5) - OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $6)) + AND ((parquet_file.min_time <= $5 AND parquet_file.max_time >= $4) + OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $5)) ;"#, ) .bind(&table_partition.sequencer_id) // $1 .bind(&table_partition.table_id) // $2 .bind(&table_partition.partition_id) // $3 - .bind(MAX_COMPACT_SIZE) // $4 - .bind(min_time) // $5 - .bind(max_time) // $6 + .bind(min_time) // $4 + .bind(max_time) // $5 .fetch_all(&mut self.inner) .await .map_err(|e| Error::SqlxError { source: e }) From 268138ceef8488460611c3852c8f88d5ab112ed2 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Sun, 13 Mar 2022 20:28:10 -0400 Subject: [PATCH 09/22] fix: Make SQL queries more consistent - Use "SELECT *" when possible - Left align - Wrap at 100 chars - Include semicolon --- iox_catalog/src/postgres.rs | 429 +++++++++++++++++++----------------- 1 file changed, 226 insertions(+), 203 deletions(-) diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index b9d953148e..382b18dfbb 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -174,8 +174,8 @@ impl Drop for PostgresTxn { if let PostgresTxnInner::Txn(Some(_)) = self.inner { warn!("Dropping PostgresTxn w/o finalizing (commit or abort)"); - // SQLx ensures that the inner transaction enqueues a rollback when it is dropped, so we don't need to spawn - // a task here to call `rollback` manually. + // SQLx ensures that the inner transaction enqueues a rollback when it is dropped, so + // we don't need to spawn a task here to call `rollback` manually. } } } @@ -214,15 +214,15 @@ impl TransactionFinalize for PostgresTxn { #[async_trait] impl Catalog for PostgresCatalog { async fn setup(&self) -> Result<(), Error> { - // We need to create the schema if we're going to set it as the first item of the search_path - // otherwise when we run the sqlx migration scripts for the first time, sqlx will create the - // `_sqlx_migrations` table in the public namespace (the only namespace that exists), but the second - // time it will create it in the `` namespace and re-run all the migrations without - // skipping the ones already applied (see #3893). + // We need to create the schema if we're going to set it as the first item of the + // search_path otherwise when we run the sqlx migration scripts for the first time, sqlx + // will create the `_sqlx_migrations` table in the public namespace (the only namespace + // that exists), but the second time it will create it in the `` namespace and + // re-run all the migrations without skipping the ones already applied (see #3893). // - // This makes the migrations/20210217134322_create_schema.sql step unnecessary; we need to keep that - // file because migration files are immutable. - let create_schema_query = format!("CREATE SCHEMA IF NOT EXISTS {}", &self.schema_name); + // This makes the migrations/20210217134322_create_schema.sql step unnecessary; we need to + // keep that file because migration files are immutable. + let create_schema_query = format!("CREATE SCHEMA IF NOT EXISTS {};", &self.schema_name); self.pool .execute(sqlx::query(&create_schema_query)) .await @@ -293,16 +293,16 @@ async fn new_raw_pool( // set as part of the DSN, and we can set it explictly. // Recall that this block is running on connection, not when creating the pool! let current_application_name: String = - sqlx::query_scalar("SELECT current_setting('application_name')") + sqlx::query_scalar("SELECT current_setting('application_name');") .fetch_one(&mut *c) .await?; if current_application_name.is_empty() { - sqlx::query("SELECT set_config('application_name', $1, false)") + sqlx::query("SELECT set_config('application_name', $1, false);") .bind(&*app_name) .execute(&mut *c) .await?; } - let search_path_query = format!("SET search_path TO {},public", schema_name); + let search_path_query = format!("SET search_path TO {},public;", schema_name); c.execute(sqlx::query(&search_path_query)).await?; Ok(()) }) @@ -395,7 +395,12 @@ async fn new_pool( current_dsn = new_dsn; } Err(e) => { - warn!(error=%e, filename=%dsn_file, "not replacing hotswap pool because of an error connecting to the new DSN"); + warn!( + error=%e, + filename=%dsn_file, + "not replacing hotswap pool because of an error \ + connecting to the new DSN" + ); } } } @@ -467,7 +472,8 @@ impl KafkaTopicRepo for PostgresTxn { INSERT INTO kafka_topic ( name ) VALUES ( $1 ) ON CONFLICT ON CONSTRAINT kafka_topic_name_unique -DO UPDATE SET name = kafka_topic.name RETURNING *; +DO UPDATE SET name = kafka_topic.name +RETURNING *; "#, ) .bind(&name) // $1 @@ -481,7 +487,9 @@ DO UPDATE SET name = kafka_topic.name RETURNING *; async fn get_by_name(&mut self, name: &str) -> Result> { let rec = sqlx::query_as::<_, KafkaTopic>( r#" -SELECT * FROM kafka_topic WHERE name = $1; +SELECT * +FROM kafka_topic +WHERE name = $1; "#, ) .bind(&name) // $1 @@ -506,7 +514,8 @@ impl QueryPoolRepo for PostgresTxn { INSERT INTO query_pool ( name ) VALUES ( $1 ) ON CONFLICT ON CONSTRAINT query_pool_name_unique -DO UPDATE SET name = query_pool.name RETURNING *; +DO UPDATE SET name = query_pool.name +RETURNING *; "#, ) .bind(&name) // $1 @@ -531,7 +540,7 @@ impl NamespaceRepo for PostgresTxn { r#" INSERT INTO namespace ( name, retention_duration, kafka_topic_id, query_pool_id ) VALUES ( $1, $2, $3, $4 ) -RETURNING * +RETURNING *; "#, ) .bind(&name) // $1 @@ -558,7 +567,8 @@ RETURNING * async fn list(&mut self) -> Result> { let rec = sqlx::query_as::<_, Namespace>( r#" -SELECT * FROM namespace; +SELECT * +FROM namespace; "#, ) .fetch_all(&mut self.inner) @@ -571,7 +581,9 @@ SELECT * FROM namespace; async fn get_by_id(&mut self, id: NamespaceId) -> Result> { let rec = sqlx::query_as::<_, Namespace>( r#" -SELECT * FROM namespace WHERE id = $1; +SELECT * +FROM namespace +WHERE id = $1; "#, ) .bind(&id) // $1 @@ -590,7 +602,9 @@ SELECT * FROM namespace WHERE id = $1; async fn get_by_name(&mut self, name: &str) -> Result> { let rec = sqlx::query_as::<_, Namespace>( r#" -SELECT * FROM namespace WHERE name = $1; +SELECT * +FROM namespace +WHERE name = $1; "#, ) .bind(&name) // $1 @@ -609,7 +623,10 @@ SELECT * FROM namespace WHERE name = $1; async fn update_table_limit(&mut self, name: &str, new_max: i32) -> Result { let rec = sqlx::query_as::<_, Namespace>( r#" -UPDATE namespace SET max_tables = $1 WHERE name = $2 RETURNING *; +UPDATE namespace +SET max_tables = $1 +WHERE name = $2 +RETURNING *; "#, ) .bind(&new_max) @@ -630,7 +647,10 @@ UPDATE namespace SET max_tables = $1 WHERE name = $2 RETURNING *; async fn update_column_limit(&mut self, name: &str, new_max: i32) -> Result { let rec = sqlx::query_as::<_, Namespace>( r#" -UPDATE namespace SET max_columns_per_table = $1 WHERE name = $2 RETURNING *; +UPDATE namespace +SET max_columns_per_table = $1 +WHERE name = $2 +RETURNING *; "#, ) .bind(&new_max) @@ -657,7 +677,8 @@ impl TableRepo for PostgresTxn { INSERT INTO table_name ( name, namespace_id ) VALUES ( $1, $2 ) ON CONFLICT ON CONSTRAINT table_name_unique -DO UPDATE SET name = table_name.name RETURNING *; +DO UPDATE SET name = table_name.name +RETURNING *; "#, ) .bind(&name) // $1 @@ -676,10 +697,16 @@ DO UPDATE SET name = table_name.name RETURNING *; } async fn get_by_id(&mut self, table_id: TableId) -> Result> { - let rec = sqlx::query_as::<_, Table>(r#"SELECT * FROM table_name WHERE id = $1;"#) - .bind(&table_id) // $1 - .fetch_one(&mut self.inner) - .await; + let rec = sqlx::query_as::<_, Table>( + r#" +SELECT * +FROM table_name +WHERE id = $1; + "#, + ) + .bind(&table_id) // $1 + .fetch_one(&mut self.inner) + .await; if let Err(sqlx::Error::RowNotFound) = rec { return Ok(None); @@ -693,7 +720,8 @@ DO UPDATE SET name = table_name.name RETURNING *; async fn list_by_namespace_id(&mut self, namespace_id: NamespaceId) -> Result> { let rec = sqlx::query_as::<_, Table>( r#" -SELECT * FROM table_name +SELECT * +FROM table_name WHERE namespace_id = $1; "#, ) @@ -713,30 +741,32 @@ WHERE namespace_id = $1; ) -> Result> { let rec = sqlx::query_as::<_, TablePersistInfo>( r#" - WITH tid as (SELECT id FROM table_name WHERE name = $2 AND namespace_id = $3) - SELECT $1 as sequencer_id, id as table_id, parquet_file.max_sequence_number AS parquet_max_sequence_number, - tombstone.sequence_number as tombstone_max_sequence_number - FROM tid - LEFT JOIN ( - SELECT tombstone.table_id, sequence_number - FROM tombstone - WHERE sequencer_id = $1 AND tombstone.table_id = (SELECT id FROM tid) - ORDER BY sequence_number DESC - LIMIT 1 - ) tombstone ON tombstone.table_id = tid.id - LEFT JOIN ( - SELECT parquet_file.table_id, max_sequence_number - FROM parquet_file - WHERE parquet_file.sequencer_id = $1 AND parquet_file.table_id = (SELECT id from tid) - ORDER BY max_sequence_number DESC - LIMIT 1 - ) parquet_file ON parquet_file.table_id = tid.id; - "#) - .bind(&sequencer_id) // $1 - .bind(&table_name) // $2 - .bind(&namespace_id) // $3 - .fetch_one(&mut self.inner) - .await; +WITH tid as (SELECT id FROM table_name WHERE name = $2 AND namespace_id = $3) +SELECT $1 as sequencer_id, id as table_id, + parquet_file.max_sequence_number AS parquet_max_sequence_number, + tombstone.sequence_number as tombstone_max_sequence_number +FROM tid +LEFT JOIN ( + SELECT tombstone.table_id, sequence_number + FROM tombstone + WHERE sequencer_id = $1 AND tombstone.table_id = (SELECT id FROM tid) + ORDER BY sequence_number DESC + LIMIT 1 +) tombstone ON tombstone.table_id = tid.id +LEFT JOIN ( + SELECT parquet_file.table_id, max_sequence_number + FROM parquet_file + WHERE parquet_file.sequencer_id = $1 AND parquet_file.table_id = (SELECT id from tid) + ORDER BY max_sequence_number DESC + LIMIT 1 +) parquet_file ON parquet_file.table_id = tid.id; + "#, + ) + .bind(&sequencer_id) // $1 + .bind(&table_name) // $2 + .bind(&namespace_id) // $3 + .fetch_one(&mut self.inner) + .await; if let Err(sqlx::Error::RowNotFound) = rec { return Ok(None); @@ -763,7 +793,8 @@ impl ColumnRepo for PostgresTxn { INSERT INTO column_name ( name, table_id, column_type ) VALUES ( $1, $2, $3 ) ON CONFLICT ON CONSTRAINT column_name_unique -DO UPDATE SET name = column_name.name RETURNING *; +DO UPDATE SET name = column_name.name +RETURNING *; "#, ) .bind(&name) // $1 @@ -821,18 +852,19 @@ WHERE table_name.namespace_id = $1; let out = sqlx::query_as::<_, Column>( r#" - INSERT INTO column_name ( name, table_id, column_type ) - SELECT name, table_id, column_type FROM UNNEST($1, $2, $3) as a(name, table_id, column_type) - ON CONFLICT ON CONSTRAINT column_name_unique - DO UPDATE SET name = column_name.name - RETURNING * +INSERT INTO column_name ( name, table_id, column_type ) +SELECT name, table_id, column_type FROM UNNEST($1, $2, $3) as a(name, table_id, column_type) +ON CONFLICT ON CONSTRAINT column_name_unique +DO UPDATE SET name = column_name.name +RETURNING *; "#, ) .bind(&v_name) .bind(&v_table_id) .bind(&v_column_type) .fetch_all(&mut self.inner) - .await.map_err(|e| { + .await + .map_err(|e| { if is_fk_violation(&e) { Error::ForeignKeyViolation { source: e } } else { @@ -869,12 +901,13 @@ impl SequencerRepo for PostgresTxn { ) -> Result { sqlx::query_as::<_, Sequencer>( r#" - INSERT INTO sequencer - ( kafka_topic_id, kafka_partition, min_unpersisted_sequence_number ) - VALUES - ( $1, $2, 0 ) - ON CONFLICT ON CONSTRAINT sequencer_unique - DO UPDATE SET kafka_topic_id = sequencer.kafka_topic_id RETURNING *; +INSERT INTO sequencer + ( kafka_topic_id, kafka_partition, min_unpersisted_sequence_number ) +VALUES + ( $1, $2, 0 ) +ON CONFLICT ON CONSTRAINT sequencer_unique +DO UPDATE SET kafka_topic_id = sequencer.kafka_topic_id +RETURNING *;; "#, ) .bind(&topic.id) // $1 @@ -897,7 +930,10 @@ impl SequencerRepo for PostgresTxn { ) -> Result> { let rec = sqlx::query_as::<_, Sequencer>( r#" -SELECT * FROM sequencer WHERE kafka_topic_id = $1 AND kafka_partition = $2; +SELECT * +FROM sequencer +WHERE kafka_topic_id = $1 + AND kafka_partition = $2; "#, ) .bind(topic_id) // $1 @@ -957,12 +993,13 @@ impl PartitionRepo for PostgresTxn { ) -> Result { let v = sqlx::query_as::<_, Partition>( r#" - INSERT INTO partition - ( partition_key, sequencer_id, table_id ) - VALUES - ( $1, $2, $3 ) - ON CONFLICT ON CONSTRAINT partition_key_unique - DO UPDATE SET partition_key = partition.partition_key RETURNING *; +INSERT INTO partition + ( partition_key, sequencer_id, table_id ) +VALUES + ( $1, $2, $3 ) +ON CONFLICT ON CONSTRAINT partition_key_unique +DO UPDATE SET partition_key = partition.partition_key +RETURNING *; "#, ) .bind(key) // $1 @@ -1014,16 +1051,21 @@ impl PartitionRepo for PostgresTxn { } async fn list_by_namespace(&mut self, namespace_id: NamespaceId) -> Result> { - sqlx::query_as::<_, Partition>(r#" -SELECT partition.id as id, partition.sequencer_id as sequencer_id, partition.table_id as table_id, partition.partition_key as partition_key + sqlx::query_as::<_, Partition>( + r#" +SELECT partition.id as id, + partition.sequencer_id as sequencer_id, + partition.table_id as table_id, + partition.partition_key as partition_key FROM table_name INNER JOIN partition on partition.table_id = table_name.id WHERE table_name.namespace_id = $1; - "#) - .bind(&namespace_id) // $1 - .fetch_all(&mut self.inner) - .await - .map_err(|e| Error::SqlxError { source: e }) + "#, + ) + .bind(&namespace_id) // $1 + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e }) } async fn partition_info_by_id( @@ -1032,12 +1074,13 @@ WHERE table_name.namespace_id = $1; ) -> Result> { let info = sqlx::query( r#" - SELECT namespace.name as namespace_name, table_name.name as table_name, partition.id, - partition.sequencer_id, partition.table_id, partition.partition_key - FROM partition - INNER JOIN table_name on table_name.id = partition.table_id - INNER JOIN namespace on namespace.id = table_name.namespace_id - WHERE partition.id = $1;"#, +SELECT namespace.name as namespace_name, table_name.name as table_name, partition.id, + partition.sequencer_id, partition.table_id, partition.partition_key +FROM partition +INNER JOIN table_name on table_name.id = partition.table_id +INNER JOIN namespace on namespace.id = table_name.namespace_id +WHERE partition.id = $1; + "#, ) .bind(&partition_id) // $1 .fetch_one(&mut self.inner) @@ -1074,12 +1117,13 @@ impl TombstoneRepo for PostgresTxn { ) -> Result { let v = sqlx::query_as::<_, Tombstone>( r#" - INSERT INTO tombstone - ( table_id, sequencer_id, sequence_number, min_time, max_time, serialized_predicate ) - VALUES - ( $1, $2, $3, $4, $5, $6 ) - ON CONFLICT ON CONSTRAINT tombstone_unique - DO UPDATE SET table_id = tombstone.table_id RETURNING *; +INSERT INTO tombstone + ( table_id, sequencer_id, sequence_number, min_time, max_time, serialized_predicate ) +VALUES + ( $1, $2, $3, $4, $5, $6 ) +ON CONFLICT ON CONSTRAINT tombstone_unique +DO UPDATE SET table_id = tombstone.table_id +RETURNING *; "#, ) .bind(&table_id) // $1 @@ -1147,12 +1191,20 @@ WHERE table_name.namespace_id = $1; sequencer_id: SequencerId, sequence_number: SequenceNumber, ) -> Result> { - sqlx::query_as::<_, Tombstone>(r#"SELECT * FROM tombstone WHERE sequencer_id = $1 AND sequence_number > $2 ORDER BY id;"#) - .bind(&sequencer_id) // $1 - .bind(&sequence_number) // $2 - .fetch_all(&mut self.inner) - .await - .map_err(|e| Error::SqlxError { source: e }) + sqlx::query_as::<_, Tombstone>( + r#" +SELECT * +FROM tombstone +WHERE sequencer_id = $1 + AND sequence_number > $2 +ORDER BY id; + "#, + ) + .bind(&sequencer_id) // $1 + .bind(&sequence_number) // $2 + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e }) } } @@ -1176,37 +1228,38 @@ impl ParquetFileRepo for PostgresTxn { let rec = sqlx::query_as::<_, ParquetFile>( r#" -INSERT INTO parquet_file ( sequencer_id, table_id, partition_id, object_store_id, min_sequence_number, max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata, row_count, compaction_level, created_at ) +INSERT INTO parquet_file ( + sequencer_id, table_id, partition_id, object_store_id, min_sequence_number, + max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata, + row_count, compaction_level, created_at ) VALUES ( $1, $2, $3, $4, $5, $6, $7, $8, false, $9, $10, $11, $12, $13 ) -RETURNING * +RETURNING *; "#, ) - .bind(sequencer_id) // $1 - .bind(table_id) // $2 - .bind(partition_id) // $3 - .bind(object_store_id) // $4 - .bind(min_sequence_number) // $5 - .bind(max_sequence_number) // $6 - .bind(min_time) // $7 - .bind(max_time) // $8 - .bind(file_size_bytes) // $9 - .bind(parquet_metadata) // $10 - .bind(row_count) // $11 - .bind(INITIAL_COMPACTION_LEVEL) // $12 - .bind(created_at) // $13 - .fetch_one(&mut self.inner) - .await - .map_err(|e| { - if is_unique_violation(&e) { - Error::FileExists { - object_store_id, - } - } else if is_fk_violation(&e) { - Error::ForeignKeyViolation { source: e } - } else { - Error::SqlxError { source: e } - } - })?; + .bind(sequencer_id) // $1 + .bind(table_id) // $2 + .bind(partition_id) // $3 + .bind(object_store_id) // $4 + .bind(min_sequence_number) // $5 + .bind(max_sequence_number) // $6 + .bind(min_time) // $7 + .bind(max_time) // $8 + .bind(file_size_bytes) // $9 + .bind(parquet_metadata) // $10 + .bind(row_count) // $11 + .bind(INITIAL_COMPACTION_LEVEL) // $12 + .bind(created_at) // $13 + .fetch_one(&mut self.inner) + .await + .map_err(|e| { + if is_unique_violation(&e) { + Error::FileExists { object_store_id } + } else if is_fk_violation(&e) { + Error::ForeignKeyViolation { source: e } + } else { + Error::SqlxError { source: e } + } + })?; Ok(rec) } @@ -1226,12 +1279,20 @@ RETURNING * sequencer_id: SequencerId, sequence_number: SequenceNumber, ) -> Result> { - sqlx::query_as::<_, ParquetFile>(r#"SELECT * FROM parquet_file WHERE sequencer_id = $1 AND max_sequence_number > $2 ORDER BY id;"#) - .bind(&sequencer_id) // $1 - .bind(&sequence_number) // $2 - .fetch_all(&mut self.inner) - .await - .map_err(|e| Error::SqlxError { source: e }) + sqlx::query_as::<_, ParquetFile>( + r#" +SELECT * +FROM parquet_file +WHERE sequencer_id = $1 + AND max_sequence_number > $2 +ORDER BY id; + "#, + ) + .bind(&sequencer_id) // $1 + .bind(&sequence_number) // $2 + .fetch_all(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e }) } async fn list_by_namespace_not_to_delete( @@ -1240,25 +1301,11 @@ RETURNING * ) -> Result> { sqlx::query_as::<_, ParquetFile>( r#" -SELECT - parquet_file.id as id, - parquet_file.sequencer_id as sequencer_id, - parquet_file.table_id as table_id, - parquet_file.partition_id as partition_id, - parquet_file.object_store_id as object_store_id, - parquet_file.min_sequence_number as min_sequence_number, - parquet_file.max_sequence_number as max_sequence_number, - parquet_file.min_time as min_time, - parquet_file.max_time as max_time, - parquet_file.to_delete as to_delete, - parquet_file.file_size_bytes as file_size_bytes, - parquet_file.parquet_metadata as parquet_metadata, - parquet_file.row_count as row_count, - parquet_file.compaction_level as compaction_level, - parquet_file.created_at as created_at +SELECT parquet_file.* FROM parquet_file INNER JOIN table_name on table_name.id = parquet_file.table_id -WHERE table_name.namespace_id = $1 AND parquet_file.to_delete = false; +WHERE table_name.namespace_id = $1 + AND parquet_file.to_delete = false; "#, ) .bind(&namespace_id) // $1 @@ -1270,27 +1317,12 @@ WHERE table_name.namespace_id = $1 AND parquet_file.to_delete = false; async fn level_0(&mut self, sequencer_id: SequencerId) -> Result> { sqlx::query_as::<_, ParquetFile>( r#" -SELECT - parquet_file.id as id, - parquet_file.sequencer_id as sequencer_id, - parquet_file.table_id as table_id, - parquet_file.partition_id as partition_id, - parquet_file.object_store_id as object_store_id, - parquet_file.min_sequence_number as min_sequence_number, - parquet_file.max_sequence_number as max_sequence_number, - parquet_file.min_time as min_time, - parquet_file.max_time as max_time, - parquet_file.to_delete as to_delete, - parquet_file.file_size_bytes as file_size_bytes, - parquet_file.parquet_metadata as parquet_metadata, - parquet_file.row_count as row_count, - parquet_file.compaction_level as compaction_level, - parquet_file.created_at as created_at +SELECT * FROM parquet_file WHERE parquet_file.sequencer_id = $1 AND parquet_file.compaction_level = 0 AND parquet_file.to_delete = false; - "#, + "#, ) .bind(&sequencer_id) // $1 .fetch_all(&mut self.inner) @@ -1306,22 +1338,7 @@ WHERE parquet_file.sequencer_id = $1 ) -> Result> { sqlx::query_as::<_, ParquetFile>( r#" -SELECT - parquet_file.id as id, - parquet_file.sequencer_id as sequencer_id, - parquet_file.table_id as table_id, - parquet_file.partition_id as partition_id, - parquet_file.object_store_id as object_store_id, - parquet_file.min_sequence_number as min_sequence_number, - parquet_file.max_sequence_number as max_sequence_number, - parquet_file.min_time as min_time, - parquet_file.max_time as max_time, - parquet_file.to_delete as to_delete, - parquet_file.file_size_bytes as file_size_bytes, - parquet_file.parquet_metadata as parquet_metadata, - parquet_file.row_count as row_count, - parquet_file.compaction_level as compaction_level, - parquet_file.created_at as created_at +SELECT * FROM parquet_file WHERE parquet_file.sequencer_id = $1 AND parquet_file.table_id = $2 @@ -1329,8 +1346,8 @@ WHERE parquet_file.sequencer_id = $1 AND parquet_file.compaction_level = 1 AND parquet_file.to_delete = false AND ((parquet_file.min_time <= $5 AND parquet_file.max_time >= $4) - OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $5)) - ;"#, + OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $5)); + "#, ) .bind(&table_partition.sequencer_id) // $1 .bind(&table_partition.table_id) // $2 @@ -1351,11 +1368,11 @@ WHERE parquet_file.sequencer_id = $1 let ids: Vec<_> = parquet_file_ids.iter().map(|p| p.get()).collect(); let updated = sqlx::query( r#" - UPDATE parquet_file - SET compaction_level = 1 - WHERE id = ANY($1) - RETURNING id - ;"#, +UPDATE parquet_file +SET compaction_level = 1 +WHERE id = ANY($1) +RETURNING id; + "#, ) .bind(&ids[..]) .fetch_all(&mut self.inner) @@ -1380,7 +1397,7 @@ WHERE parquet_file.sequencer_id = $1 async fn count(&mut self) -> Result { let read_result = - sqlx::query_as::<_, Count>(r#"SELECT count(*) as count FROM parquet_file;"#) + sqlx::query_as::<_, Count>(r#"SELECT count(*) as count FROM parquet_file;"#) .fetch_one(&mut self.inner) .await .map_err(|e| Error::SqlxError { source: e })?; @@ -1398,10 +1415,10 @@ impl ProcessedTombstoneRepo for PostgresTxn { ) -> Result { sqlx::query_as::<_, ProcessedTombstone>( r#" - INSERT INTO processed_tombstone ( tombstone_id, parquet_file_id ) - VALUES ( $1, $2 ) - RETURNING * - "#, +INSERT INTO processed_tombstone ( tombstone_id, parquet_file_id ) +VALUES ( $1, $2 ) +RETURNING *; + "#, ) .bind(tombstone_id) // $1 .bind(parquet_file_id) // $2 @@ -1427,12 +1444,18 @@ impl ProcessedTombstoneRepo for PostgresTxn { tombstone_id: TombstoneId, ) -> Result { let read_result = sqlx::query_as::<_, Count>( - r#"SELECT count(*) as count FROM processed_tombstone WHERE parquet_file_id = $1 AND tombstone_id = $2;"#) - .bind(&parquet_file_id) // $1 - .bind(&tombstone_id) // $2 - .fetch_one(&mut self.inner) - .await - .map_err(|e| Error::SqlxError { source: e })?; + r#" +SELECT count(*) as count +FROM processed_tombstone +WHERE parquet_file_id = $1 + AND tombstone_id = $2; + "#, + ) + .bind(&parquet_file_id) // $1 + .bind(&tombstone_id) // $2 + .fetch_one(&mut self.inner) + .await + .map_err(|e| Error::SqlxError { source: e })?; Ok(read_result.count > 0) } @@ -1991,9 +2014,9 @@ mod tests { paste::paste! { #[tokio::test] async fn []() { - // If running an integration test on your laptop, this requires that you have Postgres - // running and that you've done the sqlx migrations. See the README in this crate for - // info to set it up. + // If running an integration test on your laptop, this requires that you have + // Postgres running and that you've done the sqlx migrations. See the README in + // this crate for info to set it up. maybe_skip_integration!(); let postgres = setup_db().await; From 632c4953b42a487832355510715fca50ec344066 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 14 Mar 2022 09:35:38 +0000 Subject: [PATCH 10/22] feat: add addressable heap for query cache (#4016) * feat: add addressable heap for query cache This will be used as a helper data structure for TTL and LRU. It's probably not the most performant implementation but it's good enough for now. This is for #3985. * fix: test + explain tie breaking in `AddressableHeap` --- .../cache_system/backend/addressable_heap.rs | 386 ++++++++++++++++++ querier/src/cache_system/backend/mod.rs | 1 + 2 files changed, 387 insertions(+) create mode 100644 querier/src/cache_system/backend/addressable_heap.rs diff --git a/querier/src/cache_system/backend/addressable_heap.rs b/querier/src/cache_system/backend/addressable_heap.rs new file mode 100644 index 0000000000..fe13a99162 --- /dev/null +++ b/querier/src/cache_system/backend/addressable_heap.rs @@ -0,0 +1,386 @@ +use std::{ + collections::{HashMap, VecDeque}, + hash::Hash, +}; + +/// Addressable heap. +/// +/// Stores a value `V` together with a key `K` and an order `O`. Elements are sorted by `O` and the smallest element can +/// be peeked/popped. At the same time elements can be addressed via `K`. +/// +/// Note that `K` requires [`Ord`] to implement the inner data structure as a tie breaker. +/// structure. +#[derive(Debug, Clone)] +pub struct AddressableHeap +where + K: Clone + Eq + Hash + Ord, + O: Clone + Ord, +{ + /// Key to order and value. + /// + /// The order is required to lookup data within the queue. + /// + /// The value is stored here instead of the queue since HashMap entries are copied around less often than queue elements. + key_to_order_and_value: HashMap, + + /// Queue that handles the priorities. + /// + /// The order goes first, the key goes second. + /// + /// Note: This is not really a heap, but it fullfills the interface that we need. + queue: VecDeque<(O, K)>, +} + +impl AddressableHeap +where + K: Clone + Eq + Hash + Ord, + O: Clone + Ord, +{ + /// Create new, empty heap. + pub fn new() -> Self { + Self { + key_to_order_and_value: HashMap::new(), + queue: VecDeque::new(), + } + } + + /// Insert element. + /// + /// If the element (compared by `K`) already exists, it will be returned. + pub fn insert(&mut self, k: K, v: V, o: O) -> Option<(V, O)> { + // always remove the entry first so we have a clean queue + let result = self.remove(&k); + + assert!( + self.key_to_order_and_value + .insert(k.clone(), (v, o.clone())) + .is_none(), + "entry should have been removed by now" + ); + + match self.queue.binary_search_by_key(&(&o, &k), project_tuple) { + Ok(_) => unreachable!("entry should have been removed by now"), + Err(index) => { + self.queue.insert(index, (o, k)); + } + } + + result + } + + /// Peek first element (by smallest `O`). + pub fn peek(&self) -> Option<(&K, &V, &O)> { + if let Some((o, k)) = self.queue.front() { + let (v, o2) = self + .key_to_order_and_value + .get(k) + .expect("value is in queue"); + assert!(o == o2); + Some((k, v, o)) + } else { + None + } + } + + /// Pop first element (by smallest `O`) from heap. + pub fn pop(&mut self) -> Option<(K, V, O)> { + if let Some((o, k)) = self.queue.pop_front() { + let (v, o2) = self + .key_to_order_and_value + .remove(&k) + .expect("value is in queue"); + assert!(o == o2); + Some((k, v, o)) + } else { + None + } + } + + /// Get element by key. + pub fn get(&self, k: &K) -> Option<(&V, &O)> { + self.key_to_order_and_value.get(k).map(project_tuple) + } + + /// Remove element by key. + /// + /// If the element exists within the heap (addressed via `K`), the value and order will be returned. + pub fn remove(&mut self, k: &K) -> Option<(V, O)> { + if let Some((v, o)) = self.key_to_order_and_value.remove(k) { + let index = self + .queue + .binary_search_by_key(&(&o, k), project_tuple) + .expect("key was in key_to_order"); + self.queue.remove(index); + Some((v, o)) + } else { + None + } + } +} + +impl Default for AddressableHeap +where + K: Clone + Eq + Hash + Ord, + O: Clone + Ord, +{ + fn default() -> Self { + Self::new() + } +} + +/// Project tuple references. +fn project_tuple(t: &(A, B)) -> (&A, &B) { + (&t.0, &t.1) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_peek_empty() { + let heap = AddressableHeap::::new(); + + assert_eq!(heap.peek(), None); + } + + #[test] + fn test_peek_some() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(3, "c", 5); + + assert_eq!(heap.peek(), Some((&2, &"b", &3))); + } + + #[test] + fn test_peek_tie() { + let mut heap = AddressableHeap::new(); + + heap.insert(3, "a", 1); + heap.insert(1, "b", 1); + heap.insert(2, "c", 1); + + assert_eq!(heap.peek(), Some((&1, &"b", &1))); + } + + #[test] + fn test_peek_after_remove() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(3, "c", 5); + + assert_eq!(heap.peek(), Some((&2, &"b", &3))); + heap.remove(&3); + assert_eq!(heap.peek(), Some((&2, &"b", &3))); + heap.remove(&2); + assert_eq!(heap.peek(), Some((&1, &"a", &4))); + heap.remove(&1); + assert_eq!(heap.peek(), None); + } + + #[test] + fn test_peek_after_override() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(1, "c", 2); + + assert_eq!(heap.peek(), Some((&1, &"c", &2))); + } + + #[test] + fn test_pop_empty() { + let mut heap = AddressableHeap::::new(); + + assert_eq!(heap.pop(), None); + } + + #[test] + fn test_pop_all() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(3, "c", 5); + + assert_eq!(heap.pop(), Some((2, "b", 3))); + assert_eq!(heap.pop(), Some((1, "a", 4))); + assert_eq!(heap.pop(), Some((3, "c", 5))); + assert_eq!(heap.pop(), None); + } + + #[test] + fn test_pop_tie() { + let mut heap = AddressableHeap::new(); + + heap.insert(3, "a", 1); + heap.insert(1, "b", 1); + heap.insert(2, "c", 1); + + assert_eq!(heap.pop(), Some((1, "b", 1))); + assert_eq!(heap.pop(), Some((2, "c", 1))); + assert_eq!(heap.pop(), Some((3, "a", 1))); + assert_eq!(heap.pop(), None); + } + + #[test] + fn test_pop_after_insert() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(3, "c", 5); + + assert_eq!(heap.pop(), Some((2, "b", 3))); + + heap.insert(4, "d", 2); + assert_eq!(heap.pop(), Some((4, "d", 2))); + assert_eq!(heap.pop(), Some((1, "a", 4))); + } + + #[test] + fn test_pop_after_remove() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(3, "c", 5); + + heap.remove(&2); + assert_eq!(heap.pop(), Some((1, "a", 4))); + } + + #[test] + fn test_pop_after_override() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + heap.insert(1, "c", 2); + + assert_eq!(heap.pop(), Some((1, "c", 2))); + assert_eq!(heap.pop(), Some((2, "b", 3))); + assert_eq!(heap.pop(), None); + } + + #[test] + fn test_get_empty() { + let heap = AddressableHeap::::new(); + + assert_eq!(heap.get(&1), None); + } + + #[test] + fn test_get_multiple() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + + assert_eq!(heap.get(&1), Some((&"a", &4))); + assert_eq!(heap.get(&2), Some((&"b", &3))); + } + + #[test] + fn test_get_after_remove() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + + heap.remove(&1); + + assert_eq!(heap.get(&1), None); + assert_eq!(heap.get(&2), Some((&"b", &3))); + } + + #[test] + fn test_get_after_pop() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + + heap.pop(); + + assert_eq!(heap.get(&1), Some((&"a", &4))); + assert_eq!(heap.get(&2), None); + } + + #[test] + fn test_get_after_override() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(1, "b", 3); + + assert_eq!(heap.get(&1), Some((&"b", &3))); + } + + #[test] + fn test_remove_empty() { + let mut heap = AddressableHeap::::new(); + + assert_eq!(heap.remove(&1), None); + } + + #[test] + fn test_remove_some() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + + assert_eq!(heap.remove(&1), Some(("a", 4))); + assert_eq!(heap.remove(&2), Some(("b", 3))); + } + + #[test] + fn test_remove_twice() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + + assert_eq!(heap.remove(&1), Some(("a", 4))); + assert_eq!(heap.remove(&1), None); + } + + #[test] + fn test_remove_after_pop() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(2, "b", 3); + + heap.pop(); + + assert_eq!(heap.remove(&1), Some(("a", 4))); + assert_eq!(heap.remove(&2), None); + } + + #[test] + fn test_remove_after_override() { + let mut heap = AddressableHeap::new(); + + heap.insert(1, "a", 4); + heap.insert(1, "b", 3); + + assert_eq!(heap.remove(&1), Some(("b", 3))); + assert_eq!(heap.remove(&1), None); + } + + #[test] + fn test_override() { + let mut heap = AddressableHeap::new(); + + assert_eq!(heap.insert(1, "a", 4), None); + assert_eq!(heap.insert(2, "b", 3), None); + assert_eq!(heap.insert(1, "c", 5), Some(("a", 4))); + } +} diff --git a/querier/src/cache_system/backend/mod.rs b/querier/src/cache_system/backend/mod.rs index 6a4ab33c9b..8910622a23 100644 --- a/querier/src/cache_system/backend/mod.rs +++ b/querier/src/cache_system/backend/mod.rs @@ -1,5 +1,6 @@ use std::{fmt::Debug, hash::Hash}; +mod addressable_heap; pub mod hash_map; #[cfg(test)] From 3f0f090c4ea81244c0edd81ab4cb5928fd9a77ab Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 14 Mar 2022 10:33:27 +0000 Subject: [PATCH 11/22] chore(deps): Bump pretty_assertions from 1.1.0 to 1.2.0 (#4024) Bumps [pretty_assertions](https://github.com/colin-kiegel/rust-pretty-assertions) from 1.1.0 to 1.2.0. - [Release notes](https://github.com/colin-kiegel/rust-pretty-assertions/releases) - [Changelog](https://github.com/colin-kiegel/rust-pretty-assertions/blob/main/CHANGELOG.md) - [Commits](https://github.com/colin-kiegel/rust-pretty-assertions/commits) --- updated-dependencies: - dependency-name: pretty_assertions dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- iox_catalog/Cargo.toml | 2 +- router2/Cargo.toml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 205eeb46f7..a46768ff52 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4021,9 +4021,9 @@ dependencies = [ [[package]] name = "pretty_assertions" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76d5b548b725018ab5496482b45cb8bef21e9fed1858a6d674e3a8a0f0bb5d50" +checksum = "57c038cb5319b9c704bf9c227c261d275bfec0ad438118a2787ce47944fb228b" dependencies = [ "ansi_term", "ctor", diff --git a/iox_catalog/Cargo.toml b/iox_catalog/Cargo.toml index f1b34dbf86..84f2f2c93d 100644 --- a/iox_catalog/Cargo.toml +++ b/iox_catalog/Cargo.toml @@ -24,7 +24,7 @@ workspace-hack = { path = "../workspace-hack"} dotenv = "0.15.0" mutable_batch_lp = { path = "../mutable_batch_lp" } paste = "1.0.6" -pretty_assertions = "1.0.0" +pretty_assertions = "1.2.0" rand = "0.8" tempfile = "3" test_helpers = { path = "../test_helpers" } diff --git a/router2/Cargo.toml b/router2/Cargo.toml index f592d28c92..342777bd66 100644 --- a/router2/Cargo.toml +++ b/router2/Cargo.toml @@ -41,7 +41,7 @@ assert_matches = "1.5" criterion = { version = "0.3.4", features = ["async_tokio", "html_reports"] } lazy_static = "1.4.0" paste = "1.0.6" -pretty_assertions = "1.1.0" +pretty_assertions = "1.2.0" rand = "0.8.3" schema = { path = "../schema" } From 27efb66237d5d2434ab4ce85a1ab923958992398 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 14 Mar 2022 12:38:30 +0000 Subject: [PATCH 12/22] test: add proptest for `AddressableHeap` (#4025) * test: add proptest for `AddressableHeap` For #3985. * refactor: simplify code Co-authored-by: Edd Robinson Co-authored-by: Edd Robinson --- Cargo.lock | 27 +++++ querier/Cargo.toml | 1 + .../cache_system/backend/addressable_heap.rs | 113 ++++++++++++++++++ 3 files changed, 141 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index a46768ff52..ee2bb42843 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4078,6 +4078,23 @@ dependencies = [ "thiserror", ] +[[package]] +name = "proptest" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e0d9cc07f18492d879586c92b485def06bc850da3118075cd45d50e9c95b0e5" +dependencies = [ + "bitflags", + "byteorder", + "lazy_static", + "num-traits", + "quick-error", + "rand", + "rand_chacha", + "rand_xorshift", + "regex-syntax", +] + [[package]] name = "prost" version = "0.9.0" @@ -4160,6 +4177,7 @@ dependencies = [ "parquet_file", "pin-project", "predicate", + "proptest", "prost", "query", "rand", @@ -4312,6 +4330,15 @@ dependencies = [ "rand_core", ] +[[package]] +name = "rand_xorshift" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f" +dependencies = [ + "rand_core", +] + [[package]] name = "rayon" version = "1.5.1" diff --git a/querier/Cargo.toml b/querier/Cargo.toml index 4acf34626a..d7d978cee3 100644 --- a/querier/Cargo.toml +++ b/querier/Cargo.toml @@ -44,3 +44,4 @@ arrow_util = { path = "../arrow_util" } bytes = "1.0" iox_tests = { path = "../iox_tests" } mutable_batch_lp = { path = "../mutable_batch_lp" } +proptest = { version = "1", default_features = false, features = ["std"] } diff --git a/querier/src/cache_system/backend/addressable_heap.rs b/querier/src/cache_system/backend/addressable_heap.rs index fe13a99162..bb12a31efc 100644 --- a/querier/src/cache_system/backend/addressable_heap.rs +++ b/querier/src/cache_system/backend/addressable_heap.rs @@ -135,6 +135,8 @@ fn project_tuple(t: &(A, B)) -> (&A, &B) { #[cfg(test)] mod tests { + use proptest::prelude::*; + use super::*; #[test] @@ -383,4 +385,115 @@ mod tests { assert_eq!(heap.insert(2, "b", 3), None); assert_eq!(heap.insert(1, "c", 5), Some(("a", 4))); } + + /// Simple version of [`AddressableHeap`] for testing. + struct SimpleAddressableHeap { + inner: Vec<(u8, String, i8)>, + } + + impl SimpleAddressableHeap { + fn new() -> Self { + Self { inner: Vec::new() } + } + + fn insert(&mut self, k: u8, v: String, o: i8) -> Option<(String, i8)> { + let res = self.remove(&k); + self.inner.push((k, v, o)); + + res + } + + fn peek(&self) -> Option<(&u8, &String, &i8)> { + self.inner + .iter() + .min_by_key(|(k, _v, o)| (o, k)) + .map(|(k, v, o)| (k, v, o)) + } + + fn pop(&mut self) -> Option<(u8, String, i8)> { + self.inner + .iter() + .enumerate() + .min_by_key(|(_idx, (k, _v, o))| (o, k)) + .map(|(idx, _)| idx) + .map(|idx| self.inner.remove(idx)) + } + + fn get(&self, k: &u8) -> Option<(&String, &i8)> { + self.inner + .iter() + .find(|(k2, _v, _o)| k2 == k) + .map(|(_k, v, o)| (v, o)) + } + + fn remove(&mut self, k: &u8) -> Option<(String, i8)> { + self.inner + .iter() + .enumerate() + .find(|(_idx, (k2, _v, _o))| k2 == k) + .map(|(idx, _)| idx) + .map(|idx| { + let (_k, v, o) = self.inner.remove(idx); + (v, o) + }) + } + } + + #[derive(Debug, Clone)] + enum Action { + Insert { k: u8, v: String, o: i8 }, + Peek, + Pop, + Get { k: u8 }, + Remove { k: u8 }, + } + + // Use a hand-rolled strategy instead of `proptest-derive`, because the latter one is quite a heavy dependency. + fn action() -> impl Strategy { + prop_oneof![ + (any::(), ".*", any::()).prop_map(|(k, v, o)| Action::Insert { k, v, o }), + Just(Action::Peek), + Just(Action::Pop), + any::().prop_map(|k| Action::Get { k }), + any::().prop_map(|k| Action::Remove { k }), + ] + } + + proptest! { + #[test] + fn test_proptest(actions in prop::collection::vec(action(), 0..100)) { + let mut heap = AddressableHeap::new(); + let mut sim = SimpleAddressableHeap::new(); + + for action in actions { + match action { + Action::Insert{k, v, o} => { + let res1 = heap.insert(k, v.clone(), o); + let res2 = sim.insert(k, v, o); + assert_eq!(res1, res2); + } + Action::Peek => { + let res1 = heap.peek(); + let res2 = sim.peek(); + assert_eq!(res1, res2); + } + Action::Pop => { + let res1 = heap.pop(); + let res2 = sim.pop(); + assert_eq!(res1, res2); + } + Action::Get{k} => { + let res1 = heap.get(&k); + let res2 = sim.get(&k); + assert_eq!(res1, res2); + } + Action::Remove{k} => { + let res1 = heap.remove(&k); + let res2 = sim.remove(&k); + assert_eq!(res1, res2); + } + } + } + } + } } From 4c674b931aa14ef9afdeb4f3a237f04e3eef6518 Mon Sep 17 00:00:00 2001 From: Marko Mikulicic Date: Mon, 14 Mar 2022 15:12:34 +0100 Subject: [PATCH 13/22] fix: Remove partition_id from metric attributes (#4028) --- ingester/src/job.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/ingester/src/job.rs b/ingester/src/job.rs index 34628d057a..0d063cb94c 100644 --- a/ingester/src/job.rs +++ b/ingester/src/job.rs @@ -65,11 +65,5 @@ impl JobRegistry { } fn f_attributes(job: &Job) -> metric::Attributes { - let mut attributes = metric::Attributes::from(&[("name", job.name())]); - - if let Some(partition_id) = job.partition_id() { - attributes.insert("partition_id", partition_id.get().to_string()); - } - - attributes + metric::Attributes::from(&[("name", job.name())]) } From 6fd3c3f1ea539823e31264c91a609362d934d063 Mon Sep 17 00:00:00 2001 From: Marko Mikulicic Date: Mon, 14 Mar 2022 16:24:54 +0100 Subject: [PATCH 14/22] fix: Kick c2updater (#4029) --- .circleci/get-deploy-tags.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/.circleci/get-deploy-tags.sh b/.circleci/get-deploy-tags.sh index d96543e5e2..f72fa4f96d 100755 --- a/.circleci/get-deploy-tags.sh +++ b/.circleci/get-deploy-tags.sh @@ -16,6 +16,7 @@ # Will generate the deployment JSON file and output it to stdout. If # IMAGE_PROMOTION_COMMAND is set in the environment, the deployment JSON file is # piped to it at the end of execution. +# set -euo pipefail From 87e53f30d12177423f6e50c4ef7178f55b1348ef Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Mon, 14 Mar 2022 15:35:27 +0000 Subject: [PATCH 15/22] refactor: add TTL cache backend (#4027) * feat: `CacheBackend::as_any` * refactor: add TTL cache backend This is based on the new `AddressableHeap`, which simplifies the implementation quite a lot. For #3985. * refactor: `TtlBackend::{update->evict_expired}` * docs: exlain ttl cache eviction Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- querier/src/cache_system/backend/hash_map.rs | 5 + querier/src/cache_system/backend/mod.rs | 6 +- querier/src/cache_system/backend/ttl.rs | 450 +++++++++++++++++++ 3 files changed, 460 insertions(+), 1 deletion(-) create mode 100644 querier/src/cache_system/backend/ttl.rs diff --git a/querier/src/cache_system/backend/hash_map.rs b/querier/src/cache_system/backend/hash_map.rs index a26270ed9d..5b595dd2bd 100644 --- a/querier/src/cache_system/backend/hash_map.rs +++ b/querier/src/cache_system/backend/hash_map.rs @@ -1,4 +1,5 @@ use std::{ + any::Any, collections::HashMap, fmt::Debug, hash::{BuildHasher, Hash}, @@ -26,6 +27,10 @@ where fn remove(&mut self, k: &Self::K) { self.remove(k); } + + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } } #[cfg(test)] diff --git a/querier/src/cache_system/backend/mod.rs b/querier/src/cache_system/backend/mod.rs index 8910622a23..3196df5698 100644 --- a/querier/src/cache_system/backend/mod.rs +++ b/querier/src/cache_system/backend/mod.rs @@ -1,7 +1,8 @@ -use std::{fmt::Debug, hash::Hash}; +use std::{any::Any, fmt::Debug, hash::Hash}; mod addressable_heap; pub mod hash_map; +pub mod ttl; #[cfg(test)] mod test_util; @@ -28,4 +29,7 @@ pub trait CacheBackend: Debug + Send + 'static { /// /// It is OK to remove a key even when it does not exist. fn remove(&mut self, k: &Self::K); + + /// Return backend as [`Any`] which can be used to downcast to a specifc implementation. + fn as_any(&self) -> &dyn Any; } diff --git a/querier/src/cache_system/backend/ttl.rs b/querier/src/cache_system/backend/ttl.rs new file mode 100644 index 0000000000..8c8a31852d --- /dev/null +++ b/querier/src/cache_system/backend/ttl.rs @@ -0,0 +1,450 @@ +use std::{any::Any, fmt::Debug, hash::Hash, marker::PhantomData, sync::Arc, time::Duration}; + +use time::{Time, TimeProvider}; + +use super::{addressable_heap::AddressableHeap, CacheBackend}; + +/// Interface to provide TTL (time to live) data for a key-value pair. +pub trait TtlProvider: std::fmt::Debug + Send + Sync + 'static { + /// Cache key. + type K; + + /// Cached value. + type V; + + /// When should the given key-value pair expire? + /// + /// Return `None` for "never". + /// + /// The function is only called once for a newly cached key-value pair. This means: + /// - There is no need in remembering the time of a given pair (e.g. you can safely always return a constant). + /// - You cannot change the TTL after the data was cached. + /// + /// Expiration is set to take place AT OR AFTER the provided duration. + fn expires_in(&self, k: &Self::K, v: &Self::V) -> Option; +} + +/// [`TtlProvider`] that never expires. +#[derive(Default)] +pub struct NeverTtlProvider +where + K: 'static, + V: 'static, +{ + // phantom data that is Send and Sync, see https://stackoverflow.com/a/50201389 + _k: PhantomData K>, + _v: PhantomData V>, +} + +impl std::fmt::Debug for NeverTtlProvider { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("NeverTtlProvider").finish_non_exhaustive() + } +} + +impl TtlProvider for NeverTtlProvider { + type K = K; + type V = V; + + fn expires_in(&self, _k: &Self::K, _v: &Self::V) -> Option { + None + } +} + +/// Cache backend that implements Time To Life. +/// +/// # Cache Eviction +/// Every method ([`get`](CacheBackend::get), [`set`](CacheBackend::set), [`remove`](CacheBackend::remove)) causes the +/// cache to check for expired keys. This may lead to certain delays, esp. when dropping the contained values takes a +/// long time. +#[derive(Debug)] +pub struct TtlBackend +where + K: Clone + Eq + Debug + Hash + Ord + Send + 'static, + V: Clone + Debug + Send + 'static, +{ + inner_backend: Box>, + ttl_provider: Arc>, + time_provider: Arc, + expiration: AddressableHeap, +} + +impl TtlBackend +where + K: Clone + Eq + Debug + Hash + Ord + Send + 'static, + V: Clone + Debug + Send + 'static, +{ + /// Create new backend w/o any known keys. + /// + /// The inner backend SHOULD NOT contain any data at this point, otherwise we will not track any TTLs for these entries. + pub fn new( + inner_backend: Box>, + ttl_provider: Arc>, + time_provider: Arc, + ) -> Self { + Self { + inner_backend, + ttl_provider, + time_provider, + expiration: Default::default(), + } + } + + fn evict_expired(&mut self, now: Time) { + while self + .expiration + .peek() + .map(|(_k, _, t)| *t <= now) + .unwrap_or_default() + { + let (k, _, _t) = self.expiration.pop().unwrap(); + self.inner_backend.remove(&k); + } + } + + /// Reference to inner backend. + pub fn inner_backend(&self) -> &dyn CacheBackend { + self.inner_backend.as_ref() + } + + /// Reference to TTL provider. + pub fn ttl_provider(&self) -> &Arc> { + &self.ttl_provider + } +} + +impl CacheBackend for TtlBackend +where + K: Clone + Eq + Debug + Hash + Ord + Send + 'static, + V: Clone + Debug + Send + 'static, +{ + type K = K; + type V = V; + + fn get(&mut self, k: &Self::K) -> Option { + self.evict_expired(self.time_provider.now()); + + self.inner_backend.get(k) + } + + fn set(&mut self, k: Self::K, v: Self::V) { + let now = self.time_provider.now(); + self.evict_expired(now); + + if let Some(ttl) = self + .ttl_provider + .expires_in(&k, &v) + .and_then(|d| now.checked_add(d)) + { + self.expiration.insert(k.clone(), (), ttl); + } else { + // Still need to ensure that any current expiration is disabled + self.expiration.remove(&k); + } + + self.inner_backend.set(k, v); + } + + fn remove(&mut self, k: &Self::K) { + self.evict_expired(self.time_provider.now()); + + self.inner_backend.remove(k); + self.expiration.remove(k); + } + + fn as_any(&self) -> &dyn Any { + self as &dyn Any + } +} + +#[cfg(test)] +mod tests { + use std::{collections::HashMap, time::Duration}; + + use parking_lot::Mutex; + use time::MockProvider; + + use super::*; + + #[test] + fn test_expires_single() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + time_provider.inc(Duration::from_secs(1)); + assert_eq!(backend.get(&1), None); + } + + #[test] + fn test_overflow_expire() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + + // init time provider at MAX! + let time_provider = Arc::new(MockProvider::new(Time::MAX)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::MAX)); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_never_expire() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), None); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + time_provider.inc(Duration::from_secs(1)); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_expiration_uses_key_and_value() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + ttl_provider.set_expires_in(1, String::from("b"), Some(Duration::from_secs(4))); + ttl_provider.set_expires_in(2, String::from("a"), Some(Duration::from_secs(2))); + backend.set(1, String::from("b")); + + time_provider.inc(Duration::from_secs(3)); + assert_eq!(backend.get(&1), Some(String::from("b"))); + } + + #[test] + fn test_override_with_different_expiration() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(3))); + backend.set(1, String::from("a")); + + time_provider.inc(Duration::from_secs(2)); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_override_with_no_expiration() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + ttl_provider.set_expires_in(1, String::from("a"), None); + backend.set(1, String::from("a")); + + time_provider.inc(Duration::from_secs(2)); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_readd_with_different_expiration() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(3))); + backend.remove(&1); + backend.set(1, String::from("a")); + + time_provider.inc(Duration::from_secs(2)); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_readd_with_no_expiration() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + ttl_provider.set_expires_in(1, String::from("a"), None); + backend.remove(&1); + backend.set(1, String::from("a")); + + time_provider.inc(Duration::from_secs(2)); + assert_eq!(backend.get(&1), Some(String::from("a"))); + } + + #[test] + fn test_update_cleans_multiple_keys() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + ttl_provider.set_expires_in(2, String::from("b"), Some(Duration::from_secs(2))); + ttl_provider.set_expires_in(3, String::from("c"), Some(Duration::from_secs(2))); + ttl_provider.set_expires_in(4, String::from("d"), Some(Duration::from_secs(3))); + backend.set(1, String::from("a")); + backend.set(2, String::from("b")); + backend.set(3, String::from("c")); + backend.set(4, String::from("d")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + assert_eq!(backend.get(&2), Some(String::from("b"))); + assert_eq!(backend.get(&3), Some(String::from("c"))); + assert_eq!(backend.get(&4), Some(String::from("d"))); + + time_provider.inc(Duration::from_secs(2)); + assert_eq!(backend.get(&1), None); + let inner_backend = backend + .inner_backend() + .as_any() + .downcast_ref::>() + .unwrap(); + assert!(!inner_backend.contains_key(&1)); + assert!(!inner_backend.contains_key(&2)); + assert!(!inner_backend.contains_key(&3)); + assert!(inner_backend.contains_key(&4)); + assert_eq!(backend.get(&2), None); + assert_eq!(backend.get(&3), None); + assert_eq!(backend.get(&4), Some(String::from("d"))); + } + + #[test] + fn test_remove_expired_key() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + backend.set(1, String::from("a")); + assert_eq!(backend.get(&1), Some(String::from("a"))); + + time_provider.inc(Duration::from_secs(1)); + backend.remove(&1); + assert_eq!(backend.get(&1), None); + } + + #[test] + fn test_expire_removed_key() { + let ttl_provider = Arc::new(TestTtlProvider::new()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + let mut backend = TtlBackend::new( + Box::new(HashMap::::new()), + Arc::clone(&ttl_provider) as _, + Arc::clone(&time_provider) as _, + ); + + ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1))); + ttl_provider.set_expires_in(2, String::from("b"), Some(Duration::from_secs(2))); + backend.set(1, String::from("a")); + backend.remove(&1); + + time_provider.inc(Duration::from_secs(1)); + backend.set(2, String::from("b")); + assert_eq!(backend.get(&1), None); + assert_eq!(backend.get(&2), Some(String::from("b"))); + } + + #[test] + fn test_generic() { + use crate::cache_system::backend::test_util::test_generic; + + test_generic(|| { + let ttl_provider = Arc::new(NeverTtlProvider::default()); + let time_provider = Arc::new(MockProvider::new(Time::MIN)); + TtlBackend::new( + Box::new(HashMap::::new()), + ttl_provider, + time_provider, + ) + }); + } + + #[derive(Debug)] + struct TestTtlProvider { + expires_in: Mutex>>, + } + + impl TestTtlProvider { + fn new() -> Self { + Self { + expires_in: Mutex::new(HashMap::new()), + } + } + + fn set_expires_in(&self, k: u8, v: String, d: Option) { + self.expires_in.lock().insert((k, v), d); + } + } + + impl TtlProvider for TestTtlProvider { + type K = u8; + type V = String; + + fn expires_in(&self, k: &Self::K, v: &Self::V) -> Option { + *self + .expires_in + .lock() + .get(&(*k, v.clone())) + .expect("expires_in value not mocked") + } + } +} From 8f1938a482b4b837d24c8137fcc0669b47bd7ecb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 14 Mar 2022 13:24:00 -0400 Subject: [PATCH 16/22] chore: Update datafusion (#4022) * chore: Update datafusion * chore: update for change in Expr Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 8 ++++---- datafusion/Cargo.toml | 2 +- generated_types/src/ingester.rs | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ee2bb42843..4e8d9f049c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1092,7 +1092,7 @@ dependencies = [ [[package]] name = "datafusion" version = "7.0.0" -source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241" +source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2" dependencies = [ "ahash", "arrow", @@ -1122,7 +1122,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "7.0.0" -source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241" +source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2" dependencies = [ "arrow", "ordered-float 2.10.0", @@ -1133,7 +1133,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "7.0.0" -source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241" +source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2" dependencies = [ "ahash", "arrow", @@ -1144,7 +1144,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "7.0.0" -source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241" +source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2" dependencies = [ "ahash", "arrow", diff --git a/datafusion/Cargo.toml b/datafusion/Cargo.toml index 6a3cb65e9c..fb35acad02 100644 --- a/datafusion/Cargo.toml +++ b/datafusion/Cargo.toml @@ -9,5 +9,5 @@ description = "Re-exports datafusion at a specific version" # Rename to workaround doctest bug # Turn off optional datafusion features (e.g. don't get support for crypo functions or avro) -upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="0e440eaa5ca52c0261e88590b846808539198241", default-features = false, package = "datafusion" } +upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="ca952bd33402816dbb1550debb9b8cac3b13e8f2", default-features = false, package = "datafusion" } workspace-hack = { path = "../workspace-hack"} diff --git a/generated_types/src/ingester.rs b/generated_types/src/ingester.rs index 7a9f1550ad..d39ff730a5 100644 --- a/generated_types/src/ingester.rs +++ b/generated_types/src/ingester.rs @@ -639,7 +639,7 @@ fn from_expr(expr: Expr) -> Result { expr_type: Some(ExprType::AggregateExpr(aggregate_expr)), }) } - Expr::ScalarVariable(_) => unimplemented!(), + Expr::ScalarVariable(_, _) => unimplemented!(), Expr::ScalarFunction { fun, args } => { let fun = from_scalar_function(fun)?; let args: Vec = args From 4b5cf6a70ed410b65427d773f1ccc1265b5eeaa8 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 15 Mar 2022 10:28:08 +0000 Subject: [PATCH 17/22] feat: cache processed tombstones (#4030) * test: allow to mock time in `iox_test` * feat: cache processed tombstones For #3974. * refactor: introduce `TTL_NOT_PROCESSED` --- iox_tests/src/util.rs | 13 +- querier/src/cache/mod.rs | 22 ++- querier/src/cache/processed_tombstones.rs | 162 ++++++++++++++++++++++ querier/src/chunk.rs | 5 +- querier/src/database.rs | 5 +- querier/src/namespace/mod.rs | 26 ++-- querier/src/namespace/test_util.rs | 5 +- 7 files changed, 217 insertions(+), 21 deletions(-) create mode 100644 querier/src/cache/processed_tombstones.rs diff --git a/iox_tests/src/util.rs b/iox_tests/src/util.rs index abb7f78470..4c2faa9a91 100644 --- a/iox_tests/src/util.rs +++ b/iox_tests/src/util.rs @@ -24,7 +24,7 @@ pub struct TestCatalog { pub catalog: Arc, pub metric_registry: Arc, pub object_store: Arc, - pub time_provider: Arc, + pub time_provider: Arc, pub exec: Arc, } @@ -61,9 +61,18 @@ impl TestCatalog { Arc::clone(&self.object_store) } + /// Return the mockable version of the catalog's time provider. + /// + /// If you need a generic time provider, use [`time_provider`](Self::time_provider) instead. + pub fn mock_time_provider(&self) -> &MockProvider { + self.time_provider.as_ref() + } + /// Return the catalog's time provider + /// + /// If you need to mock the time, use [`mock_time_provider`](Self::mock_time_provider) instead. pub fn time_provider(&self) -> Arc { - Arc::clone(&self.time_provider) + Arc::clone(&self.time_provider) as _ } /// Return the catalog's executor diff --git a/querier/src/cache/mod.rs b/querier/src/cache/mod.rs index 2b045e34ab..1ab3cca280 100644 --- a/querier/src/cache/mod.rs +++ b/querier/src/cache/mod.rs @@ -1,11 +1,16 @@ use backoff::BackoffConfig; use iox_catalog::interface::Catalog; use std::sync::Arc; +use time::TimeProvider; -use self::{namespace::NamespaceCache, partition::PartitionCache, table::TableCache}; +use self::{ + namespace::NamespaceCache, partition::PartitionCache, + processed_tombstones::ProcessedTombstonesCache, table::TableCache, +}; pub mod namespace; pub mod partition; +pub mod processed_tombstones; pub mod table; #[cfg(test)] @@ -25,22 +30,28 @@ pub struct CatalogCache { /// Namespace cache. namespace_cache: NamespaceCache, + + /// Processed tombstone cache. + processed_tombstones: ProcessedTombstonesCache, } impl CatalogCache { /// Create empty cache. - pub fn new(catalog: Arc) -> Self { + pub fn new(catalog: Arc, time_provider: Arc) -> Self { let backoff_config = BackoffConfig::default(); let namespace_cache = NamespaceCache::new(Arc::clone(&catalog), backoff_config.clone()); let table_cache = TableCache::new(Arc::clone(&catalog), backoff_config.clone()); - let partition_cache = PartitionCache::new(Arc::clone(&catalog), backoff_config); + let partition_cache = PartitionCache::new(Arc::clone(&catalog), backoff_config.clone()); + let processed_tombstones = + ProcessedTombstonesCache::new(Arc::clone(&catalog), backoff_config, time_provider); Self { catalog, partition_cache, table_cache, namespace_cache, + processed_tombstones, } } @@ -63,4 +74,9 @@ impl CatalogCache { pub fn partition(&self) -> &PartitionCache { &self.partition_cache } + + /// Processed tombstone cache. + pub fn processed_tombstones(&self) -> &ProcessedTombstonesCache { + &self.processed_tombstones + } } diff --git a/querier/src/cache/processed_tombstones.rs b/querier/src/cache/processed_tombstones.rs new file mode 100644 index 0000000000..15a1864d04 --- /dev/null +++ b/querier/src/cache/processed_tombstones.rs @@ -0,0 +1,162 @@ +use std::{collections::HashMap, sync::Arc, time::Duration}; + +use backoff::{Backoff, BackoffConfig}; +use data_types2::{ParquetFileId, TombstoneId}; +use iox_catalog::interface::Catalog; +use time::TimeProvider; + +use crate::cache_system::{ + backend::ttl::{TtlBackend, TtlProvider}, + driver::Cache, + loader::FunctionLoader, +}; + +/// Duration to keep "tombstone is NOT processed yet". +/// +/// Marking tombstones as processed is a mere optimization, so we can keep this cache entry for a while. +pub const TTL_NOT_PROCESSED: Duration = Duration::from_secs(100); + +/// Cache for processed tombstones. +#[derive(Debug)] +pub struct ProcessedTombstonesCache { + cache: Cache<(ParquetFileId, TombstoneId), bool>, +} + +impl ProcessedTombstonesCache { + /// Create new empty cache. + pub fn new( + catalog: Arc, + backoff_config: BackoffConfig, + time_provider: Arc, + ) -> Self { + let loader = Arc::new(FunctionLoader::new( + move |(parquet_file_id, tombstone_id)| { + let catalog = Arc::clone(&catalog); + let backoff_config = backoff_config.clone(); + + async move { + Backoff::new(&backoff_config) + .retry_all_errors("processed tombstone exists", || async { + catalog + .repositories() + .await + .processed_tombstones() + .exist(parquet_file_id, tombstone_id) + .await + }) + .await + .expect("retry forever") + } + }, + )); + + let backend = Box::new(HashMap::new()); + let backend = Box::new(TtlBackend::new( + backend, + Arc::new(KeepExistsForever {}), + time_provider, + )); + + Self { + cache: Cache::new(loader, backend), + } + } + + /// Check if the specified tombstone is mark as "processed" for the given parquet file. + pub async fn exists(&self, parquet_file_id: ParquetFileId, tombstone_id: TombstoneId) -> bool { + self.cache.get((parquet_file_id, tombstone_id)).await + } +} + +#[derive(Debug)] +struct KeepExistsForever; + +impl TtlProvider for KeepExistsForever { + type K = (ParquetFileId, TombstoneId); + type V = bool; + + fn expires_in(&self, _k: &Self::K, v: &Self::V) -> Option { + if *v { + // keep forever + None + } else { + // marking tombstones as processed is a mere optimization, so we can keep this cache entry for a while + Some(TTL_NOT_PROCESSED) + } + } +} + +#[cfg(test)] +mod tests { + use iox_tests::util::TestCatalog; + use time::{MockProvider, Time}; + + use crate::cache::test_util::assert_histogram_metric_count; + + use super::*; + + #[tokio::test] + async fn test() { + let catalog = TestCatalog::new(); + + let ns = catalog.create_namespace("ns").await; + let table = ns.create_table("table").await; + let sequencer = ns.create_sequencer(1).await; + let partition = table.with_sequencer(&sequencer).create_partition("k").await; + + let file1 = partition.create_parquet_file("table foo=1 11").await; + let file2 = partition.create_parquet_file("table foo=1 11").await; + let ts1 = table + .with_sequencer(&sequencer) + .create_tombstone(1, 1, 10, "foo=1") + .await; + let ts2 = table + .with_sequencer(&sequencer) + .create_tombstone(2, 1, 10, "foo=1") + .await; + + ts1.mark_processed(&file1).await; + + let time_provider = Arc::new(MockProvider::new(Time::from_timestamp_nanos(0))); + let cache = ProcessedTombstonesCache::new( + catalog.catalog(), + BackoffConfig::default(), + Arc::clone(&time_provider) as _, + ); + + assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await); + assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await); + assert!(!cache.exists(file2.parquet_file.id, ts1.tombstone.id).await); + assert!(!cache.exists(file2.parquet_file.id, ts2.tombstone.id).await); + + assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 4); + + ts2.mark_processed(&file2).await; + + // values are cached for a while + assert!(TTL_NOT_PROCESSED > Duration::from_millis(1)); + time_provider.inc(TTL_NOT_PROCESSED - Duration::from_millis(1)); + assert!(!cache.exists(file2.parquet_file.id, ts2.tombstone.id).await); + assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 4); + + time_provider.inc(Duration::from_millis(1)); + assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await); + assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 5); + + // "true" results are cached forever + assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await); + assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 5); + + // cache key has two dimensions + assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await); + assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await); + assert!(!cache.exists(file2.parquet_file.id, ts1.tombstone.id).await); + assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await); + ts1.mark_processed(&file2).await; + time_provider.inc(TTL_NOT_PROCESSED); + assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await); + assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await); + assert!(cache.exists(file2.parquet_file.id, ts1.tombstone.id).await); + assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await); + } +} diff --git a/querier/src/chunk.rs b/querier/src/chunk.rs index 9fe8c8bde9..997822d26e 100644 --- a/querier/src/chunk.rs +++ b/querier/src/chunk.rs @@ -165,7 +165,10 @@ mod tests { let catalog = TestCatalog::new(); let adapter = ParquetChunkAdapter::new( - Arc::new(CatalogCache::new(catalog.catalog())), + Arc::new(CatalogCache::new( + catalog.catalog(), + catalog.time_provider(), + )), catalog.object_store(), catalog.metric_registry(), catalog.time_provider(), diff --git a/querier/src/database.rs b/querier/src/database.rs index a0a115dde7..99bc17d773 100644 --- a/querier/src/database.rs +++ b/querier/src/database.rs @@ -63,7 +63,10 @@ impl QuerierDatabase { time_provider: Arc, exec: Arc, ) -> Self { - let catalog_cache = Arc::new(CatalogCache::new(Arc::clone(&catalog))); + let catalog_cache = Arc::new(CatalogCache::new( + Arc::clone(&catalog), + Arc::clone(&time_provider), + )); Self { backoff_config: BackoffConfig::default(), diff --git a/querier/src/namespace/mod.rs b/querier/src/namespace/mod.rs index 8c7ff69e32..89e37130cf 100644 --- a/querier/src/namespace/mod.rs +++ b/querier/src/namespace/mod.rs @@ -575,17 +575,11 @@ impl QuerierNamespace { let mut predicates_filtered = vec![]; for (tombstone_id, predicate) in predicates { - let is_processed = Backoff::new(&self.backoff_config) - .retry_all_errors("processed tombstone exists", || async { - self.catalog - .repositories() - .await - .processed_tombstones() - .exist(parquet_file_id, *tombstone_id) - .await - }) - .await - .expect("retry forever"); + let is_processed = self + .catalog_cache + .processed_tombstones() + .exists(parquet_file_id, *tombstone_id) + .await; if !is_processed { predicates_filtered.push(Arc::clone(predicate)); @@ -606,7 +600,9 @@ impl QuerierNamespace { #[cfg(test)] mod tests { use super::*; - use crate::namespace::test_util::querier_namespace; + use crate::{ + cache::processed_tombstones::TTL_NOT_PROCESSED, namespace::test_util::querier_namespace, + }; use data_types2::{ChunkAddr, ChunkId, ColumnType, PartitionAddr}; use iox_tests::util::{TestCatalog, TestParquetFile}; use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType}; @@ -617,7 +613,10 @@ mod tests { let catalog = TestCatalog::new(); let querier_namespace = QuerierNamespace::new( - Arc::new(CatalogCache::new(catalog.catalog())), + Arc::new(CatalogCache::new( + catalog.catalog(), + catalog.time_provider(), + )), "ns".into(), NamespaceId::new(1), catalog.metric_registry(), @@ -935,6 +934,7 @@ mod tests { .with_sequencer(&sequencer1) .create_tombstone(4, 1, 10, "foo=4") .await; + catalog.mock_time_provider().inc(TTL_NOT_PROCESSED); // cache timeout for processed tombstones querier_namespace.sync().await; assert_eq!( delete_predicates(&querier_namespace), diff --git a/querier/src/namespace/test_util.rs b/querier/src/namespace/test_util.rs index 18d28dc95b..fc9350fe37 100644 --- a/querier/src/namespace/test_util.rs +++ b/querier/src/namespace/test_util.rs @@ -7,7 +7,10 @@ use super::QuerierNamespace; pub fn querier_namespace(catalog: &Arc, ns: &Arc) -> QuerierNamespace { QuerierNamespace::new( - Arc::new(CatalogCache::new(catalog.catalog())), + Arc::new(CatalogCache::new( + catalog.catalog(), + catalog.time_provider(), + )), ns.namespace.name.clone().into(), ns.namespace.id, catalog.metric_registry(), From 10831a053040ace41831031a6841eb3ba5ebb8cb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 15 Mar 2022 10:43:28 +0000 Subject: [PATCH 18/22] chore(deps): Bump reqwest from 0.11.9 to 0.11.10 (#4032) Bumps [reqwest](https://github.com/seanmonstar/reqwest) from 0.11.9 to 0.11.10. - [Release notes](https://github.com/seanmonstar/reqwest/releases) - [Changelog](https://github.com/seanmonstar/reqwest/blob/master/CHANGELOG.md) - [Commits](https://github.com/seanmonstar/reqwest/compare/v0.11.9...v0.11.10) --- updated-dependencies: - dependency-name: reqwest dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4e8d9f049c..263faecc04 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2720,7 +2720,7 @@ dependencies = [ "pem 1.0.2", "pin-project", "rustls 0.20.4", - "rustls-pemfile", + "rustls-pemfile 0.2.1", "secrecy", "serde", "serde_json", @@ -4446,9 +4446,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.9" +version = "0.11.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f242f1488a539a79bac6dbe7c8609ae43b7914b7736210f239a37cccb32525" +checksum = "46a1f7aa4f35e5e8b4160449f51afc758f0ce6454315a9fa7d0d113e958c41eb" dependencies = [ "base64 0.13.0", "bytes", @@ -4468,7 +4468,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "rustls 0.20.4", - "rustls-pemfile", + "rustls-pemfile 0.3.0", "serde", "serde_json", "serde_urlencoded", @@ -4754,7 +4754,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5ca9ebdfa27d3fc180e42879037b5338ab1c040c06affd00d8338598e7800943" dependencies = [ "openssl-probe", - "rustls-pemfile", + "rustls-pemfile 0.2.1", "schannel", "security-framework", ] @@ -4768,6 +4768,15 @@ dependencies = [ "base64 0.13.0", ] +[[package]] +name = "rustls-pemfile" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ee86d63972a7c661d1536fefe8c3c8407321c3df668891286de28abcd087360" +dependencies = [ + "base64 0.13.0", +] + [[package]] name = "rustversion" version = "1.0.6" @@ -6495,9 +6504,9 @@ checksum = "504a2476202769977a040c6364301a3f65d0cc9e3fb08600b2bda150a0488316" [[package]] name = "winreg" -version = "0.7.0" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0120db82e8a1e0b9fb3345a539c478767c0048d842860994d96113d5b667bd69" +checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" dependencies = [ "winapi", ] From eeb918ca2b2b5a4b3a716fa93f6ee321115938a5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 15 Mar 2022 10:56:52 +0000 Subject: [PATCH 19/22] chore(deps): Bump libc from 0.2.119 to 0.2.120 (#4034) Bumps [libc](https://github.com/rust-lang/libc) from 0.2.119 to 0.2.120. - [Release notes](https://github.com/rust-lang/libc/releases) - [Commits](https://github.com/rust-lang/libc/compare/0.2.119...0.2.120) --- updated-dependencies: - dependency-name: libc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 263faecc04..f7794600e5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2866,9 +2866,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.119" +version = "0.2.120" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bf2e165bb3457c8e098ea76f3e3bc9db55f87aa90d52d0e6be741470916aaa4" +checksum = "ad5c14e80759d0939d013e6ca49930e59fc53dd8e5009132f76240c179380c09" [[package]] name = "libloading" From e2cf5c63e576187dbf3c8347304d06bbd4b56d80 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 15 Mar 2022 11:29:01 +0000 Subject: [PATCH 20/22] chore(deps): Bump nom from 7.1.0 to 7.1.1 (#4035) Bumps [nom](https://github.com/Geal/nom) from 7.1.0 to 7.1.1. - [Release notes](https://github.com/Geal/nom/releases) - [Changelog](https://github.com/Geal/nom/blob/main/CHANGELOG.md) - [Commits](https://github.com/Geal/nom/compare/7.1.0...7.1.1) --- updated-dependencies: - dependency-name: nom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f7794600e5..9bd638895c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3249,13 +3249,12 @@ checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" [[package]] name = "nom" -version = "7.1.0" +version = "7.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b1d11e1ef389c76fe5b81bcaf2ea32cf88b62bc494e19f493d0b30e7a930109" +checksum = "a8903e5a29a317527874d0402f867152a3d21c908bb0b933e416c65e301d4c36" dependencies = [ "memchr", "minimal-lexical", - "version_check", ] [[package]] From 97d595e4fb8826c17e4aa40c3f7d7562ee75f117 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 15 Mar 2022 12:12:26 +0000 Subject: [PATCH 21/22] feat: `Cache::set` (#4036) * feat: `Cache::set` This will be helpful to fill caches if we got the information from somewhere else. For #3985. * docs: improve Co-authored-by: Edd Robinson * docs: explain lock gap * feat: add debug log to `Cache` Co-authored-by: Edd Robinson --- querier/src/cache_system/driver.rs | 178 +++++++++++++++++++++++------ 1 file changed, 144 insertions(+), 34 deletions(-) diff --git a/querier/src/cache_system/driver.rs b/querier/src/cache_system/driver.rs index 586a73666f..7baec1c467 100644 --- a/querier/src/cache_system/driver.rs +++ b/querier/src/cache_system/driver.rs @@ -4,8 +4,12 @@ use futures::{ future::{BoxFuture, Shared}, FutureExt, TryFutureExt, }; +use observability_deps::tracing::debug; use parking_lot::Mutex; -use tokio::{sync::oneshot::error::RecvError, task::JoinHandle}; +use tokio::{ + sync::oneshot::{error::RecvError, Sender}, + task::JoinHandle, +}; use super::{backend::CacheBackend, loader::Loader}; @@ -61,51 +65,83 @@ where } // check if there is already a query for this key running - if let Some((receiver, _handle)) = state.running_queries.get(&k) { - receiver.clone() + if let Some(running_query) = state.running_queries.get(&k) { + running_query.recv.clone() } else { // requires new query - let (tx, rx) = tokio::sync::oneshot::channel(); - let receiver = rx + let (tx_main, rx_main) = tokio::sync::oneshot::channel(); + let receiver = rx_main .map_ok(|v| Arc::new(Mutex::new(v))) .map_err(Arc::new) .boxed() .shared(); + let (tx_set, rx_set) = tokio::sync::oneshot::channel(); // need to wrap the query into a tokio task so that it doesn't get cancelled when this very request is canceled let state_captured = Arc::clone(&self.state); let loader = Arc::clone(&self.loader); let k_captured = k.clone(); let handle = tokio::spawn(async move { - // need to clone K and bind it so rustc doesn't require `K: Sync` - let k_for_loader = k_captured.clone(); + let loader_fut = async move { + // need to clone K and bind it so rustc doesn't require `K: Sync` + let k_for_loader = k_captured.clone(); - // execute the loader - // If we panic here then `tx` will be dropped and the receivers will be notified. - let v = loader.load(k_for_loader).await; + // execute the loader + // If we panic here then `tx` will be dropped and the receivers will be notified. + let v = loader.load(k_for_loader).await; + + // remove "running" state and store result + // + // Note: we need to manually drop the result of `.remove(...).expect(...)` here to convince rustc + // that we don't need the shared future within the resulting tuple. The warning we would get + // is: + // + // warning: unused `futures::future::Shared` in tuple element 0 that must be used + let mut state = state_captured.lock(); + drop( + state + .running_queries + .remove(&k_captured) + .expect("query should be running"), + ); + state.cached_entries.set(k_captured, v.clone()); + + v + }; + + let v = tokio::select! { + v = loader_fut => v, + maybe_v = rx_set => { + match maybe_v { + Ok(v) => { + // data get side-loaded via `Cache::set`. In this case, we do NOT modify the state + // because there would be a lock-gap. The `set` function will do that for us instead. + v + } + Err(_) => { + // sender side is gone, very likely the cache is shutting down + debug!( + "Sender for side-loading data into running query gone.", + ); + return; + } + } + } + }; // broadcast result // It's OK if the receiver side is gone. This might happen during shutdown - tx.send(v.clone()).ok(); - - // remove "running" state and store result - // - // Note: we need to manually drop the result of `.remove(...).expect(...)` here to convince rustc - // that we don't need the shared future within the resulting tuple. The warning we would get - // is: - // - // warning: unused `futures::future::Shared` in tuple element 0 that must be used - let mut state = state_captured.lock(); - drop( - state - .running_queries - .remove(&k_captured) - .expect("query should be running"), - ); - state.cached_entries.set(k_captured, v); + tx_main.send(v).ok(); }); - state.running_queries.insert(k, (receiver.clone(), handle)); + state.running_queries.insert( + k, + RunningQuery { + recv: receiver.clone(), + set: tx_set, + join_handle: handle, + }, + ); receiver } }; @@ -116,6 +152,24 @@ where .lock() .clone() } + + /// Side-load an entry into the cache. + /// + /// This will also complete a currently running request for this key. + pub fn set(&self, k: K, v: V) { + let mut state = self.state.lock(); + + if let Some(running_query) = state.running_queries.remove(&k) { + // it's OK when the receiver side is gone (likely panicked) + running_query.set.send(v.clone()).ok(); + + // When we side-load data into the running task, the task does NOT modify the backend, so we have to do + // that. The reason for not letting the task feed the side-loaded data back into `cached_entries` is that we + // would need to drop the state lock here before the task could acquire it, leading to a lock gap. + } + + state.cached_entries.set(k, v); + } } impl Drop for Cache @@ -124,11 +178,11 @@ where V: Clone + std::fmt::Debug + Send + 'static, { fn drop(&mut self) { - for (_k, (_receiver, handle)) in self.state.lock().running_queries.drain() { + for (_k, running_query) in self.state.lock().running_queries.drain() { // It's unlikely that anyone is still using the shared receiver at this point, because Cache::get borrow // the self. If it is still in use, aborting the task will cancel the contained future which in turn will // drop the sender of the oneshot channel. The receivers will be notified. - handle.abort(); + running_query.join_handle.abort(); } } } @@ -144,6 +198,21 @@ where /// - `Shared`: Allow the receiver to be cloned and be awaited from multiple places. type SharedReceiver = Shared>, Arc>>>; +/// State for coordinating the execution of a single running query. +#[derive(Debug)] +struct RunningQuery { + /// A receiver that can await the result as well. + recv: SharedReceiver, + + /// A sender that enables setting entries while the query is running. + set: Sender, + + /// A handle for the task that is currently executing the query. + /// + /// The handle can be used to abort the running query, e.g. when dropping the cache. + join_handle: JoinHandle<()>, +} + /// Inner cache state that is usually guarded by a lock. /// /// The state parts must be updated in a consistent manner, i.e. while using the same lock guard. @@ -153,10 +222,7 @@ struct CacheState { cached_entries: Box>, /// Currently running queries indexed by cache key. - /// - /// For each query we have a receiver that can await the result as well as a handle for the task that is currently - /// executing the query. The handle can be used to abort the running query, e.g. when dropping the cache. - running_queries: HashMap, JoinHandle<()>)>, + running_queries: HashMap>, } #[cfg(test)] @@ -308,6 +374,50 @@ mod tests { assert_eq!(Arc::strong_count(&loader), 1); } + #[tokio::test] + async fn test_set_before_request() { + let (cache, loader) = setup(); + + loader.block(); + + cache.set(1, String::from("foo")); + + // blocked loader is not used + let res = tokio::time::timeout(Duration::from_millis(10), cache.get(1)) + .await + .unwrap(); + assert_eq!(res, String::from("foo")); + assert_eq!(loader.loaded(), Vec::::new()); + } + + #[tokio::test] + async fn test_set_during_request() { + let (cache, loader) = setup(); + + loader.block(); + + let cache_captured = Arc::clone(&cache); + let handle = tokio::spawn(async move { cache_captured.get(1).await }); + tokio::time::sleep(Duration::from_millis(10)).await; + + cache.set(1, String::from("foo")); + + // request succeeds even though the loader is blocked + let res = tokio::time::timeout(Duration::from_millis(10), handle) + .await + .unwrap() + .unwrap(); + assert_eq!(res, String::from("foo")); + assert_eq!(loader.loaded(), vec![1]); + + // still cached + let res = tokio::time::timeout(Duration::from_millis(10), cache.get(1)) + .await + .unwrap(); + assert_eq!(res, String::from("foo")); + assert_eq!(loader.loaded(), vec![1]); + } + fn setup() -> (Arc>, Arc) { let loader = Arc::new(TestLoader::default()); let cache = Arc::new(Cache::new( From cc7f71126cb2cf365618de25147063e6cbcca82e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 15 Mar 2022 12:22:18 +0000 Subject: [PATCH 22/22] chore(deps): Bump syn from 1.0.86 to 1.0.88 (#4037) Bumps [syn](https://github.com/dtolnay/syn) from 1.0.86 to 1.0.88. - [Release notes](https://github.com/dtolnay/syn/releases) - [Commits](https://github.com/dtolnay/syn/compare/1.0.86...1.0.88) --- updated-dependencies: - dependency-name: syn dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9bd638895c..a331b87757 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5496,9 +5496,9 @@ dependencies = [ [[package]] name = "syn" -version = "1.0.86" +version = "1.0.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a65b3f4ffa0092e9887669db0eae07941f023991ab58ea44da8fe8e2d511c6b" +checksum = "ebd69e719f31e88618baa1eaa6ee2de5c9a1c004f1e9ecdb58e8352a13f20a01" dependencies = [ "proc-macro2", "quote",