From ea4683095471ad691cef31ba388df5e0b78e2659 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 11:37:21 -0400 Subject: [PATCH 01/31] fix: Remove iox_object_store crate; move ParquetFilePath to parquet_file --- Cargo.lock | 26 +- Cargo.toml | 1 - compactor/Cargo.toml | 1 - compactor/src/compact.rs | 26 +- compactor/src/garbage_collector.rs | 36 +- compactor/src/utils.rs | 6 +- influxdb_iox/Cargo.toml | 1 - influxdb_iox/src/commands/remote/partition.rs | 11 +- ingester/Cargo.toml | 1 - ingester/src/data.rs | 6 +- ingester/src/persist.rs | 37 +- iox_object_store/Cargo.toml | 21 - iox_object_store/src/lib.rs | 859 ------------------ iox_object_store/src/paths.rs | 233 ----- iox_object_store/src/paths/parquet_file.rs | 460 ---------- .../src/paths/transaction_file.rs | 384 -------- iox_object_store_service/Cargo.toml | 2 +- iox_object_store_service/src/lib.rs | 18 +- iox_tests/Cargo.toml | 1 - iox_tests/src/util.rs | 33 +- parquet_file/Cargo.toml | 1 - parquet_file/src/chunk.rs | 23 +- parquet_file/src/lib.rs | 90 ++ parquet_file/src/storage.rs | 34 +- querier/Cargo.toml | 1 - querier/src/chunk/mod.rs | 15 +- 26 files changed, 200 insertions(+), 2127 deletions(-) delete mode 100644 iox_object_store/Cargo.toml delete mode 100644 iox_object_store/src/lib.rs delete mode 100644 iox_object_store/src/paths.rs delete mode 100644 iox_object_store/src/paths/parquet_file.rs delete mode 100644 iox_object_store/src/paths/transaction_file.rs diff --git a/Cargo.lock b/Cargo.lock index a79154c663..f85bbd356a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -818,7 +818,6 @@ dependencies = [ "datafusion 0.1.0", "futures", "iox_catalog", - "iox_object_store", "iox_tests", "iox_time", "metric", @@ -2185,7 +2184,6 @@ dependencies = [ "influxrpc_parser", "ingester", "iox_catalog", - "iox_object_store", "iox_time", "ioxd_common", "ioxd_compactor", @@ -2344,7 +2342,6 @@ dependencies = [ "generated_types", "hyper", "iox_catalog", - "iox_object_store", "iox_time", "lazy_static", "metric", @@ -2508,24 +2505,6 @@ dependencies = [ "workspace-hack", ] -[[package]] -name = "iox_object_store" -version = "0.1.0" -dependencies = [ - "bytes", - "data_types", - "data_types2", - "futures", - "object_store", - "observability_deps", - "snafu", - "test_helpers", - "tokio", - "tokio-stream", - "uuid 0.8.2", - "workspace-hack", -] - [[package]] name = "iox_object_store_service" version = "0.1.0" @@ -2536,11 +2515,11 @@ dependencies = [ "futures", "generated_types", "iox_catalog", - "iox_object_store", "iox_time", "metric", "object_store", "observability_deps", + "parquet_file", "serde", "serde_urlencoded", "tokio", @@ -2560,7 +2539,6 @@ dependencies = [ "data_types2", "datafusion 0.1.0", "iox_catalog", - "iox_object_store", "iox_time", "metric", "mutable_batch_lp", @@ -3823,7 +3801,6 @@ dependencies = [ "datafusion_util", "futures", "generated_types", - "iox_object_store", "iox_time", "metric", "object_store", @@ -4294,7 +4271,6 @@ dependencies = [ "hyper", "influxdb_iox_client", "iox_catalog", - "iox_object_store", "iox_tests", "iox_time", "metric", diff --git a/Cargo.toml b/Cargo.toml index 490b38c4cf..63435bd1ee 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,7 +27,6 @@ members = [ "iox_catalog_service", "iox_data_generator", "iox_gitops_adapter", - "iox_object_store", "iox_object_store_service", "iox_tests", "iox_time", diff --git a/compactor/Cargo.toml b/compactor/Cargo.toml index 65c8e5953b..62b61baf60 100644 --- a/compactor/Cargo.toml +++ b/compactor/Cargo.toml @@ -14,7 +14,6 @@ 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" } diff --git a/compactor/src/compact.rs b/compactor/src/compact.rs index 4126164f66..c0d062fe2b 100644 --- a/compactor/src/compact.rs +++ b/compactor/src/compact.rs @@ -15,12 +15,14 @@ use data_types2::{ }; use datafusion::error::DataFusionError; use iox_catalog::interface::{Catalog, Transaction}; -use iox_object_store::ParquetFilePath; use iox_time::{Time, TimeProvider}; use metric::{Attributes, Metric, U64Counter, U64Gauge, U64Histogram, U64HistogramOptions}; use object_store::DynObjectStore; use observability_deps::tracing::{debug, info, warn}; -use parquet_file::metadata::{IoxMetadata, IoxParquetMetaData}; +use parquet_file::{ + metadata::{IoxMetadata, IoxParquetMetaData}, + ParquetFilePath, +}; use query::provider::overlap::group_potential_duplicates; use query::{ exec::{Executor, ExecutorType}, @@ -33,7 +35,7 @@ use snafu::{ensure, OptionExt, ResultExt, Snafu}; use std::{ cmp::{max, min, Ordering}, collections::{BTreeMap, HashSet}, - ops::DerefMut, + ops::{Deref, DerefMut}, sync::Arc, }; use uuid::Uuid; @@ -782,16 +784,7 @@ impl Compactor { .expect("record_batches.is_empty was just checked") .schema(); - // Make a fake IOx object store to conform to the parquet file - // interface, but note this isn't actually used to find parquet - // paths to write to - use iox_object_store::IoxObjectStore; - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(&object_store), - IoxObjectStore::root_path_for(&*object_store, uuid::Uuid::new_v4()), - )); - - let data = parquet_file::storage::Storage::new(Arc::clone(&iox_object_store)) + let data = parquet_file::storage::Storage::new(Arc::clone(&object_store)) .parquet_bytes(record_batches, schema, metadata) .await .context(ConvertingToBytesSnafu)?; @@ -810,16 +803,17 @@ impl Compactor { let file_size = data.len(); let bytes = Bytes::from(data); - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( metadata.namespace_id, metadata.table_id, metadata.sequencer_id, metadata.partition_id, metadata.object_store_id, ); + let path = path.object_store_path(object_store.deref()); - iox_object_store - .put_parquet_file(&path, bytes) + object_store + .put(&path, bytes) .await .context(WritingToObjectStoreSnafu)?; diff --git a/compactor/src/garbage_collector.rs b/compactor/src/garbage_collector.rs index 7241734b24..d40a19fb19 100644 --- a/compactor/src/garbage_collector.rs +++ b/compactor/src/garbage_collector.rs @@ -4,11 +4,11 @@ use data_types2::Timestamp; use iox_catalog::interface::Catalog; -use iox_object_store::ParquetFilePath; use iox_time::TimeProvider; use object_store::DynObjectStore; +use parquet_file::ParquetFilePath; use snafu::{ResultExt, Snafu}; -use std::sync::Arc; +use std::{ops::Deref, sync::Arc}; #[derive(Debug, Snafu)] #[allow(missing_copy_implementations, missing_docs)] @@ -53,15 +53,6 @@ impl GarbageCollector { /// associated object store files. /// Meant to be invoked in a background loop. pub async fn cleanup(&self, older_than: Timestamp) -> Result<()> { - // Make a fake IOx object store to conform to the parquet file - // interface, but note this isn't actually used to find parquet - // paths to write to - use iox_object_store::IoxObjectStore; - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(&self.object_store), - IoxObjectStore::root_path_for(&*self.object_store, uuid::Uuid::new_v4()), - )); - let deleted_catalog_records = self .catalog .repositories() @@ -74,15 +65,16 @@ impl GarbageCollector { let mut object_store_errors = Vec::with_capacity(deleted_catalog_records.len()); for catalog_record in deleted_catalog_records { - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( catalog_record.namespace_id, catalog_record.table_id, catalog_record.sequencer_id, catalog_record.partition_id, catalog_record.object_store_id, ); + let path = path.object_store_path(self.object_store.deref()); - if let Err(e) = iox_object_store.delete_parquet_file(&path).await { + if let Err(e) = self.object_store.delete(&path).await { object_store_errors.push(e); } } @@ -103,7 +95,6 @@ mod tests { use super::*; use data_types2::{KafkaPartition, ParquetFile, ParquetFileParams, SequenceNumber}; use iox_catalog::interface::INITIAL_COMPACTION_LEVEL; - use iox_object_store::ParquetFilePath; use iox_tests::util::TestCatalog; use object_store::ObjectStoreTestConvenience; use std::time::Duration; @@ -118,27 +109,16 @@ mod tests { ) { let bytes = "arbitrary".into(); - // Make a fake IOx object store to conform to the parquet file - // interface, but note this isn't actually used to find parquet - // paths to write to - use iox_object_store::IoxObjectStore; - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(&object_store), - IoxObjectStore::root_path_for(&*object_store, uuid::Uuid::new_v4()), - )); - - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( catalog_record.namespace_id, catalog_record.table_id, catalog_record.sequencer_id, catalog_record.partition_id, catalog_record.object_store_id, ); + let path = path.object_store_path(object_store.deref()); - iox_object_store - .put_parquet_file(&path, bytes) - .await - .unwrap(); + object_store.put(&path, bytes).await.unwrap(); } #[tokio::test] diff --git a/compactor/src/utils.rs b/compactor/src/utils.rs index cf931911cf..80dae8e7c3 100644 --- a/compactor/src/utils.rs +++ b/compactor/src/utils.rs @@ -5,7 +5,6 @@ use arrow::record_batch::RecordBatch; use data_types2::{ ParquetFileId, ParquetFileParams, ParquetFileWithMetadata, Timestamp, Tombstone, TombstoneId, }; -use iox_object_store::IoxObjectStore; use object_store::DynObjectStore; use observability_deps::tracing::*; use parquet_file::{ @@ -94,12 +93,11 @@ impl ParquetFileWithTombstone { partition_sort_key: Option, ) -> 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, ChunkMetrics::new_unregistered(), // TODO: need to add metrics - Arc::new(iox_object_store), + object_store, ); debug!( diff --git a/influxdb_iox/Cargo.toml b/influxdb_iox/Cargo.toml index b37473a81a..ce6c8b59b9 100644 --- a/influxdb_iox/Cargo.toml +++ b/influxdb_iox/Cargo.toml @@ -20,7 +20,6 @@ influxdb_line_protocol = { path = "../influxdb_line_protocol" } ingester = { path = "../ingester" } influxrpc_parser = { path = "../influxrpc_parser"} iox_catalog = { path = "../iox_catalog" } -iox_object_store = { path = "../iox_object_store" } ioxd_common = { path = "../ioxd_common"} ioxd_compactor = { path = "../ioxd_compactor"} ioxd_ingester = { path = "../ioxd_ingester"} diff --git a/influxdb_iox/src/commands/remote/partition.rs b/influxdb_iox/src/commands/remote/partition.rs index 5182b6cbb1..9540d29a19 100644 --- a/influxdb_iox/src/commands/remote/partition.rs +++ b/influxdb_iox/src/commands/remote/partition.rs @@ -18,9 +18,9 @@ use influxdb_iox_client::{ store, }; use iox_catalog::interface::{get_schema_by_name, Catalog}; -use iox_object_store::ParquetFilePath; use object_store::{DynObjectStore, ObjectStoreImpl}; -use std::sync::Arc; +use parquet_file::ParquetFilePath; +use std::{ops::Deref, sync::Arc}; use thiserror::Error; use tokio_stream::StreamExt; use uuid::Uuid; @@ -150,15 +150,14 @@ pub async fn command(connection: Connection, config: Config) -> Result<(), Error let mut handles = vec![]; let store_client = store::Client::new(connection); for parquet_file in parquet_files { - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( parquet_file.namespace_id, parquet_file.table_id, parquet_file.sequencer_id, parquet_file.partition_id, parquet_file.object_store_id, - ) - .absolute_dirs_and_file_name(); - let path = object_store.path_from_dirs_and_filename(path); + ); + let path = path.object_store_path(object_store.deref()); match object_store.get(&path).await { Ok(_) => { println!( diff --git a/ingester/Cargo.toml b/ingester/Cargo.toml index b449a61a08..669c643f8b 100644 --- a/ingester/Cargo.toml +++ b/ingester/Cargo.toml @@ -22,7 +22,6 @@ chrono = { version = "0.4", default-features = false } dml = { path = "../dml" } hyper = "0.14" iox_catalog = { path = "../iox_catalog" } -iox_object_store = { path = "../iox_object_store" } metric = { path = "../metric" } mutable_batch = { path = "../mutable_batch"} mutable_batch_lp = { path = "../mutable_batch_lp" } diff --git a/ingester/src/data.rs b/ingester/src/data.rs index 2f9d1efe4f..26248a9c1c 100644 --- a/ingester/src/data.rs +++ b/ingester/src/data.rs @@ -312,7 +312,11 @@ impl Persister for IngesterData { // save the compacted data to a parquet file in object storage let file_size_and_md = Backoff::new(&self.backoff_config) .retry_all_errors("persist to object store", || { - persist(&iox_meta, record_batches.to_vec(), &self.object_store) + persist( + &iox_meta, + record_batches.to_vec(), + Arc::clone(&self.object_store), + ) }) .await .expect("retry forever"); diff --git a/ingester/src/persist.rs b/ingester/src/persist.rs index 861e179040..7bd724d0ff 100644 --- a/ingester/src/persist.rs +++ b/ingester/src/persist.rs @@ -2,11 +2,13 @@ use arrow::record_batch::RecordBatch; use bytes::Bytes; -use iox_object_store::ParquetFilePath; use object_store::DynObjectStore; -use parquet_file::metadata::{IoxMetadata, IoxParquetMetaData}; +use parquet_file::{ + metadata::{IoxMetadata, IoxParquetMetaData}, + ParquetFilePath, +}; use snafu::{ResultExt, Snafu}; -use std::sync::Arc; +use std::{ops::Deref, sync::Arc}; #[derive(Debug, Snafu)] #[allow(missing_docs)] @@ -29,7 +31,7 @@ pub type Result = std::result::Result; pub async fn persist( metadata: &IoxMetadata, record_batches: Vec, - object_store: &Arc, + object_store: Arc, ) -> Result> { if record_batches.is_empty() { return Ok(None); @@ -39,16 +41,7 @@ pub async fn persist( .expect("record_batches.is_empty was just checked") .schema(); - // Make a fake IOx object store to conform to the parquet file - // interface, but note this isn't actually used to find parquet - // paths to write to - use iox_object_store::IoxObjectStore; - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(object_store), - IoxObjectStore::root_path_for(&**object_store, uuid::Uuid::new_v4()), - )); - - let data = parquet_file::storage::Storage::new(Arc::clone(&iox_object_store)) + let data = parquet_file::storage::Storage::new(Arc::clone(&object_store)) .parquet_bytes(record_batches, schema, metadata) .await .context(ConvertingToBytesSnafu)?; @@ -67,7 +60,7 @@ pub async fn persist( let file_size = data.len(); let bytes = Bytes::from(data); - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( metadata.namespace_id, metadata.table_id, metadata.sequencer_id, @@ -75,8 +68,10 @@ pub async fn persist( metadata.object_store_id, ); - iox_object_store - .put_parquet_file(&path, bytes) + let path = path.object_store_path(object_store.deref()); + + object_store + .put(&path, bytes) .await .context(WritingToObjectStoreSnafu)?; @@ -124,7 +119,9 @@ mod tests { }; let object_store = object_store(); - persist(&metadata, vec![], &object_store).await.unwrap(); + persist(&metadata, vec![], Arc::clone(&object_store)) + .await + .unwrap(); assert!(object_store.list_all().await.unwrap().is_empty()); } @@ -163,7 +160,9 @@ mod tests { let object_store = object_store(); - persist(&metadata, batches, &object_store).await.unwrap(); + persist(&metadata, batches, Arc::clone(&object_store)) + .await + .unwrap(); let obj_store_paths = object_store.list_all().await.unwrap(); assert_eq!(obj_store_paths.len(), 1); diff --git a/iox_object_store/Cargo.toml b/iox_object_store/Cargo.toml deleted file mode 100644 index ce004b5f02..0000000000 --- a/iox_object_store/Cargo.toml +++ /dev/null @@ -1,21 +0,0 @@ -[package] -name = "iox_object_store" -version = "0.1.0" -edition = "2021" -description = "IOx-specific semantics wrapping the general-purpose object store crate" - -[dependencies] -bytes = "1.0" -data_types = { path = "../data_types" } -data_types2 = { path = "../data_types2" } -futures = "0.3" -object_store = { path = "../object_store" } -observability_deps = { path = "../observability_deps" } -snafu = "0.7" -tokio = { version = "1.18", features = ["macros", "parking_lot", "sync", "rt"] } -tokio-stream = "0.1" -uuid = { version = "0.8", features = ["v4"] } -workspace-hack = { path = "../workspace-hack"} - -[dev-dependencies] # In alphabetical order -test_helpers = { path = "../test_helpers" } diff --git a/iox_object_store/src/lib.rs b/iox_object_store/src/lib.rs deleted file mode 100644 index 5dcae35c8d..0000000000 --- a/iox_object_store/src/lib.rs +++ /dev/null @@ -1,859 +0,0 @@ -//! Wraps the object_store crate with IOx-specific semantics. The main responsibility of this crate -//! is to be the single source of truth for the paths of files in object storage. There is a -//! specific path type for each IOx-specific reason an object storage file exists. Content of the -//! files is managed outside of this crate. - -#![deny(rustdoc::broken_intra_doc_links, rustdoc::bare_urls, rust_2018_idioms)] -#![warn( - missing_copy_implementations, - missing_debug_implementations, - missing_docs, - clippy::explicit_iter_loop, - clippy::future_not_send, - clippy::use_self, - clippy::clone_on_ref_ptr -)] - -use bytes::Bytes; -use data_types::server_id::ServerId; -use futures::{stream::BoxStream, StreamExt, TryStreamExt}; -use object_store::{path::Path, DynObjectStore, GetResult, Result}; -use observability_deps::tracing::warn; -use snafu::{ensure, ResultExt, Snafu}; -use std::{ops::Deref, sync::Arc}; -use tokio::sync::mpsc::channel; -use tokio_stream::wrappers::ReceiverStream; -use uuid::Uuid; - -mod paths; -pub use paths::{ - parquet_file::{ParquetFilePath, ParquetFilePathParseError}, - transaction_file::TransactionFilePath, -}; -use paths::{DataPath, RootPath, TransactionsPath}; - -#[derive(Debug, Snafu)] -#[allow(missing_docs)] -pub enum IoxObjectStoreError { - #[snafu(display("{}", source))] - UnderlyingObjectStoreError { source: object_store::Error }, - - #[snafu(display("Cannot create database with UUID `{}`; it already exists", uuid))] - DatabaseAlreadyExists { uuid: Uuid }, - - #[snafu(display("No rules found to load at {}", root_path))] - NoRulesFound { root_path: RootPath }, -} - -/// Handles persistence of data for a particular database. Writes within its directory/prefix. -/// -/// This wrapper on top of an `ObjectStore` maps IOx specific concepts to ObjectStore locations -#[derive(Debug)] -pub struct IoxObjectStore { - inner: Arc, - root_path: RootPath, - data_path: DataPath, - transactions_path: TransactionsPath, -} - -impl IoxObjectStore { - /// Get the data for the server config to determine the names and locations of the databases - /// that this server owns. - /// - /// TEMPORARY: Server config used to be at the top level instead of beneath `/nodes/`. Until - /// all deployments have transitioned, check both locations before reporting that the server - /// config is not found. - pub async fn get_server_config_file( - inner: &DynObjectStore, - server_id: ServerId, - ) -> Result { - let path = paths::server_config_path(inner, server_id); - let result = match inner.get(&path).await { - Err(object_store::Error::NotFound { .. }) => { - use object_store::path::ObjectStorePath; - let mut legacy_path = inner.new_path(); - legacy_path.push_dir(server_id.to_string()); - legacy_path.set_file_name(paths::SERVER_CONFIG_FILE_NAME); - - inner.get(&legacy_path).await - } - other => other, - }?; - - Ok(result.bytes().await?.into()) - } - - /// Store the data for the server config with the names and locations of the databases - /// that this server owns. - pub async fn put_server_config_file( - inner: &DynObjectStore, - server_id: ServerId, - bytes: Bytes, - ) -> Result<()> { - let path = paths::server_config_path(inner, server_id); - inner.put(&path, bytes).await - } - - /// Return the path to the server config file to be used in database ownership information to - /// identify the current server that a database thinks is its owner. - pub fn server_config_path(inner: &DynObjectStore, server_id: ServerId) -> Path { - paths::server_config_path(inner, server_id) - } - - /// Returns what the root path would be for a given database. Does not check existence or - /// validity of the path in object storage. - pub fn root_path_for(inner: &DynObjectStore, uuid: Uuid) -> RootPath { - RootPath::new(inner, uuid) - } - - /// Create a database-specific wrapper. Takes all the information needed to create a new - /// root directory of a database. Checks that there isn't already anything in this database's - /// directory in object storage. - /// - /// Caller *MUST* ensure there is at most 1 concurrent call of this function with the same - /// parameters; this function does *NOT* do any locking. - pub async fn create( - inner: Arc, - uuid: Uuid, - ) -> Result { - let root_path = Self::root_path_for(&*inner, uuid); - - let list_result = inner - .list_with_delimiter(&root_path.inner) - .await - .context(UnderlyingObjectStoreSnafu)?; - - ensure!( - list_result.objects.is_empty(), - DatabaseAlreadyExistsSnafu { uuid } - ); - - Ok(Self::existing(inner, root_path)) - } - - /// Look in object storage for an existing, active database with this UUID. - pub async fn load(inner: Arc, uuid: Uuid) -> Result { - let root_path = Self::root_path_for(&*inner, uuid); - - Self::find(inner, root_path).await - } - - /// Look in object storage for an existing database with this name and the given root path - /// that was retrieved from a server config - pub async fn load_at_root_path( - inner: Arc, - root_path_str: &str, - ) -> Result { - let root_path = RootPath::from_str(&*inner, root_path_str); - - Self::find(inner, root_path).await - } - - async fn find( - inner: Arc, - root_path: RootPath, - ) -> Result { - let list_result = inner - .list_with_delimiter(&root_path.inner) - .await - .context(UnderlyingObjectStoreSnafu)?; - - let rules_file = root_path.rules_path(); - let rules_exists = list_result - .objects - .iter() - .any(|object| object.location == rules_file.inner); - - ensure!(rules_exists, NoRulesFoundSnafu { root_path }); - - Ok(Self::existing(inner, root_path)) - } - - /// Access the database-specific object storage files for an existing database that has - /// already been located and verified to be active. Does not check object storage. - pub fn existing(inner: Arc, root_path: RootPath) -> Self { - let data_path = root_path.data_path(); - let transactions_path = root_path.transactions_path(); - - Self { - inner, - root_path, - data_path, - transactions_path, - } - } - - /// In the database's root directory, write out a file pointing to the server's config. This - /// data can serve as an extra check on which server owns this database. - pub async fn put_owner_file(&self, bytes: Bytes) -> Result<()> { - let owner_path = self.root_path.owner_path(); - - self.inner.put(&owner_path, bytes).await - } - - /// Return the contents of the owner file in the database's root directory that provides - /// information on the server that owns this database. - pub async fn get_owner_file(&self) -> Result { - let owner_path = self.root_path.owner_path(); - - Ok(self.inner.get(&owner_path).await?.bytes().await?.into()) - } - - /// Delete owner file for testing - pub async fn delete_owner_file_for_testing(&self) -> Result<()> { - let owner_path = self.root_path.owner_path(); - - self.inner.delete(&owner_path).await - } - - /// The location in object storage for all files for this database, suitable for logging or - /// debugging purposes only. Do not parse this, as its format is subject to change! - pub fn debug_database_path(&self) -> String { - self.root_path.inner.to_string() - } - - /// The possibly valid location in object storage for this database. Suitable for serialization - /// to use during initial database load, but not parsing for semantic meaning, as its format is - /// subject to change! - pub fn root_path(&self) -> String { - self.root_path.to_string() - } - - // Catalog transaction file methods =========================================================== - - /// List all the catalog transaction files in object storage for this database. - pub async fn catalog_transaction_files( - &self, - ) -> Result>>> { - Ok(self - .list(Some(&self.transactions_path.inner)) - .await? - .map_ok(move |list| { - list.into_iter() - // This `flat_map` ignores any filename in the transactions_path we couldn't - // parse as a TransactionFilePath - .flat_map(TransactionFilePath::from_absolute) - .collect() - }) - .boxed()) - } - - /// Get the catalog transaction data in this relative path in this database's object store. - pub async fn get_catalog_transaction_file( - &self, - location: &TransactionFilePath, - ) -> Result> { - let full_path = self.transactions_path.join(location); - - self.inner.get(&full_path).await - } - - /// Store the data for this parquet file in this database's object store. - pub async fn put_catalog_transaction_file( - &self, - location: &TransactionFilePath, - bytes: Bytes, - ) -> Result<()> { - let full_path = self.transactions_path.join(location); - - self.inner.put(&full_path, bytes).await - } - - /// Delete all catalog transaction files for this database. - pub async fn wipe_catalog(&self) -> Result<()> { - let mut stream = self.catalog_transaction_files().await?; - - while let Some(transaction_file_list) = stream.try_next().await? { - for transaction_file_path in &transaction_file_list { - self.delete_catalog_transaction_file(transaction_file_path) - .await?; - } - } - - Ok(()) - } - - /// Remove the data for this catalog transaction file from this database's object store - pub async fn delete_catalog_transaction_file( - &self, - location: &TransactionFilePath, - ) -> Result<()> { - let full_path = self.transactions_path.join(location); - - self.inner.delete(&full_path).await - } - - // Parquet file methods ======================================================================= - - /// List all parquet file paths in object storage for this database. - pub async fn parquet_files(&self) -> Result>>> { - Ok(self - .list(Some(&self.data_path.inner)) - .await? - .map_ok(move |list| { - list.into_iter() - // This `flat_map` ignores any filename in the data_path we couldn't parse as - // a ParquetFilePath - .flat_map(ParquetFilePath::from_absolute) - .collect() - }) - .boxed()) - } - - /// Get the parquet file data in this relative path in this database's object store. - pub async fn get_parquet_file( - &self, - location: &ParquetFilePath, - ) -> Result> { - self.inner.get(&self.full_parquet_path(location)).await - } - - /// Store the data for this parquet file in this database's object store. - pub async fn put_parquet_file(&self, location: &ParquetFilePath, bytes: Bytes) -> Result<()> { - self.inner - .put(&self.full_parquet_path(location), bytes) - .await - } - - /// Remove the data for this parquet file from this database's object store - pub async fn delete_parquet_file(&self, location: &ParquetFilePath) -> Result<()> { - self.inner.delete(&self.full_parquet_path(location)).await - } - - fn full_parquet_path(&self, location: &ParquetFilePath) -> Path { - if location.is_new_gen() { - self.inner - .deref() - .path_from_dirs_and_filename(location.absolute_dirs_and_file_name()) - } else { - self.data_path.join(location) - } - } - - // Database rule file methods ================================================================= - - // Deliberately private; this should not leak outside this crate - // so assumptions about the object store organization are confined - // (and can be changed) in this crate - fn db_rules_path(&self) -> Path { - self.root_path.rules_path().inner - } - - /// Get the data for the database rules - pub async fn get_database_rules_file(&self) -> Result { - let path = &self.db_rules_path(); - Ok(self.inner.get(path).await?.bytes().await?.into()) - } - - /// Return the database rules file content without creating an IoxObjectStore instance. Useful - /// when restoring a database given a UUID to check existence of the specified database and - /// get information such as the database name from the rules before proceeding with restoring - /// and initializing the database. - pub async fn load_database_rules(inner: Arc, uuid: Uuid) -> Result { - let root_path = Self::root_path_for(&*inner, uuid); - let db_rules_path = root_path.rules_path().inner; - - Ok(inner.get(&db_rules_path).await?.bytes().await?.into()) - } - - /// Store the data for the database rules - pub async fn put_database_rules_file(&self, bytes: Bytes) -> Result<()> { - self.inner.put(&self.db_rules_path(), bytes).await - } - - /// Delete the data for the database rules - pub async fn delete_database_rules_file(&self) -> Result<()> { - self.inner.delete(&self.db_rules_path()).await - } - - /// List the relative paths in this database's object store. - /// - // Deliberately private; this should not leak outside this crate - // so assumptions about the object store organization are confined - // (and can be changed) in this crate - /// All outside calls should go to one of the more specific listing methods. - async fn list(&self, prefix: Option<&Path>) -> Result>>> { - let (tx, rx) = channel(4); - let inner = Arc::clone(&self.inner); - let prefix = prefix.cloned(); - - // This is necessary because of the lifetime restrictions on the ObjectStoreApi trait's - // methods, which might not actually be necessary but fixing it involves changes to the - // cloud_storage crate that are longer term. - tokio::spawn(async move { - match inner.list(prefix.as_ref()).await { - Err(e) => { - let _ = tx.send(Err(e)).await; - } - Ok(mut stream) => { - while let Some(list) = stream.next().await { - let _ = tx.send(list).await; - } - } - } - }); - - Ok(ReceiverStream::new(rx).boxed()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::paths::ALL_DATABASES_DIRECTORY; - use data_types::chunk_metadata::{ChunkAddr, ChunkId}; - use data_types2::{NamespaceId, PartitionId, SequencerId, TableId}; - use object_store::{parsed_path, path::ObjectStorePath, ObjectStoreImpl}; - use test_helpers::assert_error; - use uuid::Uuid; - - /// Creates a new in-memory object store - fn make_object_store() -> Arc { - Arc::new(ObjectStoreImpl::new_in_memory()) - } - - async fn add_file(object_store: &DynObjectStore, location: &Path) { - let data = Bytes::from("arbitrary data"); - - object_store.put(location, data).await.unwrap(); - } - - async fn parquet_files(iox_object_store: &IoxObjectStore) -> Vec { - iox_object_store - .parquet_files() - .await - .unwrap() - .try_collect::>() - .await - .unwrap() - .into_iter() - .flatten() - .collect() - } - - async fn add_parquet_file(iox_object_store: &IoxObjectStore, location: &ParquetFilePath) { - let data = Bytes::from("arbitrary data"); - - iox_object_store - .put_parquet_file(location, data) - .await - .unwrap(); - } - - #[tokio::test] - async fn only_lists_relevant_parquet_files() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let uuid_string = uuid.to_string(); - let uuid_str = uuid_string.as_str(); - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - let parquet_uuid = Uuid::new_v4(); - let good_filename = format!("111.{}.parquet", parquet_uuid); - let good_filename_str = good_filename.as_str(); - - // Put a non-database file in - let path = object_store.path_from_dirs_and_filename(parsed_path!(["foo"])); - add_file(&*object_store, &path).await; - - // Put a file for some other server in - let path = object_store.path_from_dirs_and_filename(parsed_path!(["12345"])); - add_file(&*object_store, &path).await; - - // Put a file for some other database in - let other_db_uuid = Uuid::new_v4().to_string(); - let path = object_store.path_from_dirs_and_filename(parsed_path!([ - ALL_DATABASES_DIRECTORY, - other_db_uuid.as_str() - ])); - add_file(&*object_store, &path).await; - - // Put a file in the database dir but not the data dir - let path = object_store.path_from_dirs_and_filename(parsed_path!( - [ALL_DATABASES_DIRECTORY, uuid_str], - good_filename_str - )); - add_file(&*object_store, &path).await; - - // Put files in the data dir whose names are in the wrong format - let mut path = object_store.path_from_dirs_and_filename(parsed_path!( - [ALL_DATABASES_DIRECTORY, uuid_str, "data"], - "111.parquet" - )); - add_file(&*object_store, &path).await; - path.set_file_name(&format!("111.{}.xls", parquet_uuid)); - add_file(&*object_store, &path).await; - - // Parquet files should be empty - let pf = parquet_files(&iox_object_store).await; - assert!(pf.is_empty(), "{:?}", pf); - - // Add a real parquet file - let chunk_addr = ChunkAddr { - db_name: "clouds".into(), - table_name: "my_table".into(), - partition_key: "my_partition".into(), - chunk_id: ChunkId::new_test(13), - }; - let p1 = ParquetFilePath::new_old_gen(&chunk_addr); - add_parquet_file(&iox_object_store, &p1).await; - - // Only the real file should be returned - let pf = parquet_files(&iox_object_store).await; - assert_eq!(&pf, &[p1]); - } - - async fn catalog_transaction_files( - iox_object_store: &IoxObjectStore, - ) -> Vec { - iox_object_store - .catalog_transaction_files() - .await - .unwrap() - .try_collect::>() - .await - .unwrap() - .into_iter() - .flatten() - .collect() - } - - async fn add_catalog_transaction_file( - iox_object_store: &IoxObjectStore, - location: &TransactionFilePath, - ) { - let data = Bytes::from("arbitrary data"); - - iox_object_store - .put_catalog_transaction_file(location, data) - .await - .unwrap(); - } - - #[tokio::test] - async fn only_lists_relevant_catalog_transaction_files() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let uuid_string = uuid.to_string(); - let uuid_str = uuid_string.as_str(); - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - let txn_uuid = Uuid::new_v4(); - let good_txn_filename = format!("{}.txn", txn_uuid); - let good_txn_filename_str = good_txn_filename.as_str(); - - // Put a non-database file in - let path = object_store.path_from_dirs_and_filename(parsed_path!(["foo"])); - add_file(&*object_store, &path).await; - - // Put a file in a directory other than the databases directory - let path = object_store.path_from_dirs_and_filename(parsed_path!(["12345"])); - add_file(&*object_store, &path).await; - - // Put a file for some other database in - let other_db_uuid = Uuid::new_v4().to_string(); - let path = object_store.path_from_dirs_and_filename(parsed_path!([ - ALL_DATABASES_DIRECTORY, - other_db_uuid.as_str() - ])); - add_file(&*object_store, &path).await; - - // Put a file in the database dir but not the transactions dir - let path = object_store.path_from_dirs_and_filename(parsed_path!( - [ALL_DATABASES_DIRECTORY, uuid_str], - good_txn_filename_str - )); - add_file(&*object_store, &path).await; - - // Put files in the transactions dir whose names are in the wrong format - let mut path = object_store.path_from_dirs_and_filename(parsed_path!( - [ALL_DATABASES_DIRECTORY, uuid_str], - "111.parquet" - )); - add_file(&*object_store, &path).await; - path.set_file_name(&format!("{}.xls", txn_uuid)); - add_file(&*object_store, &path).await; - - // Catalog transaction files should be empty - let ctf = catalog_transaction_files(&iox_object_store).await; - assert!(ctf.is_empty(), "{:?}", ctf); - - // Add a real transaction file - let t1 = TransactionFilePath::new_transaction(123, txn_uuid); - add_catalog_transaction_file(&iox_object_store, &t1).await; - // Add a real checkpoint file - let t2 = TransactionFilePath::new_checkpoint(123, txn_uuid); - add_catalog_transaction_file(&iox_object_store, &t2).await; - - // Only the real files should be returned - let ctf = catalog_transaction_files(&iox_object_store).await; - assert_eq!(ctf.len(), 2); - assert!(ctf.contains(&t1)); - assert!(ctf.contains(&t2)); - } - - fn make_db_rules_path(object_store: &DynObjectStore, uuid: Uuid) -> Path { - let mut p = object_store.new_path(); - p.push_all_dirs(&[ALL_DATABASES_DIRECTORY, uuid.to_string().as_str()]); - p.set_file_name("rules.pb"); - p - } - - #[tokio::test] - async fn db_rules_should_be_a_file() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let rules_path = make_db_rules_path(&*object_store, uuid); - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - - // PUT - let original_file_content = Bytes::from("hello world"); - iox_object_store - .put_database_rules_file(original_file_content.clone()) - .await - .unwrap(); - - let actual_content = object_store - .get(&rules_path) - .await - .unwrap() - .bytes() - .await - .unwrap(); - - assert_eq!(original_file_content, actual_content); - - // GET - let updated_file_content = Bytes::from("goodbye moon"); - let expected_content = updated_file_content.clone(); - - object_store - .put(&rules_path, updated_file_content) - .await - .unwrap(); - - let actual_content = iox_object_store.get_database_rules_file().await.unwrap(); - - assert_eq!(expected_content, actual_content); - - // DELETE - iox_object_store.delete_database_rules_file().await.unwrap(); - - let file_count = object_store - .list(None) - .await - .unwrap() - .try_fold(0, |a, paths| async move { Ok(a + paths.len()) }) - .await - .unwrap(); - - assert_eq!(file_count, 0); - } - - fn make_owner_path(object_store: &DynObjectStore, uuid: Uuid) -> Path { - let mut p = object_store.new_path(); - p.push_all_dirs(&[ALL_DATABASES_DIRECTORY, uuid.to_string().as_str()]); - p.set_file_name("owner.pb"); - p - } - - #[tokio::test] - async fn owner_should_be_a_file() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let owner_path = make_owner_path(&*object_store, uuid); - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - - // PUT - let original_file_content = Bytes::from("hello world"); - iox_object_store - .put_owner_file(original_file_content.clone()) - .await - .unwrap(); - - let actual_content = object_store - .get(&owner_path) - .await - .unwrap() - .bytes() - .await - .unwrap(); - - assert_eq!(original_file_content, actual_content); - - // GET - let updated_file_content = Bytes::from("goodbye moon"); - let expected_content = updated_file_content.clone(); - - object_store - .put(&owner_path, updated_file_content) - .await - .unwrap(); - - let actual_content = iox_object_store.get_owner_file().await.unwrap(); - - assert_eq!(expected_content, actual_content); - } - - #[tokio::test] - async fn create_new_with_same_uuid_errors() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - - iox_object_store - .put_database_rules_file(Bytes::new()) - .await - .unwrap(); - - assert_error!( - IoxObjectStore::create(Arc::clone(&object_store), uuid).await, - IoxObjectStoreError::DatabaseAlreadyExists { uuid: err_uuid } if err_uuid == uuid, - ); - } - - #[tokio::test] - async fn create_new_with_any_files_under_uuid_errors() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - - let mut not_rules_path = object_store.new_path(); - not_rules_path.push_all_dirs(&[ALL_DATABASES_DIRECTORY, uuid.to_string().as_str()]); - not_rules_path.set_file_name("not_rules.txt"); - object_store - .put(¬_rules_path, Bytes::new()) - .await - .unwrap(); - - assert_error!( - IoxObjectStore::create(Arc::clone(&object_store), uuid).await, - IoxObjectStoreError::DatabaseAlreadyExists { uuid: err_uuid } if err_uuid == uuid, - ); - } - - async fn create_database(object_store: Arc, uuid: Uuid) -> IoxObjectStore { - let iox_object_store = IoxObjectStore::create(Arc::clone(&object_store), uuid) - .await - .unwrap(); - - iox_object_store - .put_database_rules_file(Bytes::new()) - .await - .unwrap(); - - iox_object_store - } - - #[tokio::test] - async fn cant_read_rules_if_no_rules_exist() { - let object_store = make_object_store(); - - // Create a uuid but don't create a corresponding database - let db = Uuid::new_v4(); - - // This fails, there are no rules to read - assert_error!( - IoxObjectStore::load_database_rules(object_store, db).await, - object_store::Error::NotFound { .. }, - ); - } - - #[tokio::test] - async fn test_load() { - let object_store = make_object_store(); - - // Load can't find nonexistent database - let nonexistent = Uuid::new_v4(); - assert_error!( - IoxObjectStore::load(Arc::clone(&object_store), nonexistent).await, - IoxObjectStoreError::NoRulesFound { .. }, - ); - - // Create a database - let db = Uuid::new_v4(); - create_database(Arc::clone(&object_store), db).await; - - // Load should return that database - let returned = IoxObjectStore::load(Arc::clone(&object_store), db) - .await - .unwrap(); - assert_eq!( - returned.root_path(), - format!("{}/{}/", ALL_DATABASES_DIRECTORY, db) - ); - } - - #[tokio::test] - async fn round_trip_through_object_store_root_path() { - let object_store = make_object_store(); - - // Create a new iox object store that doesn't exist yet - let uuid = Uuid::new_v4(); - let db_iox_store = create_database(Arc::clone(&object_store), uuid).await; - - // Save its root path as the server config would - let saved_root_path = db_iox_store.root_path(); - - // Simulate server restarting and reading the server config to construct iox object stores, - // the database files in object storage should be found in the same root - let restarted_iox_store = - IoxObjectStore::load_at_root_path(Arc::clone(&object_store), &saved_root_path) - .await - .unwrap(); - assert_eq!(db_iox_store.root_path(), restarted_iox_store.root_path()); - - // This should also equal root_path_for, which can be constructed even if a database - // hasn't been fully initialized yet - let alternate = IoxObjectStore::root_path_for(&*object_store, uuid).to_string(); - assert_eq!(alternate, saved_root_path); - } - - #[tokio::test] - async fn test_ng_parquet_io() { - let object_store = make_object_store(); - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(&object_store), - IoxObjectStore::root_path_for(&*object_store, uuid::Uuid::new_v4()), - )); - - let pfp = ParquetFilePath::new_new_gen( - NamespaceId::new(1), - TableId::new(2), - SequencerId::new(3), - PartitionId::new(4), - Uuid::nil(), - ); - - // file does not exist yet - iox_object_store.get_parquet_file(&pfp).await.unwrap_err(); - - // create file - let content = Bytes::from(b"foo".to_vec()); - iox_object_store - .put_parquet_file(&pfp, content.clone()) - .await - .unwrap(); - - let actual = iox_object_store - .get_parquet_file(&pfp) - .await - .unwrap() - .bytes() - .await - .unwrap(); - assert_eq!(content.to_vec(), actual); - - // delete file - iox_object_store.delete_parquet_file(&pfp).await.unwrap(); - iox_object_store.get_parquet_file(&pfp).await.unwrap_err(); - } -} diff --git a/iox_object_store/src/paths.rs b/iox_object_store/src/paths.rs deleted file mode 100644 index 8672a9cd7a..0000000000 --- a/iox_object_store/src/paths.rs +++ /dev/null @@ -1,233 +0,0 @@ -//! Paths for specific types of files within a database's object storage. - -use data_types::server_id::ServerId; -use object_store::{ - path::{ObjectStorePath, Path}, - DynObjectStore, -}; -use std::fmt; -use uuid::Uuid; - -pub mod parquet_file; -use parquet_file::ParquetFilePath; - -pub mod transaction_file; -use transaction_file::TransactionFilePath; - -pub(crate) const ALL_DATABASES_DIRECTORY: &str = "dbs"; -const ALL_SERVERS_DIRECTORY: &str = "nodes"; -pub(crate) const SERVER_CONFIG_FILE_NAME: &str = "config.pb"; -const DATABASE_OWNER_FILE_NAME: &str = "owner.pb"; - -/// The path to the server file containing the list of databases this server owns. -// TODO: this is in the process of replacing all_databases_path for the floating databases design -pub(crate) fn server_config_path(object_store: &DynObjectStore, server_id: ServerId) -> Path { - let mut path = object_store.new_path(); - path.push_dir(ALL_SERVERS_DIRECTORY); - path.push_dir(server_id.to_string()); - path.set_file_name(SERVER_CONFIG_FILE_NAME); - path -} - -/// A database-specific object store path that all `IoxObjectStore` `Path`s should be within. -/// This can be serialized to facilitate initial loading of a database from object storage, but -/// the path should not be parsed into its component parts as the format might change. -#[derive(Debug, Clone, PartialEq)] -pub struct RootPath { - pub(crate) inner: Path, -} - -impl RootPath { - /// How the root of a database is defined in object storage. - pub(crate) fn new(object_store: &DynObjectStore, uuid: Uuid) -> Self { - let mut inner = object_store.new_path(); - inner.push_dir(ALL_DATABASES_DIRECTORY); - inner.push_dir(uuid.to_string()); - Self { inner } - } - - pub(crate) fn from_str(object_store: &DynObjectStore, raw: &str) -> Self { - Self { - inner: object_store.path_from_raw(raw), - } - } - - fn join(&self, dir: &str) -> Path { - let mut result = self.inner.clone(); - result.push_dir(dir); - result - } - - pub(crate) fn owner_path(&self) -> Path { - let mut result = self.inner.clone(); - result.set_file_name(DATABASE_OWNER_FILE_NAME); - result - } - - pub(crate) fn rules_path(&self) -> RulesPath { - RulesPath::new(self) - } - - pub(crate) fn data_path(&self) -> DataPath { - DataPath::new(self) - } - - pub(crate) fn transactions_path(&self) -> TransactionsPath { - TransactionsPath::new(self) - } -} - -impl fmt::Display for RootPath { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.inner.to_raw()) - } -} - -#[derive(Debug, Clone)] -pub(crate) struct RulesPath { - pub(crate) inner: Path, -} - -impl RulesPath { - const DB_RULES_FILE_NAME: &'static str = "rules.pb"; - - /// How the rules path of a database is defined in object storage in terms of the - /// root path. - pub(crate) fn new(root_path: &RootPath) -> Self { - Self::new_from_object_store_path(&root_path.inner) - } - - /// Creating a potential rules file location given an object storage path received from - /// an object storage list operation. - pub(crate) fn new_from_object_store_path(path: &Path) -> Self { - let mut inner = path.clone(); - inner.set_file_name(Self::DB_RULES_FILE_NAME); - Self { inner } - } -} - -/// A database-specific object store path for all catalog transaction files. This should not be -/// leaked outside this crate. -#[derive(Debug, Clone)] -pub(crate) struct TransactionsPath { - pub(crate) inner: Path, -} - -impl TransactionsPath { - /// How the transactions path of a database is defined in object storage in terms of the - /// root path. - pub(crate) fn new(root_path: &RootPath) -> Self { - Self { - inner: root_path.join("transactions"), - } - } - - pub(crate) fn join(&self, transaction_file_path: &TransactionFilePath) -> Path { - let mut result = self.inner.clone(); - let relative = transaction_file_path.relative_dirs_and_file_name(); - for part in relative.directories { - result.push_dir(part.to_string()); - } - result.set_file_name( - relative - .file_name - .expect("Transaction file paths have filenames") - .to_string(), - ); - result - } -} - -/// A database-specific object store path for all data files. This should not be leaked outside -/// this crate. -#[derive(Debug, Clone)] -pub(crate) struct DataPath { - pub(crate) inner: Path, -} - -impl DataPath { - /// How the data path of a database is defined in object storage in terms of the root path. - pub(crate) fn new(root_path: &RootPath) -> Self { - Self { - inner: root_path.join("data"), - } - } - - pub(crate) fn join(&self, parquet_file_path: &ParquetFilePath) -> Path { - let mut result = self.inner.clone(); - let relative = parquet_file_path.relative_dirs_and_file_name(); - for part in relative.directories { - result.push_dir(part.to_string()); - } - result.set_file_name( - relative - .file_name - .expect("Parquet file paths have filenames") - .to_string(), - ); - result - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::IoxObjectStore; - use object_store::ObjectStoreImpl; - use std::sync::Arc; - - /// Creates a new in-memory object store. These tests rely on the `Path`s being of type - /// `DirsAndFileName` and thus using object_store::path::DELIMITER as the separator - fn make_object_store() -> Arc { - Arc::new(ObjectStoreImpl::new_in_memory()) - } - - #[test] - fn root_path_contains_dbs_and_db_uuid() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let root_path = RootPath::new(&*object_store, uuid); - - assert_eq!( - root_path.inner.to_string(), - format!("mem:{}/{}/", ALL_DATABASES_DIRECTORY, uuid) - ); - } - - #[test] - fn root_path_join_concatenates() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let root_path = RootPath::new(&*object_store, uuid); - - let path = root_path.join("foo"); - assert_eq!( - path.to_string(), - format!("mem:{}/{}/foo/", ALL_DATABASES_DIRECTORY, uuid) - ); - } - - #[test] - fn transactions_path_is_relative_to_root_path() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let root_path = RootPath::new(&*object_store, uuid); - let iox_object_store = IoxObjectStore::existing(Arc::clone(&object_store), root_path); - assert_eq!( - iox_object_store.transactions_path.inner.to_string(), - format!("mem:{}/{}/transactions/", ALL_DATABASES_DIRECTORY, uuid) - ); - } - - #[test] - fn data_path_is_relative_to_root_path() { - let object_store = make_object_store(); - let uuid = Uuid::new_v4(); - let root_path = RootPath::new(&*object_store, uuid); - let iox_object_store = IoxObjectStore::existing(Arc::clone(&object_store), root_path); - assert_eq!( - iox_object_store.data_path.inner.to_string(), - format!("mem:{}/{}/data/", ALL_DATABASES_DIRECTORY, uuid) - ); - } -} diff --git a/iox_object_store/src/paths/parquet_file.rs b/iox_object_store/src/paths/parquet_file.rs deleted file mode 100644 index 3881d10609..0000000000 --- a/iox_object_store/src/paths/parquet_file.rs +++ /dev/null @@ -1,460 +0,0 @@ -use data_types::chunk_metadata::{ChunkAddr, ChunkId}; -use data_types2::{NamespaceId, PartitionId, SequencerId, TableId}; -use object_store::{ - path::{parsed::DirsAndFileName, ObjectStorePath, Path as ObjStoPath}, - Result, -}; -use snafu::{ensure, OptionExt, ResultExt, Snafu}; -use std::sync::Arc; -use uuid::Uuid; - -#[derive(Debug, Clone, Eq, PartialEq, Hash, Ord, PartialOrd)] -enum Variant { - Old { - table_name: Arc, - partition_key: Arc, - chunk_id: ChunkId, - }, - New { - namespace_id: NamespaceId, - table_id: TableId, - sequencer_id: SequencerId, - partition_id: PartitionId, - object_store_id: Uuid, - }, -} - -/// Location of a Parquet file within a database's object store. -/// The exact format is an implementation detail and is subject to change. -#[derive(Clone, Eq, PartialEq, Hash, Ord, PartialOrd)] -pub struct ParquetFilePath(Variant); - -impl std::fmt::Debug for ParquetFilePath { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let path = match self.0 { - Variant::Old { .. } => self.relative_dirs_and_file_name().to_string(), - Variant::New { .. } => self.absolute_dirs_and_file_name().to_string(), - }; - f.debug_struct("ParquetFilePath") - .field("inner", &self.0) - .field("resolved_path", &path) - .finish() - } -} - -impl ParquetFilePath { - /// Create a location for this chunk's parquet file. Calling this twice on the same `ChunkAddr` - /// will return different `parquet_file::Path`s. - pub fn new_old_gen(chunk_addr: &ChunkAddr) -> Self { - Self(Variant::Old { - table_name: Arc::clone(&chunk_addr.table_name), - partition_key: Arc::clone(&chunk_addr.partition_key), - chunk_id: chunk_addr.chunk_id, - }) - } - - /// Create parquet file path relevant for the NG storage layout. - pub fn new_new_gen( - namespace_id: NamespaceId, - table_id: TableId, - sequencer_id: SequencerId, - partition_id: PartitionId, - object_store_id: Uuid, - ) -> Self { - Self(Variant::New { - namespace_id, - table_id, - sequencer_id, - partition_id, - object_store_id, - }) - } - - /// Checks if this is an NG-style path. - pub fn is_new_gen(&self) -> bool { - matches!(self.0, Variant::New { .. }) - } - - /// Turn this into directories and file names to be added to a root path or to be serialized - /// in protobuf. - /// - /// # Panic - /// Panics if this is an NG-style path. - pub fn relative_dirs_and_file_name(&self) -> DirsAndFileName { - match &self.0 { - Variant::Old { - table_name, - partition_key, - chunk_id, - } => { - let mut result = DirsAndFileName::default(); - result.push_all_dirs(&[table_name.as_ref(), partition_key.as_ref()]); - result.set_file_name(format!("{}.parquet", chunk_id.get())); - result - } - Variant::New { .. } => { - panic!("relative dirs don't apply to new-gen parquet file paths") - } - } - } - - /// Get absolute storage location. - /// - /// # Panic - /// Panics if this is an old-style path. - pub fn absolute_dirs_and_file_name(&self) -> DirsAndFileName { - match &self.0 { - Variant::Old { .. } => { - panic!("absolute dirs don't apply to old-gen parquet file paths") - } - Variant::New { - namespace_id, - table_id, - sequencer_id, - partition_id, - object_store_id, - } => { - let mut result = DirsAndFileName::default(); - result.push_all_dirs(&[ - namespace_id.to_string().as_str(), - table_id.to_string().as_str(), - sequencer_id.to_string().as_str(), - partition_id.to_string().as_str(), - ]); - result.set_file_name(format!("{}.parquet", object_store_id)); - result - } - } - } - - /// Create from serialized protobuf strings. - pub fn from_relative_dirs_and_file_name( - dirs_and_file_name: &DirsAndFileName, - ) -> Result { - let mut directories = dirs_and_file_name.directories.iter(); - let table_name = directories - .next() - .context(MissingTableNameSnafu)? - .to_string() - .into(); - let partition_key = directories - .next() - .context(MissingPartitionKeySnafu)? - .to_string() - .into(); - - ensure!(directories.next().is_none(), UnexpectedDirectorySnafu); - - let file_name = dirs_and_file_name - .file_name - .as_ref() - .context(MissingChunkIdSnafu)? - .to_string(); - let mut parts = file_name.split('.'); - let chunk_id = parts - .next() - .context(MissingChunkIdSnafu)? - .parse::() - .context(InvalidChunkIdSnafu)? - .into(); - let ext = parts.next().context(MissingExtensionSnafu)?; - ensure!(ext == "parquet", InvalidExtensionSnafu { ext }); - ensure!(parts.next().is_none(), UnexpectedExtensionSnafu); - - Ok(Self(Variant::Old { - table_name, - partition_key, - chunk_id, - })) - } - - // Deliberately pub(crate); this transformation should only happen within this crate - pub(crate) fn from_absolute( - absolute_path: ObjStoPath, - ) -> Result { - let absolute_path: DirsAndFileName = absolute_path.into(); - - let mut absolute_dirs = absolute_path.directories.into_iter().fuse(); - - // The number of `next`s here needs to match the total number of directories in - // iox_object_store data_paths - absolute_dirs.next(); // server id - absolute_dirs.next(); // database uuid - absolute_dirs.next(); // "data" - - let remaining = DirsAndFileName { - directories: absolute_dirs.collect(), - file_name: absolute_path.file_name, - }; - - Self::from_relative_dirs_and_file_name(&remaining) - } -} - -impl From<&Self> for ParquetFilePath { - fn from(borrowed: &Self) -> Self { - borrowed.clone() - } -} - -#[derive(Snafu, Debug, PartialEq)] -#[allow(missing_docs)] -pub enum ParquetFilePathParseError { - #[snafu(display("Could not find required table name"))] - MissingTableName, - - #[snafu(display("Could not find required partition key"))] - MissingPartitionKey, - - #[snafu(display("Too many directories found"))] - UnexpectedDirectory, - - #[snafu(display("Could not find required chunk id"))] - MissingChunkId, - - #[snafu(display("Could not parse chunk id: {}", source))] - InvalidChunkId { source: uuid::Error }, - - #[snafu(display("Could not find required file extension"))] - MissingExtension, - - #[snafu(display("Extension should have been `parquet`, instead found `{}`", ext))] - InvalidExtension { ext: String }, - - #[snafu(display("Too many extensions found"))] - UnexpectedExtension, -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::{paths::ALL_DATABASES_DIRECTORY, IoxObjectStore, RootPath}; - use object_store::{DynObjectStore, ObjectStoreImpl}; - use test_helpers::assert_error; - - /// Creates a new in-memory object store. These tests rely on the `Path`s being of type - /// `DirsAndFileName` and thus using object_store::path::DELIMITER as the separator - fn make_object_store() -> Arc { - Arc::new(ObjectStoreImpl::new_in_memory()) - } - - #[test] - fn test_parquet_file_path_deserialization() { - // Error cases - use ParquetFilePathParseError::*; - - let mut df = DirsAndFileName::default(); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!(matches!(result, Err(MissingTableName)), "got {:?}", result); - - df.push_dir("foo"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(MissingPartitionKey)), - "got {:?}", - result - ); - - df.push_dir("bar"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(MissingChunkId { .. })), - "got {:?}", - result - ); - - let mut extra = df.clone(); - extra.push_dir("nope"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&extra); - assert!( - matches!(result, Err(UnexpectedDirectory)), - "got {:?}", - result - ); - - df.set_file_name("bleh"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(InvalidChunkId { .. })), - "got {:?}", - result - ); - - df.set_file_name("00000000-0000-0000-0000-00000000000a"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!(matches!(result, Err(MissingExtension)), "got {:?}", result); - - df.set_file_name("00000000-0000-0000-0000-00000000000a.exe"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(InvalidExtension { .. })), - "got {:?}", - result - ); - - df.set_file_name("00000000-0000-0000-0000-00000000000a.parquet.v6"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(UnexpectedExtension)), - "got {:?}", - result - ); - - // Success case - df.set_file_name("00000000-0000-0000-0000-00000000000a.parquet"); - let result = ParquetFilePath::from_relative_dirs_and_file_name(&df).unwrap(); - assert_eq!( - result, - ParquetFilePath(Variant::Old { - table_name: "foo".into(), - partition_key: "bar".into(), - chunk_id: ChunkId::new_test(10), - }) - ); - let round_trip = result.relative_dirs_and_file_name(); - assert_eq!(round_trip, df); - } - - #[test] - fn parquet_file_from_absolute() { - let object_store = make_object_store(); - - // Success case - let mut path = object_store.new_path(); - path.push_all_dirs(&["server", "uuid", "data", "}*", "aoeu"]); - path.set_file_name("00000000-0000-0000-0000-00000000000a.parquet"); - let result = ParquetFilePath::from_absolute(path); - assert_eq!( - result.unwrap(), - ParquetFilePath(Variant::Old { - table_name: "}*".into(), - partition_key: "aoeu".into(), - chunk_id: ChunkId::new_test(10), - }) - ); - - // Error cases - use ParquetFilePathParseError::*; - - let mut path = object_store.new_path(); - // incorrect directories are fine, we're assuming that list(data_path) scoped to the - // right directories so we don't check again on the way out - path.push_all_dirs(&["server", "uuid", "data", "}*", "aoeu"]); - // but this file name doesn't contain a chunk id - path.set_file_name("rules.pb"); - assert_error!(ParquetFilePath::from_absolute(path), InvalidChunkId { .. }); - - let mut path = object_store.new_path(); - path.push_all_dirs(&["server", "uuid", "data", "}*", "aoeu"]); - // missing file name - assert_error!(ParquetFilePath::from_absolute(path), MissingChunkId); - } - - #[test] - fn parquet_file_relative_dirs_and_file_path() { - let pfp = ParquetFilePath(Variant::Old { - table_name: "}*".into(), - partition_key: "aoeu".into(), - chunk_id: ChunkId::new_test(10), - }); - let dirs_and_file_name = pfp.relative_dirs_and_file_name(); - assert_eq!( - dirs_and_file_name.to_string(), - "%7D%2A/aoeu/00000000-0000-0000-0000-00000000000a.parquet".to_string(), - ); - let round_trip = - ParquetFilePath::from_relative_dirs_and_file_name(&dirs_and_file_name).unwrap(); - assert_eq!(pfp, round_trip); - - assert_eq!(format!("{:?}", pfp), "ParquetFilePath { inner: Old { table_name: \"}*\", partition_key: \"aoeu\", chunk_id: ChunkId(10) }, resolved_path: \"%7D%2A/aoeu/00000000-0000-0000-0000-00000000000a.parquet\" }"); - } - - #[test] - #[should_panic(expected = "relative dirs don't apply to new-gen parquet file paths")] - fn parquet_file_relative_dirs_and_file_path_new_gen() { - let pfp = ParquetFilePath(Variant::New { - namespace_id: NamespaceId::new(1), - table_id: TableId::new(2), - sequencer_id: SequencerId::new(3), - partition_id: PartitionId::new(4), - object_store_id: Uuid::nil(), - }); - pfp.relative_dirs_and_file_name(); - } - - #[test] - fn parquet_file_absolute_dirs_and_file_path() { - let pfp = ParquetFilePath(Variant::New { - namespace_id: NamespaceId::new(1), - table_id: TableId::new(2), - sequencer_id: SequencerId::new(3), - partition_id: PartitionId::new(4), - object_store_id: Uuid::nil(), - }); - let dirs_and_file_name = pfp.absolute_dirs_and_file_name(); - assert_eq!( - dirs_and_file_name.to_string(), - "1/2/3/4/00000000-0000-0000-0000-000000000000.parquet".to_string(), - ); - - assert_eq!(format!("{:?}", pfp), "ParquetFilePath { inner: New { namespace_id: NamespaceId(1), table_id: TableId(2), sequencer_id: SequencerId(3), partition_id: PartitionId(4), object_store_id: 00000000-0000-0000-0000-000000000000 }, resolved_path: \"1/2/3/4/00000000-0000-0000-0000-000000000000.parquet\" }"); - } - - #[test] - #[should_panic(expected = "absolute dirs don't apply to old-gen parquet file paths")] - fn parquet_file_absolute_dirs_and_file_path_old_gen() { - let pfp = ParquetFilePath(Variant::Old { - table_name: "}*".into(), - partition_key: "aoeu".into(), - chunk_id: ChunkId::new_test(10), - }); - pfp.absolute_dirs_and_file_name(); - } - - #[test] - fn parquet_file_is_new_gen() { - let pfp = ParquetFilePath(Variant::Old { - table_name: "}*".into(), - partition_key: "aoeu".into(), - chunk_id: ChunkId::new_test(10), - }); - assert!(!pfp.is_new_gen()); - - let pfp = ParquetFilePath(Variant::New { - namespace_id: NamespaceId::new(1), - table_id: TableId::new(2), - sequencer_id: SequencerId::new(3), - partition_id: PartitionId::new(4), - object_store_id: Uuid::nil(), - }); - assert!(pfp.is_new_gen()); - } - - #[test] - fn data_path_join_with_parquet_file_path() { - let db_uuid = Uuid::new_v4(); - let object_store = make_object_store(); - let root_path = RootPath::new(&*object_store, db_uuid); - let iox_object_store = IoxObjectStore::existing(Arc::clone(&object_store), root_path); - - let pfp = ParquetFilePath(Variant::Old { - table_name: "}*".into(), - partition_key: "aoeu".into(), - chunk_id: ChunkId::new_test(10), - }); - - let path = iox_object_store.data_path.join(&pfp); - - let mut expected_path = object_store.new_path(); - expected_path.push_all_dirs(&[ - ALL_DATABASES_DIRECTORY, - &db_uuid.to_string(), - "data", - "}*", - "aoeu", - ]); - expected_path.set_file_name("00000000-0000-0000-0000-00000000000a.parquet"); - - assert_eq!(path, expected_path); - } -} diff --git a/iox_object_store/src/paths/transaction_file.rs b/iox_object_store/src/paths/transaction_file.rs deleted file mode 100644 index 2ec55c868f..0000000000 --- a/iox_object_store/src/paths/transaction_file.rs +++ /dev/null @@ -1,384 +0,0 @@ -use object_store::{ - path::{parsed::DirsAndFileName, ObjectStorePath, Path as ObjStoPath}, - Result, -}; -use snafu::{ensure, OptionExt, ResultExt, Snafu}; -use std::str::FromStr; -use uuid::Uuid; - -/// File suffix for transaction files in object store. -const TRANSACTION_FILE_SUFFIX: &str = "txn"; - -/// File suffix for checkpoint files in object store. -const CHECKPOINT_FILE_SUFFIX: &str = "ckpt"; - -/// Location of a catalog transaction file within a database's object store. -/// The exact format is an implementation detail and is subject to change. -#[derive(Debug, Clone, Copy, PartialEq)] -pub struct TransactionFilePath { - /// Transaction revision - pub revision_counter: u64, - /// Transaction identifier - pub uuid: Uuid, - suffix: TransactionFileSuffix, -} - -impl TransactionFilePath { - /// Create a new file path to store transaction info. - pub fn new_transaction(revision_counter: u64, uuid: Uuid) -> Self { - Self { - revision_counter, - uuid, - suffix: TransactionFileSuffix::Transaction, - } - } - - /// Create a new file path to store checkpoint info. - pub fn new_checkpoint(revision_counter: u64, uuid: Uuid) -> Self { - Self { - revision_counter, - uuid, - suffix: TransactionFileSuffix::Checkpoint, - } - } - - /// Returns true if this path is to a checkpoint file; false otherwise. - pub fn is_checkpoint(&self) -> bool { - self.suffix == TransactionFileSuffix::Checkpoint - } - - /// Turn this into directories and file names to be added to a root path - pub fn relative_dirs_and_file_name(&self) -> DirsAndFileName { - let mut result = DirsAndFileName::default(); - - // pad number: `u64::MAX.to_string().len()` is 20 - result.push_dir(format!("{:0>20}", self.revision_counter)); - - let file_name = format!("{}.{}", self.uuid, self.suffix.as_str()); - result.set_file_name(file_name); - result - } - - /// Create from serialized protobuf strings. - pub fn from_relative_dirs_and_file_name( - dirs_and_file_name: &DirsAndFileName, - ) -> Result { - let mut directories = dirs_and_file_name.directories.iter(); - - let revision_counter = directories - .next() - .context(MissingRevisionCounterSnafu)? - .to_string() - .parse() - .context(InvalidRevisionCounterSnafu)?; - - ensure!(directories.next().is_none(), UnexpectedDirectorySnafu); - - let file_name = dirs_and_file_name - .file_name - .as_ref() - .context(MissingFileNameSnafu)? - .to_string(); - let mut parts = file_name.split('.'); - - let uuid = parts - .next() - .context(MissingUuidSnafu)? - .parse() - .context(InvalidUuidSnafu)?; - - let suffix = parts - .next() - .context(MissingSuffixSnafu)? - .parse() - .context(InvalidSuffixSnafu)?; - - ensure!(parts.next().is_none(), UnexpectedExtensionSnafu); - - Ok(Self { - revision_counter, - uuid, - suffix, - }) - } - - // Deliberately pub(crate); this transformation should only happen within this crate - pub(crate) fn from_absolute( - absolute_path: ObjStoPath, - ) -> Result { - let absolute_path: DirsAndFileName = absolute_path.into(); - - let mut absolute_dirs = absolute_path.directories.into_iter().fuse(); - - // The number of `next`s here needs to match the total number of directories in - // iox_object_store transactions_path - absolute_dirs.next(); // "dbs" - absolute_dirs.next(); // database uuid - absolute_dirs.next(); // "transactions" - - let remaining = DirsAndFileName { - directories: absolute_dirs.collect(), - file_name: absolute_path.file_name, - }; - - Self::from_relative_dirs_and_file_name(&remaining) - } -} - -#[derive(Snafu, Debug, PartialEq)] -#[allow(missing_docs)] -pub enum TransactionFilePathParseError { - #[snafu(display("Could not find required revision counter"))] - MissingRevisionCounter, - - #[snafu(display("Could not parse revision counter: {}", source))] - InvalidRevisionCounter { source: std::num::ParseIntError }, - - #[snafu(display("Too many directories found"))] - UnexpectedDirectory, - - #[snafu(display("Could not find required file name"))] - MissingFileName, - - #[snafu(display("Could not find required UUID"))] - MissingUuid, - - #[snafu(display("Could not parse UUID: {}", source))] - InvalidUuid { source: uuid::Error }, - - #[snafu(display("Could not find required suffix"))] - MissingSuffix, - - #[snafu(display("Invalid suffix: {}", source))] - InvalidSuffix { - source: TransactionFileSuffixParseError, - }, - - #[snafu(display("Too many extensions found"))] - UnexpectedExtension, -} - -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -enum TransactionFileSuffix { - Transaction, - Checkpoint, -} - -impl TransactionFileSuffix { - fn as_str(&self) -> &'static str { - match self { - Self::Transaction => TRANSACTION_FILE_SUFFIX, - Self::Checkpoint => CHECKPOINT_FILE_SUFFIX, - } - } -} - -#[derive(Snafu, Debug, PartialEq)] -#[allow(missing_docs)] -pub enum TransactionFileSuffixParseError { - #[snafu(display("Unknown suffix: {}", suffix))] - UnknownSuffix { suffix: String }, -} - -impl FromStr for TransactionFileSuffix { - type Err = TransactionFileSuffixParseError; - - fn from_str(s: &str) -> Result { - match s { - TRANSACTION_FILE_SUFFIX => Ok(Self::Transaction), - CHECKPOINT_FILE_SUFFIX => Ok(Self::Checkpoint), - suffix => UnknownSuffixSnafu { suffix }.fail(), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::{paths::ALL_DATABASES_DIRECTORY, IoxObjectStore, RootPath}; - use object_store::{DynObjectStore, ObjectStoreImpl}; - use std::sync::Arc; - use test_helpers::assert_error; - - /// Creates a new in-memory object store. These tests rely on the `Path`s being of type - /// `DirsAndFileName` and thus using object_store::path::DELIMITER as the separator - fn make_object_store() -> Arc { - Arc::new(ObjectStoreImpl::new_in_memory()) - } - - #[test] - fn is_checkpoint_works() { - let uuid = Uuid::new_v4(); - - let transaction = TransactionFilePath::new_transaction(0, uuid); - assert!(!transaction.is_checkpoint()); - - let checkpoint = TransactionFilePath::new_checkpoint(0, uuid); - assert!(checkpoint.is_checkpoint()); - } - - #[test] - fn test_transaction_file_path_deserialization() { - // Error cases - use TransactionFilePathParseError::*; - - let mut df = DirsAndFileName::default(); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(MissingRevisionCounter)), - "got {:?}", - result - ); - - df.push_dir("foo"); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(InvalidRevisionCounter { .. })), - "got {:?}", - result - ); - - let mut df = DirsAndFileName::default(); - df.push_dir("00000000000000000123"); - df.push_dir("foo"); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(UnexpectedDirectory)), - "got {:?}", - result - ); - - let mut df = DirsAndFileName::default(); - df.push_dir("00000000000000000123"); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!(matches!(result, Err(MissingFileName)), "got {:?}", result); - - df.set_file_name("foo"); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(InvalidUuid { .. })), - "got {:?}", - result - ); - - let uuid = Uuid::new_v4(); - - df.set_file_name(&format!("{}", uuid)); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!(matches!(result, Err(MissingSuffix)), "got {:?}", result); - - df.set_file_name(&format!("{}.exe", uuid)); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(InvalidSuffix { .. })), - "got {:?}", - result - ); - - df.set_file_name(&format!("{}.{}.foo", uuid, TRANSACTION_FILE_SUFFIX)); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df); - assert!( - matches!(result, Err(UnexpectedExtension)), - "got {:?}", - result - ); - - // Success case - df.set_file_name(&format!("{}.{}", uuid, TRANSACTION_FILE_SUFFIX)); - let result = TransactionFilePath::from_relative_dirs_and_file_name(&df).unwrap(); - assert_eq!( - result, - TransactionFilePath { - revision_counter: 123, - uuid, - suffix: TransactionFileSuffix::Transaction, - } - ); - let round_trip = result.relative_dirs_and_file_name(); - assert_eq!(round_trip, df); - } - - #[test] - fn transaction_file_from_absolute() { - let object_store = make_object_store(); - - // Success case - let uuid = Uuid::new_v4(); - let mut path = object_store.new_path(); - path.push_all_dirs(&["dbs", "uuid", "data", "00000000000000000123"]); - path.set_file_name(&format!("{}.{}", uuid, CHECKPOINT_FILE_SUFFIX)); - let result = TransactionFilePath::from_absolute(path); - assert_eq!( - result.unwrap(), - TransactionFilePath { - revision_counter: 123, - uuid, - suffix: TransactionFileSuffix::Checkpoint, - } - ); - - // Error cases - use TransactionFilePathParseError::*; - - let mut path = object_store.new_path(); - // incorrect directories are fine, we're assuming that list(transactions_path) scoped to the - // right directories so we don't check again on the way out - path.push_all_dirs(&["foo", "bar", "baz", "}*", "aoeu", "blah"]); - path.set_file_name("rules.pb"); - assert_error!( - TransactionFilePath::from_absolute(path), - InvalidRevisionCounter { .. }, - ); - - let mut path = object_store.new_path(); - path.push_all_dirs(&["dbs", "uuid", "data", "00000000000000000123"]); - // missing file name - assert_error!(TransactionFilePath::from_absolute(path), MissingFileName,); - } - - #[test] - fn transaction_file_relative_dirs_and_file_path() { - let uuid = Uuid::new_v4(); - let tfp = TransactionFilePath { - revision_counter: 555, - uuid, - suffix: TransactionFileSuffix::Transaction, - }; - let dirs_and_file_name = tfp.relative_dirs_and_file_name(); - assert_eq!( - dirs_and_file_name.to_string(), - format!("00000000000000000555/{}.{}", uuid, TRANSACTION_FILE_SUFFIX) - ); - let round_trip = - TransactionFilePath::from_relative_dirs_and_file_name(&dirs_and_file_name).unwrap(); - assert_eq!(tfp, round_trip); - } - - #[test] - fn transactions_path_join_with_parquet_file_path() { - let db_uuid = Uuid::new_v4(); - let object_store = make_object_store(); - let root_path = RootPath::new(&*object_store, db_uuid); - let iox_object_store = IoxObjectStore::existing(Arc::clone(&object_store), root_path); - - let uuid = Uuid::new_v4(); - let tfp = TransactionFilePath { - revision_counter: 555, - uuid, - suffix: TransactionFileSuffix::Checkpoint, - }; - - let path = iox_object_store.transactions_path.join(&tfp); - - let mut expected_path = object_store.new_path(); - expected_path.push_all_dirs(&[ - ALL_DATABASES_DIRECTORY, - &db_uuid.to_string(), - "transactions", - "00000000000000000555", - ]); - expected_path.set_file_name(&format!("{}.{}", uuid, CHECKPOINT_FILE_SUFFIX)); - - assert_eq!(path, expected_path); - } -} diff --git a/iox_object_store_service/Cargo.toml b/iox_object_store_service/Cargo.toml index 5e65ca855c..9b61dada7e 100644 --- a/iox_object_store_service/Cargo.toml +++ b/iox_object_store_service/Cargo.toml @@ -11,9 +11,9 @@ data_types2 = { path = "../data_types2" } futures = "0.3" generated_types = { path = "../generated_types" } iox_catalog = { path = "../iox_catalog" } -iox_object_store = { path = "../iox_object_store" } object_store = { path = "../object_store" } observability_deps = { path = "../observability_deps" } +parquet_file = { path = "../parquet_file" } serde = "1.0" serde_urlencoded = "0.7" iox_time = { path = "../iox_time" } diff --git a/iox_object_store_service/src/lib.rs b/iox_object_store_service/src/lib.rs index 7aec2f6d50..a9a8c88dac 100644 --- a/iox_object_store_service/src/lib.rs +++ b/iox_object_store_service/src/lib.rs @@ -16,10 +16,10 @@ use futures::stream::BoxStream; use futures::StreamExt; use generated_types::influxdata::iox::object_store::v1::*; use iox_catalog::interface::Catalog; -use iox_object_store::ParquetFilePath; use object_store::DynObjectStore; use observability_deps::tracing::*; -use std::sync::Arc; +use parquet_file::ParquetFilePath; +use std::{ops::Deref, sync::Arc}; use tonic::{Request, Response, Status}; use uuid::Uuid; @@ -66,15 +66,14 @@ impl object_store_service_server::ObjectStoreService for ObjectStoreService { })? .ok_or_else(|| Status::not_found(req.uuid))?; - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( parquet_file.namespace_id, parquet_file.table_id, parquet_file.sequencer_id, parquet_file.partition_id, parquet_file.object_store_id, - ) - .absolute_dirs_and_file_name(); - let path = self.object_store.path_from_dirs_and_filename(path); + ); + let path = path.object_store_path(self.object_store.deref()); let res = self .object_store @@ -164,15 +163,14 @@ mod tests { let object_store = Arc::new(ObjectStoreImpl::new_in_memory()); - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( p1.namespace_id, p1.table_id, p1.sequencer_id, p1.partition_id, p1.object_store_id, - ) - .absolute_dirs_and_file_name(); - let path = object_store.path_from_dirs_and_filename(path); + ); + let path = path.object_store_path(object_store.deref()); let data = Bytes::from_static(b"some data"); diff --git a/iox_tests/Cargo.toml b/iox_tests/Cargo.toml index c3c0c890c3..d17fc51f77 100644 --- a/iox_tests/Cargo.toml +++ b/iox_tests/Cargo.toml @@ -11,7 +11,6 @@ bytes = "1.0" data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } iox_catalog = { path = "../iox_catalog" } -iox_object_store = { path = "../iox_object_store" } metric = { path = "../metric" } mutable_batch_lp = { path = "../mutable_batch_lp" } object_store = { path = "../object_store" } diff --git a/iox_tests/src/util.rs b/iox_tests/src/util.rs index 7b34575855..b57df919bd 100644 --- a/iox_tests/src/util.rs +++ b/iox_tests/src/util.rs @@ -15,18 +15,20 @@ use iox_catalog::{ interface::{Catalog, PartitionRepo, INITIAL_COMPACTION_LEVEL}, mem::MemCatalog, }; -use iox_object_store::{IoxObjectStore, ParquetFilePath}; use iox_time::{MockProvider, Time, TimeProvider}; use mutable_batch_lp::test_helpers::lp_to_mutable_batch; use object_store::{DynObjectStore, ObjectStoreImpl}; -use parquet_file::metadata::{IoxMetadata, IoxParquetMetaData}; +use parquet_file::{ + metadata::{IoxMetadata, IoxParquetMetaData}, + ParquetFilePath, +}; use query::{exec::Executor, provider::RecordBatchDeduplicator, util::arrow_sort_key_exprs}; use schema::{ selection::Selection, sort::{adjust_sort_key_columns, SortKey, SortKeyBuilder}, Schema, }; -use std::sync::Arc; +use std::{ops::Deref, sync::Arc}; use uuid::Uuid; /// Catalog for tests @@ -527,8 +529,12 @@ impl TestPartition { compaction_level: INITIAL_COMPACTION_LEVEL, sort_key: Some(sort_key.clone()), }; - let (parquet_metadata_bin, real_file_size_bytes) = - create_parquet_file(&self.catalog.object_store, &metadata, record_batch).await; + let (parquet_metadata_bin, real_file_size_bytes) = create_parquet_file( + Arc::clone(&self.catalog.object_store), + &metadata, + record_batch, + ) + .await; let parquet_file_params = ParquetFileParams { sequencer_id: self.sequencer.sequencer.id, @@ -640,18 +646,13 @@ async fn update_catalog_sort_key_if_needed( /// Create parquet file and return thrift-encoded and zstd-compressed parquet metadata as well as the file size. async fn create_parquet_file( - object_store: &Arc, + object_store: Arc, metadata: &IoxMetadata, record_batch: RecordBatch, ) -> (Vec, usize) { - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(object_store), - IoxObjectStore::root_path_for(&**object_store, uuid::Uuid::new_v4()), - )); - let schema = record_batch.schema(); - let data = parquet_file::storage::Storage::new(Arc::clone(&iox_object_store)) + let data = parquet_file::storage::Storage::new(Arc::clone(&object_store)) .parquet_bytes(vec![record_batch], schema, metadata) .await .unwrap(); @@ -665,18 +666,16 @@ async fn create_parquet_file( let file_size = data.len(); let bytes = Bytes::from(data); - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( metadata.namespace_id, metadata.table_id, metadata.sequencer_id, metadata.partition_id, metadata.object_store_id, ); + let path = path.object_store_path(object_store.deref()); - iox_object_store - .put_parquet_file(&path, bytes) - .await - .unwrap(); + object_store.put(&path, bytes).await.unwrap(); (parquet_md, file_size) } diff --git a/parquet_file/Cargo.toml b/parquet_file/Cargo.toml index 81a7f4a11c..2c046b0916 100644 --- a/parquet_file/Cargo.toml +++ b/parquet_file/Cargo.toml @@ -14,7 +14,6 @@ datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } futures = "0.3" generated_types = { path = "../generated_types" } -iox_object_store = { path = "../iox_object_store" } metric = { path = "../metric" } object_store = { path = "../object_store" } observability_deps = { path = "../observability_deps" } diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index 9d01e2ac8a..1dee0f3d0b 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -1,6 +1,7 @@ use crate::{ metadata::{DecodedIoxParquetMetaData, IoxMetadata, IoxParquetMetaData}, storage::Storage, + ParquetFilePath, }; use data_types::{ partition_metadata::{Statistics, TableSummary}, @@ -8,7 +9,7 @@ use data_types::{ }; use data_types2::{ParquetFile, ParquetFileWithMetadata}; use datafusion::physical_plan::SendableRecordBatchStream; -use iox_object_store::{IoxObjectStore, ParquetFilePath}; +use object_store::DynObjectStore; use observability_deps::tracing::*; use predicate::Predicate; use schema::selection::Selection; @@ -90,7 +91,7 @@ pub struct ParquetChunk { timestamp_min_max: Option, /// Persists the parquet file within a database's relative path - iox_object_store: Arc, + object_store: Arc, /// Path in the database's object store. path: ParquetFilePath, @@ -112,7 +113,7 @@ impl ParquetChunk { /// Creates new chunk from given parquet metadata. pub fn new( path: &ParquetFilePath, - iox_object_store: Arc, + object_store: Arc, file_size_bytes: usize, parquet_metadata: Arc, metrics: ChunkMetrics, @@ -133,7 +134,7 @@ impl ParquetChunk { Arc::new(table_summary), schema, path, - iox_object_store, + object_store, file_size_bytes, parquet_metadata, rows, @@ -148,7 +149,7 @@ impl ParquetChunk { table_summary: Arc, schema: Arc, path: &ParquetFilePath, - iox_object_store: Arc, + object_store: Arc, file_size_bytes: usize, parquet_metadata: Arc, rows: usize, @@ -160,8 +161,8 @@ impl ParquetChunk { table_summary, schema, timestamp_min_max, - iox_object_store, path: path.into(), + object_store, file_size_bytes, parquet_metadata, rows, @@ -237,8 +238,8 @@ impl ParquetChunk { predicate, selection, Arc::clone(&self.schema.as_arrow()), - self.path.clone(), - Arc::clone(&self.iox_object_store), + self.path, + Arc::clone(&self.object_store), ) .context(ReadParquetSnafu) } @@ -306,10 +307,10 @@ impl DecodedParquetFile { pub fn new_parquet_chunk( decoded_parquet_file: &DecodedParquetFile, metrics: ChunkMetrics, - iox_object_store: Arc, + object_store: Arc, ) -> ParquetChunk { let iox_metadata = &decoded_parquet_file.iox_metadata; - let path = ParquetFilePath::new_new_gen( + let path = ParquetFilePath::new( iox_metadata.namespace_id, iox_metadata.table_id, iox_metadata.sequencer_id, @@ -322,7 +323,7 @@ pub fn new_parquet_chunk( ParquetChunk::new( &path, - iox_object_store, + object_store, file_size_bytes, Arc::clone(&decoded_parquet_file.parquet_metadata), metrics, diff --git a/parquet_file/src/lib.rs b/parquet_file/src/lib.rs index da58aa56bb..ad3c9830e8 100644 --- a/parquet_file/src/lib.rs +++ b/parquet_file/src/lib.rs @@ -11,3 +11,93 @@ pub mod chunk; pub mod metadata; pub mod storage; + +use data_types2::{NamespaceId, PartitionId, SequencerId, TableId}; +use object_store::{ + path::{parsed::DirsAndFileName, ObjectStorePath, Path}, + DynObjectStore, +}; +use uuid::Uuid; + +/// Location of a Parquet file within a database's object store. +/// The exact format is an implementation detail and is subject to change. +#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash, Ord, PartialOrd)] +pub struct ParquetFilePath { + namespace_id: NamespaceId, + table_id: TableId, + sequencer_id: SequencerId, + partition_id: PartitionId, + object_store_id: Uuid, +} + +impl ParquetFilePath { + /// Create parquet file path relevant for the NG storage layout. + pub fn new( + namespace_id: NamespaceId, + table_id: TableId, + sequencer_id: SequencerId, + partition_id: PartitionId, + object_store_id: Uuid, + ) -> Self { + Self { + namespace_id, + table_id, + sequencer_id, + partition_id, + object_store_id, + } + } + + /// Get absolute storage location. + fn absolute_dirs_and_file_name(&self) -> DirsAndFileName { + let Self { + namespace_id, + table_id, + sequencer_id, + partition_id, + object_store_id, + } = self; + + let mut result = DirsAndFileName::default(); + result.push_all_dirs(&[ + namespace_id.to_string().as_str(), + table_id.to_string().as_str(), + sequencer_id.to_string().as_str(), + partition_id.to_string().as_str(), + ]); + result.set_file_name(format!("{}.parquet", object_store_id)); + result + } + + /// Get object-store specific absolute path. + pub fn object_store_path(&self, object_store: &DynObjectStore) -> Path { + object_store.path_from_dirs_and_filename(self.absolute_dirs_and_file_name()) + } +} + +impl From<&Self> for ParquetFilePath { + fn from(borrowed: &Self) -> Self { + *borrowed + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn parquet_file_absolute_dirs_and_file_path() { + let pfp = ParquetFilePath::new( + NamespaceId::new(1), + TableId::new(2), + SequencerId::new(3), + PartitionId::new(4), + Uuid::nil(), + ); + let dirs_and_file_name = pfp.absolute_dirs_and_file_name(); + assert_eq!( + dirs_and_file_name.to_string(), + "1/2/3/4/00000000-0000-0000-0000-000000000000.parquet".to_string(), + ); + } +} diff --git a/parquet_file/src/storage.rs b/parquet_file/src/storage.rs index 7d9e871236..bf1fe5368e 100644 --- a/parquet_file/src/storage.rs +++ b/parquet_file/src/storage.rs @@ -1,6 +1,10 @@ -use crate::metadata::{IoxMetadata, METADATA_KEY}; -/// This module responsible to write given data to specify object store and -/// read them back +//! This module is responsible for writing the given data to the specified object store and reading +//! it back. + +use crate::{ + metadata::{IoxMetadata, METADATA_KEY}, + ParquetFilePath, +}; use arrow::{ datatypes::{Schema, SchemaRef}, error::{ArrowError, Result as ArrowResult}, @@ -10,8 +14,7 @@ use bytes::Bytes; use datafusion::physical_plan::SendableRecordBatchStream; use datafusion_util::AdapterStream; use futures::{stream, StreamExt}; -use iox_object_store::{IoxObjectStore, ParquetFilePath}; -use object_store::GetResult; +use object_store::{DynObjectStore, GetResult}; use observability_deps::tracing::*; use parking_lot::Mutex; use parquet::arrow::{ArrowReader, ParquetFileArrowReader}; @@ -27,6 +30,7 @@ use schema::selection::Selection; use snafu::{OptionExt, ResultExt, Snafu}; use std::{ io::{Cursor, Seek, SeekFrom, Write}, + ops::Deref, sync::Arc, }; @@ -83,16 +87,16 @@ pub type Result = std::result::Result; #[derive(Debug, Clone)] pub struct Storage { - iox_object_store: Arc, + object_store: Arc, // If `Some`, restricts the size of the row groups created in the parquet file max_row_group_size: Option, } impl Storage { - pub fn new(iox_object_store: Arc) -> Self { + pub fn new(object_store: Arc) -> Self { Self { - iox_object_store, + object_store, max_row_group_size: None, } } @@ -154,9 +158,10 @@ impl Storage { /// Put the given vector of bytes to the specified location pub async fn to_object_store(&self, data: Vec, path: &ParquetFilePath) -> Result<()> { let data = Bytes::from(data); + let path = path.object_store_path(self.object_store.deref()); - self.iox_object_store - .put_parquet_file(path, data) + self.object_store + .put(&path, data) .await .context(WritingToObjectStoreSnafu) } @@ -187,13 +192,14 @@ impl Storage { fn download_and_scan_parquet( projection: Vec, path: ParquetFilePath, - store: Arc, + object_store: Arc, tx: tokio::sync::mpsc::Sender>, ) -> Result<()> { // Size of each batch let batch_size = 1024; // Todo: make a constant or policy for this + let path = path.object_store_path(object_store.deref()); - let read_stream = futures::executor::block_on(store.get_parquet_file(&path)) + let read_stream = futures::executor::block_on(object_store.get(&path)) .context(ReadingObjectStoreSnafu)?; let file = match read_stream { @@ -243,7 +249,7 @@ impl Storage { selection: Selection<'_>, schema: SchemaRef, path: ParquetFilePath, - store: Arc, + object_store: Arc, ) -> Result { // Indices of columns in the schema needed to read let projection: Vec = Self::column_indices(selection, Arc::clone(&schema)); @@ -263,7 +269,7 @@ impl Storage { // not silently ignored tokio::task::spawn_blocking(move || { let download_result = - Self::download_and_scan_parquet(projection, path, store, tx.clone()); + Self::download_and_scan_parquet(projection, path, object_store, tx.clone()); // If there was an error returned from download_and_scan_parquet send it back to the receiver. if let Err(e) = download_result { diff --git a/querier/Cargo.toml b/querier/Cargo.toml index 04b4ebadf2..e20e2270cc 100644 --- a/querier/Cargo.toml +++ b/querier/Cargo.toml @@ -19,7 +19,6 @@ generated_types = { path = "../generated_types" } hyper = "0.14" influxdb_iox_client = { path = "../influxdb_iox_client" } 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" } diff --git a/querier/src/chunk/mod.rs b/querier/src/chunk/mod.rs index 1b9b509f7e..f649ff7cce 100644 --- a/querier/src/chunk/mod.rs +++ b/querier/src/chunk/mod.rs @@ -9,7 +9,6 @@ use data_types2::{ }; use futures::StreamExt; use iox_catalog::interface::Catalog; -use iox_object_store::IoxObjectStore; use iox_time::TimeProvider; use object_store::DynObjectStore; use parquet_file::chunk::{ @@ -175,8 +174,8 @@ pub struct ParquetChunkAdapter { /// Cache catalog_cache: Arc, - /// Old-gen object store. - iox_object_store: Arc, + /// Object store. + object_store: Arc, /// Metric registry. metric_registry: Arc, @@ -193,15 +192,9 @@ impl ParquetChunkAdapter { metric_registry: Arc, time_provider: Arc, ) -> Self { - // create a virtual IOx object store, the UUID won't be used anyways - let iox_object_store = Arc::new(IoxObjectStore::existing( - Arc::clone(&object_store), - IoxObjectStore::root_path_for(&*object_store, uuid::Uuid::new_v4()), - )); - Self { catalog_cache, - iox_object_store, + object_store, metric_registry, time_provider, } @@ -229,7 +222,7 @@ impl ParquetChunkAdapter { Some(new_parquet_chunk( decoded_parquet_file, metrics, - Arc::clone(&self.iox_object_store), + Arc::clone(&self.object_store), )) } From c221960ebd5e0b3f78427d665132d2870d4a1c2f Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 14:28:38 -0400 Subject: [PATCH 02/31] fix: Move chunk metadata types to data_types2 --- Cargo.lock | 1 + data_types2/src/lib.rs | 142 +++++++++++++++++++++++++- query/src/frontend/influxrpc.rs | 53 +++++----- query/src/lib.rs | 9 +- query/src/test.rs | 9 +- service_grpc_influxrpc/Cargo.toml | 1 + service_grpc_influxrpc/src/service.rs | 9 +- 7 files changed, 173 insertions(+), 51 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f85bbd356a..24b7119e84 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5216,6 +5216,7 @@ dependencies = [ "arrow", "async-trait", "data_types", + "data_types2", "datafusion 0.1.0", "futures", "generated_types", diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index a921c109a6..c93e1b82bb 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -17,13 +17,13 @@ use std::{ collections::BTreeMap, convert::TryFrom, fmt::{Debug, Formatter}, + num::NonZeroU32, ops::{Add, Sub}, sync::Arc, }; use uuid::Uuid; pub use data_types::{ - chunk_metadata::{ChunkAddr, ChunkId, ChunkOrder, ChunkSummary}, database_rules::{PartitionTemplate, TemplatePart}, delete_predicate::{DeleteExpr, DeletePredicate, Op, Scalar}, names::{org_and_bucket_to_database, OrgBucketMappingError}, @@ -32,7 +32,7 @@ pub use data_types::{ ColumnSummary, InfluxDbType, PartitionAddr, StatValues, Statistics, TableSummary, }, sequence::Sequence, - timestamp::TimestampRange, + timestamp::{TimestampMinMax, TimestampRange}, DatabaseName, }; @@ -953,3 +953,141 @@ impl IngesterQueryRequest { } } } + +/// Address of the chunk within the catalog +#[derive(Debug, Clone, Eq, PartialEq, PartialOrd, Ord, Hash)] +pub struct ChunkAddr { + /// Database name + pub db_name: Arc, + + /// What table does the chunk belong to? + pub table_name: Arc, + + /// What partition does the chunk belong to? + pub partition_key: Arc, + + /// The ID of the chunk + pub chunk_id: ChunkId, +} + +impl std::fmt::Display for ChunkAddr { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Chunk('{}':'{}':'{}':{})", + self.db_name, + self.table_name, + self.partition_key, + self.chunk_id.get() + ) + } +} + +/// ID of a chunk. +/// +/// This ID is unique within a single partition. +#[derive(Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct ChunkId(Uuid); + +impl ChunkId { + /// Create new, random ID. + #[allow(clippy::new_without_default)] // `new` creates non-deterministic result + pub fn new() -> Self { + Self(Uuid::new_v4()) + } + + /// **TESTING ONLY:** Create new ID from integer. + /// + /// Since this can easily lead to ID collissions (which in turn can lead to panics), this must + /// only be used for testing purposes! + pub fn new_test(id: u128) -> Self { + Self(Uuid::from_u128(id)) + } + + /// The chunk id is only effective in case the chunk's order is the same with another chunk. + /// Hence collisions are safe in that context. + pub fn new_id_for_ng(id: u128) -> Self { + Self(Uuid::from_u128(id)) + } + + /// Get inner UUID. + pub fn get(&self) -> Uuid { + self.0 + } +} + +impl std::fmt::Debug for ChunkId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + ::fmt(self, f) + } +} + +impl std::fmt::Display for ChunkId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + if (self.0.get_variant() == Some(uuid::Variant::RFC4122)) + && (self.0.get_version() == Some(uuid::Version::Random)) + { + f.debug_tuple("ChunkId").field(&self.0).finish() + } else { + f.debug_tuple("ChunkId").field(&self.0.as_u128()).finish() + } + } +} + +impl From for ChunkId { + fn from(uuid: Uuid) -> Self { + Self(uuid) + } +} + +/// Order of a chunk. +/// +/// This is used for: +/// 1. **upsert order:** chunks with higher order overwrite data in chunks with lower order +/// 2. **locking order:** chunks must be locked in consistent (ascending) order +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct ChunkOrder(NonZeroU32); + +impl ChunkOrder { + /// The minimum ordering value a chunk could have. Currently only used in testing. + // TODO: remove `unsafe` once https://github.com/rust-lang/rust/issues/51999 is fixed + pub const MIN: Self = Self(unsafe { NonZeroU32::new_unchecked(1) }); + + /// Create a ChunkOrder from the given value. + pub fn new(order: u32) -> Option { + NonZeroU32::new(order).map(Self) + } + +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_chunk_id_new() { + // `ChunkId::new()` create new random ID + assert_ne!(ChunkId::new(), ChunkId::new()); + } + + #[test] + fn test_chunk_id_new_test() { + // `ChunkId::new_test(...)` creates deterministic ID + assert_eq!(ChunkId::new_test(1), ChunkId::new_test(1)); + assert_ne!(ChunkId::new_test(1), ChunkId::new_test(2)); + } + + #[test] + fn test_chunk_id_debug_and_display() { + // Random chunk IDs use UUID-format + let id_random = ChunkId::new(); + let inner: Uuid = id_random.get(); + assert_eq!(format!("{:?}", id_random), format!("ChunkId({})", inner)); + assert_eq!(format!("{}", id_random), format!("ChunkId({})", inner)); + + // Deterministic IDs use integer format + let id_test = ChunkId::new_test(42); + assert_eq!(format!("{:?}", id_test), "ChunkId(42)"); + assert_eq!(format!("{}", id_test), "ChunkId(42)"); + } +} diff --git a/query/src/frontend/influxrpc.rs b/query/src/frontend/influxrpc.rs index 748d87dc1b..58237502b4 100644 --- a/query/src/frontend/influxrpc.rs +++ b/query/src/frontend/influxrpc.rs @@ -1,33 +1,4 @@ //! Query frontend for InfluxDB Storage gRPC requests -use std::{ - collections::{BTreeMap, BTreeSet}, - sync::Arc, -}; - -use arrow::datatypes::DataType; -use data_types::chunk_metadata::ChunkId; -use datafusion::{ - error::DataFusionError, - logical_plan::{ - col, when, DFSchemaRef, Expr, ExprRewritable, ExprSchemable, LogicalPlan, - LogicalPlanBuilder, - }, -}; -use datafusion_util::AsExpr; - -use hashbrown::HashSet; -use observability_deps::tracing::{debug, trace}; -use predicate::rpc_predicate::InfluxRpcPredicate; -use predicate::{Predicate, PredicateMatch}; -use query_functions::{ - group_by::{Aggregate, WindowDuration}, - make_window_bound_expr, - selectors::{selector_first, selector_last, selector_max, selector_min, SelectorOutput}, -}; -use schema::selection::Selection; -use schema::{InfluxColumnType, Schema, TIME_COLUMN_NAME}; -use snafu::{ensure, OptionExt, ResultExt, Snafu}; - use crate::{ exec::{field::FieldColumns, make_non_null_checker, make_schema_pivot, IOxSessionContext}, plan::{ @@ -39,6 +10,30 @@ use crate::{ util::MissingColumnsToNull, QueryChunk, QueryDatabase, }; +use arrow::datatypes::DataType; +use data_types2::ChunkId; +use datafusion::{ + error::DataFusionError, + logical_plan::{ + col, when, DFSchemaRef, Expr, ExprRewritable, ExprSchemable, LogicalPlan, + LogicalPlanBuilder, + }, +}; +use datafusion_util::AsExpr; +use hashbrown::HashSet; +use observability_deps::tracing::{debug, trace}; +use predicate::{rpc_predicate::InfluxRpcPredicate, Predicate, PredicateMatch}; +use query_functions::{ + group_by::{Aggregate, WindowDuration}, + make_window_bound_expr, + selectors::{selector_first, selector_last, selector_max, selector_min, SelectorOutput}, +}; +use schema::{selection::Selection, InfluxColumnType, Schema, TIME_COLUMN_NAME}; +use snafu::{ensure, OptionExt, ResultExt, Snafu}; +use std::{ + collections::{BTreeMap, BTreeSet}, + sync::Arc, +}; #[derive(Debug, Snafu)] pub enum Error { diff --git a/query/src/lib.rs b/query/src/lib.rs index f19a4abb2d..b09b27444b 100644 --- a/query/src/lib.rs +++ b/query/src/lib.rs @@ -9,13 +9,10 @@ )] use async_trait::async_trait; -use data_types::{ - chunk_metadata::{ChunkAddr, ChunkId, ChunkOrder}, - delete_predicate::DeletePredicate, - partition_metadata::{InfluxDbType, TableSummary}, - timestamp::TimestampMinMax, +use data_types2::{ + ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, InfluxDbType, PartitionId, TableSummary, + TimestampMinMax, }; -use data_types2::PartitionId; use datafusion::physical_plan::SendableRecordBatchStream; use exec::{stringset::StringSet, IOxSessionContext}; use observability_deps::tracing::{debug, trace}; diff --git a/query/src/test.rs b/query/src/test.rs index b75edfe481..25222244be 100644 --- a/query/src/test.rs +++ b/query/src/test.rs @@ -16,13 +16,10 @@ use arrow::{ record_batch::RecordBatch, }; use async_trait::async_trait; -use data_types::chunk_metadata::{ChunkAddr, ChunkId, ChunkOrder}; -use data_types::timestamp::TimestampMinMax; -use data_types::{ - delete_predicate::DeletePredicate, - partition_metadata::{ColumnSummary, InfluxDbType, StatValues, Statistics, TableSummary}, +use data_types2::{ + ChunkAddr, ChunkId, ChunkOrder, ColumnSummary, DeletePredicate, InfluxDbType, PartitionId, + StatValues, Statistics, TableSummary, TimestampMinMax, }; -use data_types2::PartitionId; use datafusion::physical_plan::SendableRecordBatchStream; use datafusion_util::stream_from_batches; use futures::StreamExt; diff --git a/service_grpc_influxrpc/Cargo.toml b/service_grpc_influxrpc/Cargo.toml index 21478f498b..b9630afd4c 100644 --- a/service_grpc_influxrpc/Cargo.toml +++ b/service_grpc_influxrpc/Cargo.toml @@ -9,6 +9,7 @@ edition = "2021" # Workspace dependencies, in alphabetical order datafusion = { path = "../datafusion" } data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } generated_types = { path = "../generated_types" } observability_deps = { path = "../observability_deps" } predicate = { path = "../predicate" } diff --git a/service_grpc_influxrpc/src/service.rs b/service_grpc_influxrpc/src/service.rs index cc38efb262..94e4471159 100644 --- a/service_grpc_influxrpc/src/service.rs +++ b/service_grpc_influxrpc/src/service.rs @@ -5,13 +5,11 @@ use std::{ collections::{BTreeSet, HashMap}, sync::Arc, }; - use service_common::QueryDatabaseProvider; use snafu::{OptionExt, ResultExt, Snafu}; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; use tonic::Status; - use data_types::{error::ErrorLogger, names::org_and_bucket_to_database, DatabaseName}; use generated_types::{ google::protobuf::Empty, literal_or_regex::Value as RegexOrLiteralValue, @@ -33,7 +31,6 @@ use query::{ }, QueryDatabase, QueryText, }; - use crate::{ data::{ fieldlist_to_measurement_fields_response, series_or_groups_to_read_response, @@ -44,7 +41,6 @@ use crate::{ StorageService, }; use service_common::planner::Planner; - use super::{TAG_KEY_FIELD, TAG_KEY_MEASUREMENT}; #[derive(Debug, Snafu)] @@ -1374,14 +1370,12 @@ mod tests { num::NonZeroU64, sync::Arc, }; - use async_trait::async_trait; - use data_types::chunk_metadata::ChunkId; + use data_types2::ChunkId; use generated_types::{i_ox_testing_client::IOxTestingClient, tag_key_predicate::Value}; use parking_lot::Mutex; use service_common::QueryDatabaseProvider; use tokio_stream::wrappers::TcpListenerStream; - use datafusion::logical_plan::{col, lit, Expr}; use influxdb_storage_client::{ connection::{Builder as ConnectionBuilder, Connection}, @@ -1395,7 +1389,6 @@ mod tests { test::{TestChunk, TestDatabase}, }; use test_helpers::{assert_contains, tracing::TracingCapture}; - use super::*; use metric::{Attributes, Metric, U64Counter}; From 12793bffbfae66d72c821f528cd20110ead6a7ca Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 14:49:37 -0400 Subject: [PATCH 03/31] fix: Move Partition Template types to data_types2 --- Cargo.lock | 2 +- data_types2/Cargo.toml | 1 - data_types2/src/lib.rs | 62 ++++++++++++++++++++- iox_catalog/src/lib.rs | 2 +- mutable_batch/Cargo.toml | 1 + mutable_batch/src/payload.rs | 2 +- mutable_batch/src/payload/partition.rs | 2 +- mutable_batch/tests/writer_fuzz.rs | 2 +- service_grpc_influxrpc/src/service.rs | 76 +++++++++++++------------- 9 files changed, 103 insertions(+), 47 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 24b7119e84..ac002def53 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1174,7 +1174,6 @@ version = "0.1.0" dependencies = [ "data_types", "influxdb_line_protocol", - "mutable_batch", "predicate", "schema", "sqlx", @@ -3301,6 +3300,7 @@ dependencies = [ "arrow_util", "chrono", "data_types", + "data_types2", "hashbrown 0.12.0", "itertools", "rand", diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index f08ae8d125..4f03457c81 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -7,7 +7,6 @@ description = "Shared data types in the Iox NG architecture" [dependencies] data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } -mutable_batch = { path = "../mutable_batch" } predicate = { path = "../predicate" } schema = { path = "../schema" } sqlx = { version = "0.5", features = ["runtime-tokio-rustls", "postgres", "uuid"] } diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index c93e1b82bb..3860a32253 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -24,7 +24,6 @@ use std::{ use uuid::Uuid; pub use data_types::{ - database_rules::{PartitionTemplate, TemplatePart}, delete_predicate::{DeleteExpr, DeletePredicate, Op, Scalar}, names::{org_and_bucket_to_database, OrgBucketMappingError}, non_empty::NonEmptyString, @@ -492,8 +491,8 @@ impl ColumnSchema { } /// Returns true if `mb_column` is of the same type as `self`. - pub fn matches_type(&self, mb_column: &mutable_batch::column::Column) -> bool { - self.column_type == mb_column.influx_type() + pub fn matches_type(&self, mb_column_influx_type: InfluxColumnType) -> bool { + self.column_type == mb_column_influx_type } } @@ -1057,7 +1056,64 @@ impl ChunkOrder { pub fn new(order: u32) -> Option { NonZeroU32::new(order).map(Self) } +} +/// `PartitionTemplate` is used to compute the partition key of each row that +/// gets written. It can consist of the table name, a column name and its value, +/// a formatted time, or a string column and regex captures of its value. For +/// columns that do not appear in the input row, a blank value is output. +/// +/// The key is constructed in order of the template parts; thus ordering changes +/// what partition key is generated. +#[derive(Debug, Default, Eq, PartialEq, Clone)] +#[allow(missing_docs)] +pub struct PartitionTemplate { + pub parts: Vec, +} + +/// `TemplatePart` specifies what part of a row should be used to compute this +/// part of a partition key. +#[derive(Debug, Eq, PartialEq, Clone)] +pub enum TemplatePart { + /// The name of a table + Table, + /// The value in a named column + Column(String), + /// Applies a `strftime` format to the "time" column. + /// + /// For example, a time format of "%Y-%m-%d %H:%M:%S" will produce + /// partition key parts such as "2021-03-14 12:25:21" and + /// "2021-04-14 12:24:21" + TimeFormat(String), + /// Applies a regex to the value in a string column + RegexCapture(RegexCapture), + /// Applies a `strftime` pattern to some column other than "time" + StrftimeColumn(StrftimeColumn), +} + +/// `RegexCapture` is for pulling parts of a string column into the partition +/// key. +#[derive(Debug, Eq, PartialEq, Clone)] +#[allow(missing_docs)] +pub struct RegexCapture { + pub column: String, + pub regex: String, +} + +/// [`StrftimeColumn`] is used to create a time based partition key off some +/// column other than the builtin `time` column. +/// +/// The value of the named column is formatted using a `strftime` +/// style string. +/// +/// For example, a time format of "%Y-%m-%d %H:%M:%S" will produce +/// partition key parts such as "2021-03-14 12:25:21" and +/// "2021-04-14 12:24:21" +#[derive(Debug, Eq, PartialEq, Clone)] +#[allow(missing_docs)] +pub struct StrftimeColumn { + pub column: String, + pub format: String, } #[cfg(test)] diff --git a/iox_catalog/src/lib.rs b/iox_catalog/src/lib.rs index 27c08ed6c7..f53100c2bf 100644 --- a/iox_catalog/src/lib.rs +++ b/iox_catalog/src/lib.rs @@ -125,7 +125,7 @@ where // If it does, validate it. If it does not exist, create it and insert // it into the cached schema. match table.columns.get(name.as_str()) { - Some(existing) if existing.matches_type(col) => { + Some(existing) if existing.matches_type(col.influx_type()) => { // No action is needed as the column matches the existing column // schema. } diff --git a/mutable_batch/Cargo.toml b/mutable_batch/Cargo.toml index d080315805..1a7b78d192 100644 --- a/mutable_batch/Cargo.toml +++ b/mutable_batch/Cargo.toml @@ -9,6 +9,7 @@ arrow = { version = "13", features = ["prettyprint"] } arrow_util = { path = "../arrow_util" } chrono = { version = "0.4", default-features = false } data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } schema = { path = "../schema" } snafu = "0.7" hashbrown = "0.12" diff --git a/mutable_batch/src/payload.rs b/mutable_batch/src/payload.rs index c03dd7950e..3d19893092 100644 --- a/mutable_batch/src/payload.rs +++ b/mutable_batch/src/payload.rs @@ -2,7 +2,7 @@ use crate::column::ColumnData; use crate::{MutableBatch, Result}; -use data_types::database_rules::PartitionTemplate; +use data_types2::PartitionTemplate; use hashbrown::HashMap; use schema::TIME_COLUMN_NAME; use std::num::NonZeroUsize; diff --git a/mutable_batch/src/payload/partition.rs b/mutable_batch/src/payload/partition.rs index 0d88704a3c..dfc948278d 100644 --- a/mutable_batch/src/payload/partition.rs +++ b/mutable_batch/src/payload/partition.rs @@ -5,7 +5,7 @@ use crate::column::{Column, ColumnData}; use crate::MutableBatch; use chrono::{format::StrftimeItems, TimeZone, Utc}; -use data_types::database_rules::{PartitionTemplate, TemplatePart}; +use data_types2::{PartitionTemplate, TemplatePart}; use schema::TIME_COLUMN_NAME; use std::ops::Range; diff --git a/mutable_batch/tests/writer_fuzz.rs b/mutable_batch/tests/writer_fuzz.rs index 08b7418d84..e3bf11c6c9 100644 --- a/mutable_batch/tests/writer_fuzz.rs +++ b/mutable_batch/tests/writer_fuzz.rs @@ -19,8 +19,8 @@ use hashbrown::HashSet; use rand::prelude::*; use arrow_util::bitset::BitSet; -use data_types::database_rules::{PartitionTemplate, TemplatePart}; use data_types::partition_metadata::{IsNan, StatValues, Statistics}; +use data_types2::{PartitionTemplate, TemplatePart}; use mutable_batch::writer::Writer; use mutable_batch::{MutableBatch, PartitionWrite, WritePayload}; use schema::selection::Selection; diff --git a/service_grpc_influxrpc/src/service.rs b/service_grpc_influxrpc/src/service.rs index 94e4471159..6be36c7a1f 100644 --- a/service_grpc_influxrpc/src/service.rs +++ b/service_grpc_influxrpc/src/service.rs @@ -1,15 +1,16 @@ //! This module contains implementations for the storage gRPC service //! implemented in terms of the [`QueryDatabase`](query::QueryDatabase). -use std::{ - collections::{BTreeSet, HashMap}, - sync::Arc, +use super::{TAG_KEY_FIELD, TAG_KEY_MEASUREMENT}; +use crate::{ + data::{ + fieldlist_to_measurement_fields_response, series_or_groups_to_read_response, + tag_keys_to_byte_vecs, + }, + expr::{self, GroupByAndAggregate, InfluxRpcPredicateBuilder, Loggable, SpecialTagKeys}, + input::GrpcInputs, + StorageService, }; -use service_common::QueryDatabaseProvider; -use snafu::{OptionExt, ResultExt, Snafu}; -use tokio::sync::mpsc; -use tokio_stream::wrappers::ReceiverStream; -use tonic::Status; use data_types::{error::ErrorLogger, names::org_and_bucket_to_database, DatabaseName}; use generated_types::{ google::protobuf::Empty, literal_or_regex::Value as RegexOrLiteralValue, @@ -31,17 +32,16 @@ use query::{ }, QueryDatabase, QueryText, }; -use crate::{ - data::{ - fieldlist_to_measurement_fields_response, series_or_groups_to_read_response, - tag_keys_to_byte_vecs, - }, - expr::{self, GroupByAndAggregate, InfluxRpcPredicateBuilder, Loggable, SpecialTagKeys}, - input::GrpcInputs, - StorageService, -}; use service_common::planner::Planner; -use super::{TAG_KEY_FIELD, TAG_KEY_MEASUREMENT}; +use service_common::QueryDatabaseProvider; +use snafu::{OptionExt, ResultExt, Snafu}; +use std::{ + collections::{BTreeSet, HashMap}, + sync::Arc, +}; +use tokio::sync::mpsc; +use tokio_stream::wrappers::ReceiverStream; +use tonic::Status; #[derive(Debug, Snafu)] pub enum Error { @@ -1364,33 +1364,33 @@ where #[cfg(test)] mod tests { + use super::*; + use async_trait::async_trait; + use data_types2::ChunkId; + use datafusion::logical_plan::{col, lit, Expr}; + use generated_types::{i_ox_testing_client::IOxTestingClient, tag_key_predicate::Value}; + use influxdb_storage_client::{ + connection::{Builder as ConnectionBuilder, Connection}, + generated_types::*, + Client as StorageClient, OrgAndBucket, + }; + use metric::{Attributes, Metric, U64Counter}; + use panic_logging::SendPanicsToTracing; + use parking_lot::Mutex; + use predicate::{PredicateBuilder, PredicateMatch}; + use query::{ + exec::Executor, + test::{TestChunk, TestDatabase}, + }; + use service_common::QueryDatabaseProvider; use std::{ collections::BTreeMap, net::{IpAddr, Ipv4Addr, SocketAddr}, num::NonZeroU64, sync::Arc, }; - use async_trait::async_trait; - use data_types2::ChunkId; - use generated_types::{i_ox_testing_client::IOxTestingClient, tag_key_predicate::Value}; - use parking_lot::Mutex; - use service_common::QueryDatabaseProvider; - use tokio_stream::wrappers::TcpListenerStream; - use datafusion::logical_plan::{col, lit, Expr}; - use influxdb_storage_client::{ - connection::{Builder as ConnectionBuilder, Connection}, - generated_types::*, - Client as StorageClient, OrgAndBucket, - }; - use panic_logging::SendPanicsToTracing; - use predicate::{PredicateBuilder, PredicateMatch}; - use query::{ - exec::Executor, - test::{TestChunk, TestDatabase}, - }; use test_helpers::{assert_contains, tracing::TracingCapture}; - use super::*; - use metric::{Attributes, Metric, U64Counter}; + use tokio_stream::wrappers::TcpListenerStream; fn to_str_vec(s: &[&str]) -> Vec { s.iter().map(|s| s.to_string()).collect() From e9a42c418ac0ad96b81e2c3e2f25b0c9b17f1d70 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 15:18:08 -0400 Subject: [PATCH 04/31] fix: Only use data_types2 in generated_types --- Cargo.lock | 1 - generated_types/Cargo.toml | 4 +--- generated_types/src/delete_predicate.rs | 5 +---- generated_types/src/ingester.rs | 3 +-- 4 files changed, 3 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ac002def53..9f7c35caff 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1731,7 +1731,6 @@ name = "generated_types" version = "0.1.0" dependencies = [ "bytes", - "data_types", "data_types2", "datafusion 0.1.0", "iox_time", diff --git a/generated_types/Cargo.toml b/generated_types/Cargo.toml index c122009c9d..f791d52100 100644 --- a/generated_types/Cargo.toml +++ b/generated_types/Cargo.toml @@ -6,7 +6,6 @@ edition = "2021" [dependencies] # In alphabetical order bytes = "1.0" -data_types = { path = "../data_types", optional = true } data_types2 = { path = "../data_types2", optional = true } datafusion = { path = "../datafusion", optional = true } observability_deps = { path = "../observability_deps" } @@ -22,7 +21,6 @@ iox_time = { path = "../iox_time" } workspace-hack = { path = "../workspace-hack"} [dev-dependencies] -data_types = { path = "../data_types" } num_cpus = "1.13.0" [build-dependencies] # In alphabetical order @@ -32,4 +30,4 @@ pbjson-build = "0.3" [features] default = ["data_types_conversions"] -data_types_conversions = ["data_types", "data_types2", "datafusion", "predicate"] +data_types_conversions = ["data_types2", "datafusion", "predicate"] diff --git a/generated_types/src/delete_predicate.rs b/generated_types/src/delete_predicate.rs index 2fa7b2126d..4c4183de9a 100644 --- a/generated_types/src/delete_predicate.rs +++ b/generated_types/src/delete_predicate.rs @@ -11,10 +11,7 @@ use crate::google::{FieldViolation, FromOptionalField, FromRepeatedField, Option use crate::influxdata::iox::predicate::v1 as proto; use crate::influxdata::iox::predicate::v1::scalar::Value; use crate::influxdata::iox::predicate::v1::{Expr, Predicate}; -use data_types::{ - delete_predicate::{DeleteExpr, DeletePredicate, Op, Scalar}, - timestamp::TimestampRange, -}; +use data_types2::{DeleteExpr, DeletePredicate, Op, Scalar, TimestampRange}; impl From for proto::Predicate { fn from(predicate: DeletePredicate) -> Self { diff --git a/generated_types/src/ingester.rs b/generated_types/src/ingester.rs index d721d825f7..a75a1c2989 100644 --- a/generated_types/src/ingester.rs +++ b/generated_types/src/ingester.rs @@ -1,6 +1,5 @@ use crate::{google::FieldViolation, influxdata::iox::ingester::v1 as proto}; -use data_types::timestamp::TimestampRange; -use data_types2::IngesterQueryRequest; +use data_types2::{IngesterQueryRequest, TimestampRange}; use datafusion::{ common::DataFusionError, datafusion_proto::bytes::Serializeable, logical_plan::Expr, }; From 485d6edb8fea30950f4a340318f5fcaed2d06434 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 15:43:25 -0400 Subject: [PATCH 05/31] refactor: Move IngesterQueryRequest to generated_types --- data_types2/src/lib.rs | 33 +------------------ generated_types/src/ingester.rs | 33 ++++++++++++++++++- .../tests/end_to_end_ng_cases/ingester.rs | 11 +++---- ingester/src/handler.rs | 4 +-- ingester/src/querier_handler.rs | 2 +- querier/src/ingester/flight_client.rs | 15 ++++----- querier/src/ingester/mod.rs | 11 +++---- query_tests/src/scenarios/util.rs | 8 ++--- 8 files changed, 57 insertions(+), 60 deletions(-) diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 3860a32253..5e594251d7 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -11,7 +11,7 @@ )] use influxdb_line_protocol::FieldValue; -use predicate::{delete_predicate::parse_delete_predicate, Predicate}; +use predicate::delete_predicate::parse_delete_predicate; use schema::{builder::SchemaBuilder, sort::SortKey, InfluxColumnType, InfluxFieldType, Schema}; use std::{ collections::BTreeMap, @@ -922,37 +922,6 @@ pub struct ProcessedTombstone { pub parquet_file_id: ParquetFileId, } -/// Request from the querier service to the ingester service -#[derive(Debug, PartialEq, Clone)] -pub struct IngesterQueryRequest { - /// namespace to search - pub namespace: String, - /// Table to search - pub table: String, - /// Columns the query service is interested in - pub columns: Vec, - /// Predicate for filtering - pub predicate: Option, -} - -impl IngesterQueryRequest { - /// Make a request to return data for a specified table for - /// all sequencers an ingester is responsible for - pub fn new( - namespace: String, - table: String, - columns: Vec, - predicate: Option, - ) -> Self { - Self { - namespace, - table, - columns, - predicate, - } - } -} - /// Address of the chunk within the catalog #[derive(Debug, Clone, Eq, PartialEq, PartialOrd, Ord, Hash)] pub struct ChunkAddr { diff --git a/generated_types/src/ingester.rs b/generated_types/src/ingester.rs index a75a1c2989..1f4a30544d 100644 --- a/generated_types/src/ingester.rs +++ b/generated_types/src/ingester.rs @@ -1,5 +1,5 @@ use crate::{google::FieldViolation, influxdata::iox::ingester::v1 as proto}; -use data_types2::{IngesterQueryRequest, TimestampRange}; +use data_types2::TimestampRange; use datafusion::{ common::DataFusionError, datafusion_proto::bytes::Serializeable, logical_plan::Expr, }; @@ -19,6 +19,37 @@ fn expr_from_bytes_violation(field: impl Into, e: DataFusionError) -> Fi } } +/// Request from the querier service to the ingester service +#[derive(Debug, PartialEq, Clone)] +pub struct IngesterQueryRequest { + /// namespace to search + pub namespace: String, + /// Table to search + pub table: String, + /// Columns the query service is interested in + pub columns: Vec, + /// Predicate for filtering + pub predicate: Option, +} + +impl IngesterQueryRequest { + /// Make a request to return data for a specified table for + /// all sequencers an ingester is responsible for + pub fn new( + namespace: String, + table: String, + columns: Vec, + predicate: Option, + ) -> Self { + Self { + namespace, + table, + columns, + predicate, + } + } +} + impl TryFrom for IngesterQueryRequest { type Error = FieldViolation; diff --git a/influxdb_iox/tests/end_to_end_ng_cases/ingester.rs b/influxdb_iox/tests/end_to_end_ng_cases/ingester.rs index f662ade123..c9dd3f45f1 100644 --- a/influxdb_iox/tests/end_to_end_ng_cases/ingester.rs +++ b/influxdb_iox/tests/end_to_end_ng_cases/ingester.rs @@ -1,14 +1,13 @@ -use std::collections::BTreeMap; - -use generated_types::influxdata::iox::ingester::v1::PartitionStatus; +use arrow_util::assert_batches_sorted_eq; +use generated_types::{ + influxdata::iox::ingester::v1::PartitionStatus, ingester::IngesterQueryRequest, +}; use http::StatusCode; +use std::collections::BTreeMap; use test_helpers_end_to_end_ng::{ get_write_token, maybe_skip_integration, wait_for_readable, MiniCluster, }; -use arrow_util::assert_batches_sorted_eq; -use data_types2::IngesterQueryRequest; - #[tokio::test] async fn ingester_flight_api() { test_helpers::maybe_start_logging(); diff --git a/ingester/src/handler.rs b/ingester/src/handler.rs index e6a25dc1ef..e5e174796a 100644 --- a/ingester/src/handler.rs +++ b/ingester/src/handler.rs @@ -13,13 +13,13 @@ use crate::{ }; use async_trait::async_trait; use backoff::BackoffConfig; -use data_types2::{IngesterQueryRequest, KafkaPartition, KafkaTopic, Sequencer}; - +use data_types2::{KafkaPartition, KafkaTopic, Sequencer}; use futures::{ future::{BoxFuture, Shared}, stream::FuturesUnordered, FutureExt, StreamExt, TryFutureExt, }; +use generated_types::ingester::IngesterQueryRequest; use iox_catalog::interface::Catalog; use iox_time::SystemProvider; use object_store::DynObjectStore; diff --git a/ingester/src/querier_handler.rs b/ingester/src/querier_handler.rs index 16f5bd8bb7..c8f383ea6a 100644 --- a/ingester/src/querier_handler.rs +++ b/ingester/src/querier_handler.rs @@ -5,7 +5,6 @@ use crate::data::{ }; use arrow::record_batch::RecordBatch; use arrow_util::util::merge_record_batches; -use data_types2::IngesterQueryRequest; use datafusion::{ error::DataFusionError, physical_plan::{ @@ -14,6 +13,7 @@ use datafusion::{ SendableRecordBatchStream, }, }; +use generated_types::ingester::IngesterQueryRequest; use predicate::Predicate; use query::{ exec::{Executor, ExecutorType}, diff --git a/querier/src/ingester/flight_client.rs b/querier/src/ingester/flight_client.rs index 7cb4c03409..41fc3dddc7 100644 --- a/querier/src/ingester/flight_client.rs +++ b/querier/src/ingester/flight_client.rs @@ -1,17 +1,16 @@ +use arrow::{datatypes::Schema, record_batch::RecordBatch}; +use async_trait::async_trait; +use client_util::connection; +use generated_types::ingester::IngesterQueryRequest; +use influxdb_iox_client::flight::{self, generated_types::IngesterQueryResponseMetadata}; +use observability_deps::tracing::debug; +use snafu::{ResultExt, Snafu}; use std::{ fmt::Debug, ops::{Deref, DerefMut}, sync::Arc, }; -use arrow::{datatypes::Schema, record_batch::RecordBatch}; -use async_trait::async_trait; -use client_util::connection; -use data_types2::IngesterQueryRequest; -use influxdb_iox_client::flight::{self, generated_types::IngesterQueryResponseMetadata}; -use observability_deps::tracing::debug; -use snafu::{ResultExt, Snafu}; - pub use flight::{Error as FlightError, PerformQuery}; #[derive(Debug, Snafu)] diff --git a/querier/src/ingester/mod.rs b/querier/src/ingester/mod.rs index 797f7b5e20..529d1a6105 100644 --- a/querier/src/ingester/mod.rs +++ b/querier/src/ingester/mod.rs @@ -1,14 +1,14 @@ -use std::{any::Any, collections::HashMap, sync::Arc}; - +use crate::cache::CatalogCache; use arrow::{datatypes::DataType, error::ArrowError, record_batch::RecordBatch}; use async_trait::async_trait; use data_types::timestamp::TimestampMinMax; use data_types2::{ - ChunkAddr, ChunkId, ChunkOrder, ColumnSummary, InfluxDbType, IngesterQueryRequest, PartitionId, - SequenceNumber, SequencerId, StatValues, Statistics, TableSummary, + ChunkAddr, ChunkId, ChunkOrder, ColumnSummary, InfluxDbType, PartitionId, SequenceNumber, + SequencerId, StatValues, Statistics, TableSummary, }; use datafusion_util::MemoryStream; use futures::{stream::FuturesUnordered, TryStreamExt}; +use generated_types::ingester::IngesterQueryRequest; use observability_deps::tracing::{debug, trace}; use predicate::{Predicate, PredicateMatch}; use query::{ @@ -18,8 +18,7 @@ use query::{ }; use schema::{selection::Selection, sort::SortKey, InfluxColumnType, InfluxFieldType, Schema}; use snafu::{ensure, OptionExt, ResultExt, Snafu}; - -use crate::cache::CatalogCache; +use std::{any::Any, collections::HashMap, sync::Arc}; use self::{ flight_client::{Error as FlightClientError, FlightClient, FlightClientImpl, FlightError}, diff --git a/query_tests/src/scenarios/util.rs b/query_tests/src/scenarios/util.rs index d3d6caf041..480a47a6fe 100644 --- a/query_tests/src/scenarios/util.rs +++ b/query_tests/src/scenarios/util.rs @@ -5,13 +5,13 @@ use async_trait::async_trait; use backoff::BackoffConfig; use data_types::delete_predicate::DeletePredicate; use data_types2::{ - IngesterQueryRequest, NonEmptyString, PartitionId, Sequence, SequenceNumber, SequencerId, - TombstoneId, + NonEmptyString, PartitionId, Sequence, SequenceNumber, SequencerId, TombstoneId, }; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; use futures::StreamExt; -use generated_types::influxdata::iox::ingester::v1::{ - IngesterQueryResponseMetadata, PartitionStatus, +use generated_types::{ + influxdata::iox::ingester::v1::{IngesterQueryResponseMetadata, PartitionStatus}, + ingester::IngesterQueryRequest, }; use influxdb_iox_client::flight::Error as FlightError; use ingester::data::{IngesterData, IngesterQueryResponse, Persister, SequencerData}; From eb31b347b0b42ae039c0c6a353cbce7676048514 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 15:56:17 -0400 Subject: [PATCH 06/31] refactor: Move tombstones_to_delete_predicates to the predicate crate --- Cargo.lock | 2 +- compactor/src/query.rs | 6 +++--- data_types2/Cargo.toml | 1 - data_types2/src/lib.rs | 22 --------------------- ingester/src/query.rs | 9 ++++++--- predicate/Cargo.toml | 1 + predicate/src/delete_predicate.rs | 33 +++++++++++++++++++++++-------- 7 files changed, 36 insertions(+), 38 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9f7c35caff..c7278dabde 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1174,7 +1174,6 @@ version = "0.1.0" dependencies = [ "data_types", "influxdb_line_protocol", - "predicate", "schema", "sqlx", "uuid 0.8.2", @@ -4054,6 +4053,7 @@ dependencies = [ "arrow", "chrono", "data_types", + "data_types2", "datafusion 0.1.0", "datafusion_util", "itertools", diff --git a/compactor/src/query.rs b/compactor/src/query.rs index 7d882f346e..fe179c17d4 100644 --- a/compactor/src/query.rs +++ b/compactor/src/query.rs @@ -4,13 +4,13 @@ use std::sync::Arc; use data_types::timestamp::TimestampMinMax; use data_types2::{ - tombstones_to_delete_predicates, ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, - SequenceNumber, TableSummary, Timestamp, Tombstone, + ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, + Timestamp, Tombstone, }; use datafusion::physical_plan::SendableRecordBatchStream; use observability_deps::tracing::trace; use parquet_file::chunk::ParquetChunk; -use predicate::{Predicate, PredicateMatch}; +use predicate::{delete_predicate::tombstones_to_delete_predicates, Predicate, PredicateMatch}; use query::{ exec::{stringset::StringSet, IOxSessionContext}, QueryChunk, QueryChunkError, QueryChunkMeta, diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index 4f03457c81..5e0b6c55c1 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -7,7 +7,6 @@ description = "Shared data types in the Iox NG architecture" [dependencies] data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } -predicate = { path = "../predicate" } schema = { path = "../schema" } sqlx = { version = "0.5", features = ["runtime-tokio-rustls", "postgres", "uuid"] } uuid = { version = "0.8", features = ["v4"] } diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 5e594251d7..763d3c8df8 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -11,7 +11,6 @@ )] use influxdb_line_protocol::FieldValue; -use predicate::delete_predicate::parse_delete_predicate; use schema::{builder::SchemaBuilder, sort::SortKey, InfluxColumnType, InfluxFieldType, Schema}; use std::{ collections::BTreeMap, @@ -699,27 +698,6 @@ 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, Copy, PartialEq, sqlx::FromRow)] pub struct ParquetFile { diff --git a/ingester/src/query.rs b/ingester/src/query.rs index aff10d9b28..6d3a3b5ff4 100644 --- a/ingester/src/query.rs +++ b/ingester/src/query.rs @@ -5,8 +5,8 @@ use arrow::record_batch::RecordBatch; use arrow_util::util::merge_record_batches; use data_types::timestamp::TimestampMinMax; use data_types2::{ - tombstones_to_delete_predicates, tombstones_to_delete_predicates_iter, ChunkAddr, ChunkId, - ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, Tombstone, + ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, + Tombstone, }; use datafusion::{ logical_plan::ExprRewritable, @@ -18,7 +18,10 @@ use datafusion::{ }; use datafusion_util::batch_filter; use observability_deps::tracing::{debug, trace}; -use predicate::{Predicate, PredicateMatch}; +use predicate::{ + delete_predicate::{tombstones_to_delete_predicates, tombstones_to_delete_predicates_iter}, + Predicate, PredicateMatch, +}; use query::{ exec::{stringset::StringSet, IOxSessionContext}, util::{df_physical_expr_from_schema_and_expr, MissingColumnsToNull}, diff --git a/predicate/Cargo.toml b/predicate/Cargo.toml index 763e383be1..098ec60c7d 100644 --- a/predicate/Cargo.toml +++ b/predicate/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" arrow = { version = "13", features = ["prettyprint"] } chrono = { version = "0.4", default-features = false } data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } itertools = "0.10" diff --git a/predicate/src/delete_predicate.rs b/predicate/src/delete_predicate.rs index 024de46ca0..a97d9ba5b5 100644 --- a/predicate/src/delete_predicate.rs +++ b/predicate/src/delete_predicate.rs @@ -1,18 +1,14 @@ +use crate::delete_expr::{df_to_expr, expr_to_df}; use chrono::DateTime; +use data_types2::{DeleteExpr, DeletePredicate, TimestampRange, Tombstone}; +use datafusion::logical_plan::{lit, Column, Expr, Operator}; use snafu::{ResultExt, Snafu}; use sqlparser::{ ast::{BinaryOperator, Expr as SqlParserExpr, Ident, Statement, Value}, dialect::GenericDialect, parser::Parser, }; - -use data_types::{ - delete_predicate::{DeleteExpr, DeletePredicate}, - timestamp::TimestampRange, -}; -use datafusion::logical_plan::{lit, Column, Expr, Operator}; - -use crate::delete_expr::{df_to_expr, expr_to_df}; +use std::sync::Arc; const FLUX_TABLE: &str = "_measurement"; @@ -80,6 +76,27 @@ impl From for crate::Predicate { } } +/// 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"), + ) + }) +} + /// Parse and convert the delete grpc API into ParseDeletePredicate to send to server pub fn parse_delete_predicate( start_time: &str, From f39b0933572790d5bd5f0dbfe6b10556b066ca0e Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 15:59:00 -0400 Subject: [PATCH 07/31] fix: Only use data_types2 in predicate --- Cargo.lock | 1 - data_types2/src/lib.rs | 2 +- predicate/Cargo.toml | 1 - predicate/src/delete_expr.rs | 5 ++--- predicate/src/delete_predicate.rs | 3 +-- predicate/src/lib.rs | 2 +- 6 files changed, 5 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c7278dabde..65c96318b8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4052,7 +4052,6 @@ version = "0.1.0" dependencies = [ "arrow", "chrono", - "data_types", "data_types2", "datafusion 0.1.0", "datafusion_util", diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 763d3c8df8..a0fc2f610b 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -30,7 +30,7 @@ pub use data_types::{ ColumnSummary, InfluxDbType, PartitionAddr, StatValues, Statistics, TableSummary, }, sequence::Sequence, - timestamp::{TimestampMinMax, TimestampRange}, + timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}, DatabaseName, }; diff --git a/predicate/Cargo.toml b/predicate/Cargo.toml index 098ec60c7d..224ed5ea5f 100644 --- a/predicate/Cargo.toml +++ b/predicate/Cargo.toml @@ -6,7 +6,6 @@ edition = "2021" [dependencies] arrow = { version = "13", features = ["prettyprint"] } chrono = { version = "0.4", default-features = false } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } diff --git a/predicate/src/delete_expr.rs b/predicate/src/delete_expr.rs index f17b977c00..8ca028363b 100644 --- a/predicate/src/delete_expr.rs +++ b/predicate/src/delete_expr.rs @@ -1,7 +1,6 @@ -use std::ops::Deref; - -use data_types::delete_predicate::{DeleteExpr, Op, Scalar}; +use data_types2::{DeleteExpr, Op, Scalar}; use snafu::{ResultExt, Snafu}; +use std::ops::Deref; pub(crate) fn expr_to_df(expr: DeleteExpr) -> datafusion::logical_plan::Expr { use datafusion::logical_plan::Expr; diff --git a/predicate/src/delete_predicate.rs b/predicate/src/delete_predicate.rs index a97d9ba5b5..397fd286ea 100644 --- a/predicate/src/delete_predicate.rs +++ b/predicate/src/delete_predicate.rs @@ -396,9 +396,8 @@ pub fn parse_http_delete_request(input: &str) -> Result { #[cfg(test)] mod tests { - use data_types::delete_predicate::{Op, Scalar}; - use super::*; + use data_types2::{Op, Scalar}; #[test] fn test_time_range_valid() { diff --git a/predicate/src/lib.rs b/predicate/src/lib.rs index 4e71bd9480..28c4efbf38 100644 --- a/predicate/src/lib.rs +++ b/predicate/src/lib.rs @@ -13,7 +13,7 @@ pub mod delete_predicate; pub mod rewrite; pub mod rpc_predicate; -use data_types::timestamp::{TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}; +use data_types2::{TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}; use datafusion::{ error::DataFusionError, logical_plan::{binary_expr, col, lit_timestamp_nano, Expr, Operator}, From 3ab0788a947bf5ae92ab4b1c9598803ad5e8e5ee Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 16:41:09 -0400 Subject: [PATCH 08/31] fix: Move DeletePredicate types to data_types2 --- Cargo.lock | 3 + data_types/src/delete_predicate.rs | 357 -------------------------- data_types/src/lib.rs | 1 - data_types2/Cargo.toml | 3 +- data_types2/src/lib.rs | 358 ++++++++++++++++++++++++++- dml/Cargo.toml | 1 + dml/src/lib.rs | 23 +- query_tests/src/scenarios/delete.rs | 70 ++---- query_tests/src/scenarios/library.rs | 29 +-- query_tests/src/scenarios/util.rs | 27 +- write_buffer/Cargo.toml | 1 + write_buffer/src/codec.rs | 29 +-- write_buffer/src/kafka/mod.rs | 52 ++-- 13 files changed, 458 insertions(+), 496 deletions(-) delete mode 100644 data_types/src/delete_predicate.rs diff --git a/Cargo.lock b/Cargo.lock index 65c96318b8..1a8f7d86b4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1174,6 +1174,7 @@ version = "0.1.0" dependencies = [ "data_types", "influxdb_line_protocol", + "ordered-float 3.0.0", "schema", "sqlx", "uuid 0.8.2", @@ -1409,6 +1410,7 @@ version = "0.1.0" dependencies = [ "arrow_util", "data_types", + "data_types2", "hashbrown 0.12.0", "iox_time", "mutable_batch", @@ -6784,6 +6786,7 @@ version = "0.1.0" dependencies = [ "async-trait", "data_types", + "data_types2", "dml", "dotenv", "futures", diff --git a/data_types/src/delete_predicate.rs b/data_types/src/delete_predicate.rs deleted file mode 100644 index e10dab7112..0000000000 --- a/data_types/src/delete_predicate.rs +++ /dev/null @@ -1,357 +0,0 @@ -use crate::timestamp::TimestampRange; -use std::{fmt::Write, num::FpCategory}; - -/// Represents a parsed delete predicate for evaluation by the InfluxDB IOx -/// query engine. -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct DeletePredicate { - /// Only rows within this range are included in - /// results. Other rows are excluded. - pub range: TimestampRange, - - /// Optional arbitrary predicates, represented as list of - /// expressions applied a logical conjunction (aka they - /// are 'AND'ed together). Only rows that evaluate to TRUE for all - /// these expressions should be returned. Other rows are excluded - /// from the results. - pub exprs: Vec, -} - -impl DeletePredicate { - /// Format expr to SQL string. - pub fn expr_sql_string(&self) -> String { - let mut out = String::new(); - for expr in &self.exprs { - if !out.is_empty() { - write!(&mut out, " AND ").expect("writing to a string shouldn't fail"); - } - write!(&mut out, "{}", expr).expect("writing to a string shouldn't fail"); - } - out - } - - /// Return the approximate memory size of the predicate, in bytes. - /// - /// This includes `Self`. - pub fn size(&self) -> usize { - std::mem::size_of::() + self.exprs.iter().map(|expr| expr.size()).sum::() - } -} - -/// Single expression to be used as parts of a predicate. -/// -/// Only very simple expression of the type ` ` are supported. -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct DeleteExpr { - /// Column (w/o table name). - pub column: String, - - /// Operator. - pub op: Op, - - /// Scalar value. - pub scalar: Scalar, -} - -impl DeleteExpr { - /// Create a new [`DeleteExpr`] - pub fn new(column: String, op: Op, scalar: Scalar) -> Self { - Self { column, op, scalar } - } - - /// Column (w/o table name). - pub fn column(&self) -> &str { - &self.column - } - - /// Operator. - pub fn op(&self) -> Op { - self.op - } - - /// Scalar value. - pub fn scalar(&self) -> &Scalar { - &self.scalar - } - - /// Return the approximate memory size of the expression, in bytes. - /// - /// This includes `Self`. - pub fn size(&self) -> usize { - std::mem::size_of::() + self.column.capacity() + self.scalar.size() - } -} - -impl std::fmt::Display for DeleteExpr { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - r#""{}"{}{}"#, - self.column().replace('\\', r#"\\"#).replace('"', r#"\""#), - self.op(), - self.scalar(), - ) - } -} - -/// Binary operator that can be evaluated on a column and a scalar value. -#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub enum Op { - /// Strict equality (`=`). - Eq, - - /// Inequality (`!=`). - Ne, -} - -impl std::fmt::Display for Op { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Self::Eq => write!(f, "="), - Self::Ne => write!(f, "!="), - } - } -} - -/// Scalar value of a certain type. -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] -#[allow(missing_docs)] -pub enum Scalar { - Bool(bool), - I64(i64), - F64(ordered_float::OrderedFloat), - String(String), -} - -impl Scalar { - /// Return the approximate memory size of the scalar, in bytes. - /// - /// This includes `Self`. - pub fn size(&self) -> usize { - std::mem::size_of::() - + match &self { - Self::Bool(_) | Self::I64(_) | Self::F64(_) => 0, - Self::String(s) => s.capacity(), - } - } -} - -impl std::fmt::Display for Scalar { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Scalar::Bool(value) => value.fmt(f), - Scalar::I64(value) => value.fmt(f), - Scalar::F64(value) => match value.classify() { - FpCategory::Nan => write!(f, "'NaN'"), - FpCategory::Infinite if *value.as_ref() < 0.0 => write!(f, "'-Infinity'"), - FpCategory::Infinite => write!(f, "'Infinity'"), - _ => write!(f, "{:?}", value.as_ref()), - }, - Scalar::String(value) => { - write!( - f, - "'{}'", - value.replace('\\', r#"\\"#).replace('\'', r#"\'"#), - ) - } - } - } -} - -#[cfg(test)] -mod tests { - use ordered_float::OrderedFloat; - - use super::*; - - #[test] - fn test_expr_to_sql_no_expressions() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![], - }; - assert_eq!(&pred.expr_sql_string(), ""); - } - - #[test] - fn test_expr_to_sql_operators() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col1"), - op: Op::Eq, - scalar: Scalar::I64(1), - }, - DeleteExpr { - column: String::from("col2"), - op: Op::Ne, - scalar: Scalar::I64(2), - }, - ], - }; - assert_eq!(&pred.expr_sql_string(), r#""col1"=1 AND "col2"!=2"#); - } - - #[test] - fn test_expr_to_sql_column_escape() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col 1"), - op: Op::Eq, - scalar: Scalar::I64(1), - }, - DeleteExpr { - column: String::from(r#"col\2"#), - op: Op::Eq, - scalar: Scalar::I64(2), - }, - DeleteExpr { - column: String::from(r#"col"3"#), - op: Op::Eq, - scalar: Scalar::I64(3), - }, - ], - }; - assert_eq!( - &pred.expr_sql_string(), - r#""col 1"=1 AND "col\\2"=2 AND "col\"3"=3"# - ); - } - - #[test] - fn test_expr_to_sql_bool() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col1"), - op: Op::Eq, - scalar: Scalar::Bool(false), - }, - DeleteExpr { - column: String::from("col2"), - op: Op::Eq, - scalar: Scalar::Bool(true), - }, - ], - }; - assert_eq!(&pred.expr_sql_string(), r#""col1"=false AND "col2"=true"#); - } - - #[test] - fn test_expr_to_sql_i64() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col1"), - op: Op::Eq, - scalar: Scalar::I64(0), - }, - DeleteExpr { - column: String::from("col2"), - op: Op::Eq, - scalar: Scalar::I64(-1), - }, - DeleteExpr { - column: String::from("col3"), - op: Op::Eq, - scalar: Scalar::I64(1), - }, - DeleteExpr { - column: String::from("col4"), - op: Op::Eq, - scalar: Scalar::I64(i64::MIN), - }, - DeleteExpr { - column: String::from("col5"), - op: Op::Eq, - scalar: Scalar::I64(i64::MAX), - }, - ], - }; - assert_eq!( - &pred.expr_sql_string(), - r#""col1"=0 AND "col2"=-1 AND "col3"=1 AND "col4"=-9223372036854775808 AND "col5"=9223372036854775807"# - ); - } - - #[test] - fn test_expr_to_sql_f64() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col1"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(0.0)), - }, - DeleteExpr { - column: String::from("col2"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(-0.0)), - }, - DeleteExpr { - column: String::from("col3"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(1.0)), - }, - DeleteExpr { - column: String::from("col4"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(f64::INFINITY)), - }, - DeleteExpr { - column: String::from("col5"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(f64::NEG_INFINITY)), - }, - DeleteExpr { - column: String::from("col6"), - op: Op::Eq, - scalar: Scalar::F64(OrderedFloat::from(f64::NAN)), - }, - ], - }; - assert_eq!( - &pred.expr_sql_string(), - r#""col1"=0.0 AND "col2"=-0.0 AND "col3"=1.0 AND "col4"='Infinity' AND "col5"='-Infinity' AND "col6"='NaN'"# - ); - } - - #[test] - fn test_expr_to_sql_string() { - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![ - DeleteExpr { - column: String::from("col1"), - op: Op::Eq, - scalar: Scalar::String(String::from("")), - }, - DeleteExpr { - column: String::from("col2"), - op: Op::Eq, - scalar: Scalar::String(String::from("foo")), - }, - DeleteExpr { - column: String::from("col3"), - op: Op::Eq, - scalar: Scalar::String(String::from(r#"fo\o"#)), - }, - DeleteExpr { - column: String::from("col4"), - op: Op::Eq, - scalar: Scalar::String(String::from(r#"fo'o"#)), - }, - ], - }; - assert_eq!( - &pred.expr_sql_string(), - r#""col1"='' AND "col2"='foo' AND "col3"='fo\\o' AND "col4"='fo\'o'"# - ); - } -} diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 0ee1d52f33..622fad2327 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -15,7 +15,6 @@ pub mod chunk_metadata; pub mod consistent_hasher; mod database_name; pub mod database_rules; -pub mod delete_predicate; pub mod error; pub mod job; pub mod names; diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index 5e0b6c55c1..5ce2b0a42b 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -2,11 +2,12 @@ name = "data_types2" version = "0.1.0" edition = "2021" -description = "Shared data types in the Iox NG architecture" +description = "Shared data types in the IOx NG architecture" [dependencies] data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } +ordered-float = "3" schema = { path = "../schema" } sqlx = { version = "0.5", features = ["runtime-tokio-rustls", "postgres", "uuid"] } uuid = { version = "0.8", features = ["v4"] } diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index a0fc2f610b..978c7811ac 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -1,4 +1,4 @@ -//! Shared data types in the Iox NG architecture +//! Shared data types in the IOx NG architecture #![warn( missing_copy_implementations, @@ -15,15 +15,14 @@ use schema::{builder::SchemaBuilder, sort::SortKey, InfluxColumnType, InfluxFiel use std::{ collections::BTreeMap, convert::TryFrom, - fmt::{Debug, Formatter}, - num::NonZeroU32, + fmt::Write, + num::{FpCategory, NonZeroU32}, ops::{Add, Sub}, sync::Arc, }; use uuid::Uuid; pub use data_types::{ - delete_predicate::{DeleteExpr, DeletePredicate, Op, Scalar}, names::{org_and_bucket_to_database, OrgBucketMappingError}, non_empty::NonEmptyString, partition_metadata::{ @@ -535,7 +534,7 @@ impl ColumnType { } impl std::fmt::Display for ColumnType { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let s = self.as_str(); write!(f, "{}", s) @@ -1063,9 +1062,167 @@ pub struct StrftimeColumn { pub format: String, } +/// Represents a parsed delete predicate for evaluation by the InfluxDB IOx +/// query engine. +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct DeletePredicate { + /// Only rows within this range are included in + /// results. Other rows are excluded. + pub range: TimestampRange, + + /// Optional arbitrary predicates, represented as list of + /// expressions applied a logical conjunction (aka they + /// are 'AND'ed together). Only rows that evaluate to TRUE for all + /// these expressions should be returned. Other rows are excluded + /// from the results. + pub exprs: Vec, +} + +impl DeletePredicate { + /// Format expr to SQL string. + pub fn expr_sql_string(&self) -> String { + let mut out = String::new(); + for expr in &self.exprs { + if !out.is_empty() { + write!(&mut out, " AND ").expect("writing to a string shouldn't fail"); + } + write!(&mut out, "{}", expr).expect("writing to a string shouldn't fail"); + } + out + } + + /// Return the approximate memory size of the predicate, in bytes. + /// + /// This includes `Self`. + pub fn size(&self) -> usize { + std::mem::size_of::() + self.exprs.iter().map(|expr| expr.size()).sum::() + } +} + +/// Single expression to be used as parts of a predicate. +/// +/// Only very simple expression of the type ` ` are supported. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct DeleteExpr { + /// Column (w/o table name). + pub column: String, + + /// Operator. + pub op: Op, + + /// Scalar value. + pub scalar: Scalar, +} + +impl DeleteExpr { + /// Create a new [`DeleteExpr`] + pub fn new(column: String, op: Op, scalar: Scalar) -> Self { + Self { column, op, scalar } + } + + /// Column (w/o table name). + pub fn column(&self) -> &str { + &self.column + } + + /// Operator. + pub fn op(&self) -> Op { + self.op + } + + /// Scalar value. + pub fn scalar(&self) -> &Scalar { + &self.scalar + } + + /// Return the approximate memory size of the expression, in bytes. + /// + /// This includes `Self`. + pub fn size(&self) -> usize { + std::mem::size_of::() + self.column.capacity() + self.scalar.size() + } +} + +impl std::fmt::Display for DeleteExpr { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + r#""{}"{}{}"#, + self.column().replace('\\', r#"\\"#).replace('"', r#"\""#), + self.op(), + self.scalar(), + ) + } +} + +/// Binary operator that can be evaluated on a column and a scalar value. +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub enum Op { + /// Strict equality (`=`). + Eq, + + /// Inequality (`!=`). + Ne, +} + +impl std::fmt::Display for Op { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::Eq => write!(f, "="), + Self::Ne => write!(f, "!="), + } + } +} + +/// Scalar value of a certain type. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +#[allow(missing_docs)] +pub enum Scalar { + Bool(bool), + I64(i64), + F64(ordered_float::OrderedFloat), + String(String), +} + +impl Scalar { + /// Return the approximate memory size of the scalar, in bytes. + /// + /// This includes `Self`. + pub fn size(&self) -> usize { + std::mem::size_of::() + + match &self { + Self::Bool(_) | Self::I64(_) | Self::F64(_) => 0, + Self::String(s) => s.capacity(), + } + } +} + +impl std::fmt::Display for Scalar { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Scalar::Bool(value) => value.fmt(f), + Scalar::I64(value) => value.fmt(f), + Scalar::F64(value) => match value.classify() { + FpCategory::Nan => write!(f, "'NaN'"), + FpCategory::Infinite if *value.as_ref() < 0.0 => write!(f, "'-Infinity'"), + FpCategory::Infinite => write!(f, "'Infinity'"), + _ => write!(f, "{:?}", value.as_ref()), + }, + Scalar::String(value) => { + write!( + f, + "'{}'", + value.replace('\\', r#"\\"#).replace('\'', r#"\'"#), + ) + } + } + } +} + #[cfg(test)] mod tests { use super::*; + use ordered_float::OrderedFloat; #[test] fn test_chunk_id_new() { @@ -1093,4 +1250,195 @@ mod tests { assert_eq!(format!("{:?}", id_test), "ChunkId(42)"); assert_eq!(format!("{}", id_test), "ChunkId(42)"); } + + #[test] + fn test_expr_to_sql_no_expressions() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![], + }; + assert_eq!(&pred.expr_sql_string(), ""); + } + + #[test] + fn test_expr_to_sql_operators() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col1"), + op: Op::Eq, + scalar: Scalar::I64(1), + }, + DeleteExpr { + column: String::from("col2"), + op: Op::Ne, + scalar: Scalar::I64(2), + }, + ], + }; + assert_eq!(&pred.expr_sql_string(), r#""col1"=1 AND "col2"!=2"#); + } + + #[test] + fn test_expr_to_sql_column_escape() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col 1"), + op: Op::Eq, + scalar: Scalar::I64(1), + }, + DeleteExpr { + column: String::from(r#"col\2"#), + op: Op::Eq, + scalar: Scalar::I64(2), + }, + DeleteExpr { + column: String::from(r#"col"3"#), + op: Op::Eq, + scalar: Scalar::I64(3), + }, + ], + }; + assert_eq!( + &pred.expr_sql_string(), + r#""col 1"=1 AND "col\\2"=2 AND "col\"3"=3"# + ); + } + + #[test] + fn test_expr_to_sql_bool() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col1"), + op: Op::Eq, + scalar: Scalar::Bool(false), + }, + DeleteExpr { + column: String::from("col2"), + op: Op::Eq, + scalar: Scalar::Bool(true), + }, + ], + }; + assert_eq!(&pred.expr_sql_string(), r#""col1"=false AND "col2"=true"#); + } + + #[test] + fn test_expr_to_sql_i64() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col1"), + op: Op::Eq, + scalar: Scalar::I64(0), + }, + DeleteExpr { + column: String::from("col2"), + op: Op::Eq, + scalar: Scalar::I64(-1), + }, + DeleteExpr { + column: String::from("col3"), + op: Op::Eq, + scalar: Scalar::I64(1), + }, + DeleteExpr { + column: String::from("col4"), + op: Op::Eq, + scalar: Scalar::I64(i64::MIN), + }, + DeleteExpr { + column: String::from("col5"), + op: Op::Eq, + scalar: Scalar::I64(i64::MAX), + }, + ], + }; + assert_eq!( + &pred.expr_sql_string(), + r#""col1"=0 AND "col2"=-1 AND "col3"=1 AND "col4"=-9223372036854775808 AND "col5"=9223372036854775807"# + ); + } + + #[test] + fn test_expr_to_sql_f64() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col1"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(0.0)), + }, + DeleteExpr { + column: String::from("col2"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(-0.0)), + }, + DeleteExpr { + column: String::from("col3"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(1.0)), + }, + DeleteExpr { + column: String::from("col4"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(f64::INFINITY)), + }, + DeleteExpr { + column: String::from("col5"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(f64::NEG_INFINITY)), + }, + DeleteExpr { + column: String::from("col6"), + op: Op::Eq, + scalar: Scalar::F64(OrderedFloat::from(f64::NAN)), + }, + ], + }; + assert_eq!( + &pred.expr_sql_string(), + r#""col1"=0.0 AND "col2"=-0.0 AND "col3"=1.0 AND "col4"='Infinity' AND "col5"='-Infinity' AND "col6"='NaN'"# + ); + } + + #[test] + fn test_expr_to_sql_string() { + let pred = DeletePredicate { + range: TimestampRange::new(1, 2), + exprs: vec![ + DeleteExpr { + column: String::from("col1"), + op: Op::Eq, + scalar: Scalar::String(String::from("")), + }, + DeleteExpr { + column: String::from("col2"), + op: Op::Eq, + scalar: Scalar::String(String::from("foo")), + }, + DeleteExpr { + column: String::from("col3"), + op: Op::Eq, + scalar: Scalar::String(String::from(r#"fo\o"#)), + }, + DeleteExpr { + column: String::from("col4"), + op: Op::Eq, + scalar: Scalar::String(String::from(r#"fo'o"#)), + }, + ], + }; + assert_eq!( + &pred.expr_sql_string(), + r#""col1"='' AND "col2"='foo' AND "col3"='fo\\o' AND "col4"='fo\'o'"# + ); + } } diff --git a/dml/Cargo.toml b/dml/Cargo.toml index 25a62fb2cd..4606ad7ab2 100644 --- a/dml/Cargo.toml +++ b/dml/Cargo.toml @@ -7,6 +7,7 @@ description = "DML types" [dependencies] arrow_util = { path = "../arrow_util" } data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } hashbrown = "0.12" mutable_batch = { path = "../mutable_batch" } ordered-float = "3" diff --git a/dml/src/lib.rs b/dml/src/lib.rs index c52db75bf1..29a6b9e10f 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -11,17 +11,17 @@ clippy::clone_on_ref_ptr )] -use std::collections::{BTreeMap, HashSet}; - -use data_types::router::{ShardConfig, ShardId}; +use data_types::{ + non_empty::NonEmptyString, + partition_metadata::{StatValues, Statistics}, + router::{ShardConfig, ShardId}, + sequence::Sequence, +}; +use data_types2::DeletePredicate; use hashbrown::HashMap; - -use data_types::delete_predicate::DeletePredicate; -use data_types::non_empty::NonEmptyString; -use data_types::partition_metadata::{StatValues, Statistics}; -use data_types::sequence::Sequence; use iox_time::Time; use mutable_batch::MutableBatch; +use std::collections::{BTreeMap, HashSet}; use trace::ctx::SpanContext; /// Metadata information about a DML operation @@ -487,9 +487,10 @@ pub mod test_util { #[cfg(test)] mod tests { + use super::*; + use crate::test_util::assert_writes_eq; use data_types::{ consistent_hasher::ConsistentHasher, - delete_predicate::DeletePredicate, non_empty::NonEmptyString, router::{HashRing, Matcher, MatcherToShard}, timestamp::TimestampRange, @@ -497,10 +498,6 @@ mod tests { use mutable_batch_lp::lines_to_batches; use regex::Regex; - use crate::test_util::assert_writes_eq; - - use super::*; - #[test] fn test_write_sharding() { let config = ShardConfig { diff --git a/query_tests/src/scenarios/delete.rs b/query_tests/src/scenarios/delete.rs index a6b480224f..05d80c6433 100644 --- a/query_tests/src/scenarios/delete.rs +++ b/query_tests/src/scenarios/delete.rs @@ -1,16 +1,16 @@ //! This module contains testing scenarios for Delete -use data_types::delete_predicate::{DeleteExpr, DeletePredicate}; -use data_types::timestamp::TimestampRange; - -use async_trait::async_trait; - -use super::util::{make_n_chunks_scenario_new, ChunkDataNew, DeleteTimeNew, PredNew}; -use super::{DbScenario, DbSetup}; +use super::{ + util::{make_n_chunks_scenario_new, ChunkDataNew, DeleteTimeNew, PredNew}, + DbScenario, DbSetup, +}; use crate::scenarios::util::all_scenarios_for_one_chunk; +use async_trait::async_trait; +use data_types2::{DeleteExpr, DeletePredicate, Op, Scalar, TimestampRange}; -// ========================================================================================================================= -// DELETE TEST SETUPS: chunk lp data, how many chunks, their types, how many delete predicates and when they happen +// ================================================================================================ +// DELETE TEST SETUPS: chunk lp data, how many chunks, their types, how many delete predicates and +// when they happen #[derive(Debug)] /// Setup for delete query test with one table and one chunk moved from MUB to RUB to OS @@ -53,8 +53,8 @@ impl DbSetup for OneDeleteSimpleExprOneChunk { range: TimestampRange::new(0, 15), exprs: vec![DeleteExpr::new( "bar".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::F64((1.0).into()), + Op::Eq, + Scalar::F64((1.0).into()), )], }; @@ -104,16 +104,8 @@ impl DbSetup for OneDeleteMultiExprsOneChunk { let pred = DeletePredicate { range: TimestampRange::new(0, 30), exprs: vec![ - DeleteExpr::new( - "bar".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::F64((1.0).into()), - ), - DeleteExpr::new( - "foo".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("me".to_string()), - ), + DeleteExpr::new("bar".to_string(), Op::Eq, Scalar::F64((1.0).into())), + DeleteExpr::new("foo".to_string(), Op::Eq, Scalar::String("me".to_string())), ], }; @@ -147,16 +139,8 @@ impl DbSetup for TwoDeletesMultiExprsOneChunk { let pred1 = DeletePredicate { range: TimestampRange::new(0, 32), exprs: vec![ - DeleteExpr::new( - "bar".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::F64((1.0).into()), - ), - DeleteExpr::new( - "foo".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("me".to_string()), - ), + DeleteExpr::new("bar".to_string(), Op::Eq, Scalar::F64((1.0).into())), + DeleteExpr::new("foo".to_string(), Op::Eq, Scalar::String("me".to_string())), ], }; @@ -165,8 +149,8 @@ impl DbSetup for TwoDeletesMultiExprsOneChunk { range: TimestampRange::new(10, 40), exprs: vec![DeleteExpr::new( "bar".to_string(), - data_types::delete_predicate::Op::Ne, - data_types::delete_predicate::Scalar::F64((1.0).into()), + Op::Ne, + Scalar::F64((1.0).into()), )], }; @@ -204,16 +188,8 @@ impl DbSetup for ThreeDeleteThreeChunks { let pred1 = DeletePredicate { range: TimestampRange::new(0, 30), exprs: vec![ - DeleteExpr::new( - "bar".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::F64((1.0).into()), - ), - DeleteExpr::new( - "foo".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("me".to_string()), - ), + DeleteExpr::new("bar".to_string(), Op::Eq, Scalar::F64((1.0).into())), + DeleteExpr::new("foo".to_string(), Op::Eq, Scalar::String("me".to_string())), ], }; @@ -229,8 +205,8 @@ impl DbSetup for ThreeDeleteThreeChunks { range: TimestampRange::new(20, 45), exprs: vec![DeleteExpr::new( "foo".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("you".to_string()), + Op::Eq, + Scalar::String("you".to_string()), )], }; @@ -246,8 +222,8 @@ impl DbSetup for ThreeDeleteThreeChunks { range: TimestampRange::new(75, 95), exprs: vec![DeleteExpr::new( "bar".to_string(), - data_types::delete_predicate::Op::Ne, - data_types::delete_predicate::Scalar::F64((7.0).into()), + Op::Ne, + Scalar::F64((7.0).into()), )], }; diff --git a/query_tests/src/scenarios/library.rs b/query_tests/src/scenarios/library.rs index ddbe85f38a..fc62201eba 100644 --- a/query_tests/src/scenarios/library.rs +++ b/query_tests/src/scenarios/library.rs @@ -5,10 +5,7 @@ use super::{ }; use crate::scenarios::util::{make_n_chunks_scenario_new, ChunkDataNew}; use async_trait::async_trait; -use data_types::{ - delete_predicate::{DeleteExpr, DeletePredicate}, - timestamp::TimestampRange, -}; +use data_types2::{DeleteExpr, DeletePredicate, Op, Scalar, TimestampRange}; use query::frontend::sql::SqlQueryPlanner; #[derive(Debug)] @@ -112,8 +109,8 @@ impl DbSetup for OneMeasurementManyNullTagsWithDelete { range: TimestampRange::new(400, 602), exprs: vec![DeleteExpr::new( "state".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String(("NY").to_string()), + Op::Eq, + Scalar::String(("NY").to_string()), )], }; @@ -205,8 +202,8 @@ impl DbSetup for TwoMeasurementsWithDelete { range: TimestampRange::new(120, 160), exprs: vec![DeleteExpr::new( "region".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("west".to_string()), + Op::Eq, + Scalar::String("west".to_string()), )], }; @@ -237,8 +234,8 @@ impl DbSetup for TwoMeasurementsWithDeleteAll { range: TimestampRange::new(120, 160), exprs: vec![DeleteExpr::new( "region".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String("west".to_string()), + Op::Eq, + Scalar::String("west".to_string()), )], }; @@ -661,8 +658,8 @@ impl DbSetup for EndToEndTestWithDelete { range: TimestampRange::new(6000, 6000), exprs: vec![DeleteExpr::new( "name".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String(("disk0").to_string()), + Op::Eq, + Scalar::String(("disk0").to_string()), )], }; @@ -835,8 +832,8 @@ impl DbSetup for MeasurementsSortableTagsWithDelete { range: TimestampRange::new(120, 350), exprs: vec![DeleteExpr::new( "state".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::String(("CA").to_string()), + Op::Eq, + Scalar::String(("CA").to_string()), )], }; @@ -896,8 +893,8 @@ impl DbSetup for OneMeasurementNoTagsWithDelete { range: TimestampRange::new(1, 1), exprs: vec![DeleteExpr::new( "foo".to_string(), - data_types::delete_predicate::Op::Eq, - data_types::delete_predicate::Scalar::F64((1.0).into()), + Op::Eq, + Scalar::F64((1.0).into()), )], }; diff --git a/query_tests/src/scenarios/util.rs b/query_tests/src/scenarios/util.rs index 480a47a6fe..22df97bac9 100644 --- a/query_tests/src/scenarios/util.rs +++ b/query_tests/src/scenarios/util.rs @@ -3,9 +3,9 @@ use super::DbScenario; use arrow::record_batch::RecordBatch; use async_trait::async_trait; use backoff::BackoffConfig; -use data_types::delete_predicate::DeletePredicate; use data_types2::{ - NonEmptyString, PartitionId, Sequence, SequenceNumber, SequencerId, TombstoneId, + DeletePredicate, NonEmptyString, PartitionId, Sequence, SequenceNumber, SequencerId, + TombstoneId, }; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; use futures::StreamExt; @@ -14,10 +14,12 @@ use generated_types::{ ingester::IngesterQueryRequest, }; use influxdb_iox_client::flight::Error as FlightError; -use ingester::data::{IngesterData, IngesterQueryResponse, Persister, SequencerData}; -use ingester::lifecycle::LifecycleHandle; -use ingester::partioning::{Partitioner, PartitionerError}; -use ingester::querier_handler::prepare_data_to_querier; +use ingester::{ + data::{IngesterData, IngesterQueryResponse, Persister, SequencerData}, + lifecycle::LifecycleHandle, + partioning::{Partitioner, PartitionerError}, + querier_handler::prepare_data_to_querier, +}; use iox_catalog::interface::get_schema_by_name; use iox_tests::util::{TestCatalog, TestNamespace, TestSequencer}; use itertools::Itertools; @@ -28,11 +30,14 @@ use querier::{ IngesterFlightClientQueryData, QuerierCatalogCache, QuerierNamespace, }; use schema::selection::Selection; -use std::cmp::Ordering; -use std::collections::{BTreeMap, HashMap}; -use std::fmt::Write; -use std::sync::Mutex; -use std::{fmt::Display, sync::Arc}; +use std::{ + cmp::Ordering, + collections::{BTreeMap, HashMap}, + fmt::Display, + fmt::Write, + sync::Arc, + sync::Mutex, +}; // Structs, enums, and functions used to exhaust all test scenarios of chunk life cycle // & when delete predicates are applied diff --git a/write_buffer/Cargo.toml b/write_buffer/Cargo.toml index c35ffe981b..61e2a20a43 100644 --- a/write_buffer/Cargo.toml +++ b/write_buffer/Cargo.toml @@ -6,6 +6,7 @@ edition = "2021" [dependencies] async-trait = "0.1" data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } dml = { path = "../dml" } dotenv = "0.15.0" futures = "0.3" diff --git a/write_buffer/src/codec.rs b/write_buffer/src/codec.rs index f5bcb900c3..2e318d49a1 100644 --- a/write_buffer/src/codec.rs +++ b/write_buffer/src/codec.rs @@ -1,26 +1,23 @@ //! Encode/Decode for messages -use std::borrow::Cow; -use std::sync::Arc; - -use data_types::non_empty::NonEmptyString; -use http::{HeaderMap, HeaderValue}; -use prost::Message; - -use data_types::sequence::Sequence; +use crate::core::WriteBufferError; +use data_types::{non_empty::NonEmptyString, sequence::Sequence}; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; -use generated_types::google::FromOptionalField; -use generated_types::influxdata::iox::delete::v1::DeletePayload; -use generated_types::influxdata::iox::write_buffer::v1::write_buffer_payload::Payload; -use generated_types::influxdata::iox::write_buffer::v1::WriteBufferPayload; +use generated_types::{ + google::FromOptionalField, + influxdata::iox::{ + delete::v1::DeletePayload, + write_buffer::v1::{write_buffer_payload::Payload, WriteBufferPayload}, + }, +}; +use http::{HeaderMap, HeaderValue}; use iox_time::Time; use mutable_batch_pb::decode::decode_database_batch; -use trace::ctx::SpanContext; -use trace::TraceCollector; +use prost::Message; +use std::{borrow::Cow, sync::Arc}; +use trace::{ctx::SpanContext, TraceCollector}; use trace_http::ctx::{format_jaeger_trace_context, TraceHeaderParser}; -use crate::core::WriteBufferError; - /// Pbdata based content type pub const CONTENT_TYPE_PROTOBUF: &str = r#"application/x-protobuf; schema="influxdata.iox.write_buffer.v1.WriteBufferPayload""#; diff --git a/write_buffer/src/kafka/mod.rs b/write_buffer/src/kafka/mod.rs index ec6c7cde2f..8387b0911f 100644 --- a/write_buffer/src/kafka/mod.rs +++ b/write_buffer/src/kafka/mod.rs @@ -1,11 +1,14 @@ -use std::{ - collections::{BTreeMap, BTreeSet}, - sync::{ - atomic::{AtomicBool, Ordering}, - Arc, +use self::{ + aggregator::DmlAggregator, + config::{ClientConfig, ConsumerConfig, ProducerConfig, TopicCreationConfig}, +}; +use crate::{ + codec::IoxHeaders, + core::{ + WriteBufferError, WriteBufferErrorKind, WriteBufferReading, WriteBufferStreamHandler, + WriteBufferWriting, }, }; - use async_trait::async_trait; use data_types::{sequence::Sequence, write_buffer::WriteBufferCreationConfig}; use dml::{DmlMeta, DmlOperation}; @@ -19,20 +22,14 @@ use rskafka::client::{ producer::{BatchProducer, BatchProducerBuilder}, ClientBuilder, }; -use trace::TraceCollector; - -use crate::{ - codec::IoxHeaders, - core::{ - WriteBufferError, WriteBufferErrorKind, WriteBufferReading, WriteBufferStreamHandler, - WriteBufferWriting, +use std::{ + collections::{BTreeMap, BTreeSet}, + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, }, }; - -use self::{ - aggregator::DmlAggregator, - config::{ClientConfig, ConsumerConfig, ProducerConfig, TopicCreationConfig}, -}; +use trace::TraceCollector; mod aggregator; mod config; @@ -361,15 +358,7 @@ async fn setup_topic( #[cfg(test)] mod tests { - use std::num::NonZeroU32; - - use data_types::{delete_predicate::DeletePredicate, timestamp::TimestampRange}; - use dml::{test_util::assert_write_op_eq, DmlDelete, DmlWrite}; - use futures::{stream::FuturesUnordered, TryStreamExt}; - use rskafka::{client::partition::Compression, record::Record}; - use test_helpers::assert_contains; - use trace::{ctx::SpanContext, RingBufferTraceCollector}; - + use super::*; use crate::{ core::test_utils::{ assert_span_context_eq_or_linked, perform_generic_tests, random_topic_name, @@ -377,8 +366,13 @@ mod tests { }, maybe_skip_kafka_integration, }; - - use super::*; + use data_types2::{DeletePredicate, TimestampRange}; + use dml::{test_util::assert_write_op_eq, DmlDelete, DmlWrite}; + use futures::{stream::FuturesUnordered, TryStreamExt}; + use rskafka::{client::partition::Compression, record::Record}; + use std::num::NonZeroU32; + use test_helpers::assert_contains; + use trace::{ctx::SpanContext, RingBufferTraceCollector}; struct RSKafkaTestAdapter { conn: String, From 6b0e7ae46a8d45beb0d26aff47919445fff3bbe5 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 16:55:15 -0400 Subject: [PATCH 09/31] fix: Move name parsing code to data_types2 --- Cargo.lock | 3 + data_types/src/lib.rs | 1 - data_types/src/names.rs | 90 --------------------------- data_types2/Cargo.toml | 2 + data_types2/src/lib.rs | 87 +++++++++++++++++++++++++- ioxd_common/Cargo.toml | 1 + ioxd_common/src/http/dml.rs | 13 ++-- service_grpc_influxrpc/src/service.rs | 3 +- 8 files changed, 98 insertions(+), 102 deletions(-) delete mode 100644 data_types/src/names.rs diff --git a/Cargo.lock b/Cargo.lock index 1a8f7d86b4..0d2ec75941 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1175,7 +1175,9 @@ dependencies = [ "data_types", "influxdb_line_protocol", "ordered-float 3.0.0", + "percent-encoding", "schema", + "snafu", "sqlx", "uuid 0.8.2", "workspace-hack", @@ -2568,6 +2570,7 @@ dependencies = [ "clap 3.1.12", "clap_blocks", "data_types", + "data_types2", "dml", "flate2", "futures", diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 622fad2327..bf3d4cb7e0 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -17,7 +17,6 @@ mod database_name; pub mod database_rules; pub mod error; pub mod job; -pub mod names; pub mod non_empty; pub mod partition_metadata; pub mod router; diff --git a/data_types/src/names.rs b/data_types/src/names.rs deleted file mode 100644 index 1384824051..0000000000 --- a/data_types/src/names.rs +++ /dev/null @@ -1,90 +0,0 @@ -use std::borrow::Cow; - -use crate::{DatabaseName, DatabaseNameError}; -use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC}; -use snafu::{ResultExt, Snafu}; - -#[derive(Debug, Snafu)] -pub enum OrgBucketMappingError { - #[snafu(display("Invalid database name: {}", source))] - InvalidDatabaseName { source: DatabaseNameError }, - - #[snafu(display("missing org/bucket value"))] - NotSpecified, -} - -/// Map an InfluxDB 2.X org & bucket into an IOx DatabaseName. -/// -/// This function ensures the mapping is unambiguous by requiring both `org` and -/// `bucket` to not contain the `_` character in addition to the -/// [`DatabaseName`] validation. -pub fn org_and_bucket_to_database<'a, O: AsRef, B: AsRef>( - org: O, - bucket: B, -) -> Result, OrgBucketMappingError> { - const SEPARATOR: char = '_'; - - let org: Cow<'_, str> = utf8_percent_encode(org.as_ref(), NON_ALPHANUMERIC).into(); - let bucket: Cow<'_, str> = utf8_percent_encode(bucket.as_ref(), NON_ALPHANUMERIC).into(); - - // An empty org or bucket is not acceptable. - if org.is_empty() || bucket.is_empty() { - return Err(OrgBucketMappingError::NotSpecified); - } - - let db_name = format!("{}{}{}", org.as_ref(), SEPARATOR, bucket.as_ref()); - - DatabaseName::new(db_name).context(InvalidDatabaseNameSnafu) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_org_bucket_map_db_ok() { - let got = org_and_bucket_to_database("org", "bucket").expect("failed on valid DB mapping"); - - assert_eq!(got.as_str(), "org_bucket"); - } - - #[test] - fn test_org_bucket_map_db_contains_underscore() { - let got = org_and_bucket_to_database("my_org", "bucket").unwrap(); - assert_eq!(got.as_str(), "my%5Forg_bucket"); - - let got = org_and_bucket_to_database("org", "my_bucket").unwrap(); - assert_eq!(got.as_str(), "org_my%5Fbucket"); - - let got = org_and_bucket_to_database("org", "my__bucket").unwrap(); - assert_eq!(got.as_str(), "org_my%5F%5Fbucket"); - - let got = org_and_bucket_to_database("my_org", "my_bucket").unwrap(); - assert_eq!(got.as_str(), "my%5Forg_my%5Fbucket"); - } - - #[test] - fn test_org_bucket_map_db_contains_underscore_and_percent() { - let got = org_and_bucket_to_database("my%5Forg", "bucket").unwrap(); - assert_eq!(got.as_str(), "my%255Forg_bucket"); - - let got = org_and_bucket_to_database("my%5Forg_", "bucket").unwrap(); - assert_eq!(got.as_str(), "my%255Forg%5F_bucket"); - } - - #[test] - fn test_bad_database_name_is_encoded() { - let got = org_and_bucket_to_database("org", "bucket?").unwrap(); - assert_eq!(got.as_str(), "org_bucket%3F"); - - let got = org_and_bucket_to_database("org!", "bucket").unwrap(); - assert_eq!(got.as_str(), "org%21_bucket"); - } - - #[test] - fn test_empty_org_bucket() { - let err = org_and_bucket_to_database("", "") - .expect_err("should fail with empty org/bucket valuese"); - assert!(matches!(err, OrgBucketMappingError::NotSpecified)); - } -} diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index 5ce2b0a42b..e160be0bb2 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -8,7 +8,9 @@ description = "Shared data types in the IOx NG architecture" data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } ordered-float = "3" +percent-encoding = "2.1.0" schema = { path = "../schema" } +snafu = "0.7" sqlx = { version = "0.5", features = ["runtime-tokio-rustls", "postgres", "uuid"] } uuid = { version = "0.8", features = ["v4"] } workspace-hack = { path = "../workspace-hack"} diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 978c7811ac..ee4d171941 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -11,8 +11,11 @@ )] use influxdb_line_protocol::FieldValue; +use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC}; use schema::{builder::SchemaBuilder, sort::SortKey, InfluxColumnType, InfluxFieldType, Schema}; +use snafu::{ResultExt, Snafu}; use std::{ + borrow::Cow, collections::BTreeMap, convert::TryFrom, fmt::Write, @@ -23,14 +26,13 @@ use std::{ use uuid::Uuid; pub use data_types::{ - names::{org_and_bucket_to_database, OrgBucketMappingError}, non_empty::NonEmptyString, partition_metadata::{ ColumnSummary, InfluxDbType, PartitionAddr, StatValues, Statistics, TableSummary, }, sequence::Sequence, timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}, - DatabaseName, + DatabaseName, DatabaseNameError, }; /// Unique ID for a `Namespace` @@ -1219,6 +1221,40 @@ impl std::fmt::Display for Scalar { } } +#[derive(Debug, Snafu)] +#[allow(missing_docs)] +pub enum OrgBucketMappingError { + #[snafu(display("Invalid database name: {}", source))] + InvalidDatabaseName { source: DatabaseNameError }, + + #[snafu(display("missing org/bucket value"))] + NotSpecified, +} + +/// Map an InfluxDB 2.X org & bucket into an IOx DatabaseName. +/// +/// This function ensures the mapping is unambiguous by requiring both `org` and +/// `bucket` to not contain the `_` character in addition to the +/// [`DatabaseName`] validation. +pub fn org_and_bucket_to_database<'a, O: AsRef, B: AsRef>( + org: O, + bucket: B, +) -> Result, OrgBucketMappingError> { + const SEPARATOR: char = '_'; + + let org: Cow<'_, str> = utf8_percent_encode(org.as_ref(), NON_ALPHANUMERIC).into(); + let bucket: Cow<'_, str> = utf8_percent_encode(bucket.as_ref(), NON_ALPHANUMERIC).into(); + + // An empty org or bucket is not acceptable. + if org.is_empty() || bucket.is_empty() { + return Err(OrgBucketMappingError::NotSpecified); + } + + let db_name = format!("{}{}{}", org.as_ref(), SEPARATOR, bucket.as_ref()); + + DatabaseName::new(db_name).context(InvalidDatabaseNameSnafu) +} + #[cfg(test)] mod tests { use super::*; @@ -1441,4 +1477,51 @@ mod tests { r#""col1"='' AND "col2"='foo' AND "col3"='fo\\o' AND "col4"='fo\'o'"# ); } + + #[test] + fn test_org_bucket_map_db_ok() { + let got = org_and_bucket_to_database("org", "bucket").expect("failed on valid DB mapping"); + + assert_eq!(got.as_str(), "org_bucket"); + } + + #[test] + fn test_org_bucket_map_db_contains_underscore() { + let got = org_and_bucket_to_database("my_org", "bucket").unwrap(); + assert_eq!(got.as_str(), "my%5Forg_bucket"); + + let got = org_and_bucket_to_database("org", "my_bucket").unwrap(); + assert_eq!(got.as_str(), "org_my%5Fbucket"); + + let got = org_and_bucket_to_database("org", "my__bucket").unwrap(); + assert_eq!(got.as_str(), "org_my%5F%5Fbucket"); + + let got = org_and_bucket_to_database("my_org", "my_bucket").unwrap(); + assert_eq!(got.as_str(), "my%5Forg_my%5Fbucket"); + } + + #[test] + fn test_org_bucket_map_db_contains_underscore_and_percent() { + let got = org_and_bucket_to_database("my%5Forg", "bucket").unwrap(); + assert_eq!(got.as_str(), "my%255Forg_bucket"); + + let got = org_and_bucket_to_database("my%5Forg_", "bucket").unwrap(); + assert_eq!(got.as_str(), "my%255Forg%5F_bucket"); + } + + #[test] + fn test_bad_database_name_is_encoded() { + let got = org_and_bucket_to_database("org", "bucket?").unwrap(); + assert_eq!(got.as_str(), "org_bucket%3F"); + + let got = org_and_bucket_to_database("org!", "bucket").unwrap(); + assert_eq!(got.as_str(), "org%21_bucket"); + } + + #[test] + fn test_empty_org_bucket() { + let err = org_and_bucket_to_database("", "") + .expect_err("should fail with empty org/bucket valuese"); + assert!(matches!(err, OrgBucketMappingError::NotSpecified)); + } } diff --git a/ioxd_common/Cargo.toml b/ioxd_common/Cargo.toml index ebd57753e9..3582869dcb 100644 --- a/ioxd_common/Cargo.toml +++ b/ioxd_common/Cargo.toml @@ -11,6 +11,7 @@ edition = "2021" # Workspace dependencies, in alphabetical order clap_blocks = { path = "../clap_blocks" } data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } dml = { path = "../dml" } metric = { path = "../metric" } observability_deps = { path = "../observability_deps" } diff --git a/ioxd_common/src/http/dml.rs b/ioxd_common/src/http/dml.rs index 361296921d..1edf6fba6c 100644 --- a/ioxd_common/src/http/dml.rs +++ b/ioxd_common/src/http/dml.rs @@ -2,19 +2,16 @@ use std::sync::Arc; use async_trait::async_trait; use chrono::Utc; -use hyper::{Body, Method, Request, Response, StatusCode}; -use serde::Deserialize; -use snafu::{OptionExt, ResultExt, Snafu}; - -use data_types::{ - names::{org_and_bucket_to_database, OrgBucketMappingError}, - non_empty::NonEmptyString, - DatabaseName, +use data_types2::{ + org_and_bucket_to_database, DatabaseName, NonEmptyString, OrgBucketMappingError, }; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; +use hyper::{Body, Method, Request, Response, StatusCode}; use mutable_batch_lp::LinesConverter; use observability_deps::tracing::debug; use predicate::delete_predicate::{parse_delete_predicate, parse_http_delete_request}; +use serde::Deserialize; +use snafu::{OptionExt, ResultExt, Snafu}; use crate::{http::utils::parse_body, server_type::ServerType}; diff --git a/service_grpc_influxrpc/src/service.rs b/service_grpc_influxrpc/src/service.rs index 6be36c7a1f..3e06d80c98 100644 --- a/service_grpc_influxrpc/src/service.rs +++ b/service_grpc_influxrpc/src/service.rs @@ -11,7 +11,8 @@ use crate::{ input::GrpcInputs, StorageService, }; -use data_types::{error::ErrorLogger, names::org_and_bucket_to_database, DatabaseName}; +use data_types::error::ErrorLogger; +use data_types2::{org_and_bucket_to_database, DatabaseName}; use generated_types::{ google::protobuf::Empty, literal_or_regex::Value as RegexOrLiteralValue, offsets_response::PartitionOffsetResponse, storage_server::Storage, tag_key_predicate, From 1ea4a40b1faf6b7ea5dae96ee40613ddc73af254 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 16:57:50 -0400 Subject: [PATCH 10/31] fix: Move NonEmptyString to data_types2 --- data_types/src/lib.rs | 1 - data_types/src/non_empty.rs | 29 ----------------------------- data_types2/src/lib.rs | 31 +++++++++++++++++++++++++++++-- dml/src/lib.rs | 4 +--- write_buffer/src/codec.rs | 2 +- 5 files changed, 31 insertions(+), 36 deletions(-) delete mode 100644 data_types/src/non_empty.rs diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index bf3d4cb7e0..07b479f8c1 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -17,7 +17,6 @@ mod database_name; pub mod database_rules; pub mod error; pub mod job; -pub mod non_empty; pub mod partition_metadata; pub mod router; pub mod sequence; diff --git a/data_types/src/non_empty.rs b/data_types/src/non_empty.rs deleted file mode 100644 index a053f060fd..0000000000 --- a/data_types/src/non_empty.rs +++ /dev/null @@ -1,29 +0,0 @@ -use std::ops::Deref; - -/// A string that cannot be empty -/// -/// This is particularly useful for types that map to/from protobuf, where string fields -/// are not nullable - that is they default to an empty string if not specified -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct NonEmptyString(Box); - -impl NonEmptyString { - /// Create a new `NonEmptyString` from the provided `String` - /// - /// Returns None if empty - pub fn new(s: impl Into) -> Option { - let s = s.into(); - match s.is_empty() { - true => None, - false => Some(Self(s.into_boxed_str())), - } - } -} - -impl Deref for NonEmptyString { - type Target = str; - - fn deref(&self) -> &Self::Target { - self.0.as_ref() - } -} diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index ee4d171941..dba8b548fb 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -20,13 +20,12 @@ use std::{ convert::TryFrom, fmt::Write, num::{FpCategory, NonZeroU32}, - ops::{Add, Sub}, + ops::{Add, Deref, Sub}, sync::Arc, }; use uuid::Uuid; pub use data_types::{ - non_empty::NonEmptyString, partition_metadata::{ ColumnSummary, InfluxDbType, PartitionAddr, StatValues, Statistics, TableSummary, }, @@ -1255,6 +1254,34 @@ pub fn org_and_bucket_to_database<'a, O: AsRef, B: AsRef>( DatabaseName::new(db_name).context(InvalidDatabaseNameSnafu) } +/// A string that cannot be empty +/// +/// This is particularly useful for types that map to/from protobuf, where string fields +/// are not nullable - that is they default to an empty string if not specified +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct NonEmptyString(Box); + +impl NonEmptyString { + /// Create a new `NonEmptyString` from the provided `String` + /// + /// Returns None if empty + pub fn new(s: impl Into) -> Option { + let s = s.into(); + match s.is_empty() { + true => None, + false => Some(Self(s.into_boxed_str())), + } + } +} + +impl Deref for NonEmptyString { + type Target = str; + + fn deref(&self) -> &Self::Target { + self.0.as_ref() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/dml/src/lib.rs b/dml/src/lib.rs index 29a6b9e10f..2f33b45887 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -12,12 +12,11 @@ )] use data_types::{ - non_empty::NonEmptyString, partition_metadata::{StatValues, Statistics}, router::{ShardConfig, ShardId}, sequence::Sequence, }; -use data_types2::DeletePredicate; +use data_types2::{DeletePredicate, NonEmptyString}; use hashbrown::HashMap; use iox_time::Time; use mutable_batch::MutableBatch; @@ -491,7 +490,6 @@ mod tests { use crate::test_util::assert_writes_eq; use data_types::{ consistent_hasher::ConsistentHasher, - non_empty::NonEmptyString, router::{HashRing, Matcher, MatcherToShard}, timestamp::TimestampRange, }; diff --git a/write_buffer/src/codec.rs b/write_buffer/src/codec.rs index 2e318d49a1..84b1ec04ec 100644 --- a/write_buffer/src/codec.rs +++ b/write_buffer/src/codec.rs @@ -1,7 +1,7 @@ //! Encode/Decode for messages use crate::core::WriteBufferError; -use data_types::{non_empty::NonEmptyString, sequence::Sequence}; +use data_types2::{NonEmptyString, Sequence}; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; use generated_types::{ google::FromOptionalField, From afdff2b1db83662c9a50b0f2acaf54582e8170bb Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 4 May 2022 17:04:29 -0400 Subject: [PATCH 11/31] fix: Move DatabaseName to data_types2 --- Cargo.lock | 3 +- data_types/src/database_name.rs | 187 --------------------- data_types/src/database_rules.rs | 276 ------------------------------- data_types/src/lib.rs | 3 - data_types2/Cargo.toml | 3 + data_types2/src/lib.rs | 188 ++++++++++++++++++++- service_grpc_flight/Cargo.toml | 2 +- service_grpc_flight/src/lib.rs | 19 +-- write_buffer/src/config.rs | 3 +- 9 files changed, 202 insertions(+), 482 deletions(-) delete mode 100644 data_types/src/database_name.rs delete mode 100644 data_types/src/database_rules.rs diff --git a/Cargo.lock b/Cargo.lock index 0d2ec75941..402acad833 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1179,6 +1179,7 @@ dependencies = [ "schema", "snafu", "sqlx", + "test_helpers", "uuid 0.8.2", "workspace-hack", ] @@ -5195,7 +5196,7 @@ dependencies = [ "arrow", "arrow-flight", "bytes", - "data_types", + "data_types2", "datafusion 0.1.0", "futures", "generated_types", diff --git a/data_types/src/database_name.rs b/data_types/src/database_name.rs deleted file mode 100644 index eff74ac764..0000000000 --- a/data_types/src/database_name.rs +++ /dev/null @@ -1,187 +0,0 @@ -use snafu::Snafu; -use std::{borrow::Cow, ops::RangeInclusive}; - -/// Length constraints for a database name. -/// -/// A `RangeInclusive` is a closed interval, covering [1, 64] -const LENGTH_CONSTRAINT: RangeInclusive = 1..=64; - -/// Database name validation errors. -#[derive(Debug, Snafu)] -pub enum DatabaseNameError { - #[snafu(display( - "Database name {} length must be between {} and {} characters", - name, - LENGTH_CONSTRAINT.start(), - LENGTH_CONSTRAINT.end() - ))] - LengthConstraint { name: String }, - - #[snafu(display( - "Database name '{}' contains invalid character. Character number {} is a control which is not allowed.", name, bad_char_offset - ))] - BadChars { - bad_char_offset: usize, - name: String, - }, -} - -/// A correctly formed database name. -/// -/// Using this wrapper type allows the consuming code to enforce the invariant -/// that only valid names are provided. -/// -/// This type derefs to a `str` and therefore can be used in place of anything -/// that is expecting a `str`: -/// -/// ```rust -/// # use data_types::DatabaseName; -/// fn print_database(s: &str) { -/// println!("database name: {}", s); -/// } -/// -/// let db = DatabaseName::new("data").unwrap(); -/// print_database(&db); -/// ``` -/// -/// But this is not reciprocal - functions that wish to accept only -/// pre-validated names can use `DatabaseName` as a parameter. -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] -pub struct DatabaseName<'a>(Cow<'a, str>); - -impl<'a> DatabaseName<'a> { - pub fn new>>(name: T) -> Result { - let name: Cow<'a, str> = name.into(); - - if !LENGTH_CONSTRAINT.contains(&name.len()) { - return Err(DatabaseNameError::LengthConstraint { - name: name.to_string(), - }); - } - - // Validate the name contains only valid characters. - // - // NOTE: If changing these characters, please update the error message - // above. - if let Some(bad_char_offset) = name.chars().position(|c| c.is_control()) { - return BadCharsSnafu { - bad_char_offset, - name, - } - .fail(); - }; - - Ok(Self(name)) - } - - pub fn as_str(&self) -> &str { - self.0.as_ref() - } -} - -impl<'a> std::convert::From> for String { - fn from(name: DatabaseName<'a>) -> Self { - name.0.to_string() - } -} - -impl<'a> std::convert::From<&DatabaseName<'a>> for String { - fn from(name: &DatabaseName<'a>) -> Self { - name.to_string() - } -} - -impl<'a> std::convert::TryFrom<&'a str> for DatabaseName<'a> { - type Error = DatabaseNameError; - - fn try_from(v: &'a str) -> Result { - Self::new(v) - } -} - -impl<'a> std::convert::TryFrom for DatabaseName<'a> { - type Error = DatabaseNameError; - - fn try_from(v: String) -> Result { - Self::new(v) - } -} - -impl<'a> std::ops::Deref for DatabaseName<'a> { - type Target = str; - - fn deref(&self) -> &Self::Target { - self.as_str() - } -} - -impl<'a> std::fmt::Display for DatabaseName<'a> { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - self.0.fmt(f) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use std::convert::TryFrom; - use test_helpers::assert_contains; - - #[test] - fn test_deref() { - let db = DatabaseName::new("my_example_name").unwrap(); - assert_eq!(&*db, "my_example_name"); - } - - #[test] - fn test_too_short() { - let name = "".to_string(); - let got = DatabaseName::try_from(name).unwrap_err(); - - assert!(matches!( - got, - DatabaseNameError::LengthConstraint { name: _n } - )); - } - - #[test] - fn test_too_long() { - let name = "my_example_name_that_is_quite_a_bit_longer_than_allowed_even_though_database_names_can_be_quite_long_bananas".to_string(); - let got = DatabaseName::try_from(name).unwrap_err(); - - assert!(matches!( - got, - DatabaseNameError::LengthConstraint { name: _n } - )); - } - - #[test] - fn test_bad_chars_null() { - let got = DatabaseName::new("example\x00").unwrap_err(); - assert_contains!(got.to_string() , "Database name 'example\x00' contains invalid character. Character number 7 is a control which is not allowed."); - } - - #[test] - fn test_bad_chars_high_control() { - let got = DatabaseName::new("\u{007f}example").unwrap_err(); - assert_contains!(got.to_string() , "Database name '\u{007f}example' contains invalid character. Character number 0 is a control which is not allowed."); - } - - #[test] - fn test_bad_chars_tab() { - let got = DatabaseName::new("example\tdb").unwrap_err(); - assert_contains!(got.to_string() , "Database name 'example\tdb' contains invalid character. Character number 7 is a control which is not allowed."); - } - - #[test] - fn test_bad_chars_newline() { - let got = DatabaseName::new("my_example\ndb").unwrap_err(); - assert_contains!(got.to_string() , "Database name 'my_example\ndb' contains invalid character. Character number 10 is a control which is not allowed."); - } - - #[test] - fn test_ok_chars() { - let db = DatabaseName::new("my-example-db_with_underscores and spaces").unwrap(); - assert_eq!(&*db, "my-example-db_with_underscores and spaces"); - } -} diff --git a/data_types/src/database_rules.rs b/data_types/src/database_rules.rs deleted file mode 100644 index 8b350eaabe..0000000000 --- a/data_types/src/database_rules.rs +++ /dev/null @@ -1,276 +0,0 @@ -use crate::{write_buffer::WriteBufferConnection, DatabaseName}; -use snafu::Snafu; -use std::{ - num::{NonZeroU32, NonZeroU64, NonZeroUsize}, - time::Duration, -}; - -#[derive(Debug, Snafu)] -pub enum Error { - #[snafu(display("Error in {}: {}", source_module, source))] - PassThrough { - source_module: &'static str, - source: Box, - }, - - #[snafu(display("No sharding rule matches table: {}", table))] - NoShardingRuleMatches { table: String }, - - #[snafu(display("No shards defined"))] - NoShardsDefined, -} - -pub type Result = std::result::Result; - -/// `DatabaseRules` contains the rules for replicating data, sending data to -/// subscribers, and querying data for a single database. This information is -/// provided by and exposed to operators. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct DatabaseRules { - /// The name of the database - pub name: DatabaseName<'static>, - - /// Template that generates a partition key for each row inserted into the - /// db - pub partition_template: PartitionTemplate, - - /// Configure how data flows through the system - pub lifecycle_rules: LifecycleRules, - - /// Duration for which the cleanup loop should sleep on average. - /// Defaults to 500 seconds. - pub worker_cleanup_avg_sleep: Duration, - - /// An optional connection string to a write buffer for either writing or reading. - pub write_buffer_connection: Option, -} - -impl DatabaseRules { - pub fn new(name: DatabaseName<'static>) -> Self { - Self { - name, - partition_template: Default::default(), - lifecycle_rules: Default::default(), - worker_cleanup_avg_sleep: Duration::from_secs(500), - write_buffer_connection: None, - } - } - - pub fn db_name(&self) -> &str { - self.name.as_str() - } -} - -pub const DEFAULT_WORKER_BACKOFF_MILLIS: u64 = 1_000; -pub const DEFAULT_CATALOG_TRANSACTIONS_UNTIL_CHECKPOINT: u64 = 100; -pub const DEFAULT_CATALOG_TRANSACTION_PRUNE_AGE: Duration = Duration::from_secs(24 * 60 * 60); -pub const DEFAULT_MUB_ROW_THRESHOLD: usize = 100_000; -pub const DEFAULT_PERSIST_ROW_THRESHOLD: usize = 1_000_000; -pub const DEFAULT_PERSIST_AGE_THRESHOLD_SECONDS: u32 = 30 * 60; -pub const DEFAULT_LATE_ARRIVE_WINDOW_SECONDS: u32 = 5 * 60; - -/// Configures how data automatically flows through the system -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct LifecycleRules { - /// Once the total amount of buffered data in memory reaches this size start - /// dropping data from memory - pub buffer_size_soft: Option, - - /// Once the amount of data in memory reaches this size start - /// rejecting writes - pub buffer_size_hard: Option, - - /// Persists chunks to object storage. - pub persist: bool, - - /// Do not allow writing new data to this database - pub immutable: bool, - - /// If the background worker doesn't find anything to do it - /// will sleep for this many milliseconds before looking again - pub worker_backoff_millis: NonZeroU64, - - /// The maximum number of permitted concurrently executing compactions. - pub max_active_compactions: MaxActiveCompactions, - - /// After how many transactions should IOx write a new checkpoint? - pub catalog_transactions_until_checkpoint: NonZeroU64, - - /// Prune catalog transactions older than the given age. - /// - /// Keeping old transaction can be useful for debugging. - pub catalog_transaction_prune_age: Duration, - - /// Once a partition hasn't received a write for this period of time, - /// it will be compacted and, if set, persisted. Writers will generally - /// have this amount of time to send late arriving writes or this could - /// be their clock skew. - pub late_arrive_window_seconds: NonZeroU32, - - /// Maximum number of rows before triggering persistence - pub persist_row_threshold: NonZeroUsize, - - /// Maximum age of a write before triggering persistence - pub persist_age_threshold_seconds: NonZeroU32, - - /// Maximum number of rows to buffer in a MUB chunk before compacting it - pub mub_row_threshold: NonZeroUsize, - - /// Use up to this amount of space in bytes for caching Parquet files. None - /// will disable Parquet file caching. - pub parquet_cache_limit: Option, -} - -#[derive(Debug, PartialEq, Clone)] -pub enum MaxActiveCompactions { - /// The maximum number of permitted concurrently executing compactions. - /// It is not currently possible to set a limit that disables compactions - /// entirely, nor is it possible to set an "unlimited" value. - MaxActiveCompactions(NonZeroU32), - - // The maximum number of concurrent active compactions that can run - // expressed as a fraction of the available cpus (rounded to the next smallest non-zero integer). - MaxActiveCompactionsCpuFraction { - fraction: f32, - effective: NonZeroU32, - }, -} - -impl MaxActiveCompactions { - pub fn new(fraction: f32) -> Self { - let cpus = num_cpus::get() as f32 * fraction; - let effective = (cpus as u32).saturating_sub(1) + 1; - let effective = NonZeroU32::new(effective).unwrap(); - Self::MaxActiveCompactionsCpuFraction { - fraction, - effective, - } - } - - pub fn get(&self) -> u32 { - match self { - Self::MaxActiveCompactions(effective) => effective, - Self::MaxActiveCompactionsCpuFraction { effective, .. } => effective, - } - .get() - } -} - -// Defaults to number of CPUs. -impl Default for MaxActiveCompactions { - fn default() -> Self { - Self::new(1.0) - } -} - -// Required because database rules must be Eq but cannot derive Eq for Self -// since f32 is not Eq. -impl Eq for MaxActiveCompactions {} - -impl LifecycleRules { - /// The max timestamp skew across concurrent writers before persisted chunks might overlap - pub fn late_arrive_window(&self) -> Duration { - Duration::from_secs(self.late_arrive_window_seconds.get() as u64) - } -} - -impl Default for LifecycleRules { - fn default() -> Self { - Self { - buffer_size_soft: None, - buffer_size_hard: None, - persist: false, - immutable: false, - worker_backoff_millis: NonZeroU64::new(DEFAULT_WORKER_BACKOFF_MILLIS).unwrap(), - max_active_compactions: Default::default(), - catalog_transactions_until_checkpoint: NonZeroU64::new( - DEFAULT_CATALOG_TRANSACTIONS_UNTIL_CHECKPOINT, - ) - .unwrap(), - catalog_transaction_prune_age: DEFAULT_CATALOG_TRANSACTION_PRUNE_AGE, - late_arrive_window_seconds: NonZeroU32::new(DEFAULT_LATE_ARRIVE_WINDOW_SECONDS) - .unwrap(), - persist_row_threshold: NonZeroUsize::new(DEFAULT_PERSIST_ROW_THRESHOLD).unwrap(), - persist_age_threshold_seconds: NonZeroU32::new(DEFAULT_PERSIST_AGE_THRESHOLD_SECONDS) - .unwrap(), - mub_row_threshold: NonZeroUsize::new(DEFAULT_MUB_ROW_THRESHOLD).unwrap(), - parquet_cache_limit: None, - } - } -} - -/// `PartitionTemplate` is used to compute the partition key of each row that -/// gets written. It can consist of the table name, a column name and its value, -/// a formatted time, or a string column and regex captures of its value. For -/// columns that do not appear in the input row, a blank value is output. -/// -/// The key is constructed in order of the template parts; thus ordering changes -/// what partition key is generated. -#[derive(Debug, Default, Eq, PartialEq, Clone)] -pub struct PartitionTemplate { - pub parts: Vec, -} - -/// `TemplatePart` specifies what part of a row should be used to compute this -/// part of a partition key. -#[derive(Debug, Eq, PartialEq, Clone)] -pub enum TemplatePart { - /// The name of a table - Table, - /// The value in a named column - Column(String), - /// Applies a `strftime` format to the "time" column. - /// - /// For example, a time format of "%Y-%m-%d %H:%M:%S" will produce - /// partition key parts such as "2021-03-14 12:25:21" and - /// "2021-04-14 12:24:21" - TimeFormat(String), - /// Applies a regex to the value in a string column - RegexCapture(RegexCapture), - /// Applies a `strftime` pattern to some column other than "time" - StrftimeColumn(StrftimeColumn), -} - -/// `RegexCapture` is for pulling parts of a string column into the partition -/// key. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct RegexCapture { - pub column: String, - pub regex: String, -} - -/// [`StrftimeColumn`] is used to create a time based partition key off some -/// column other than the builtin `time` column. -/// -/// The value of the named column is formatted using a `strftime` -/// style string. -/// -/// For example, a time format of "%Y-%m-%d %H:%M:%S" will produce -/// partition key parts such as "2021-03-14 12:25:21" and -/// "2021-04-14 12:24:21" -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct StrftimeColumn { - pub column: String, - pub format: String, -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_max_active_compactions_cpu_fraction() { - let n = MaxActiveCompactions::new(1.0); - let cpus = n.get(); - - let n = MaxActiveCompactions::new(0.5); - let half_cpus = n.get(); - - assert_eq!(half_cpus, cpus / 2); - - let n = MaxActiveCompactions::new(0.0); - let non_zero = n.get(); - - assert_eq!(non_zero, 1); - } -} diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 07b479f8c1..22b92dd497 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -13,8 +13,6 @@ pub mod boolean_flag; pub mod chunk_metadata; pub mod consistent_hasher; -mod database_name; -pub mod database_rules; pub mod error; pub mod job; pub mod partition_metadata; @@ -24,4 +22,3 @@ pub mod server_id; pub mod timestamp; pub mod write_buffer; pub mod write_summary; -pub use database_name::*; diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index e160be0bb2..77dea8c888 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -14,3 +14,6 @@ snafu = "0.7" sqlx = { version = "0.5", features = ["runtime-tokio-rustls", "postgres", "uuid"] } uuid = { version = "0.8", features = ["v4"] } workspace-hack = { path = "../workspace-hack"} + +[dev-dependencies] # In alphabetical order +test_helpers = { path = "../test_helpers" } diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index dba8b548fb..07828173e9 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -20,7 +20,7 @@ use std::{ convert::TryFrom, fmt::Write, num::{FpCategory, NonZeroU32}, - ops::{Add, Deref, Sub}, + ops::{Add, Deref, RangeInclusive, Sub}, sync::Arc, }; use uuid::Uuid; @@ -31,7 +31,6 @@ pub use data_types::{ }, sequence::Sequence, timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}, - DatabaseName, DatabaseNameError, }; /// Unique ID for a `Namespace` @@ -1282,10 +1281,137 @@ impl Deref for NonEmptyString { } } +/// Length constraints for a database name. +/// +/// A `RangeInclusive` is a closed interval, covering [1, 64] +const LENGTH_CONSTRAINT: RangeInclusive = 1..=64; + +/// Database name validation errors. +#[derive(Debug, Snafu)] +#[allow(missing_docs)] +pub enum DatabaseNameError { + #[snafu(display( + "Database name {} length must be between {} and {} characters", + name, + LENGTH_CONSTRAINT.start(), + LENGTH_CONSTRAINT.end() + ))] + LengthConstraint { name: String }, + + #[snafu(display( + "Database name '{}' contains invalid character. \ + Character number {} is a control which is not allowed.", + name, + bad_char_offset + ))] + BadChars { + bad_char_offset: usize, + name: String, + }, +} + +/// A correctly formed database name. +/// +/// Using this wrapper type allows the consuming code to enforce the invariant +/// that only valid names are provided. +/// +/// This type derefs to a `str` and therefore can be used in place of anything +/// that is expecting a `str`: +/// +/// ```rust +/// # use data_types2::DatabaseName; +/// fn print_database(s: &str) { +/// println!("database name: {}", s); +/// } +/// +/// let db = DatabaseName::new("data").unwrap(); +/// print_database(&db); +/// ``` +/// +/// But this is not reciprocal - functions that wish to accept only +/// pre-validated names can use `DatabaseName` as a parameter. +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct DatabaseName<'a>(Cow<'a, str>); + +impl<'a> DatabaseName<'a> { + /// Create a new, valid DatabaseName. + pub fn new>>(name: T) -> Result { + let name: Cow<'a, str> = name.into(); + + if !LENGTH_CONSTRAINT.contains(&name.len()) { + return Err(DatabaseNameError::LengthConstraint { + name: name.to_string(), + }); + } + + // Validate the name contains only valid characters. + // + // NOTE: If changing these characters, please update the error message + // above. + if let Some(bad_char_offset) = name.chars().position(|c| c.is_control()) { + return BadCharsSnafu { + bad_char_offset, + name, + } + .fail(); + }; + + Ok(Self(name)) + } + + /// Borrow a string slice of the name. + pub fn as_str(&self) -> &str { + self.0.as_ref() + } +} + +impl<'a> std::convert::From> for String { + fn from(name: DatabaseName<'a>) -> Self { + name.0.to_string() + } +} + +impl<'a> std::convert::From<&DatabaseName<'a>> for String { + fn from(name: &DatabaseName<'a>) -> Self { + name.to_string() + } +} + +impl<'a> std::convert::TryFrom<&'a str> for DatabaseName<'a> { + type Error = DatabaseNameError; + + fn try_from(v: &'a str) -> Result { + Self::new(v) + } +} + +impl<'a> std::convert::TryFrom for DatabaseName<'a> { + type Error = DatabaseNameError; + + fn try_from(v: String) -> Result { + Self::new(v) + } +} + +impl<'a> std::ops::Deref for DatabaseName<'a> { + type Target = str; + + fn deref(&self) -> &Self::Target { + self.as_str() + } +} + +impl<'a> std::fmt::Display for DatabaseName<'a> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.0.fmt(f) + } +} + #[cfg(test)] mod tests { use super::*; use ordered_float::OrderedFloat; + use test_helpers::assert_contains; #[test] fn test_chunk_id_new() { @@ -1551,4 +1677,62 @@ mod tests { .expect_err("should fail with empty org/bucket valuese"); assert!(matches!(err, OrgBucketMappingError::NotSpecified)); } + + #[test] + fn test_deref() { + let db = DatabaseName::new("my_example_name").unwrap(); + assert_eq!(&*db, "my_example_name"); + } + + #[test] + fn test_too_short() { + let name = "".to_string(); + let got = DatabaseName::try_from(name).unwrap_err(); + + assert!(matches!( + got, + DatabaseNameError::LengthConstraint { name: _n } + )); + } + + #[test] + fn test_too_long() { + let name = "my_example_name_that_is_quite_a_bit_longer_than_allowed_even_though_database_names_can_be_quite_long_bananas".to_string(); + let got = DatabaseName::try_from(name).unwrap_err(); + + assert!(matches!( + got, + DatabaseNameError::LengthConstraint { name: _n } + )); + } + + #[test] + fn test_bad_chars_null() { + let got = DatabaseName::new("example\x00").unwrap_err(); + assert_contains!(got.to_string() , "Database name 'example\x00' contains invalid character. Character number 7 is a control which is not allowed."); + } + + #[test] + fn test_bad_chars_high_control() { + let got = DatabaseName::new("\u{007f}example").unwrap_err(); + assert_contains!(got.to_string() , "Database name '\u{007f}example' contains invalid character. Character number 0 is a control which is not allowed."); + } + + #[test] + fn test_bad_chars_tab() { + let got = DatabaseName::new("example\tdb").unwrap_err(); + assert_contains!(got.to_string() , "Database name 'example\tdb' contains invalid character. Character number 7 is a control which is not allowed."); + } + + #[test] + fn test_bad_chars_newline() { + let got = DatabaseName::new("my_example\ndb").unwrap_err(); + assert_contains!(got.to_string() , "Database name 'my_example\ndb' contains invalid character. Character number 10 is a control which is not allowed."); + } + + #[test] + fn test_ok_chars() { + let db = DatabaseName::new("my-example-db_with_underscores and spaces").unwrap(); + assert_eq!(&*db, "my-example-db_with_underscores and spaces"); + } } diff --git a/service_grpc_flight/Cargo.toml b/service_grpc_flight/Cargo.toml index 9c4ba72b85..4799c2f7ed 100644 --- a/service_grpc_flight/Cargo.toml +++ b/service_grpc_flight/Cargo.toml @@ -7,7 +7,7 @@ edition = "2021" [dependencies] # Workspace dependencies, in alphabetical order -data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } generated_types = { path = "../generated_types" } observability_deps = { path = "../observability_deps" } diff --git a/service_grpc_flight/src/lib.rs b/service_grpc_flight/src/lib.rs index 89e067c9c7..e3129997b9 100644 --- a/service_grpc_flight/src/lib.rs +++ b/service_grpc_flight/src/lib.rs @@ -1,7 +1,4 @@ //! Implements the native gRPC IOx query API using Arrow Flight -use std::fmt::Debug; -use std::task::Poll; -use std::{pin::Pin, sync::Arc}; use arrow::{ array::{make_array, ArrayRef, MutableArrayData}, @@ -15,24 +12,24 @@ use arrow_flight::{ HandshakeRequest, HandshakeResponse, PutResult, SchemaAsIpc, SchemaResult, Ticket, }; use bytes::{Bytes, BytesMut}; +use data_types2::{DatabaseName, DatabaseNameError}; use datafusion::physical_plan::ExecutionPlan; use futures::{SinkExt, Stream, StreamExt}; use generated_types::influxdata::iox::querier::v1 as proto; +use observability_deps::tracing::{info, warn}; use pin_project::{pin_project, pinned_drop}; use prost::Message; -use query::{QueryCompletedToken, QueryDatabase}; +use query::{ + exec::{ExecutionContextProvider, IOxSessionContext}, + QueryCompletedToken, QueryDatabase, +}; use serde::Deserialize; -use service_common::QueryDatabaseProvider; +use service_common::{planner::Planner, QueryDatabaseProvider}; use snafu::{ResultExt, Snafu}; +use std::{fmt::Debug, pin::Pin, sync::Arc, task::Poll}; use tokio::task::JoinHandle; use tonic::{Request, Response, Streaming}; -use data_types::{DatabaseName, DatabaseNameError}; -use observability_deps::tracing::{info, warn}; -use query::exec::{ExecutionContextProvider, IOxSessionContext}; - -use service_common::planner::Planner; - #[allow(clippy::enum_variant_names)] #[derive(Debug, Snafu)] pub enum Error { diff --git a/write_buffer/src/config.rs b/write_buffer/src/config.rs index 8b8fe139c2..140f175e53 100644 --- a/write_buffer/src/config.rs +++ b/write_buffer/src/config.rs @@ -193,7 +193,8 @@ mod tests { core::test_utils::random_topic_name, maybe_skip_kafka_integration, mock::MockBufferSharedState, }; - use data_types::{write_buffer::WriteBufferCreationConfig, DatabaseName}; + use data_types::write_buffer::WriteBufferCreationConfig; + use data_types2::DatabaseName; use std::{convert::TryFrom, num::NonZeroU32}; use tempfile::TempDir; From d2671355c36184aca9414875bdf43d8c17ec3ac0 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 09:49:34 -0400 Subject: [PATCH 12/31] fix: Move partition metadata types to data_types2 --- Cargo.lock | 2 + data_types2/Cargo.toml | 1 + data_types2/src/lib.rs | 1141 ++++++++++++++++++++++++++- dml/src/lib.rs | 3 +- mutable_batch/src/column.rs | 15 +- mutable_batch/src/writer.rs | 2 +- mutable_batch/tests/extend.rs | 8 +- mutable_batch/tests/extend_range.rs | 8 +- mutable_batch/tests/writer.rs | 2 +- mutable_batch/tests/writer_fuzz.rs | 33 +- parquet_file/src/chunk.rs | 9 +- parquet_file/src/metadata.rs | 10 +- querier/src/chunk/query_access.rs | 5 +- query/src/provider/overlap.rs | 19 +- query/src/provider/physical.rs | 14 +- query/src/pruning.rs | 19 +- query/src/statistics.rs | 9 +- query/src/test.rs | 2 +- read_buffer/Cargo.toml | 1 + read_buffer/src/chunk.rs | 12 +- read_buffer/src/schema.rs | 4 +- read_buffer/src/table.rs | 14 +- 22 files changed, 1220 insertions(+), 113 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 402acad833..3173fce4b3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1174,6 +1174,7 @@ version = "0.1.0" dependencies = [ "data_types", "influxdb_line_protocol", + "observability_deps", "ordered-float 3.0.0", "percent-encoding", "schema", @@ -4522,6 +4523,7 @@ dependencies = [ "criterion", "croaring", "data_types", + "data_types2", "datafusion 0.1.0", "either", "hashbrown 0.12.0", diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index 77dea8c888..84c58e76d8 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -7,6 +7,7 @@ description = "Shared data types in the IOx NG architecture" [dependencies] data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } +observability_deps = { path = "../observability_deps" } ordered-float = "3" percent-encoding = "2.1.0" schema = { path = "../schema" } diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 07828173e9..55ef129408 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -11,24 +11,23 @@ )] use influxdb_line_protocol::FieldValue; +use observability_deps::tracing::warn; use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC}; use schema::{builder::SchemaBuilder, sort::SortKey, InfluxColumnType, InfluxFieldType, Schema}; use snafu::{ResultExt, Snafu}; use std::{ - borrow::Cow, + borrow::{Borrow, Cow}, collections::BTreeMap, convert::TryFrom, fmt::Write, - num::{FpCategory, NonZeroU32}, + mem, + num::{FpCategory, NonZeroU32, NonZeroU64}, ops::{Add, Deref, RangeInclusive, Sub}, sync::Arc, }; use uuid::Uuid; pub use data_types::{ - partition_metadata::{ - ColumnSummary, InfluxDbType, PartitionAddr, StatValues, Statistics, TableSummary, - }, sequence::Sequence, timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}, }; @@ -1407,6 +1406,523 @@ impl<'a> std::fmt::Display for DatabaseName<'a> { } } +/// Column name, statistics which encode type information +#[derive(Debug, PartialEq, Clone)] +pub struct ColumnSummary { + /// Column name + pub name: String, + + /// Column's Influx data model type (if any) + pub influxdb_type: Option, + + /// Per column + pub stats: Statistics, +} + +impl ColumnSummary { + /// Returns the total number of rows (including nulls) in this column + pub fn total_count(&self) -> u64 { + self.stats.total_count() + } + + /// Updates statistics from other if the same type, otherwise a noop + pub fn update_from(&mut self, other: &Self) { + match (&mut self.stats, &other.stats) { + (Statistics::F64(s), Statistics::F64(o)) => { + s.update_from(o); + } + (Statistics::I64(s), Statistics::I64(o)) => { + s.update_from(o); + } + (Statistics::Bool(s), Statistics::Bool(o)) => { + s.update_from(o); + } + (Statistics::String(s), Statistics::String(o)) => { + s.update_from(o); + } + (Statistics::U64(s), Statistics::U64(o)) => { + s.update_from(o); + } + // do catch alls for the specific types, that way if a new type gets added, the compiler + // will complain. + (Statistics::F64(_), _) => unreachable!(), + (Statistics::I64(_), _) => unreachable!(), + (Statistics::U64(_), _) => unreachable!(), + (Statistics::Bool(_), _) => unreachable!(), + (Statistics::String(_), _) => unreachable!(), + } + } + + /// Updates these statistics so that that the total length of this + /// column is `len` rows, padding it with trailing NULLs if + /// necessary + pub fn update_to_total_count(&mut self, len: u64) { + let total_count = self.total_count(); + assert!( + total_count <= len, + "trying to shrink column stats from {} to {}", + total_count, + len + ); + let delta = len - total_count; + self.stats.update_for_nulls(delta); + } + + /// Return size in bytes of this Column metadata (not the underlying column) + pub fn size(&self) -> usize { + mem::size_of::() + self.name.len() + self.stats.size() + } +} + +// Replicate this enum here as it can't be derived from the existing statistics +#[derive(Debug, PartialEq, Clone, Copy)] +#[allow(missing_docs)] +pub enum InfluxDbType { + Tag, + Field, + Timestamp, +} + +/// Address of the chunk within the catalog +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct PartitionAddr { + /// Database name + pub db_name: Arc, + + /// What table does the chunk belong to? + pub table_name: Arc, + + /// What partition does the chunk belong to? + pub partition_key: Arc, +} + +/// Summary statistics for a column. +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct StatValues { + /// minimum (non-NaN, non-NULL) value, if any + pub min: Option, + + /// maximum (non-NaN, non-NULL) value, if any + pub max: Option, + + /// total number of values in this column, including null values + pub total_count: u64, + + /// number of null values in this column + pub null_count: Option, + + /// number of distinct values in this column if known + /// + /// This includes NULLs and NANs + pub distinct_count: Option, +} + +/// Represents the result of comparing the min/max ranges of two [`StatValues`] +#[derive(Debug, PartialEq, Copy, Clone)] +pub enum StatOverlap { + /// There is at least one value that exists in both ranges + NonZero, + + /// There are zero values that exists in both ranges + Zero, + + /// It is not known if there are any intersections (e.g. because + /// one of the bounds is not Known / is None) + Unknown, +} + +impl StatValues +where + T: PartialOrd, +{ + /// returns information about the overlap between two `StatValues` + pub fn overlaps(&self, other: &Self) -> StatOverlap { + match (&self.min, &self.max, &other.min, &other.max) { + (Some(self_min), Some(self_max), Some(other_min), Some(other_max)) => { + if self_min <= other_max && self_max >= other_min { + StatOverlap::NonZero + } else { + StatOverlap::Zero + } + } + // At least one of the values was None + _ => StatOverlap::Unknown, + } + } +} + +impl Default for StatValues { + fn default() -> Self { + Self { + min: None, + max: None, + total_count: 0, + null_count: None, + distinct_count: None, + } + } +} + +impl StatValues { + /// Create new statistics with no values + pub fn new_empty() -> Self { + Self { + min: None, + max: None, + total_count: 0, + null_count: Some(0), + distinct_count: None, + } + } + + /// Returns true if both the min and max values are None (aka not known) + pub fn is_none(&self) -> bool { + self.min.is_none() && self.max.is_none() + } + + /// Update the statistics values to account for `num_nulls` additional null values + pub fn update_for_nulls(&mut self, num_nulls: u64) { + self.total_count += num_nulls; + self.null_count = self.null_count.map(|x| x + num_nulls); + } + + /// updates the statistics keeping the min, max and incrementing count. + /// + /// The type plumbing exists to allow calling with &str on a StatValues + pub fn update(&mut self, other: &U) + where + T: Borrow, + U: ToOwned + PartialOrd + IsNan, + { + self.total_count += 1; + self.distinct_count = None; + + if !other.is_nan() { + match &self.min { + None => self.min = Some(other.to_owned()), + Some(s) => { + if s.borrow() > other { + self.min = Some(other.to_owned()); + } + } + } + + match &self.max { + None => { + self.max = Some(other.to_owned()); + } + Some(s) => { + if other > s.borrow() { + self.max = Some(other.to_owned()); + } + } + } + } + } +} + +impl StatValues +where + T: Clone + PartialOrd, +{ + /// Updates statistics from other + pub fn update_from(&mut self, other: &Self) { + self.total_count += other.total_count; + self.null_count = self.null_count.zip(other.null_count).map(|(a, b)| a + b); + + // No way to accurately aggregate counts + self.distinct_count = None; + + match (&self.min, &other.min) { + (None, None) | (Some(_), None) => {} + (None, Some(o)) => self.min = Some(o.clone()), + (Some(s), Some(o)) => { + if s > o { + self.min = Some(o.clone()); + } + } + } + + match (&self.max, &other.max) { + (None, None) | (Some(_), None) => {} + (None, Some(o)) => self.max = Some(o.clone()), + (Some(s), Some(o)) => { + if o > s { + self.max = Some(o.clone()); + } + } + }; + } +} + +impl StatValues +where + T: IsNan + PartialOrd, +{ + /// Create new statistics with the specified count and null count + pub fn new(min: Option, max: Option, total_count: u64, null_count: Option) -> Self { + let distinct_count = None; + Self::new_with_distinct(min, max, total_count, null_count, distinct_count) + } + + /// Create statistics for a column that only has nulls up to now + pub fn new_all_null(total_count: u64, distinct_count: Option) -> Self { + let min = None; + let max = None; + let null_count = Some(total_count); + + if let Some(count) = distinct_count { + assert!(count > 0); + } + Self::new_with_distinct( + min, + max, + total_count, + null_count, + distinct_count.map(|c| NonZeroU64::new(c).unwrap()), + ) + } + + /// Create statistics for a column with zero nulls and unknown distinct count + pub fn new_non_null(min: Option, max: Option, total_count: u64) -> Self { + let null_count = Some(0); + let distinct_count = None; + Self::new_with_distinct(min, max, total_count, null_count, distinct_count) + } + + /// Create new statistics with the specified count and null count and distinct values + pub fn new_with_distinct( + min: Option, + max: Option, + total_count: u64, + null_count: Option, + distinct_count: Option, + ) -> Self { + if let Some(min) = &min { + assert!(!min.is_nan()); + } + if let Some(max) = &max { + assert!(!max.is_nan()); + } + if let (Some(min), Some(max)) = (&min, &max) { + assert!(min <= max); + } + + Self { + min, + max, + total_count, + null_count, + distinct_count, + } + } +} + +/// Whether a type is NaN or not. +pub trait IsNan { + /// Test for NaNess. + fn is_nan(&self) -> bool; +} + +impl IsNan for &T { + fn is_nan(&self) -> bool { + (*self).is_nan() + } +} + +macro_rules! impl_is_nan_false { + ($t:ty) => { + impl IsNan for $t { + fn is_nan(&self) -> bool { + false + } + } + }; +} + +impl_is_nan_false!(bool); +impl_is_nan_false!(str); +impl_is_nan_false!(String); +impl_is_nan_false!(i8); +impl_is_nan_false!(i16); +impl_is_nan_false!(i32); +impl_is_nan_false!(i64); +impl_is_nan_false!(u8); +impl_is_nan_false!(u16); +impl_is_nan_false!(u32); +impl_is_nan_false!(u64); + +impl IsNan for f64 { + fn is_nan(&self) -> bool { + Self::is_nan(*self) + } +} + +/// Statistics and type information for a column. +#[derive(Debug, PartialEq, Clone)] +#[allow(missing_docs)] +pub enum Statistics { + I64(StatValues), + U64(StatValues), + F64(StatValues), + Bool(StatValues), + String(StatValues), +} + +impl Statistics { + /// Returns the total number of rows in this column + pub fn total_count(&self) -> u64 { + match self { + Self::I64(s) => s.total_count, + Self::U64(s) => s.total_count, + Self::F64(s) => s.total_count, + Self::Bool(s) => s.total_count, + Self::String(s) => s.total_count, + } + } + + /// Returns true if both the min and max values are None (aka not known) + pub fn is_none(&self) -> bool { + match self { + Self::I64(v) => v.is_none(), + Self::U64(v) => v.is_none(), + Self::F64(v) => v.is_none(), + Self::Bool(v) => v.is_none(), + Self::String(v) => v.is_none(), + } + } + + /// Returns the number of null rows in this column + pub fn null_count(&self) -> Option { + match self { + Self::I64(s) => s.null_count, + Self::U64(s) => s.null_count, + Self::F64(s) => s.null_count, + Self::Bool(s) => s.null_count, + Self::String(s) => s.null_count, + } + } + + /// Returns the distinct count if known + pub fn distinct_count(&self) -> Option { + match self { + Self::I64(s) => s.distinct_count, + Self::U64(s) => s.distinct_count, + Self::F64(s) => s.distinct_count, + Self::Bool(s) => s.distinct_count, + Self::String(s) => s.distinct_count, + } + } + + /// Update the statistics values to account for `num_nulls` additional null values + pub fn update_for_nulls(&mut self, num_nulls: u64) { + match self { + Self::I64(v) => v.update_for_nulls(num_nulls), + Self::U64(v) => v.update_for_nulls(num_nulls), + Self::F64(v) => v.update_for_nulls(num_nulls), + Self::Bool(v) => v.update_for_nulls(num_nulls), + Self::String(v) => v.update_for_nulls(num_nulls), + } + } + + /// Return the size in bytes of this stats instance + pub fn size(&self) -> usize { + match self { + Self::String(v) => std::mem::size_of::() + v.string_size(), + _ => std::mem::size_of::(), + } + } + + /// Return a human interpretable description of this type + pub fn type_name(&self) -> &'static str { + match self { + Self::I64(_) => "I64", + Self::U64(_) => "U64", + Self::F64(_) => "F64", + Self::Bool(_) => "Bool", + Self::String(_) => "String", + } + } +} + +impl StatValues { + /// Returns the bytes associated by storing min/max string values + pub fn string_size(&self) -> usize { + self.min.as_ref().map(|x| x.len()).unwrap_or(0) + + self.max.as_ref().map(|x| x.len()).unwrap_or(0) + } +} + +/// Metadata and statistics information for a table. This can be +/// either for the portion of a Table stored within a single chunk or +/// aggregated across chunks. +#[derive(Debug, PartialEq, Clone, Default)] +pub struct TableSummary { + /// Per column statistics + pub columns: Vec, +} + +impl TableSummary { + /// Get the column summary by name. + pub fn column(&self, name: &str) -> Option<&ColumnSummary> { + self.columns.iter().find(|c| c.name == name) + } + + /// Returns the total number of rows in the columns of this summary + pub fn total_count(&self) -> u64 { + // Assumes that all tables have the same number of rows, so + // pick the first one + let count = self.columns.get(0).map(|c| c.total_count()).unwrap_or(0); + + // Validate that the counts are consistent across columns + for c in &self.columns { + // Restore to assert when https://github.com/influxdata/influxdb_iox/issues/2124 is fixed + if c.total_count() != count { + warn!(column_name=%c.name, + column_count=c.total_count(), previous_count=count, + "Mismatch in statistics count, see #2124"); + } + } + count + } + + /// Updates the table summary with combined stats from the other. Counts are + /// treated as non-overlapping so they're just added together. If the + /// type of a column differs between the two tables, no update is done + /// on that column. Columns that only exist in the other are cloned into + /// this table summary. + pub fn update_from(&mut self, other: &Self) { + let new_total_count = self.total_count() + other.total_count(); + + // update all existing columns + for col in &mut self.columns { + if let Some(other_col) = other.column(&col.name) { + col.update_from(other_col); + } else { + col.update_to_total_count(new_total_count); + } + } + + // Add any columns that were new + for col in &other.columns { + if self.column(&col.name).is_none() { + let mut new_col = col.clone(); + // ensure the count is consistent + new_col.update_to_total_count(new_total_count); + self.columns.push(new_col); + } + } + } + + /// Total size of all ColumnSummaries that belong to this table which include + /// column names and their stats + pub fn size(&self) -> usize { + let size: usize = self.columns.iter().map(|c| c.size()).sum(); + size + mem::size_of::() // Add size of this struct that points to + // table and ColumnSummary + } +} + #[cfg(test)] mod tests { use super::*; @@ -1735,4 +2251,619 @@ mod tests { let db = DatabaseName::new("my-example-db_with_underscores and spaces").unwrap(); assert_eq!(&*db, "my-example-db_with_underscores and spaces"); } + + #[test] + fn statistics_new_non_null() { + let actual = StatValues::new_non_null(Some(-1i64), Some(1i64), 3); + let expected = StatValues { + min: Some(-1i64), + max: Some(1i64), + total_count: 3, + null_count: Some(0), + distinct_count: None, + }; + assert_eq!(actual, expected); + } + + #[test] + fn statistics_new_all_null() { + // i64 values do not have a distinct count + let actual = StatValues::::new_all_null(3, None); + let expected = StatValues { + min: None, + max: None, + total_count: 3, + null_count: Some(3), + distinct_count: None, + }; + assert_eq!(actual, expected); + + // string columns can have a distinct count + let actual = StatValues::::new_all_null(3, Some(1_u64)); + let expected = StatValues { + min: None, + max: None, + total_count: 3, + null_count: Some(3), + distinct_count: Some(NonZeroU64::try_from(1_u64).unwrap()), + }; + assert_eq!(actual, expected); + } + + impl StatValues + where + T: IsNan + PartialOrd + Clone, + { + fn new_with_value(starting_value: T) -> Self { + let starting_value = if starting_value.is_nan() { + None + } else { + Some(starting_value) + }; + + let min = starting_value.clone(); + let max = starting_value; + let total_count = 1; + let null_count = Some(0); + let distinct_count = None; + Self::new_with_distinct(min, max, total_count, null_count, distinct_count) + } + } + + impl Statistics { + /// Return the minimum value, if any, formatted as a string + fn min_as_str(&self) -> Option> { + match self { + Self::I64(v) => v.min.map(|x| Cow::Owned(x.to_string())), + Self::U64(v) => v.min.map(|x| Cow::Owned(x.to_string())), + Self::F64(v) => v.min.map(|x| Cow::Owned(x.to_string())), + Self::Bool(v) => v.min.map(|x| Cow::Owned(x.to_string())), + Self::String(v) => v.min.as_deref().map(Cow::Borrowed), + } + } + + /// Return the maximum value, if any, formatted as a string + fn max_as_str(&self) -> Option> { + match self { + Self::I64(v) => v.max.map(|x| Cow::Owned(x.to_string())), + Self::U64(v) => v.max.map(|x| Cow::Owned(x.to_string())), + Self::F64(v) => v.max.map(|x| Cow::Owned(x.to_string())), + Self::Bool(v) => v.max.map(|x| Cow::Owned(x.to_string())), + Self::String(v) => v.max.as_deref().map(Cow::Borrowed), + } + } + } + + #[test] + fn statistics_update() { + let mut stat = StatValues::new_with_value(23); + assert_eq!(stat.min, Some(23)); + assert_eq!(stat.max, Some(23)); + assert_eq!(stat.total_count, 1); + + stat.update(&55); + assert_eq!(stat.min, Some(23)); + assert_eq!(stat.max, Some(55)); + assert_eq!(stat.total_count, 2); + + stat.update(&6); + assert_eq!(stat.min, Some(6)); + assert_eq!(stat.max, Some(55)); + assert_eq!(stat.total_count, 3); + + stat.update(&30); + assert_eq!(stat.min, Some(6)); + assert_eq!(stat.max, Some(55)); + assert_eq!(stat.total_count, 4); + } + + #[test] + fn statistics_default() { + let mut stat = StatValues::default(); + assert_eq!(stat.min, None); + assert_eq!(stat.max, None); + assert_eq!(stat.total_count, 0); + + stat.update(&55); + assert_eq!(stat.min, Some(55)); + assert_eq!(stat.max, Some(55)); + assert_eq!(stat.total_count, 1); + + let mut stat = StatValues::::default(); + assert_eq!(stat.min, None); + assert_eq!(stat.max, None); + assert_eq!(stat.total_count, 0); + + stat.update("cupcakes"); + assert_eq!(stat.min, Some("cupcakes".to_string())); + assert_eq!(stat.max, Some("cupcakes".to_string())); + assert_eq!(stat.total_count, 1); + + stat.update("woo"); + assert_eq!(stat.min, Some("cupcakes".to_string())); + assert_eq!(stat.max, Some("woo".to_string())); + assert_eq!(stat.total_count, 2); + } + + #[test] + fn statistics_is_none() { + let mut stat = StatValues::default(); + assert!(stat.is_none()); + stat.min = Some(0); + assert!(!stat.is_none()); + stat.max = Some(1); + assert!(!stat.is_none()); + } + + #[test] + fn statistics_overlaps() { + let stat1 = StatValues { + min: Some(10), + max: Some(20), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat1), StatOverlap::NonZero); + + // [--stat1--] + // [--stat2--] + let stat2 = StatValues { + min: Some(5), + max: Some(15), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat2), StatOverlap::NonZero); + assert_eq!(stat2.overlaps(&stat1), StatOverlap::NonZero); + + // [--stat1--] + // [--stat3--] + let stat3 = StatValues { + min: Some(15), + max: Some(25), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat3), StatOverlap::NonZero); + assert_eq!(stat3.overlaps(&stat1), StatOverlap::NonZero); + + // [--stat1--] + // [--stat4--] + let stat4 = StatValues { + min: Some(25), + max: Some(35), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat4), StatOverlap::Zero); + assert_eq!(stat4.overlaps(&stat1), StatOverlap::Zero); + + // [--stat1--] + // [--stat5--] + let stat5 = StatValues { + min: Some(0), + max: Some(5), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat5), StatOverlap::Zero); + assert_eq!(stat5.overlaps(&stat1), StatOverlap::Zero); + } + + #[test] + fn statistics_overlaps_none() { + let stat1 = StatValues { + min: Some(10), + max: Some(20), + ..Default::default() + }; + + let stat2 = StatValues { + min: None, + max: Some(20), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat2), StatOverlap::Unknown); + assert_eq!(stat2.overlaps(&stat1), StatOverlap::Unknown); + + let stat3 = StatValues { + min: Some(10), + max: None, + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat3), StatOverlap::Unknown); + assert_eq!(stat3.overlaps(&stat1), StatOverlap::Unknown); + + let stat4 = StatValues { + min: None, + max: None, + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat4), StatOverlap::Unknown); + assert_eq!(stat4.overlaps(&stat1), StatOverlap::Unknown); + } + + #[test] + fn statistics_overlaps_mixed_none() { + let stat1 = StatValues { + min: Some(10), + max: None, + ..Default::default() + }; + + let stat2 = StatValues { + min: None, + max: Some(5), + ..Default::default() + }; + assert_eq!(stat1.overlaps(&stat2), StatOverlap::Unknown); + assert_eq!(stat2.overlaps(&stat1), StatOverlap::Unknown); + } + + #[test] + fn update_string() { + let mut stat = StatValues::new_with_value("bbb".to_string()); + assert_eq!(stat.min, Some("bbb".to_string())); + assert_eq!(stat.max, Some("bbb".to_string())); + assert_eq!(stat.total_count, 1); + + stat.update("aaa"); + assert_eq!(stat.min, Some("aaa".to_string())); + assert_eq!(stat.max, Some("bbb".to_string())); + assert_eq!(stat.total_count, 2); + + stat.update("z"); + assert_eq!(stat.min, Some("aaa".to_string())); + assert_eq!(stat.max, Some("z".to_string())); + assert_eq!(stat.total_count, 3); + + stat.update("p"); + assert_eq!(stat.min, Some("aaa".to_string())); + assert_eq!(stat.max, Some("z".to_string())); + assert_eq!(stat.total_count, 4); + } + + #[test] + fn stats_is_none() { + let stat = Statistics::I64(StatValues::new_non_null(Some(-1), Some(100), 1)); + assert!(!stat.is_none()); + + let stat = Statistics::I64(StatValues::new_non_null(None, Some(100), 1)); + assert!(!stat.is_none()); + + let stat = Statistics::I64(StatValues::new_non_null(None, None, 0)); + assert!(stat.is_none()); + } + + #[test] + fn stats_as_str_i64() { + let stat = Statistics::I64(StatValues::new_non_null(Some(-1), Some(100), 1)); + assert_eq!(stat.min_as_str(), Some("-1".into())); + assert_eq!(stat.max_as_str(), Some("100".into())); + + let stat = Statistics::I64(StatValues::new_non_null(None, None, 1)); + assert_eq!(stat.min_as_str(), None); + assert_eq!(stat.max_as_str(), None); + } + + #[test] + fn stats_as_str_u64() { + let stat = Statistics::U64(StatValues::new_non_null(Some(1), Some(100), 1)); + assert_eq!(stat.min_as_str(), Some("1".into())); + assert_eq!(stat.max_as_str(), Some("100".into())); + + let stat = Statistics::U64(StatValues::new_non_null(None, None, 1)); + assert_eq!(stat.min_as_str(), None); + assert_eq!(stat.max_as_str(), None); + } + + #[test] + fn stats_as_str_f64() { + let stat = Statistics::F64(StatValues::new_non_null(Some(99.0), Some(101.0), 1)); + assert_eq!(stat.min_as_str(), Some("99".into())); + assert_eq!(stat.max_as_str(), Some("101".into())); + + let stat = Statistics::F64(StatValues::new_non_null(None, None, 1)); + assert_eq!(stat.min_as_str(), None); + assert_eq!(stat.max_as_str(), None); + } + + #[test] + fn stats_as_str_bool() { + let stat = Statistics::Bool(StatValues::new_non_null(Some(false), Some(true), 1)); + assert_eq!(stat.min_as_str(), Some("false".into())); + assert_eq!(stat.max_as_str(), Some("true".into())); + + let stat = Statistics::Bool(StatValues::new_non_null(None, None, 1)); + assert_eq!(stat.min_as_str(), None); + assert_eq!(stat.max_as_str(), None); + } + + #[test] + fn stats_as_str_str() { + let stat = Statistics::String(StatValues::new_non_null( + Some("a".to_string()), + Some("zz".to_string()), + 1, + )); + assert_eq!(stat.min_as_str(), Some("a".into())); + assert_eq!(stat.max_as_str(), Some("zz".into())); + + let stat = Statistics::String(StatValues::new_non_null(None, None, 1)); + assert_eq!(stat.min_as_str(), None); + assert_eq!(stat.max_as_str(), None); + } + + #[test] + fn table_update_from() { + let mut string_stats = StatValues::new_with_value("foo".to_string()); + string_stats.update("bar"); + let string_col = ColumnSummary { + name: "string".to_string(), + influxdb_type: None, + stats: Statistics::String(string_stats), + }; + + let mut int_stats = StatValues::new_with_value(1); + int_stats.update(&5); + let int_col = ColumnSummary { + name: "int".to_string(), + influxdb_type: None, + stats: Statistics::I64(int_stats), + }; + + let mut float_stats = StatValues::new_with_value(9.1); + float_stats.update(&1.3); + let float_col = ColumnSummary { + name: "float".to_string(), + influxdb_type: None, + stats: Statistics::F64(float_stats), + }; + + let mut table_a = TableSummary { + columns: vec![string_col, int_col, float_col], + }; + + let mut string_stats = StatValues::new_with_value("aaa".to_string()); + string_stats.update("zzz"); + let string_col = ColumnSummary { + name: "string".to_string(), + influxdb_type: None, + stats: Statistics::String(string_stats), + }; + + let mut int_stats = StatValues::new_with_value(3); + int_stats.update(&9); + let int_col = ColumnSummary { + name: "int".to_string(), + influxdb_type: None, + stats: Statistics::I64(int_stats), + }; + + let mut table_b = TableSummary { + columns: vec![int_col, string_col], + }; + + // keep this to test joining the other way + let table_c = table_a.clone(); + + table_a.update_from(&table_b); + let col = table_a.column("string").unwrap(); + assert_eq!( + col.stats, + Statistics::String(StatValues::new_non_null( + Some("aaa".to_string()), + Some("zzz".to_string()), + 4, + )) + ); + + let col = table_a.column("int").unwrap(); + assert_eq!( + col.stats, + Statistics::I64(StatValues::new_non_null(Some(1), Some(9), 4)) + ); + + let col = table_a.column("float").unwrap(); + assert_eq!( + col.stats, + Statistics::F64(StatValues::new(Some(1.3), Some(9.1), 4, Some(2))) + ); + + table_b.update_from(&table_c); + let col = table_b.column("string").unwrap(); + assert_eq!( + col.stats, + Statistics::String(StatValues::new_non_null( + Some("aaa".to_string()), + Some("zzz".to_string()), + 4, + )) + ); + + let col = table_b.column("int").unwrap(); + assert_eq!( + col.stats, + Statistics::I64(StatValues::new_non_null(Some(1), Some(9), 4)) + ); + + let col = table_b.column("float").unwrap(); + assert_eq!( + col.stats, + Statistics::F64(StatValues::new(Some(1.3), Some(9.1), 4, Some(2))) + ); + } + + #[test] + fn table_update_from_new_column() { + let string_stats = StatValues::new_with_value("bar".to_string()); + let string_col = ColumnSummary { + name: "string".to_string(), + influxdb_type: None, + stats: Statistics::String(string_stats), + }; + + let int_stats = StatValues::new_with_value(5); + let int_col = ColumnSummary { + name: "int".to_string(), + influxdb_type: None, + stats: Statistics::I64(int_stats), + }; + + // table summary that does not have the "string" col + let table1 = TableSummary { + columns: vec![int_col.clone()], + }; + + // table summary that has both columns + let table2 = TableSummary { + columns: vec![int_col, string_col], + }; + + // Statistics should be the same regardless of the order we update the stats + + let expected_string_stats = Statistics::String(StatValues::new( + Some("bar".to_string()), + Some("bar".to_string()), + 2, // total count is 2 even though did not appear in the update + Some(1), // 1 null + )); + + let expected_int_stats = Statistics::I64(StatValues::new( + Some(5), + Some(5), + 2, + Some(0), // no nulls + )); + + // update table 1 with table 2 + let mut table = table1.clone(); + table.update_from(&table2); + + assert_eq!( + &table.column("string").unwrap().stats, + &expected_string_stats + ); + + assert_eq!(&table.column("int").unwrap().stats, &expected_int_stats); + + // update table 2 with table 1 + let mut table = table2; + table.update_from(&table1); + + assert_eq!( + &table.column("string").unwrap().stats, + &expected_string_stats + ); + + assert_eq!(&table.column("int").unwrap().stats, &expected_int_stats); + } + + #[test] + fn column_update_from_boolean() { + let bool_false = ColumnSummary { + name: "b".to_string(), + influxdb_type: None, + stats: Statistics::Bool(StatValues::new(Some(false), Some(false), 1, Some(1))), + }; + let bool_true = ColumnSummary { + name: "b".to_string(), + influxdb_type: None, + stats: Statistics::Bool(StatValues::new(Some(true), Some(true), 1, Some(2))), + }; + + let expected_stats = Statistics::Bool(StatValues::new(Some(false), Some(true), 2, Some(3))); + + let mut b = bool_false.clone(); + b.update_from(&bool_true); + assert_eq!(b.stats, expected_stats); + + let mut b = bool_true; + b.update_from(&bool_false); + assert_eq!(b.stats, expected_stats); + } + + #[test] + fn column_update_from_u64() { + let mut min = ColumnSummary { + name: "foo".to_string(), + influxdb_type: None, + stats: Statistics::U64(StatValues::new(Some(5), Some(23), 1, Some(1))), + }; + + let max = ColumnSummary { + name: "foo".to_string(), + influxdb_type: None, + stats: Statistics::U64(StatValues::new(Some(6), Some(506), 43, Some(2))), + }; + + min.update_from(&max); + + let expected = Statistics::U64(StatValues::new(Some(5), Some(506), 44, Some(3))); + assert_eq!(min.stats, expected); + } + + #[test] + fn nans() { + let mut stat = StatValues::default(); + assert_eq!(stat.min, None); + assert_eq!(stat.max, None); + assert_eq!(stat.total_count, 0); + + stat.update(&f64::NAN); + assert_eq!(stat.min, None); + assert_eq!(stat.max, None); + assert_eq!(stat.total_count, 1); + + stat.update(&1.0); + assert_eq!(stat.min, Some(1.0)); + assert_eq!(stat.max, Some(1.0)); + assert_eq!(stat.total_count, 2); + + stat.update(&2.0); + assert_eq!(stat.min, Some(1.0)); + assert_eq!(stat.max, Some(2.0)); + assert_eq!(stat.total_count, 3); + + stat.update(&f64::INFINITY); + assert_eq!(stat.min, Some(1.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat.total_count, 4); + + stat.update(&-1.0); + assert_eq!(stat.min, Some(-1.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat.total_count, 5); + + // =========== + + let mut stat = StatValues::new_with_value(2.0); + stat.update(&f64::INFINITY); + assert_eq!(stat.min, Some(2.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat.total_count, 2); + + stat.update(&f64::NAN); + assert_eq!(stat.min, Some(2.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat.total_count, 3); + + // =========== + + let mut stat2 = StatValues::new_with_value(1.0); + stat2.update_from(&stat); + assert_eq!(stat2.min, Some(1.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat2.total_count, 4); + + // =========== + + let stat2 = StatValues::new_with_value(1.0); + stat.update_from(&stat2); + assert_eq!(stat.min, Some(1.0)); + assert_eq!(stat.max, Some(f64::INFINITY)); + assert_eq!(stat.total_count, 4); + + // =========== + + let stat = StatValues::new_with_value(f64::NAN); + assert_eq!(stat.min, None); + assert_eq!(stat.max, None); + assert_eq!(stat.total_count, 1); + } } diff --git a/dml/src/lib.rs b/dml/src/lib.rs index 2f33b45887..d818252ad4 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -12,11 +12,10 @@ )] use data_types::{ - partition_metadata::{StatValues, Statistics}, router::{ShardConfig, ShardId}, sequence::Sequence, }; -use data_types2::{DeletePredicate, NonEmptyString}; +use data_types2::{DeletePredicate, NonEmptyString, StatValues, Statistics}; use hashbrown::HashMap; use iox_time::Time; use mutable_batch::MutableBatch; diff --git a/mutable_batch/src/column.rs b/mutable_batch/src/column.rs index 0f18038b99..e2631dcce2 100644 --- a/mutable_batch/src/column.rs +++ b/mutable_batch/src/column.rs @@ -1,23 +1,18 @@ //! A [`Column`] stores the rows for a given column name -use std::fmt::Formatter; -use std::mem; -use std::sync::Arc; - -use arrow::error::ArrowError; use arrow::{ array::{ ArrayDataBuilder, ArrayRef, BooleanArray, Float64Array, Int64Array, TimestampNanosecondArray, UInt64Array, }, datatypes::DataType, + error::ArrowError, }; -use snafu::{ResultExt, Snafu}; - -use arrow_util::bitset::BitSet; -use arrow_util::string::PackedStringArray; -use data_types::partition_metadata::{StatValues, Statistics}; +use arrow_util::{bitset::BitSet, string::PackedStringArray}; +use data_types2::{StatValues, Statistics}; use schema::{InfluxColumnType, InfluxFieldType, TIME_DATA_TYPE}; +use snafu::{ResultExt, Snafu}; +use std::{fmt::Formatter, mem, sync::Arc}; /// A "dictionary ID" (DID) is a compact numeric representation of an interned /// string in the dictionary. The same string always maps the same DID. diff --git a/mutable_batch/src/writer.rs b/mutable_batch/src/writer.rs index 90824cc898..c322594f2e 100644 --- a/mutable_batch/src/writer.rs +++ b/mutable_batch/src/writer.rs @@ -3,7 +3,7 @@ use crate::column::{Column, ColumnData, INVALID_DID}; use crate::MutableBatch; use arrow_util::bitset::{iter_set_positions, iter_set_positions_with_offset, BitSet}; -use data_types::partition_metadata::{IsNan, StatValues, Statistics}; +use data_types2::{IsNan, StatValues, Statistics}; use schema::{InfluxColumnType, InfluxFieldType}; use snafu::Snafu; use std::num::NonZeroU64; diff --git a/mutable_batch/tests/extend.rs b/mutable_batch/tests/extend.rs index 00b4fbffbf..42325fcdff 100644 --- a/mutable_batch/tests/extend.rs +++ b/mutable_batch/tests/extend.rs @@ -1,10 +1,8 @@ use arrow_util::assert_batches_eq; -use data_types::partition_metadata::{StatValues, Statistics}; -use mutable_batch::writer::Writer; -use mutable_batch::MutableBatch; +use data_types2::{StatValues, Statistics}; +use mutable_batch::{writer::Writer, MutableBatch}; use schema::selection::Selection; -use std::collections::BTreeMap; -use std::num::NonZeroU64; +use std::{collections::BTreeMap, num::NonZeroU64}; #[test] fn test_extend() { diff --git a/mutable_batch/tests/extend_range.rs b/mutable_batch/tests/extend_range.rs index d37bd26a1b..68437141d5 100644 --- a/mutable_batch/tests/extend_range.rs +++ b/mutable_batch/tests/extend_range.rs @@ -1,10 +1,8 @@ use arrow_util::assert_batches_eq; -use data_types::partition_metadata::{StatValues, Statistics}; -use mutable_batch::writer::Writer; -use mutable_batch::MutableBatch; +use data_types2::{StatValues, Statistics}; +use mutable_batch::{writer::Writer, MutableBatch}; use schema::selection::Selection; -use std::collections::BTreeMap; -use std::num::NonZeroU64; +use std::{collections::BTreeMap, num::NonZeroU64}; #[test] fn test_extend_range() { diff --git a/mutable_batch/tests/writer.rs b/mutable_batch/tests/writer.rs index 670a7638e0..20b97a702b 100644 --- a/mutable_batch/tests/writer.rs +++ b/mutable_batch/tests/writer.rs @@ -1,6 +1,6 @@ use arrow_util::assert_batches_eq; -use data_types::partition_metadata::{StatValues, Statistics}; use data_types::write_summary::TimestampSummary; +use data_types2::{StatValues, Statistics}; use mutable_batch::writer::Writer; use mutable_batch::MutableBatch; use schema::selection::Selection; diff --git a/mutable_batch/tests/writer_fuzz.rs b/mutable_batch/tests/writer_fuzz.rs index e3bf11c6c9..6475faf9ad 100644 --- a/mutable_batch/tests/writer_fuzz.rs +++ b/mutable_batch/tests/writer_fuzz.rs @@ -1,3 +1,16 @@ +use arrow::{ + array::{ + ArrayRef, BooleanArray, Float64Array, Int64Array, StringArray, TimestampNanosecondArray, + UInt64Array, + }, + record_batch::RecordBatch, +}; +use arrow_util::bitset::BitSet; +use data_types2::{IsNan, PartitionTemplate, StatValues, Statistics, TemplatePart}; +use hashbrown::HashSet; +use mutable_batch::{writer::Writer, MutableBatch, PartitionWrite, WritePayload}; +use rand::prelude::*; +use schema::selection::Selection; /// A fuzz test of the [`mutable_batch::Writer`] interface: /// /// - column writes - `write_i64`, `write_tag`, etc... @@ -5,25 +18,7 @@ /// - batch writes with ranges - `write_batch_ranges` /// /// Verifies that the rows and statistics are as expected after a number of interleaved writes -use std::collections::BTreeMap; -use std::num::NonZeroU64; -use std::ops::Range; -use std::sync::Arc; - -use arrow::array::{ - ArrayRef, BooleanArray, Float64Array, Int64Array, StringArray, TimestampNanosecondArray, - UInt64Array, -}; -use arrow::record_batch::RecordBatch; -use hashbrown::HashSet; -use rand::prelude::*; - -use arrow_util::bitset::BitSet; -use data_types::partition_metadata::{IsNan, StatValues, Statistics}; -use data_types2::{PartitionTemplate, TemplatePart}; -use mutable_batch::writer::Writer; -use mutable_batch::{MutableBatch, PartitionWrite, WritePayload}; -use schema::selection::Selection; +use std::{collections::BTreeMap, num::NonZeroU64, ops::Range, sync::Arc}; fn make_rng() -> StdRng { let seed = rand::rngs::OsRng::default().next_u64(); diff --git a/parquet_file/src/chunk.rs b/parquet_file/src/chunk.rs index 1dee0f3d0b..3a9f2c749d 100644 --- a/parquet_file/src/chunk.rs +++ b/parquet_file/src/chunk.rs @@ -3,17 +3,14 @@ use crate::{ storage::Storage, ParquetFilePath, }; -use data_types::{ - partition_metadata::{Statistics, TableSummary}, - timestamp::{TimestampMinMax, TimestampRange}, +use data_types2::{ + ParquetFile, ParquetFileWithMetadata, Statistics, TableSummary, TimestampMinMax, TimestampRange, }; -use data_types2::{ParquetFile, ParquetFileWithMetadata}; use datafusion::physical_plan::SendableRecordBatchStream; use object_store::DynObjectStore; use observability_deps::tracing::*; use predicate::Predicate; -use schema::selection::Selection; -use schema::{Schema, TIME_COLUMN_NAME}; +use schema::{selection::Selection, Schema, TIME_COLUMN_NAME}; use snafu::{ResultExt, Snafu}; use std::{collections::BTreeSet, mem, sync::Arc}; diff --git a/parquet_file/src/metadata.rs b/parquet_file/src/metadata.rs index 92a09a868a..b359b3ab01 100644 --- a/parquet_file/src/metadata.rs +++ b/parquet_file/src/metadata.rs @@ -86,9 +86,9 @@ //! [Apache Parquet]: https://parquet.apache.org/ //! [Apache Thrift]: https://thrift.apache.org/ //! [Thrift Compact Protocol]: https://github.com/apache/thrift/blob/master/doc/specs/thrift-compact-protocol.md -use data_types::partition_metadata::{ColumnSummary, InfluxDbType, StatValues, Statistics}; use data_types2::{ - NamespaceId, ParquetFileParams, PartitionId, SequenceNumber, SequencerId, TableId, Timestamp, + ColumnSummary, InfluxDbType, NamespaceId, ParquetFileParams, PartitionId, SequenceNumber, + SequencerId, StatValues, Statistics, TableId, Timestamp, }; use generated_types::influxdata::iox::ingester::v1 as proto; use iox_time::Time; @@ -106,8 +106,10 @@ use parquet::{ schema::types::SchemaDescriptor as ParquetSchemaDescriptor, }; use prost::Message; -use schema::sort::{SortKey, SortKeyBuilder}; -use schema::{InfluxColumnType, InfluxFieldType, Schema}; +use schema::{ + sort::{SortKey, SortKeyBuilder}, + InfluxColumnType, InfluxFieldType, Schema, +}; use snafu::{ensure, OptionExt, ResultExt, Snafu}; use std::{convert::TryInto, sync::Arc}; use thrift::protocol::{TCompactInputProtocol, TCompactOutputProtocol, TOutputProtocol}; diff --git a/querier/src/chunk/query_access.rs b/querier/src/chunk/query_access.rs index 6479344a1f..8d6d17da0a 100644 --- a/querier/src/chunk/query_access.rs +++ b/querier/src/chunk/query_access.rs @@ -1,7 +1,8 @@ use std::sync::Arc; -use data_types::timestamp::TimestampMinMax; -use data_types2::{ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary}; +use data_types2::{ + ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary, TimestampMinMax, +}; use observability_deps::tracing::debug; use predicate::PredicateMatch; use query::{QueryChunk, QueryChunkError, QueryChunkMeta}; diff --git a/query/src/provider/overlap.rs b/query/src/provider/overlap.rs index fba36af4d0..ae08d22c7c 100644 --- a/query/src/provider/overlap.rs +++ b/query/src/provider/overlap.rs @@ -1,21 +1,18 @@ -//! Contains the algorithm to determine which chunks may contain -//! "duplicate" primary keys (that is where data with the same -//! combination of "tag" columns and timestamp in the InfluxDB -//! DataModel have been written in via multiple distinct line protocol -//! writes (and thus are stored in separate rows) +//! Contains the algorithm to determine which chunks may contain "duplicate" primary keys (that is +//! where data with the same combination of "tag" columns and timestamp in the InfluxDB DataModel +//! have been written in via multiple distinct line protocol writes (and thus are stored in +//! separate rows) -use data_types::{ - partition_metadata::{ColumnSummary, StatOverlap, Statistics}, - timestamp::TimestampMinMax, +use crate::{QueryChunk, QueryChunkMeta}; +use data_types2::{ + ColumnSummary, DeletePredicate, ParquetFileWithMetadata, PartitionId, StatOverlap, Statistics, + TableSummary, TimestampMinMax, }; -use data_types2::{DeletePredicate, ParquetFileWithMetadata, PartitionId, TableSummary}; use observability_deps::tracing::debug; use schema::{sort::SortKey, Schema, TIME_COLUMN_NAME}; use snafu::Snafu; use std::{cmp::Ordering, sync::Arc}; -use crate::{QueryChunk, QueryChunkMeta}; - #[derive(Debug, Snafu)] pub enum Error { #[snafu(display( diff --git a/query/src/provider/physical.rs b/query/src/provider/physical.rs index 165c148ace..98f661393f 100644 --- a/query/src/provider/physical.rs +++ b/query/src/provider/physical.rs @@ -1,9 +1,9 @@ //! Implementation of a DataFusion PhysicalPlan node across partition chunks -use std::{fmt, sync::Arc}; - +use super::adapter::SchemaAdapterStream; +use crate::{exec::IOxSessionContext, QueryChunk}; use arrow::datatypes::SchemaRef; -use data_types::partition_metadata::TableSummary; +use data_types2::TableSummary; use datafusion::{ error::DataFusionError, execution::context::TaskContext, @@ -14,13 +14,9 @@ use datafusion::{ }, }; use observability_deps::tracing::debug; -use schema::selection::Selection; -use schema::Schema; - -use crate::{exec::IOxSessionContext, QueryChunk}; use predicate::Predicate; - -use super::adapter::SchemaAdapterStream; +use schema::{selection::Selection, Schema}; +use std::{fmt, sync::Arc}; /// Implements the DataFusion physical plan interface #[derive(Debug)] diff --git a/query/src/pruning.rs b/query/src/pruning.rs index 0274aa6eb3..d505b9b0ad 100644 --- a/query/src/pruning.rs +++ b/query/src/pruning.rs @@ -1,23 +1,22 @@ //! Implementation of statistics based pruning -use std::sync::Arc; - -use arrow::array::{ - ArrayRef, BooleanArray, DictionaryArray, Float64Array, Int64Array, StringArray, UInt64Array, +use crate::QueryChunk; +use arrow::{ + array::{ + ArrayRef, BooleanArray, DictionaryArray, Float64Array, Int64Array, StringArray, UInt64Array, + }, + datatypes::{DataType, Int32Type, TimeUnit}, }; -use arrow::datatypes::{DataType, Int32Type, TimeUnit}; - -use data_types::partition_metadata::{StatValues, Statistics}; +use data_types2::{StatValues, Statistics}; use datafusion::{ logical_plan::Column, physical_optimizer::pruning::{PruningPredicate, PruningStatistics}, }; use observability_deps::tracing::{debug, trace}; use predicate::Predicate; -use schema::Schema; - -use crate::QueryChunk; use query_functions::group_by::Aggregate; +use schema::Schema; +use std::sync::Arc; /// Something that cares to be notified when pruning of chunks occurs pub trait PruningObserver { diff --git a/query/src/statistics.rs b/query/src/statistics.rs index 7f9d75b0ef..7268c52a44 100644 --- a/query/src/statistics.rs +++ b/query/src/statistics.rs @@ -1,8 +1,6 @@ //! Code to translate IOx statistics to DataFusion statistics -use data_types::partition_metadata::{ - ColumnSummary, InfluxDbType, Statistics as IOxStatistics, TableSummary, -}; +use data_types2::{ColumnSummary, InfluxDbType, Statistics as IOxStatistics, TableSummary}; use datafusion::{ physical_plan::{ColumnStatistics, Statistics as DFStatistics}, scalar::ScalarValue, @@ -107,11 +105,10 @@ fn df_from_iox_col(col: &ColumnSummary) -> ColumnStatistics { #[cfg(test)] mod test { - use std::num::NonZeroU64; - use super::*; - use data_types::partition_metadata::{InfluxDbType, StatValues}; + use data_types2::{InfluxDbType, StatValues}; use schema::{builder::SchemaBuilder, InfluxFieldType}; + use std::num::NonZeroU64; macro_rules! assert_nice_eq { ($actual:ident, $expected:ident) => { diff --git a/query/src/test.rs b/query/src/test.rs index 25222244be..8a2b85781c 100644 --- a/query/src/test.rs +++ b/query/src/test.rs @@ -283,7 +283,7 @@ impl TestChunk { Self { table_name, schema: Arc::new(SchemaBuilder::new().build().unwrap()), - table_summary: TableSummary::new(), + table_summary: TableSummary::default(), id: ChunkId::new_test(0), may_contain_pk_duplicates: Default::default(), predicates: Default::default(), diff --git a/read_buffer/Cargo.toml b/read_buffer/Cargo.toml index 815c98dc0c..c28185807c 100644 --- a/read_buffer/Cargo.toml +++ b/read_buffer/Cargo.toml @@ -15,6 +15,7 @@ arrow = { version = "13", features = ["prettyprint"] } arrow_util = { path = "../arrow_util" } croaring = "0.6" data_types = { path = "../data_types" } +data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } either = "1.6.1" hashbrown = "0.12" diff --git a/read_buffer/src/chunk.rs b/read_buffer/src/chunk.rs index 8fa6f8a8d1..f2412a2d35 100644 --- a/read_buffer/src/chunk.rs +++ b/read_buffer/src/chunk.rs @@ -5,11 +5,10 @@ use crate::{ table::{self, Table}, }; use arrow::{error::ArrowError, record_batch::RecordBatch}; -use data_types::{chunk_metadata::ChunkColumnSummary, partition_metadata::TableSummary}; - +use data_types::chunk_metadata::ChunkColumnSummary; +use data_types2::TableSummary; use observability_deps::tracing::debug; -use schema::selection::Selection; -use schema::{builder::Error as SchemaError, Schema}; +use schema::{builder::Error as SchemaError, selection::Selection, Schema}; use snafu::{ResultExt, Snafu}; use std::{ collections::{BTreeMap, BTreeSet}, @@ -527,11 +526,10 @@ mod test { Int32Type, }, }; - use data_types::partition_metadata::{ColumnSummary, InfluxDbType, StatValues, Statistics}; + use data_types2::{ColumnSummary, InfluxDbType, StatValues, Statistics}; use metric::{Attributes, MetricKind, Observation, ObservationSet, RawReporter}; use schema::builder::SchemaBuilder; - use std::iter::FromIterator; - use std::{num::NonZeroU64, sync::Arc}; + use std::{iter::FromIterator, num::NonZeroU64, sync::Arc}; // helper to make the `add_remove_tables` test simpler to read. fn gen_recordbatch() -> RecordBatch { diff --git a/read_buffer/src/schema.rs b/read_buffer/src/schema.rs index 1ea39cb3eb..981dce42c6 100644 --- a/read_buffer/src/schema.rs +++ b/read_buffer/src/schema.rs @@ -241,8 +241,8 @@ impl ColumnType { } } - pub fn as_influxdb_type(&self) -> Option { - use data_types::partition_metadata::InfluxDbType; + pub fn as_influxdb_type(&self) -> Option { + use data_types2::InfluxDbType; match self { Self::Tag(_) => Some(InfluxDbType::Tag), Self::Field(_) => Some(InfluxDbType::Field), diff --git a/read_buffer/src/table.rs b/read_buffer/src/table.rs index addab2105e..b6fbbd6d4f 100644 --- a/read_buffer/src/table.rs +++ b/read_buffer/src/table.rs @@ -6,7 +6,8 @@ use crate::{ BinaryExpr, }; use arrow::record_batch::RecordBatch; -use data_types::{chunk_metadata::ChunkColumnSummary, partition_metadata::TableSummary}; +use data_types::chunk_metadata::ChunkColumnSummary; +use data_types2::TableSummary; use parking_lot::RwLock; use schema::selection::Selection; use snafu::{ensure, Snafu}; @@ -782,7 +783,7 @@ impl MetaData { } pub fn to_summary(&self) -> TableSummary { - use data_types::partition_metadata::{ColumnSummary, StatValues, Statistics}; + use data_types2::{ColumnSummary, StatValues, Statistics}; let columns = self .columns .iter() @@ -850,8 +851,8 @@ impl MetaData { fn make_null_stats( total_count: u64, logical_data_type: &LogicalDataType, -) -> data_types::partition_metadata::Statistics { - use data_types::partition_metadata::{StatValues, Statistics}; +) -> data_types2::Statistics { + use data_types2::{StatValues, Statistics}; use LogicalDataType::*; match logical_data_type { @@ -1103,9 +1104,6 @@ impl std::fmt::Display for DisplayReadAggregateResults<'_> { #[cfg(test)] mod test { - use arrow::array::BooleanArray; - use data_types::partition_metadata::{StatValues, Statistics}; - use super::*; use crate::{ column::Column, @@ -1113,6 +1111,8 @@ mod test { schema::{self, LogicalDataType}, value::{AggregateVec, OwnedValue, Scalar}, }; + use arrow::array::BooleanArray; + use data_types2::{StatValues, Statistics}; #[test] fn meta_data_update_with() { From 236edb9181053c44897649976a6bff4ba62bc7ec Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 10:33:10 -0400 Subject: [PATCH 13/31] fix: Move Sequence type to data_types2 --- data_types/src/lib.rs | 1 - data_types/src/sequence.rs | 16 ---------------- data_types2/src/lib.rs | 24 ++++++++++++++++++++---- dml/src/lib.rs | 7 ++----- write_buffer/src/file.rs | 24 +++++++++++------------- write_buffer/src/kafka/aggregator.rs | 8 +++----- write_buffer/src/kafka/mod.rs | 3 ++- write_buffer/src/mock.rs | 23 ++++++++++------------- write_summary/src/lib.rs | 13 +++++-------- 9 files changed, 53 insertions(+), 66 deletions(-) delete mode 100644 data_types/src/sequence.rs diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 22b92dd497..7bfbadcbeb 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -17,7 +17,6 @@ pub mod error; pub mod job; pub mod partition_metadata; pub mod router; -pub mod sequence; pub mod server_id; pub mod timestamp; pub mod write_buffer; diff --git a/data_types/src/sequence.rs b/data_types/src/sequence.rs deleted file mode 100644 index 2c8bbeac75..0000000000 --- a/data_types/src/sequence.rs +++ /dev/null @@ -1,16 +0,0 @@ -#[derive(Debug, Copy, Clone, Eq, PartialEq)] -pub struct Sequence { - /// The sequencer id (kafka partition id) - pub sequencer_id: u32, - /// The sequence number (kafka offset) - pub sequence_number: u64, -} - -impl Sequence { - pub fn new(sequencer_id: u32, sequence_number: u64) -> Self { - Self { - sequencer_id, - sequence_number, - } - } -} diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index 55ef129408..c3369c8624 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -27,10 +27,7 @@ use std::{ }; use uuid::Uuid; -pub use data_types::{ - sequence::Sequence, - timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}, -}; +pub use data_types::timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}; /// Unique ID for a `Namespace` #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)] @@ -1923,6 +1920,25 @@ impl TableSummary { } } +/// Kafka partition ID plus offset +#[derive(Debug, Copy, Clone, Eq, PartialEq)] +pub struct Sequence { + /// The sequencer id (kafka partition id) + pub sequencer_id: u32, + /// The sequence number (kafka offset) + pub sequence_number: u64, +} + +impl Sequence { + /// Create a new Sequence + pub fn new(sequencer_id: u32, sequence_number: u64) -> Self { + Self { + sequencer_id, + sequence_number, + } + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/dml/src/lib.rs b/dml/src/lib.rs index d818252ad4..2eb0d96fd4 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -11,11 +11,8 @@ clippy::clone_on_ref_ptr )] -use data_types::{ - router::{ShardConfig, ShardId}, - sequence::Sequence, -}; -use data_types2::{DeletePredicate, NonEmptyString, StatValues, Statistics}; +use data_types::router::{ShardConfig, ShardId}; +use data_types2::{DeletePredicate, NonEmptyString, Sequence, StatValues, Statistics}; use hashbrown::HashMap; use iox_time::Time; use mutable_batch::MutableBatch; diff --git a/write_buffer/src/file.rs b/write_buffer/src/file.rs index 5fb0db4d84..2b11af6c0a 100644 --- a/write_buffer/src/file.rs +++ b/write_buffer/src/file.rs @@ -108,6 +108,17 @@ //! [`rename(2)`]: https://man7.org/linux/man-pages/man2/rename.2.html //! [`symlink(2)`]: https://man7.org/linux/man-pages/man2/symlink.2.html //! [`unlink(2)`]: https://man7.org/linux/man-pages/man2/unlink.2.html +use crate::{ + codec::{ContentType, IoxHeaders}, + core::{WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting}, +}; +use async_trait::async_trait; +use data_types::write_buffer::WriteBufferCreationConfig; +use data_types2::Sequence; +use dml::{DmlMeta, DmlOperation}; +use futures::{stream::BoxStream, Stream, StreamExt}; +use iox_time::{Time, TimeProvider}; +use pin_project::pin_project; use std::{ collections::{BTreeMap, BTreeSet}, path::{Path, PathBuf}, @@ -118,23 +129,10 @@ use std::{ Arc, }, }; - -use crate::{ - codec::{ContentType, IoxHeaders}, - core::WriteBufferStreamHandler, -}; -use async_trait::async_trait; -use data_types::{sequence::Sequence, write_buffer::WriteBufferCreationConfig}; -use dml::{DmlMeta, DmlOperation}; -use futures::{stream::BoxStream, Stream, StreamExt}; -use iox_time::{Time, TimeProvider}; -use pin_project::pin_project; use tokio_util::sync::ReusableBoxFuture; use trace::TraceCollector; use uuid::Uuid; -use crate::core::{WriteBufferError, WriteBufferReading, WriteBufferWriting}; - /// Header used to declare the creation time of the message. pub const HEADER_TIME: &str = "last-modified"; diff --git a/write_buffer/src/kafka/aggregator.rs b/write_buffer/src/kafka/aggregator.rs index e8478dec63..795ecefb53 100644 --- a/write_buffer/src/kafka/aggregator.rs +++ b/write_buffer/src/kafka/aggregator.rs @@ -1,6 +1,5 @@ -use std::sync::Arc; - -use data_types::sequence::Sequence; +use crate::codec::{ContentType, IoxHeaders}; +use data_types2::Sequence; use dml::{DmlMeta, DmlOperation, DmlWrite}; use hashbrown::{hash_map::Entry, HashMap}; use iox_time::{Time, TimeProvider}; @@ -11,14 +10,13 @@ use rskafka::{ record::Record, }; use schema::selection::Selection; +use std::sync::Arc; use trace::{ ctx::SpanContext, span::{Span, SpanRecorder}, TraceCollector, }; -use crate::codec::{ContentType, IoxHeaders}; - /// Newtype wrapper for tags given back to the aggregator framework. /// /// We cannot just use a simple `usize` to get the offsets from the produced records because we can have writes for diff --git a/write_buffer/src/kafka/mod.rs b/write_buffer/src/kafka/mod.rs index 8387b0911f..f50a9c67cd 100644 --- a/write_buffer/src/kafka/mod.rs +++ b/write_buffer/src/kafka/mod.rs @@ -10,7 +10,8 @@ use crate::{ }, }; use async_trait::async_trait; -use data_types::{sequence::Sequence, write_buffer::WriteBufferCreationConfig}; +use data_types::write_buffer::WriteBufferCreationConfig; +use data_types2::Sequence; use dml::{DmlMeta, DmlOperation}; use futures::{stream::BoxStream, StreamExt}; use iox_time::{Time, TimeProvider}; diff --git a/write_buffer/src/mock.rs b/write_buffer/src/mock.rs index bcc3c7081e..1d6ec69f75 100644 --- a/write_buffer/src/mock.rs +++ b/write_buffer/src/mock.rs @@ -1,3 +1,13 @@ +use crate::core::{ + WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting, +}; +use async_trait::async_trait; +use data_types::write_buffer::WriteBufferCreationConfig; +use data_types2::Sequence; +use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; +use futures::{stream::BoxStream, StreamExt}; +use iox_time::TimeProvider; +use parking_lot::Mutex; use std::{ collections::{BTreeMap, BTreeSet}, num::NonZeroU32, @@ -5,19 +15,6 @@ use std::{ task::{Poll, Waker}, }; -use async_trait::async_trait; -use futures::{stream::BoxStream, StreamExt}; -use parking_lot::Mutex; - -use data_types::sequence::Sequence; -use data_types::write_buffer::WriteBufferCreationConfig; -use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; -use iox_time::TimeProvider; - -use crate::core::{ - WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting, -}; - #[derive(Debug, Default)] struct WriteResVec { /// The maximum sequence number in the entries diff --git a/write_summary/src/lib.rs b/write_summary/src/lib.rs index 25741964af..047074f2c8 100644 --- a/write_summary/src/lib.rs +++ b/write_summary/src/lib.rs @@ -1,14 +1,11 @@ -use std::collections::BTreeMap; - use data_types2::{KafkaPartition, SequenceNumber}; -use observability_deps::tracing::debug; - +use dml::DmlMeta; /// Protobuf to/from conversion use generated_types::influxdata::iox::write_summary::v1 as proto; - -use dml::DmlMeta; - +use observability_deps::tracing::debug; use snafu::{OptionExt, Snafu}; +use std::collections::BTreeMap; + mod progress; pub use progress::SequencerProgress; @@ -203,7 +200,7 @@ impl TryFrom for WriteSummary { #[cfg(test)] mod tests { use super::*; - use data_types::sequence::Sequence; + use data_types2::Sequence; #[test] fn empty() { From 2ef44f20241a1d829882a59b14dfd564c8e64847 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 10:41:42 -0400 Subject: [PATCH 14/31] fix: Move timestamp types to data_types2 --- Cargo.lock | 1 - compactor/src/query.rs | 6 +- data_types2/src/lib.rs | 177 ++++++++++++++++++++- dml/src/lib.rs | 2 +- ingester/src/query.rs | 3 +- querier/src/chunk/mod.rs | 3 +- querier/src/chunk/query_access.rs | 6 +- querier/src/ingester/mod.rs | 12 +- query_tests/Cargo.toml | 3 - query_tests/src/influxrpc/field_columns.rs | 8 +- query_tests/src/influxrpc/table_names.rs | 8 +- query_tests/src/influxrpc/tag_keys.rs | 8 +- 12 files changed, 196 insertions(+), 41 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3173fce4b3..185c34f342 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4363,7 +4363,6 @@ dependencies = [ "arrow_util", "async-trait", "backoff 0.1.0", - "data_types", "data_types2", "datafusion 0.1.0", "dml", diff --git a/compactor/src/query.rs b/compactor/src/query.rs index fe179c17d4..f770474a3a 100644 --- a/compactor/src/query.rs +++ b/compactor/src/query.rs @@ -1,11 +1,8 @@ //! Queryable Compactor Data -use std::sync::Arc; - -use data_types::timestamp::TimestampMinMax; use data_types2::{ ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, - Timestamp, Tombstone, + Timestamp, TimestampMinMax, Tombstone, }; use datafusion::physical_plan::SendableRecordBatchStream; use observability_deps::tracing::trace; @@ -17,6 +14,7 @@ use query::{ }; use schema::{merge::SchemaMerger, selection::Selection, sort::SortKey, Schema}; use snafu::{ResultExt, Snafu}; +use std::sync::Arc; #[derive(Debug, Snafu)] #[allow(missing_copy_implementations, missing_docs)] diff --git a/data_types2/src/lib.rs b/data_types2/src/lib.rs index c3369c8624..2e97d0ae52 100644 --- a/data_types2/src/lib.rs +++ b/data_types2/src/lib.rs @@ -27,8 +27,6 @@ use std::{ }; use uuid::Uuid; -pub use data_types::timestamp::{TimestampMinMax, TimestampRange, MAX_NANO_TIME, MIN_NANO_TIME}; - /// Unique ID for a `Namespace` #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)] #[sqlx(transparent)] @@ -1939,6 +1937,105 @@ impl Sequence { } } +/// minimum time that can be represented. +/// +/// 1677-09-21 00:12:43.145224194 +0000 UTC +/// +/// The two lowest minimum integers are used as sentinel values. The +/// minimum value needs to be used as a value lower than any other value for +/// comparisons and another separate value is needed to act as a sentinel +/// default value that is unusable by the user, but usable internally. +/// Because these two values need to be used for a special purpose, we do +/// not allow users to write points at these two times. +/// +/// Source: [influxdb](https://github.com/influxdata/influxdb/blob/540bb66e1381a48a6d1ede4fc3e49c75a7d9f4af/models/time.go#L12-L34) +pub const MIN_NANO_TIME: i64 = i64::MIN + 2; + +/// maximum time that can be represented. +/// +/// 2262-04-11 23:47:16.854775806 +0000 UTC +/// +/// The highest time represented by a nanosecond needs to be used for an +/// exclusive range in the shard group, so the maximum time needs to be one +/// less than the possible maximum number of nanoseconds representable by an +/// int64 so that we don't lose a point at that one time. +/// Source: [influxdb](https://github.com/influxdata/influxdb/blob/540bb66e1381a48a6d1ede4fc3e49c75a7d9f4af/models/time.go#L12-L34) +pub const MAX_NANO_TIME: i64 = i64::MAX - 1; + +/// Specifies a continuous range of nanosecond timestamps. Timestamp +/// predicates are so common and critical to performance of timeseries +/// databases in general, and IOx in particular, that they are handled +/// specially +/// +/// Timestamp ranges are defined such that a value `v` is within the +/// range iff: +/// +/// ```text +/// range.start <= v < range.end +/// ``` +#[derive(Clone, PartialEq, Eq, PartialOrd, Ord, Copy, Debug, Hash)] +pub struct TimestampRange { + /// Start defines the inclusive lower bound. Minimum value is [MIN_NANO_TIME] + start: i64, + /// End defines the inclusive upper bound. Maximum value is [MAX_NANO_TIME] + end: i64, +} + +impl TimestampRange { + /// Create a new TimestampRange. Clamps to MIN_NANO_TIME/MAX_NANO_TIME. + pub fn new(start: i64, end: i64) -> Self { + debug_assert!(end >= start); + let start = start.max(MIN_NANO_TIME); + let end = end.min(MAX_NANO_TIME); + Self { start, end } + } + + #[inline] + /// Returns true if this range contains the value v + pub fn contains(&self, v: i64) -> bool { + self.start <= v && v < self.end + } + + /// Return the timestamp range's end. + pub fn end(&self) -> i64 { + self.end + } + + /// Return the timestamp range's start. + pub fn start(&self) -> i64 { + self.start + } +} + +/// Specifies a min/max timestamp value. +/// +/// Note this differs subtlety (but critically) from a +/// `TimestampRange` as the minimum and maximum values are included +#[derive(Clone, Debug, Copy)] +pub struct TimestampMinMax { + /// The minimum timestamp value + pub min: i64, + /// the maximum timestamp value + pub max: i64, +} + +impl TimestampMinMax { + /// Create a new TimestampMinMax. Panics if min > max. + pub fn new(min: i64, max: i64) -> Self { + assert!(min <= max, "expected min ({}) <= max ({})", min, max); + Self { min, max } + } + + #[inline] + /// Returns true if any of the values between min / max + /// (inclusive) are contained within the specified timestamp range + pub fn overlaps(&self, range: TimestampRange) -> bool { + range.contains(self.min) + || range.contains(self.max) + || (self.min <= range.start && self.max >= range.end) + } +} + #[cfg(test)] mod tests { use super::*; @@ -2882,4 +2979,80 @@ mod tests { assert_eq!(stat.max, None); assert_eq!(stat.total_count, 1); } + + #[test] + fn test_timestamp_nano_min_max() { + let cases = vec![ + ( + "MIN/MAX Nanos", + TimestampRange::new(MIN_NANO_TIME, MAX_NANO_TIME), + ), + ("MIN/MAX i64", TimestampRange::new(i64::MIN, i64::MAX)), + ]; + + for (name, range) in cases { + println!("case: {}", name); + assert!(!range.contains(i64::MIN)); + assert!(range.contains(MIN_NANO_TIME)); + assert!(range.contains(MIN_NANO_TIME + 1)); + assert!(range.contains(MAX_NANO_TIME - 1)); + assert!(!range.contains(MAX_NANO_TIME)); + assert!(!range.contains(i64::MAX)); + } + } + + #[test] + fn test_timestamp_i64_min_max_offset() { + let range = TimestampRange::new(MIN_NANO_TIME + 1, MAX_NANO_TIME - 1); + + assert!(!range.contains(i64::MIN)); + assert!(!range.contains(MIN_NANO_TIME)); + assert!(range.contains(MIN_NANO_TIME + 1)); + assert!(range.contains(MAX_NANO_TIME - 2)); + assert!(!range.contains(MAX_NANO_TIME - 1)); + assert!(!range.contains(MAX_NANO_TIME)); + assert!(!range.contains(i64::MAX)); + } + + #[test] + fn test_timestamp_range_contains() { + let range = TimestampRange::new(100, 200); + assert!(!range.contains(99)); + assert!(range.contains(100)); + assert!(range.contains(101)); + assert!(range.contains(199)); + assert!(!range.contains(200)); + assert!(!range.contains(201)); + } + + #[test] + fn test_timestamp_range_overlaps() { + let range = TimestampRange::new(100, 200); + assert!(!TimestampMinMax::new(0, 99).overlaps(range)); + assert!(TimestampMinMax::new(0, 100).overlaps(range)); + assert!(TimestampMinMax::new(0, 101).overlaps(range)); + + assert!(TimestampMinMax::new(0, 200).overlaps(range)); + assert!(TimestampMinMax::new(0, 201).overlaps(range)); + assert!(TimestampMinMax::new(0, 300).overlaps(range)); + + assert!(TimestampMinMax::new(100, 101).overlaps(range)); + assert!(TimestampMinMax::new(100, 200).overlaps(range)); + assert!(TimestampMinMax::new(100, 201).overlaps(range)); + + assert!(TimestampMinMax::new(101, 101).overlaps(range)); + assert!(TimestampMinMax::new(101, 200).overlaps(range)); + assert!(TimestampMinMax::new(101, 201).overlaps(range)); + + assert!(!TimestampMinMax::new(200, 200).overlaps(range)); + assert!(!TimestampMinMax::new(200, 201).overlaps(range)); + + assert!(!TimestampMinMax::new(201, 300).overlaps(range)); + } + + #[test] + #[should_panic(expected = "expected min (2) <= max (1)")] + fn test_timestamp_min_max_invalid() { + TimestampMinMax::new(2, 1); + } } diff --git a/dml/src/lib.rs b/dml/src/lib.rs index 2eb0d96fd4..c7750fdfcf 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -487,8 +487,8 @@ mod tests { use data_types::{ consistent_hasher::ConsistentHasher, router::{HashRing, Matcher, MatcherToShard}, - timestamp::TimestampRange, }; + use data_types2::TimestampRange; use mutable_batch_lp::lines_to_batches; use regex::Regex; diff --git a/ingester/src/query.rs b/ingester/src/query.rs index 6d3a3b5ff4..e0de111ae9 100644 --- a/ingester/src/query.rs +++ b/ingester/src/query.rs @@ -3,10 +3,9 @@ use crate::data::{QueryableBatch, SnapshotBatch}; use arrow::record_batch::RecordBatch; use arrow_util::util::merge_record_batches; -use data_types::timestamp::TimestampMinMax; use data_types2::{ ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, - Tombstone, + TimestampMinMax, Tombstone, }; use datafusion::{ logical_plan::ExprRewritable, diff --git a/querier/src/chunk/mod.rs b/querier/src/chunk/mod.rs index f649ff7cce..cd502ab42d 100644 --- a/querier/src/chunk/mod.rs +++ b/querier/src/chunk/mod.rs @@ -2,10 +2,9 @@ use crate::cache::CatalogCache; use arrow::record_batch::RecordBatch; -use data_types::timestamp::TimestampMinMax; use data_types2::{ ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, ParquetFile, ParquetFileId, - ParquetFileWithMetadata, PartitionId, SequenceNumber, SequencerId, + ParquetFileWithMetadata, PartitionId, SequenceNumber, SequencerId, TimestampMinMax, }; use futures::StreamExt; use iox_catalog::interface::Catalog; diff --git a/querier/src/chunk/query_access.rs b/querier/src/chunk/query_access.rs index 8d6d17da0a..9596ad230f 100644 --- a/querier/src/chunk/query_access.rs +++ b/querier/src/chunk/query_access.rs @@ -1,5 +1,4 @@ -use std::sync::Arc; - +use crate::chunk::{ChunkStorage, QuerierChunk}; use data_types2::{ ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary, TimestampMinMax, }; @@ -8,8 +7,7 @@ use predicate::PredicateMatch; use query::{QueryChunk, QueryChunkError, QueryChunkMeta}; use schema::{sort::SortKey, Schema}; use snafu::{ResultExt, Snafu}; - -use crate::chunk::{ChunkStorage, QuerierChunk}; +use std::sync::Arc; #[derive(Debug, Snafu)] pub enum Error { diff --git a/querier/src/ingester/mod.rs b/querier/src/ingester/mod.rs index 529d1a6105..9f814b53e5 100644 --- a/querier/src/ingester/mod.rs +++ b/querier/src/ingester/mod.rs @@ -1,10 +1,13 @@ +use self::{ + flight_client::{Error as FlightClientError, FlightClient, FlightClientImpl, FlightError}, + test_util::MockIngesterConnection, +}; use crate::cache::CatalogCache; use arrow::{datatypes::DataType, error::ArrowError, record_batch::RecordBatch}; use async_trait::async_trait; -use data_types::timestamp::TimestampMinMax; use data_types2::{ ChunkAddr, ChunkId, ChunkOrder, ColumnSummary, InfluxDbType, PartitionId, SequenceNumber, - SequencerId, StatValues, Statistics, TableSummary, + SequencerId, StatValues, Statistics, TableSummary, TimestampMinMax, }; use datafusion_util::MemoryStream; use futures::{stream::FuturesUnordered, TryStreamExt}; @@ -20,11 +23,6 @@ use schema::{selection::Selection, sort::SortKey, InfluxColumnType, InfluxFieldT use snafu::{ensure, OptionExt, ResultExt, Snafu}; use std::{any::Any, collections::HashMap, sync::Arc}; -use self::{ - flight_client::{Error as FlightClientError, FlightClient, FlightClientImpl, FlightError}, - test_util::MockIngesterConnection, -}; - pub(crate) mod flight_client; pub(crate) mod test_util; diff --git a/query_tests/Cargo.toml b/query_tests/Cargo.toml index 56397021ab..9850de37b0 100644 --- a/query_tests/Cargo.toml +++ b/query_tests/Cargo.toml @@ -5,13 +5,10 @@ authors = ["Andrew Lamb "] edition = "2021" description = "Tests of the query engine against different database configurations" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - [dependencies] arrow = { version = "13", features = ["prettyprint"] } async-trait = "0.1" backoff = { path = "../backoff" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } dml = { path = "../dml" } diff --git a/query_tests/src/influxrpc/field_columns.rs b/query_tests/src/influxrpc/field_columns.rs index 6ae01e351c..237d94875c 100644 --- a/query_tests/src/influxrpc/field_columns.rs +++ b/query_tests/src/influxrpc/field_columns.rs @@ -1,15 +1,13 @@ +use crate::scenarios::*; use arrow::datatypes::DataType; -use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME}; +use data_types2::{MAX_NANO_TIME, MIN_NANO_TIME}; use datafusion::logical_plan::{col, lit}; -use predicate::rpc_predicate::InfluxRpcPredicate; -use predicate::PredicateBuilder; +use predicate::{rpc_predicate::InfluxRpcPredicate, PredicateBuilder}; use query::{ exec::fieldlist::{Field, FieldList}, frontend::influxrpc::InfluxRpcPlanner, }; -use crate::scenarios::*; - /// Creates and loads several database scenarios using the db_setup /// function. /// diff --git a/query_tests/src/influxrpc/table_names.rs b/query_tests/src/influxrpc/table_names.rs index 63f10232c0..03b5dedd61 100644 --- a/query_tests/src/influxrpc/table_names.rs +++ b/query_tests/src/influxrpc/table_names.rs @@ -1,15 +1,13 @@ //! Tests for the Influx gRPC queries -use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME}; +use crate::scenarios::*; +use data_types2::{MAX_NANO_TIME, MIN_NANO_TIME}; use datafusion::logical_plan::{col, lit}; -use predicate::rpc_predicate::InfluxRpcPredicate; -use predicate::PredicateBuilder; +use predicate::{rpc_predicate::InfluxRpcPredicate, PredicateBuilder}; use query::{ exec::stringset::{IntoStringSet, StringSetRef}, frontend::influxrpc::InfluxRpcPlanner, }; -use crate::scenarios::*; - /// runs table_names(predicate) and compares it to the expected /// output async fn run_table_names_test_case( diff --git a/query_tests/src/influxrpc/tag_keys.rs b/query_tests/src/influxrpc/tag_keys.rs index 68e11c0daa..0288063ab6 100644 --- a/query_tests/src/influxrpc/tag_keys.rs +++ b/query_tests/src/influxrpc/tag_keys.rs @@ -1,14 +1,12 @@ -use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME}; +use crate::scenarios::*; +use data_types2::{MAX_NANO_TIME, MIN_NANO_TIME}; use datafusion::logical_plan::{col, lit}; -use predicate::rpc_predicate::InfluxRpcPredicate; -use predicate::PredicateBuilder; +use predicate::{rpc_predicate::InfluxRpcPredicate, PredicateBuilder}; use query::{ exec::stringset::{IntoStringSet, StringSetRef}, frontend::influxrpc::InfluxRpcPlanner, }; -use crate::scenarios::*; - /// Creates and loads several database scenarios using the db_setup /// function. /// From 94be7407ba90ed6fc48ac745fb6247de3a9aa751 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 11:05:53 -0400 Subject: [PATCH 15/31] refactor: Move BooleanFlag to the only place it's used --- data_types/src/boolean_flag.rs | 46 ----------------------------- data_types/src/lib.rs | 1 - query/src/exec/query_tracing.rs | 52 +++++++++++++++++++++++++++------ 3 files changed, 43 insertions(+), 56 deletions(-) delete mode 100644 data_types/src/boolean_flag.rs diff --git a/data_types/src/boolean_flag.rs b/data_types/src/boolean_flag.rs deleted file mode 100644 index 22e5c29683..0000000000 --- a/data_types/src/boolean_flag.rs +++ /dev/null @@ -1,46 +0,0 @@ -/// Boolean flag that works with environment variables. -#[derive(Debug, Clone, Copy)] -pub enum BooleanFlag { - True, - False, -} - -impl std::str::FromStr for BooleanFlag { - type Err = String; - - fn from_str(s: &str) -> Result { - match s.to_ascii_lowercase().as_str() { - "yes" | "y" | "true" | "t" | "1" => Ok(Self::True), - "no" | "n" | "false" | "f" | "0" => Ok(Self::False), - _ => Err(format!( - "Invalid boolean flag '{}'. Valid options: yes, no, y, n, true, false, t, f, 1, 0", - s - )), - } - } -} - -impl From for bool { - fn from(yes_no: BooleanFlag) -> Self { - matches!(yes_no, BooleanFlag::True) - } -} - -#[cfg(test)] -mod tests { - use std::str::FromStr; - - use super::*; - - #[test] - fn test_parsing() { - assert!(bool::from(BooleanFlag::from_str("yes").unwrap())); - assert!(bool::from(BooleanFlag::from_str("Yes").unwrap())); - assert!(bool::from(BooleanFlag::from_str("YES").unwrap())); - - assert!(!bool::from(BooleanFlag::from_str("No").unwrap())); - assert!(!bool::from(BooleanFlag::from_str("FaLse").unwrap())); - - BooleanFlag::from_str("foo").unwrap_err(); - } -} diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 7bfbadcbeb..74d600006b 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -10,7 +10,6 @@ clippy::clone_on_ref_ptr )] -pub mod boolean_flag; pub mod chunk_metadata; pub mod consistent_hasher; pub mod error; diff --git a/query/src/exec/query_tracing.rs b/query/src/exec/query_tracing.rs index f124ddefe4..7d53e80186 100644 --- a/query/src/exec/query_tracing.rs +++ b/query/src/exec/query_tracing.rs @@ -1,11 +1,8 @@ //! This module contains the code to map DataFusion metrics to `Span`s //! for use in distributed tracing (e.g. Jaeger) -use std::{fmt, sync::Arc}; - use arrow::record_batch::RecordBatch; use chrono::{DateTime, Utc}; -use data_types::boolean_flag::BooleanFlag; use datafusion::physical_plan::{ metrics::{MetricValue, MetricsSet}, DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, @@ -13,6 +10,7 @@ use datafusion::physical_plan::{ use futures::StreamExt; use hashbrown::HashMap; use observability_deps::tracing::debug; +use std::{fmt, sync::Arc}; use trace::span::{Span, SpanRecorder}; const PER_PARTITION_TRACING_ENABLE_ENV: &str = "INFLUXDB_IOX_PER_PARTITION_TRACING"; @@ -310,8 +308,37 @@ fn get_timestamps(metrics: &MetricsSet) -> (Option>, Option Result { + match s.to_ascii_lowercase().as_str() { + "yes" | "y" | "true" | "t" | "1" => Ok(Self::True), + "no" | "n" | "false" | "f" | "0" => Ok(Self::False), + _ => Err(format!( + "Invalid boolean flag '{}'. Valid options: yes, no, y, n, true, false, t, f, 1, 0", + s + )), + } + } +} + +impl From for bool { + fn from(yes_no: BooleanFlag) -> Self { + matches!(yes_no, BooleanFlag::True) + } +} + #[cfg(test)] mod tests { + use super::*; use chrono::TimeZone; use datafusion::{ execution::context::TaskContext, @@ -321,14 +348,9 @@ mod tests { Metric, }, }; - - use std::collections::BTreeMap; - use std::{sync::Arc, time::Duration}; - + use std::{collections::BTreeMap, str::FromStr, sync::Arc, time::Duration}; use trace::{ctx::SpanContext, span::MetaValue, RingBufferTraceCollector}; - use super::*; - #[test] fn name_truncation() { let name = "Foo: expr nonsense"; @@ -667,4 +689,16 @@ mod tests { write!(f, "TestExec - {}", self.name) } } + + #[test] + fn test_parsing() { + assert!(bool::from(BooleanFlag::from_str("yes").unwrap())); + assert!(bool::from(BooleanFlag::from_str("Yes").unwrap())); + assert!(bool::from(BooleanFlag::from_str("YES").unwrap())); + + assert!(!bool::from(BooleanFlag::from_str("No").unwrap())); + assert!(!bool::from(BooleanFlag::from_str("FaLse").unwrap())); + + BooleanFlag::from_str("foo").unwrap_err(); + } } From b76c1e1ad61a85ef7edd458e1009b92b35109dc3 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 11:16:19 -0400 Subject: [PATCH 16/31] fix: Remove now-unused DML sharding and related types --- data_types/src/lib.rs | 1 - data_types/src/router.rs | 141 ------------------ dml/src/lib.rs | 302 --------------------------------------- 3 files changed, 444 deletions(-) delete mode 100644 data_types/src/router.rs diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 74d600006b..7da948f108 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -15,7 +15,6 @@ pub mod consistent_hasher; pub mod error; pub mod job; pub mod partition_metadata; -pub mod router; pub mod server_id; pub mod timestamp; pub mod write_buffer; diff --git a/data_types/src/router.rs b/data_types/src/router.rs deleted file mode 100644 index 5b1ff68a27..0000000000 --- a/data_types/src/router.rs +++ /dev/null @@ -1,141 +0,0 @@ -use std::collections::BTreeMap; - -use regex::Regex; - -use crate::{ - consistent_hasher::ConsistentHasher, server_id::ServerId, write_buffer::WriteBufferConnection, -}; - -#[derive(Debug, Eq, PartialEq, Hash, PartialOrd, Ord, Clone, Copy)] -pub struct ShardId(u32); - -impl ShardId { - pub fn new(id: u32) -> Self { - Self(id) - } - - pub fn get(&self) -> u32 { - self.0 - } -} - -impl std::fmt::Display for ShardId { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "ShardId({})", self.get()) - } -} - -/// ShardConfig defines rules for assigning a line/row to an individual -/// host or a group of hosts. A shard -/// is a logical concept, but the usage is meant to split data into -/// mutually exclusive areas. The rough order of organization is: -/// database -> shard -> partition -> chunk. For example, you could shard -/// based on table name and assign to 1 of 10 shards. Within each -/// shard you would have partitions, which would likely be based off time. -/// This makes it possible to horizontally scale out writes. -#[derive(Debug, Eq, PartialEq, Clone, Default)] -pub struct ShardConfig { - /// Each matcher, if any, is evaluated in order. - /// If there is a match, the route will be evaluated to - /// the given targets, otherwise the hash ring will be evaluated. - /// This is useful for overriding the hashring function on some hot spot. For - /// example, if you use the table name as the input to the hash function - /// and your ring has 4 slots. If two tables that are very hot get - /// assigned to the same slot you can override that by putting in a - /// specific matcher to pull that table over to a different node. - pub specific_targets: Vec, - - /// An optional default hasher which will route to one in a collection of - /// nodes. - pub hash_ring: Option, -} - -/// Maps a matcher with specific shard. If the line/row matches -/// it should be sent to the group. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct MatcherToShard { - pub matcher: Matcher, - pub shard: ShardId, -} - -/// HashRing is a rule for creating a hash key for a row and mapping that to -/// an individual node on a ring. -#[derive(Debug, Eq, PartialEq, Clone, Default)] -pub struct HashRing { - /// ring of shard ids - pub shards: ConsistentHasher, -} - -/// A matcher is used to match routing rules or subscriptions on a row-by-row -/// (or line) basis. -#[derive(Debug, Clone, Default)] -pub struct Matcher { - /// if provided, match if the table name matches against the regex - pub table_name_regex: Option, -} - -impl PartialEq for Matcher { - fn eq(&self, other: &Self) -> bool { - // this is kind of janky, but it's only used during tests and should get the job - // done - format!("{:?}", self.table_name_regex) == format!("{:?}", other.table_name_regex) - } -} -impl Eq for Matcher {} - -/// Sinks for query requests. -/// -/// Queries are sent to one of these sinks and the resulting data is received from it. -/// -/// Note that the query results are flowing into the opposite direction (aka a query sink is a result source). -#[derive(Debug, Eq, PartialEq, Clone, Default)] -pub struct QuerySinks { - pub grpc_remotes: Vec, -} - -#[derive(Debug, Eq, PartialEq, Clone)] -pub enum WriteSinkVariant { - /// gRPC-based remote, addressed by its server ID. - GrpcRemote(ServerId), - - /// Write buffer connection. - WriteBuffer(WriteBufferConnection), -} - -/// Sink of write requests aka new data. -/// -/// Data is sent to this sink and a status is received from it. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct WriteSink { - pub sink: WriteSinkVariant, - - /// If set, errors during writing to this sink are ignored and do NOT lead to an overall failure. - pub ignore_errors: bool, -} - -/// Set of write sinks. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct WriteSinkSet { - /// Sinks within the set. - pub sinks: Vec, -} - -/// Router for writes and queries. -#[derive(Debug, Eq, PartialEq, Clone)] -pub struct Router { - /// Router name. - /// - /// The name corresponds to the database name on the database node. - /// - /// The router name is unique for this router node. - pub name: String, - - /// Write sharder. - pub write_sharder: ShardConfig, - - /// Sinks for write requests. - pub write_sinks: BTreeMap, - - /// Sinks for query requests. - pub query_sinks: QuerySinks, -} diff --git a/dml/src/lib.rs b/dml/src/lib.rs index c7750fdfcf..dcd4239a9b 100644 --- a/dml/src/lib.rs +++ b/dml/src/lib.rs @@ -11,12 +11,10 @@ clippy::clone_on_ref_ptr )] -use data_types::router::{ShardConfig, ShardId}; use data_types2::{DeletePredicate, NonEmptyString, Sequence, StatValues, Statistics}; use hashbrown::HashMap; use iox_time::Time; use mutable_batch::MutableBatch; -use std::collections::{BTreeMap, HashSet}; use trace::ctx::SpanContext; /// Metadata information about a DML operation @@ -126,22 +124,6 @@ impl DmlOperation { } } - /// Shards this [`DmlOperation`] - pub fn shard(self, config: &ShardConfig) -> BTreeMap { - match self { - DmlOperation::Write(write) => write - .shard(config) - .into_iter() - .map(|(shard, write)| (shard, Self::Write(write))) - .collect(), - DmlOperation::Delete(delete) => delete - .shard(config) - .into_iter() - .map(|(shard, delete)| (shard, Self::Delete(delete))) - .collect(), - } - } - /// Return the approximate memory size of the operation, in bytes. /// /// This includes `Self`. @@ -278,31 +260,6 @@ impl DmlWrite { self.max_timestamp } - /// Shards this [`DmlWrite`] - pub fn shard(self, config: &ShardConfig) -> BTreeMap { - let mut batches: HashMap> = HashMap::new(); - - for (table, batch) in self.tables { - if let Some(shard_id) = shard_table(&table, config) { - assert!(batches - .entry(shard_id) - .or_default() - .insert(table, batch.clone()) - .is_none()); - } - } - - batches - .into_iter() - .map(|(shard_id, tables)| { - ( - shard_id, - Self::new(&self.namespace, tables, self.meta.clone()), - ) - }) - .collect() - } - /// Return the approximate memory size of the write, in bytes. /// /// This includes `Self`. @@ -368,32 +325,6 @@ impl DmlDelete { self.meta = meta } - /// Shards this [`DmlDelete`] - pub fn shard(self, config: &ShardConfig) -> BTreeMap { - if let Some(table) = self.table_name() { - if let Some(shard_id) = shard_table(table, config) { - BTreeMap::from([(shard_id, self)]) - } else { - BTreeMap::default() - } - } else { - let shards: HashSet = - config - .specific_targets - .iter() - .map(|matcher2shard| matcher2shard.shard) - .chain(config.hash_ring.iter().flat_map(|hashring| { - Vec::::from(hashring.shards.clone()).into_iter() - })) - .collect(); - - shards - .into_iter() - .map(|shard| (shard, self.clone())) - .collect() - } - } - /// Return the approximate memory size of the delete, in bytes. /// /// This includes `Self`. @@ -409,25 +340,6 @@ impl DmlDelete { } } -/// Shard only based on table name -fn shard_table(table: &str, config: &ShardConfig) -> Option { - for matcher2shard in &config.specific_targets { - if let Some(regex) = &matcher2shard.matcher.table_name_regex { - if regex.is_match(table) { - return Some(matcher2shard.shard); - } - } - } - - if let Some(hash_ring) = &config.hash_ring { - if let Some(id) = hash_ring.shards.find(table) { - return Some(id); - } - } - - None -} - /// Test utilities pub mod test_util { use arrow_util::display::pretty_format_batches; @@ -479,217 +391,3 @@ pub mod test_util { } } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::test_util::assert_writes_eq; - use data_types::{ - consistent_hasher::ConsistentHasher, - router::{HashRing, Matcher, MatcherToShard}, - }; - use data_types2::TimestampRange; - use mutable_batch_lp::lines_to_batches; - use regex::Regex; - - #[test] - fn test_write_sharding() { - let config = ShardConfig { - specific_targets: vec![ - MatcherToShard { - matcher: Matcher { - table_name_regex: None, - }, - shard: ShardId::new(1), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("some_foo").unwrap()), - }, - shard: ShardId::new(2), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("other").unwrap()), - }, - shard: ShardId::new(3), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("some_.*").unwrap()), - }, - shard: ShardId::new(4), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("baz").unwrap()), - }, - shard: ShardId::new(2), - }, - ], - hash_ring: Some(HashRing { - shards: ConsistentHasher::new(&[ - ShardId::new(11), - ShardId::new(12), - ShardId::new(13), - ]), - }), - }; - - let meta = DmlMeta::unsequenced(None); - let write = db_write( - &[ - "some_foo x=1 10", - "some_foo x=2 20", - "some_bar y=3 30", - "other z=4 40", - "rnd1 r=5 50", - "rnd2 r=6 60", - "rnd3 r=7 70", - "baz b=8 80", - ], - &meta, - ); - - let actual = write.shard(&config); - let expected = BTreeMap::from([ - ( - ShardId::new(2), - db_write(&["some_foo x=1 10", "some_foo x=2 20", "baz b=8 80"], &meta), - ), - (ShardId::new(3), db_write(&["other z=4 40"], &meta)), - (ShardId::new(4), db_write(&["some_bar y=3 30"], &meta)), - (ShardId::new(11), db_write(&["rnd1 r=5 50"], &meta)), - (ShardId::new(12), db_write(&["rnd3 r=7 70"], &meta)), - (ShardId::new(13), db_write(&["rnd2 r=6 60"], &meta)), - ]); - - let actual_shard_ids: Vec<_> = actual.keys().cloned().collect(); - let expected_shard_ids: Vec<_> = expected.keys().cloned().collect(); - assert_eq!(actual_shard_ids, expected_shard_ids); - - for (actual_write, expected_write) in actual.values().zip(expected.values()) { - assert_writes_eq(actual_write, expected_write); - } - } - - #[test] - fn test_write_no_match() { - let config = ShardConfig::default(); - - let meta = DmlMeta::default(); - let write = db_write(&["foo x=1 10"], &meta); - - let actual = write.shard(&config); - assert!(actual.is_empty()); - } - - #[test] - fn test_delete_sharding() { - let config = ShardConfig { - specific_targets: vec![ - MatcherToShard { - matcher: Matcher { - table_name_regex: None, - }, - shard: ShardId::new(1), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("some_foo").unwrap()), - }, - shard: ShardId::new(2), - }, - MatcherToShard { - matcher: Matcher { - table_name_regex: Some(Regex::new("some_.*").unwrap()), - }, - shard: ShardId::new(3), - }, - ], - hash_ring: Some(HashRing { - shards: ConsistentHasher::new(&[ - ShardId::new(11), - ShardId::new(12), - ShardId::new(13), - ]), - }), - }; - - // Deletes w/o table name go to all shards - let meta = DmlMeta::unsequenced(None); - let delete = DmlDelete::new( - "test_db", - DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![], - }, - None, - meta, - ); - - let actual = delete.clone().shard(&config); - let expected = BTreeMap::from([ - (ShardId::new(1), delete.clone()), - (ShardId::new(2), delete.clone()), - (ShardId::new(3), delete.clone()), - (ShardId::new(11), delete.clone()), - (ShardId::new(12), delete.clone()), - (ShardId::new(13), delete), - ]); - assert_sharded_deletes_eq(&actual, &expected); - - // Deletes are matched by table name regex - let meta = DmlMeta::unsequenced(None); - let delete = DmlDelete::new( - "test_db", - DeletePredicate { - range: TimestampRange::new(3, 4), - exprs: vec![], - }, - Some(NonEmptyString::new("some_foo").unwrap()), - meta, - ); - - let actual = delete.clone().shard(&config); - let expected = BTreeMap::from([(ShardId::new(2), delete)]); - assert_sharded_deletes_eq(&actual, &expected); - - // Deletes can be matched by hash-ring - let meta = DmlMeta::unsequenced(None); - let delete = DmlDelete::new( - "test_db", - DeletePredicate { - range: TimestampRange::new(5, 6), - exprs: vec![], - }, - Some(NonEmptyString::new("bar").unwrap()), - meta, - ); - - let actual = delete.clone().shard(&config); - let expected = BTreeMap::from([(ShardId::new(13), delete)]); - assert_sharded_deletes_eq(&actual, &expected); - } - - fn db_write(lines: &[&str], meta: &DmlMeta) -> DmlWrite { - DmlWrite::new( - "test_db", - lines_to_batches(&lines.join("\n"), 0).unwrap(), - meta.clone(), - ) - } - - fn assert_sharded_deletes_eq( - actual: &BTreeMap, - expected: &BTreeMap, - ) { - let actual_shard_ids: Vec<_> = actual.keys().cloned().collect(); - let expected_shard_ids: Vec<_> = expected.keys().cloned().collect(); - assert_eq!(actual_shard_ids, expected_shard_ids); - - for (actual_delete, expected_delete) in actual.values().zip(expected.values()) { - assert_eq!(actual_delete, expected_delete); - } - } -} From fb8f8d22c03ff8c9655e27cd1cafadc269fd26f5 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 11:20:28 -0400 Subject: [PATCH 17/31] fix: Remove now-unused ServerId. Fixes #4451 --- clap_blocks/src/lib.rs | 1 - clap_blocks/src/run_config.rs | 19 +----- clap_blocks/src/server_id.rs | 14 ----- data_types/src/lib.rs | 1 - data_types/src/server_id.rs | 106 ---------------------------------- generated_types/src/google.rs | 12 ---- 6 files changed, 1 insertion(+), 152 deletions(-) delete mode 100644 clap_blocks/src/server_id.rs delete mode 100644 data_types/src/server_id.rs diff --git a/clap_blocks/src/lib.rs b/clap_blocks/src/lib.rs index 22d75fbc29..a9db994261 100644 --- a/clap_blocks/src/lib.rs +++ b/clap_blocks/src/lib.rs @@ -7,6 +7,5 @@ pub mod ingester; pub mod object_store; pub mod querier; pub mod run_config; -pub mod server_id; pub mod socket_addr; pub mod write_buffer; diff --git a/clap_blocks/src/run_config.rs b/clap_blocks/src/run_config.rs index 799a3a2429..93685ae621 100644 --- a/clap_blocks/src/run_config.rs +++ b/clap_blocks/src/run_config.rs @@ -1,7 +1,7 @@ use trace_exporters::TracingConfig; use trogging::cli::LoggingConfig; -use crate::{object_store::ObjectStoreConfig, server_id::ServerIdConfig, socket_addr::SocketAddr}; +use crate::{object_store::ObjectStoreConfig, socket_addr::SocketAddr}; /// The default bind address for the HTTP API. pub const DEFAULT_API_BIND_ADDR: &str = "127.0.0.1:8080"; @@ -20,10 +20,6 @@ pub struct RunConfig { #[clap(flatten)] pub(crate) tracing_config: TracingConfig, - /// object store config - #[clap(flatten)] - pub(crate) server_id_config: ServerIdConfig, - /// The address on which IOx will serve HTTP API requests. #[clap( long = "--api-bind", @@ -69,21 +65,11 @@ impl RunConfig { &mut self.tracing_config } - /// Get a reference to the run config's server id config. - pub fn server_id_config(&self) -> &ServerIdConfig { - &self.server_id_config - } - /// Get a reference to the run config's logging config. pub fn logging_config(&self) -> &LoggingConfig { &self.logging_config } - /// Get a mutable reference to the run config's server id config. - pub fn server_id_config_mut(&mut self) -> &mut ServerIdConfig { - &mut self.server_id_config - } - /// set the http bind address pub fn with_http_bind_address(mut self, http_bind_address: SocketAddr) -> Self { self.http_bind_address = http_bind_address; @@ -108,9 +94,6 @@ impl RunConfig { Self { logging_config, tracing_config, - // TODO: server_id isn't used in NG; this field should be removed when OG is removed - // https://github.com/influxdata/influxdb_iox/issues/4451 - server_id_config: ServerIdConfig { server_id: None }, http_bind_address, grpc_bind_address, max_http_request_size, diff --git a/clap_blocks/src/server_id.rs b/clap_blocks/src/server_id.rs deleted file mode 100644 index 042b303bf7..0000000000 --- a/clap_blocks/src/server_id.rs +++ /dev/null @@ -1,14 +0,0 @@ -use data_types::server_id::ServerId; - -/// CLI config for server ID. -#[derive(Debug, Clone, clap::Parser)] -pub struct ServerIdConfig { - /// The identifier for the server. - /// - /// Used for writing to object storage and as an identifier that is added to - /// replicated writes, write buffer segments, and Chunks. Must be unique in - /// a group of connected or semi-connected IOx servers. Must be a nonzero - /// number that can be represented by a 32-bit unsigned integer. - #[clap(long = "--server-id", env = "INFLUXDB_IOX_ID")] - pub server_id: Option, -} diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 7da948f108..29f1a56182 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -15,7 +15,6 @@ pub mod consistent_hasher; pub mod error; pub mod job; pub mod partition_metadata; -pub mod server_id; pub mod timestamp; pub mod write_buffer; pub mod write_summary; diff --git a/data_types/src/server_id.rs b/data_types/src/server_id.rs deleted file mode 100644 index 2a8606e6b0..0000000000 --- a/data_types/src/server_id.rs +++ /dev/null @@ -1,106 +0,0 @@ -use snafu::{OptionExt, ResultExt, Snafu}; -use std::{ - convert::TryFrom, - fmt, - num::{NonZeroU32, ParseIntError}, - str::FromStr, -}; - -#[derive(Debug, Copy, Clone, Eq, PartialEq, Ord, PartialOrd)] -pub struct ServerId(NonZeroU32); - -impl ServerId { - pub fn new(id: NonZeroU32) -> Self { - Self(id) - } - - pub fn get(&self) -> NonZeroU32 { - self.0 - } - - pub fn get_u32(&self) -> u32 { - self.0.get() - } -} - -impl FromStr for ServerId { - type Err = Error; - - fn from_str(value: &str) -> Result { - let value: u32 = value.parse().context(UnableToParseSnafu { value })?; - Self::try_from(value) - } -} - -impl TryFrom for ServerId { - type Error = Error; - - fn try_from(value: u32) -> Result { - NonZeroU32::new(value) - .map(Self) - .context(ValueMayNotBeZeroSnafu) - .map_err(Into::into) - } -} - -impl fmt::Display for ServerId { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.0.fmt(f) - } -} - -#[derive(Debug, Snafu)] -pub struct Error(InnerError); - -#[derive(Debug, Snafu)] -enum InnerError { - #[snafu(display("The server ID may not be zero"))] - ValueMayNotBeZero, - - #[snafu(display("Could not parse {} as a non-zero 32-bit unsigned number", value))] - UnableToParse { - source: ParseIntError, - value: String, - }, -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn cannot_be_zero() { - assert!(matches!( - ServerId::try_from(0), - Err(Error(InnerError::ValueMayNotBeZero)) - )); - } - - #[test] - fn can_be_nonzero() { - let value = 2; - let server_id = ServerId::try_from(value).unwrap(); - assert_eq!(server_id.get_u32(), value); - } - - #[test] - fn can_be_parsed_from_a_string() { - assert!(matches!( - "0".parse::(), - Err(Error(InnerError::ValueMayNotBeZero)), - )); - assert!(matches!( - "moo".parse::(), - Err(Error(InnerError::UnableToParse { source: _, value })) if value == "moo", - )); - - let server_id = "1337".parse::().unwrap(); - assert_eq!(server_id.get_u32(), 1337); - } - - #[test] - fn can_be_displayed() { - let server_id = ServerId::try_from(42).unwrap(); - assert_eq!("42", format!("{}", server_id)); - } -} diff --git a/generated_types/src/google.rs b/generated_types/src/google.rs index 521f23b75b..6c92b29da2 100644 --- a/generated_types/src/google.rs +++ b/generated_types/src/google.rs @@ -233,7 +233,6 @@ pub enum ResourceType { DatabaseUuid, Job, Router, - ServerId, Unknown(String), } @@ -247,7 +246,6 @@ impl ResourceType { Self::Chunk => "chunk", Self::Job => "job", Self::Router => "router", - Self::ServerId => "server_id", Self::Unknown(unknown) => unknown, } } @@ -263,7 +261,6 @@ impl From for ResourceType { "chunk" => Self::Chunk, "job" => Self::Job, "router" => Self::Router, - "server_id" => Self::ServerId, _ => Self::Unknown(s), } } @@ -417,8 +414,6 @@ pub fn decode_not_found(status: &tonic::Status) -> impl Iterator String { match self { - Self::ServerIdNotSet => "server id must be set".to_string(), Self::DatabaseImmutable => "database must be mutable".to_string(), Self::ServerInvalidState(description) => description.clone(), Self::DatabaseInvalidState(description) => description.clone(), @@ -460,11 +454,6 @@ impl PreconditionViolation { impl From for rpc::precondition_failure::Violation { fn from(v: PreconditionViolation) -> Self { match v { - PreconditionViolation::ServerIdNotSet => Self { - r#type: "server_id".to_string(), - subject: "influxdata.com/iox".to_string(), - description: v.description(), - }, PreconditionViolation::ServerInvalidState(_) => Self { r#type: "state".to_string(), subject: "influxdata.com/iox".to_string(), @@ -516,7 +505,6 @@ impl From for rpc::precondition_failure::Violation { impl From for PreconditionViolation { fn from(v: rpc::precondition_failure::Violation) -> Self { match (v.r#type.as_str(), v.subject.as_str()) { - ("server_id", "influxdata.com/iox") => PreconditionViolation::ServerIdNotSet, ("state", "influxdata.com/iox") => { PreconditionViolation::ServerInvalidState(v.description) } From 4c56ba1e25a07ebe9dc0c7258987d2b486065412 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 11:25:16 -0400 Subject: [PATCH 18/31] fix: Move ErrorLogger trait to the only place it's used --- data_types/src/error.rs | 30 --------------------------- data_types/src/lib.rs | 1 - service_grpc_influxrpc/src/service.rs | 27 +++++++++++++++++++++++- 3 files changed, 26 insertions(+), 32 deletions(-) delete mode 100644 data_types/src/error.rs diff --git a/data_types/src/error.rs b/data_types/src/error.rs deleted file mode 100644 index ef7131298b..0000000000 --- a/data_types/src/error.rs +++ /dev/null @@ -1,30 +0,0 @@ -//! Common error utilities -use std::fmt::Debug; - -use observability_deps::tracing::error; - -/// Add ability for Results to log error messages via `error!` logs. -/// This is useful when using async tasks that may not have any code -/// checking their return values. -pub trait ErrorLogger { - /// Log the contents of self with a string of context. The context - /// should appear in a message such as - /// - /// "Error : - fn log_if_error(self, context: &str) -> Self; - - /// Provided method to log an error via the `error!` macro - fn log_error(context: &str, e: E) { - error!("Error {}: {:?}", context, e); - } -} - -/// Implement logging for all results -impl ErrorLogger for Result { - fn log_if_error(self, context: &str) -> Self { - if let Err(e) = &self { - Self::log_error(context, e); - } - self - } -} diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index 29f1a56182..bd3a990d97 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -12,7 +12,6 @@ pub mod chunk_metadata; pub mod consistent_hasher; -pub mod error; pub mod job; pub mod partition_metadata; pub mod timestamp; diff --git a/service_grpc_influxrpc/src/service.rs b/service_grpc_influxrpc/src/service.rs index 3e06d80c98..ca0c2ab29b 100644 --- a/service_grpc_influxrpc/src/service.rs +++ b/service_grpc_influxrpc/src/service.rs @@ -11,7 +11,6 @@ use crate::{ input::GrpcInputs, StorageService, }; -use data_types::error::ErrorLogger; use data_types2::{org_and_bucket_to_database, DatabaseName}; use generated_types::{ google::protobuf::Empty, literal_or_regex::Value as RegexOrLiteralValue, @@ -1363,6 +1362,32 @@ where Box::new(DeferredToJson { s: s.clone() }) } +/// Add ability for Results to log error messages via `error!` logs. +/// This is useful when using async tasks that may not have any code +/// checking their return values. +pub trait ErrorLogger { + /// Log the contents of self with a string of context. The context + /// should appear in a message such as + /// + /// "Error : + fn log_if_error(self, context: &str) -> Self; + + /// Provided method to log an error via the `error!` macro + fn log_error(context: &str, e: E) { + error!("Error {}: {:?}", context, e); + } +} + +/// Implement logging for all results +impl ErrorLogger for Result { + fn log_if_error(self, context: &str) -> Self { + if let Err(e) = &self { + Self::log_error(context, e); + } + self + } +} + #[cfg(test)] mod tests { use super::*; From d7304c111468d2d2ac311bc7e65d7570e609a870 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 11:50:06 -0400 Subject: [PATCH 19/31] fix: Move TimestampSummary to the only place it's used --- Cargo.lock | 1 + data_types/src/write_summary.rs | 54 ------------------------------- mutable_batch/Cargo.toml | 1 + mutable_batch/src/lib.rs | 57 ++++++++++++++++++++++++++------- mutable_batch/tests/writer.rs | 4 +-- 5 files changed, 48 insertions(+), 69 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 185c34f342..14b44d8e35 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3307,6 +3307,7 @@ dependencies = [ "data_types", "data_types2", "hashbrown 0.12.0", + "iox_time", "itertools", "rand", "schema", diff --git a/data_types/src/write_summary.rs b/data_types/src/write_summary.rs index ad3be67d42..2b88c394d2 100644 --- a/data_types/src/write_summary.rs +++ b/data_types/src/write_summary.rs @@ -1,4 +1,3 @@ -use crate::partition_metadata::StatValues; use iox_time::Time; /// A description of a set of writes @@ -19,56 +18,3 @@ pub struct WriteSummary { /// The number of rows in this summary pub row_count: usize, } - -/// A description of the distribution of timestamps in a -/// set of writes, bucketed based on minute within the hour -#[derive(Debug, Clone, Eq, PartialEq)] -pub struct TimestampSummary { - /// Stores the count of how many rows in the set of writes have a timestamp - /// with a minute matching a given index - /// - /// E.g. a row with timestamp 12:31:12 would store a count at index 31 - pub counts: [u32; 60], - - /// Standard timestamp statistics - pub stats: StatValues, -} - -impl Default for TimestampSummary { - fn default() -> Self { - Self { - counts: [0; 60], - stats: Default::default(), - } - } -} - -impl TimestampSummary { - /// Returns an iterator returning cumulative counts suitable for exposing - /// as a cumulative histogram - pub fn cumulative_counts(&self) -> impl Iterator + '_ { - let mut acc = 0_u64; - self.counts.iter().enumerate().map(move |(idx, count)| { - acc += *count as u64; - (idx, acc) - }) - } - - /// Merges the counts from the provided summary into this - pub fn merge(&mut self, other: &Self) { - for (a, b) in self.counts.iter_mut().zip(&other.counts) { - *a += *b - } - } - - /// Records a timestamp value - pub fn record(&mut self, timestamp: Time) { - self.counts[timestamp.minute() as usize] += 1; - self.stats.update(×tamp.timestamp_nanos()) - } - - /// Records a timestamp value from nanos - pub fn record_nanos(&mut self, timestamp_nanos: i64) { - self.record(Time::from_timestamp_nanos(timestamp_nanos)) - } -} diff --git a/mutable_batch/Cargo.toml b/mutable_batch/Cargo.toml index 1a7b78d192..1abfb504eb 100644 --- a/mutable_batch/Cargo.toml +++ b/mutable_batch/Cargo.toml @@ -10,6 +10,7 @@ arrow_util = { path = "../arrow_util" } chrono = { version = "0.4", default-features = false } data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } +iox_time = { path = "../iox_time" } schema = { path = "../schema" } snafu = "0.7" hashbrown = "0.12" diff --git a/mutable_batch/src/lib.rs b/mutable_batch/src/lib.rs index 2dd520cb9b..cc7dbb9e33 100644 --- a/mutable_batch/src/lib.rs +++ b/mutable_batch/src/lib.rs @@ -13,20 +13,17 @@ //! //! Can be viewed as a mutable version of [`RecordBatch`] that remains the exclusive //! owner of its buffers, permitting mutability. The in-memory layout is similar, however, -//! permitting fast conversion to [`RecordBatch`] -//! - -use std::ops::Range; - -use arrow::record_batch::RecordBatch; -use hashbrown::HashMap; -use snafu::{OptionExt, ResultExt, Snafu}; - -use data_types::write_summary::TimestampSummary; -use schema::selection::Selection; -use schema::{builder::SchemaBuilder, Schema, TIME_COLUMN_NAME}; +//! permitting fast conversion to [`RecordBatch`]. use crate::column::{Column, ColumnData}; +use arrow::record_batch::RecordBatch; +use data_types2::StatValues; +use hashbrown::HashMap; +use iox_time::Time; +use schema::selection::Selection; +use schema::{builder::SchemaBuilder, Schema, TIME_COLUMN_NAME}; +use snafu::{OptionExt, ResultExt, Snafu}; +use std::ops::Range; pub mod column; pub mod payload; @@ -208,3 +205,39 @@ impl MutableBatch { + self.columns.iter().map(|c| c.size()).sum::() } } + +/// A description of the distribution of timestamps in a +/// set of writes, bucketed based on minute within the hour +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct TimestampSummary { + /// Stores the count of how many rows in the set of writes have a timestamp + /// with a minute matching a given index + /// + /// E.g. a row with timestamp 12:31:12 would store a count at index 31 + pub counts: [u32; 60], + + /// Standard timestamp statistics + pub stats: StatValues, +} + +impl Default for TimestampSummary { + fn default() -> Self { + Self { + counts: [0; 60], + stats: Default::default(), + } + } +} + +impl TimestampSummary { + /// Records a timestamp value + pub fn record(&mut self, timestamp: Time) { + self.counts[timestamp.minute() as usize] += 1; + self.stats.update(×tamp.timestamp_nanos()) + } + + /// Records a timestamp value from nanos + pub fn record_nanos(&mut self, timestamp_nanos: i64) { + self.record(Time::from_timestamp_nanos(timestamp_nanos)) + } +} diff --git a/mutable_batch/tests/writer.rs b/mutable_batch/tests/writer.rs index 20b97a702b..d30bf868b7 100644 --- a/mutable_batch/tests/writer.rs +++ b/mutable_batch/tests/writer.rs @@ -1,8 +1,6 @@ use arrow_util::assert_batches_eq; -use data_types::write_summary::TimestampSummary; use data_types2::{StatValues, Statistics}; -use mutable_batch::writer::Writer; -use mutable_batch::MutableBatch; +use mutable_batch::{writer::Writer, MutableBatch, TimestampSummary}; use schema::selection::Selection; use std::num::NonZeroU64; From 44209faa8ea7d715cfc36054a749634d6d22393c Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 14:00:48 -0400 Subject: [PATCH 20/31] fix: Move write buffer data types to write_buffer crate --- clap_blocks/src/write_buffer.rs | 3 +- data_types/src/lib.rs | 1 - data_types/src/write_buffer.rs | 64 ---------------------------- write_buffer/src/config.rs | 67 ++++++++++++++++++++++++++++- write_buffer/src/file.rs | 73 +++++++++++++++++++------------- write_buffer/src/kafka/config.rs | 8 ++-- write_buffer/src/kafka/mod.rs | 2 +- write_buffer/src/mock.rs | 6 +-- 8 files changed, 117 insertions(+), 107 deletions(-) delete mode 100644 data_types/src/write_buffer.rs diff --git a/clap_blocks/src/write_buffer.rs b/clap_blocks/src/write_buffer.rs index da89fc54b5..5bfcbd317b 100644 --- a/clap_blocks/src/write_buffer.rs +++ b/clap_blocks/src/write_buffer.rs @@ -1,11 +1,10 @@ -use data_types::write_buffer::{WriteBufferConnection, WriteBufferCreationConfig}; use iox_time::SystemProvider; use observability_deps::tracing::*; use std::{collections::BTreeMap, num::NonZeroU32, path::PathBuf, sync::Arc}; use tempfile::TempDir; use trace::TraceCollector; use write_buffer::{ - config::WriteBufferConfigFactory, + config::{WriteBufferConfigFactory, WriteBufferConnection, WriteBufferCreationConfig}, core::{WriteBufferError, WriteBufferReading, WriteBufferWriting}, }; diff --git a/data_types/src/lib.rs b/data_types/src/lib.rs index bd3a990d97..e607e49c77 100644 --- a/data_types/src/lib.rs +++ b/data_types/src/lib.rs @@ -15,5 +15,4 @@ pub mod consistent_hasher; pub mod job; pub mod partition_metadata; pub mod timestamp; -pub mod write_buffer; pub mod write_summary; diff --git a/data_types/src/write_buffer.rs b/data_types/src/write_buffer.rs deleted file mode 100644 index 1b01ef8ada..0000000000 --- a/data_types/src/write_buffer.rs +++ /dev/null @@ -1,64 +0,0 @@ -use std::{collections::BTreeMap, num::NonZeroU32}; - -pub const DEFAULT_N_SEQUENCERS: u32 = 1; - -/// Configures the use of a write buffer. -#[derive(Debug, Eq, PartialEq, Clone, Hash)] -pub struct WriteBufferConnection { - /// Which type should be used (e.g. "kafka", "mock") - pub type_: String, - - /// Connection string, depends on [`type_`](Self::type_). - pub connection: String, - - /// Special configs to be applied when establishing the connection. - /// - /// This depends on [`type_`](Self::type_) and can configure aspects like timeouts. - /// - /// Note: This config should be a [`BTreeMap`] to ensure that a stable hash. - pub connection_config: BTreeMap, - - /// Specifies if the sequencers (e.g. for Kafka in form of a topic) should be automatically created if they do not - /// existing prior to reading or writing. - pub creation_config: Option, -} - -impl Default for WriteBufferConnection { - fn default() -> Self { - Self { - type_: "unspecified".to_string(), - connection: Default::default(), - connection_config: Default::default(), - creation_config: Default::default(), - } - } -} - -/// Configs sequencer auto-creation for write buffers. -/// -/// What that means depends on the used write buffer, e.g. for Kafka this will create a new topic w/ -/// [`n_sequencers`](Self::n_sequencers) partitions. -#[derive(Debug, Eq, PartialEq, Clone, Hash)] -pub struct WriteBufferCreationConfig { - /// Number of sequencers. - /// - /// How they are implemented depends on [type](WriteBufferConnection::type_), e.g. for Kafka this is mapped to the - /// number of partitions. - pub n_sequencers: NonZeroU32, - - /// Special configs to by applied when sequencers are created. - /// - /// This depends on [type](WriteBufferConnection::type_) and can setup parameters like retention policy. - /// - /// Note: This config should be a [`BTreeMap`] to ensure that a stable hash. - pub options: BTreeMap, -} - -impl Default for WriteBufferCreationConfig { - fn default() -> Self { - Self { - n_sequencers: NonZeroU32::try_from(DEFAULT_N_SEQUENCERS).unwrap(), - options: Default::default(), - } - } -} diff --git a/write_buffer/src/config.rs b/write_buffer/src/config.rs index 140f175e53..f99649af3e 100644 --- a/write_buffer/src/config.rs +++ b/write_buffer/src/config.rs @@ -7,22 +7,86 @@ use crate::{ MockBufferForWritingThatAlwaysErrors, MockBufferSharedState, }, }; -use data_types::write_buffer::WriteBufferConnection; use iox_time::TimeProvider; use parking_lot::RwLock; use std::{ collections::{btree_map::Entry, BTreeMap}, + num::NonZeroU32, path::PathBuf, sync::Arc, }; use trace::TraceCollector; +pub const DEFAULT_N_SEQUENCERS: u32 = 1; + #[derive(Debug, Clone)] enum Mock { Normal(MockBufferSharedState), AlwaysFailing, } +/// Configures the use of a write buffer. +#[derive(Debug, Eq, PartialEq, Clone, Hash)] +pub struct WriteBufferConnection { + /// Which type should be used (e.g. "kafka", "mock") + pub type_: String, + + /// Connection string, depends on [`type_`](Self::type_). + pub connection: String, + + /// Special configs to be applied when establishing the connection. + /// + /// This depends on [`type_`](Self::type_) and can configure aspects like timeouts. + /// + /// Note: This config should be a [`BTreeMap`] to ensure that a stable hash. + pub connection_config: BTreeMap, + + /// Specifies if the sequencers (e.g. for Kafka in form of a topic) should be automatically + /// created if they do not existing prior to reading or writing. + pub creation_config: Option, +} + +impl Default for WriteBufferConnection { + fn default() -> Self { + Self { + type_: "unspecified".to_string(), + connection: Default::default(), + connection_config: Default::default(), + creation_config: Default::default(), + } + } +} + +/// Configs sequencer auto-creation for write buffers. +/// +/// What that means depends on the used write buffer, e.g. for Kafka this will create a new topic w/ +/// [`n_sequencers`](Self::n_sequencers) partitions. +#[derive(Debug, Eq, PartialEq, Clone, Hash)] +pub struct WriteBufferCreationConfig { + /// Number of sequencers. + /// + /// How they are implemented depends on [type](WriteBufferConnection::type_), e.g. for Kafka + /// this is mapped to the number of partitions. + pub n_sequencers: NonZeroU32, + + /// Special configs to by applied when sequencers are created. + /// + /// This depends on [type](WriteBufferConnection::type_) and can setup parameters like + /// retention policy. + /// + /// Note: This config should be a [`BTreeMap`] to ensure that a stable hash. + pub options: BTreeMap, +} + +impl Default for WriteBufferCreationConfig { + fn default() -> Self { + Self { + n_sequencers: NonZeroU32::try_from(DEFAULT_N_SEQUENCERS).unwrap(), + options: Default::default(), + } + } +} + /// Factory that creates [`WriteBufferReading`] and [`WriteBufferWriting`] /// from [`WriteBufferConnection`]. #[derive(Debug)] @@ -193,7 +257,6 @@ mod tests { core::test_utils::random_topic_name, maybe_skip_kafka_integration, mock::MockBufferSharedState, }; - use data_types::write_buffer::WriteBufferCreationConfig; use data_types2::DatabaseName; use std::{convert::TryFrom, num::NonZeroU32}; use tempfile::TempDir; diff --git a/write_buffer/src/file.rs b/write_buffer/src/file.rs index 2b11af6c0a..701d3b1100 100644 --- a/write_buffer/src/file.rs +++ b/write_buffer/src/file.rs @@ -1,11 +1,12 @@ //! Write buffer that uses files to encode messages. //! -//! This implementation can be used by multiple readers and writers at the same time. It is ideal for local end2end -//! testing. However it might not perform extremely well when dealing with large messages and (currently) does not -//! implement any message pruning. +//! This implementation can be used by multiple readers and writers at the same time. It is ideal +//! for local end2end testing. However it might not perform extremely well when dealing with large +//! messages and (currently) does not implement any message pruning. //! //! # Format -//! Given a root path, the database name and the number of sequencers, the directory structure looks like this: +//! Given a root path, the database name and the number of sequencers, the directory structure +//! looks like this: //! //! ```text //! // @@ -54,50 +55,63 @@ //! //! ``` //! -//! The payload is binary data. The headers contain metadata about it (like timestamp, format, tracing information). -//! +//! The payload is binary data. The headers contain metadata about it (like timestamp, format, +//! tracing information). //! //! # Implementation Notes +//! //! Some notes about file system functionality that shaped this implementation //! //! ## Atomic File Creation -//! It is quite easy to create a file and ensure that it did not exist beforehand using [`open(2)`] together with -//! `O_CREAT` and `O_EXCL`. However writing actual content to that file requires time and a reader could already see an -//! incomplete version of that. A workaround is to use a scratchpad file at a temporary location, write the entire -//! desired content to it and then move the file to the target location. This assumes that the target location and the -//! file content are independent, e.g. that the file itself does not contain the `sequence_number`. Now we need to find -//! a way to make this move operation reliable though. //! -//! Files can be renamed using [`rename(2)`]. There is the `RENAME_NOREPLACE` flag that prevents that we silently -//! overwrite the target file. This however is only implemented for a handful of filesystems (notable NOT [NFS]). So to -//! use [`rename(2)`] we would need some additional locking. +//! It is quite easy to create a file and ensure that it did not exist beforehand using [`open(2)`] +//! together with `O_CREAT` and `O_EXCL`. However writing actual content to that file requires time +//! and a reader could already see an incomplete version of that. A workaround is to use a +//! scratchpad file at a temporary location, write the entire desired content to it and then move +//! the file to the target location. This assumes that the target location and the file content are +//! independent, e.g. that the file itself does not contain the `sequence_number`. Now we need to +//! find a way to make this move operation reliable though. //! -//! Then there is [`link(2)`] which creates a new link to an existing file. It explicitly states that the target is -//! NEVER overwritten. According to this should even work properly on [NFS]. -//! We then need to use [`unlink(2)`] to clean the scratchpad file. +//! Files can be renamed using [`rename(2)`]. There is the `RENAME_NOREPLACE` flag that prevents +//! that we silently overwrite the target file. This however is only implemented for a handful of +//! filesystems (notable NOT [NFS]). So to use [`rename(2)`] we would need some additional locking. +//! +//! Then there is [`link(2)`] which creates a new link to an existing file. It explicitly states +//! that the target is NEVER overwritten. According to +//! this should even work properly on [NFS]. We then need to use [`unlink(2)`] to clean the +//! scratchpad file. //! //! ## Atomic Directory Creation -//! To setup a new sequencer config we need to create the directory structure in an atomic way. Hardlinks don't work for -//! directories, but [`symlink(2)`] does and -- like [`link(2)`] -- does not overwrite existing targets. +//! +//! To setup a new sequencer config we need to create the directory structure in an atomic way. +//! Hardlinks don't work for directories, but [`symlink(2)`] does and -- like [`link(2)`] -- does +//! not overwrite existing targets. //! //! ## File Locking -//! Instead of atomic operations we could also use file locking. Under Linux there are a few ways this can be archived: +//! +//! Instead of atomic operations we could also use file locking. Under Linux there are a few ways +//! this can be archived: //! //! - **[`fcntl(2)`] via `F_SETLK`, `F_SETLKW`, `F_GETLK`:**
-//! Works on [NFS], but is process-bound (aka if you have multiple writers within the same process, only one can +//! Works on [NFS], but is process-bound (aka if you have multiple writers within the same +//! process, only one can //! acquire the lock). //! - **[`fcntl(2)`] via `F_OFD_SETLK`, `F_OFD_SETLKW`, `F_OFD_GETLK`:**
//! Works on [NFS] and is file-descriptor-bound. //! - **[`flock(2)`]:**
-//! Works on [NFS] but is technically emulated via [`fcntl(2)`] so the latter should probably be preferred. +//! Works on [NFS] but is technically emulated via [`fcntl(2)`] so the latter should probably be +//! preferred. //! -//! The biggest issue with file locking is what happens when an operation fails while a lock is being held. Either the -//! resulting state is obviously unfinished (e.g. due to some checksum or size mismatch, due to some missing marker) or -//! we would need to implement some form of lock poisoning. Since this can get quite tricky, I have decided that atomic -//! file and directory operations are easier to reason about. +//! The biggest issue with file locking is what happens when an operation fails while a lock is +//! being held. Either the resulting state is obviously unfinished (e.g. due to some checksum or +//! size mismatch, due to some missing marker) or we would need to implement some form of lock +//! poisoning. Since this can get quite tricky, I have decided that atomic file and directory +//! operations are easier to reason about. //! //! ## Message Metadata -//! We are NOT using any file-based metadata (like `mtime` or extended attributes) because they are often broken. +//! +//! We are NOT using any file-based metadata (like `mtime` or extended attributes) because they are +//! often broken. //! //! //! [`fcntl(2)`]: https://www.man7.org/linux/man-pages/man2/fcntl.2.html @@ -108,12 +122,13 @@ //! [`rename(2)`]: https://man7.org/linux/man-pages/man2/rename.2.html //! [`symlink(2)`]: https://man7.org/linux/man-pages/man2/symlink.2.html //! [`unlink(2)`]: https://man7.org/linux/man-pages/man2/unlink.2.html + use crate::{ codec::{ContentType, IoxHeaders}, + config::WriteBufferCreationConfig, core::{WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting}, }; use async_trait::async_trait; -use data_types::write_buffer::WriteBufferCreationConfig; use data_types2::Sequence; use dml::{DmlMeta, DmlOperation}; use futures::{stream::BoxStream, Stream, StreamExt}; diff --git a/write_buffer/src/kafka/config.rs b/write_buffer/src/kafka/config.rs index b8ac6c0673..193a3066fc 100644 --- a/write_buffer/src/kafka/config.rs +++ b/write_buffer/src/kafka/config.rs @@ -1,10 +1,8 @@ +use crate::{config::WriteBufferCreationConfig, core::WriteBufferError}; use std::{collections::BTreeMap, fmt::Display, str::FromStr, time::Duration}; -use data_types::write_buffer::WriteBufferCreationConfig; - -use crate::core::WriteBufferError; - -/// Generic client config that is used for consumers, producers as well as admin operations (like "create topic"). +/// Generic client config that is used for consumers, producers as well as admin operations (like +/// "create topic"). #[derive(Debug, PartialEq, Eq)] pub struct ClientConfig { /// Maximum message size in bytes. diff --git a/write_buffer/src/kafka/mod.rs b/write_buffer/src/kafka/mod.rs index f50a9c67cd..f0db8dc717 100644 --- a/write_buffer/src/kafka/mod.rs +++ b/write_buffer/src/kafka/mod.rs @@ -4,13 +4,13 @@ use self::{ }; use crate::{ codec::IoxHeaders, + config::WriteBufferCreationConfig, core::{ WriteBufferError, WriteBufferErrorKind, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting, }, }; use async_trait::async_trait; -use data_types::write_buffer::WriteBufferCreationConfig; use data_types2::Sequence; use dml::{DmlMeta, DmlOperation}; use futures::{stream::BoxStream, StreamExt}; diff --git a/write_buffer/src/mock.rs b/write_buffer/src/mock.rs index 1d6ec69f75..a5b2e86711 100644 --- a/write_buffer/src/mock.rs +++ b/write_buffer/src/mock.rs @@ -1,8 +1,8 @@ -use crate::core::{ - WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting, +use crate::{ + config::WriteBufferCreationConfig, + core::{WriteBufferError, WriteBufferReading, WriteBufferStreamHandler, WriteBufferWriting}, }; use async_trait::async_trait; -use data_types::write_buffer::WriteBufferCreationConfig; use data_types2::Sequence; use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; use futures::{stream::BoxStream, StreamExt}; From bf1d08333d7bb221edea79fa55ad1a176f842032 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 6 May 2022 14:40:58 -0400 Subject: [PATCH 21/31] fix: Move ChunkColumnSummary to read_buffer, the only place it's used --- Cargo.lock | 1 - read_buffer/Cargo.toml | 1 - read_buffer/src/chunk.rs | 14 ++++++++++++-- read_buffer/src/table.rs | 4 ++-- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 14b44d8e35..2ebfbf5e32 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4522,7 +4522,6 @@ dependencies = [ "arrow_util", "criterion", "croaring", - "data_types", "data_types2", "datafusion 0.1.0", "either", diff --git a/read_buffer/Cargo.toml b/read_buffer/Cargo.toml index c28185807c..046c91d5e9 100644 --- a/read_buffer/Cargo.toml +++ b/read_buffer/Cargo.toml @@ -14,7 +14,6 @@ edition = "2021" arrow = { version = "13", features = ["prettyprint"] } arrow_util = { path = "../arrow_util" } croaring = "0.6" -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } either = "1.6.1" diff --git a/read_buffer/src/chunk.rs b/read_buffer/src/chunk.rs index f2412a2d35..cd804a71a8 100644 --- a/read_buffer/src/chunk.rs +++ b/read_buffer/src/chunk.rs @@ -5,7 +5,6 @@ use crate::{ table::{self, Table}, }; use arrow::{error::ArrowError, record_batch::RecordBatch}; -use data_types::chunk_metadata::ChunkColumnSummary; use data_types2::TableSummary; use observability_deps::tracing::debug; use schema::{builder::Error as SchemaError, selection::Selection, Schema}; @@ -13,6 +12,7 @@ use snafu::{ResultExt, Snafu}; use std::{ collections::{BTreeMap, BTreeSet}, convert::TryFrom, + sync::Arc, }; // The desired minimum row group size, used as the default for the `ChunkBuilder`. @@ -96,7 +96,7 @@ impl Chunk { /// Return the estimated size for each column in the table. /// Note there may be multiple entries for each column. - pub fn column_sizes(&self) -> Vec { + pub(crate) fn column_sizes(&self) -> Vec { self.table.column_sizes() } @@ -508,6 +508,16 @@ impl ChunkBuilder { } } +/// Represents metadata about the physical storage of a column in a chunk +#[derive(Debug, PartialEq, Eq, Clone)] +pub(crate) struct ChunkColumnSummary { + /// Column name + pub(crate) name: Arc, + + /// Estimated size, in bytes, consumed by this column. + pub(crate) memory_bytes: usize, +} + #[cfg(test)] mod test { use super::*; diff --git a/read_buffer/src/table.rs b/read_buffer/src/table.rs index b6fbbd6d4f..5f33847fb0 100644 --- a/read_buffer/src/table.rs +++ b/read_buffer/src/table.rs @@ -1,4 +1,5 @@ use crate::{ + chunk::ChunkColumnSummary, column, row_group::{self, ColumnName, Literal, Predicate, RowGroup}, schema::{AggregateType, ColumnType, LogicalDataType, ResultSchema}, @@ -6,7 +7,6 @@ use crate::{ BinaryExpr, }; use arrow::record_batch::RecordBatch; -use data_types::chunk_metadata::ChunkColumnSummary; use data_types2::TableSummary; use parking_lot::RwLock; use schema::selection::Selection; @@ -147,7 +147,7 @@ impl Table { } /// The estimated size for each column in this table. - pub fn column_sizes(&self) -> Vec { + pub(crate) fn column_sizes(&self) -> Vec { self.table_data .read() .data From 0541c6e40f0bec300c47917d58822e3dd99c03c1 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 10:44:05 -0400 Subject: [PATCH 22/31] fix: Remove data_types crate where it's no longer used --- Cargo.lock | 15 --------------- clap_blocks/Cargo.toml | 1 - compactor/Cargo.toml | 1 - data_types2/Cargo.toml | 1 - dml/Cargo.toml | 1 - influxdb_iox/Cargo.toml | 1 - ingester/Cargo.toml | 1 - iox_data_generator/Cargo.toml | 1 - ioxd_common/Cargo.toml | 1 - mutable_batch/Cargo.toml | 1 - parquet_file/Cargo.toml | 1 - querier/Cargo.toml | 1 - query/Cargo.toml | 1 - service_grpc_influxrpc/Cargo.toml | 1 - write_buffer/Cargo.toml | 1 - write_summary/Cargo.toml | 16 ++-------------- 16 files changed, 2 insertions(+), 43 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2ebfbf5e32..aa02a7fb58 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -685,7 +685,6 @@ name = "clap_blocks" version = "0.1.0" dependencies = [ "clap 3.1.12", - "data_types", "futures", "humantime", "iox_catalog", @@ -813,7 +812,6 @@ dependencies = [ "async-trait", "backoff 0.1.0", "bytes", - "data_types", "data_types2", "datafusion 0.1.0", "futures", @@ -1172,7 +1170,6 @@ dependencies = [ name = "data_types2" version = "0.1.0" dependencies = [ - "data_types", "influxdb_line_protocol", "observability_deps", "ordered-float 3.0.0", @@ -1413,7 +1410,6 @@ name = "dml" version = "0.1.0" dependencies = [ "arrow_util", - "data_types", "data_types2", "hashbrown 0.12.0", "iox_time", @@ -2168,7 +2164,6 @@ dependencies = [ "compactor", "console-subscriber", "csv", - "data_types", "data_types2", "datafusion 0.1.0", "dml", @@ -2336,7 +2331,6 @@ dependencies = [ "bitflags", "bytes", "chrono", - "data_types", "data_types2", "datafusion 0.1.0", "datafusion_util", @@ -2458,7 +2452,6 @@ dependencies = [ "chrono-english", "clap 3.1.12", "criterion", - "data_types", "futures", "handlebars", "humantime", @@ -2571,7 +2564,6 @@ dependencies = [ "chrono", "clap 3.1.12", "clap_blocks", - "data_types", "data_types2", "dml", "flate2", @@ -3304,7 +3296,6 @@ dependencies = [ "arrow", "arrow_util", "chrono", - "data_types", "data_types2", "hashbrown 0.12.0", "iox_time", @@ -3801,7 +3792,6 @@ dependencies = [ "arrow_util", "base64 0.13.0", "bytes", - "data_types", "data_types2", "datafusion 0.1.0", "datafusion_util", @@ -4268,7 +4258,6 @@ dependencies = [ "bytes", "client_util", "criterion", - "data_types", "data_types2", "datafusion 0.1.0", "datafusion_util", @@ -4313,7 +4302,6 @@ dependencies = [ "async-trait", "chrono", "croaring", - "data_types", "data_types2", "datafusion 0.1.0", "datafusion_util", @@ -5220,7 +5208,6 @@ version = "0.1.0" dependencies = [ "arrow", "async-trait", - "data_types", "data_types2", "datafusion 0.1.0", "futures", @@ -6790,7 +6777,6 @@ name = "write_buffer" version = "0.1.0" dependencies = [ "async-trait", - "data_types", "data_types2", "dml", "dotenv", @@ -6825,7 +6811,6 @@ name = "write_summary" version = "0.1.0" dependencies = [ "base64 0.13.0", - "data_types", "data_types2", "dml", "generated_types", diff --git a/clap_blocks/Cargo.toml b/clap_blocks/Cargo.toml index ff507f92b9..b24945f600 100644 --- a/clap_blocks/Cargo.toml +++ b/clap_blocks/Cargo.toml @@ -5,7 +5,6 @@ edition = "2021" [dependencies] clap = { version = "3", features = ["derive", "env"] } -data_types = { path = "../data_types" } futures = "0.3" humantime = "2.1.0" iox_catalog = { path = "../iox_catalog" } diff --git a/compactor/Cargo.toml b/compactor/Cargo.toml index 62b61baf60..e21d11f287 100644 --- a/compactor/Cargo.toml +++ b/compactor/Cargo.toml @@ -9,7 +9,6 @@ arrow = { version = "13", features = ["prettyprint"] } async-trait = "0.1.53" backoff = { path = "../backoff" } bytes = "1.0" -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } futures = "0.3" diff --git a/data_types2/Cargo.toml b/data_types2/Cargo.toml index 84c58e76d8..4ef50bff7c 100644 --- a/data_types2/Cargo.toml +++ b/data_types2/Cargo.toml @@ -5,7 +5,6 @@ edition = "2021" description = "Shared data types in the IOx NG architecture" [dependencies] -data_types = { path = "../data_types" } influxdb_line_protocol = { path = "../influxdb_line_protocol" } observability_deps = { path = "../observability_deps" } ordered-float = "3" diff --git a/dml/Cargo.toml b/dml/Cargo.toml index 4606ad7ab2..2d43fd9cf1 100644 --- a/dml/Cargo.toml +++ b/dml/Cargo.toml @@ -6,7 +6,6 @@ description = "DML types" [dependencies] arrow_util = { path = "../arrow_util" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } hashbrown = "0.12" mutable_batch = { path = "../mutable_batch" } diff --git a/influxdb_iox/Cargo.toml b/influxdb_iox/Cargo.toml index ce6c8b59b9..6c902b2c89 100644 --- a/influxdb_iox/Cargo.toml +++ b/influxdb_iox/Cargo.toml @@ -9,7 +9,6 @@ default-run = "influxdb_iox" # Workspace dependencies, in alphabetical order clap_blocks = { path = "../clap_blocks" } compactor = { path = "../compactor" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } dml = { path = "../dml" } diff --git a/ingester/Cargo.toml b/ingester/Cargo.toml index 669c643f8b..08cb2f1fb5 100644 --- a/ingester/Cargo.toml +++ b/ingester/Cargo.toml @@ -14,7 +14,6 @@ base64 = "0.13" bytes = "1.0" datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } futures = "0.3" generated_types = { path = "../generated_types" } diff --git a/iox_data_generator/Cargo.toml b/iox_data_generator/Cargo.toml index a8d94347e9..adb72661af 100644 --- a/iox_data_generator/Cargo.toml +++ b/iox_data_generator/Cargo.toml @@ -12,7 +12,6 @@ clap = { version = "3", features = ["cargo"] } futures = "0.3" handlebars = "4.2.2" humantime = "2.1.0" -data_types = { path = "../data_types" } influxdb2_client = { path = "../influxdb2_client" } influxdb_iox_client = { path = "../influxdb_iox_client" } itertools = "0.10.0" diff --git a/ioxd_common/Cargo.toml b/ioxd_common/Cargo.toml index 3582869dcb..e2d6f4533c 100644 --- a/ioxd_common/Cargo.toml +++ b/ioxd_common/Cargo.toml @@ -10,7 +10,6 @@ edition = "2021" [dependencies] # Workspace dependencies, in alphabetical order clap_blocks = { path = "../clap_blocks" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } dml = { path = "../dml" } metric = { path = "../metric" } diff --git a/mutable_batch/Cargo.toml b/mutable_batch/Cargo.toml index 1abfb504eb..020f7bf353 100644 --- a/mutable_batch/Cargo.toml +++ b/mutable_batch/Cargo.toml @@ -8,7 +8,6 @@ description = "A mutable arrow RecordBatch" arrow = { version = "13", features = ["prettyprint"] } arrow_util = { path = "../arrow_util" } chrono = { version = "0.4", default-features = false } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } iox_time = { path = "../iox_time" } schema = { path = "../schema" } diff --git a/parquet_file/Cargo.toml b/parquet_file/Cargo.toml index 2c046b0916..464f9924f0 100644 --- a/parquet_file/Cargo.toml +++ b/parquet_file/Cargo.toml @@ -8,7 +8,6 @@ edition = "2021" arrow = { version = "13", features = ["prettyprint"] } base64 = "0.13" bytes = "1.0" -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } diff --git a/querier/Cargo.toml b/querier/Cargo.toml index e20e2270cc..077c8c002a 100644 --- a/querier/Cargo.toml +++ b/querier/Cargo.toml @@ -10,7 +10,6 @@ async-trait = "0.1.53" backoff = { path = "../backoff" } bytes = "1.0" client_util = { path = "../client_util" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } diff --git a/query/Cargo.toml b/query/Cargo.toml index a60448cb7c..8c5703a43e 100644 --- a/query/Cargo.toml +++ b/query/Cargo.toml @@ -19,7 +19,6 @@ arrow_util = { path = "../arrow_util" } async-trait = "0.1" chrono = { version = "0.4", default-features = false } croaring = "0.6" -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } datafusion = { path = "../datafusion" } datafusion_util = { path = "../datafusion_util" } diff --git a/service_grpc_influxrpc/Cargo.toml b/service_grpc_influxrpc/Cargo.toml index b9630afd4c..78a79a87cf 100644 --- a/service_grpc_influxrpc/Cargo.toml +++ b/service_grpc_influxrpc/Cargo.toml @@ -8,7 +8,6 @@ edition = "2021" [dependencies] # Workspace dependencies, in alphabetical order datafusion = { path = "../datafusion" } -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } generated_types = { path = "../generated_types" } observability_deps = { path = "../observability_deps" } diff --git a/write_buffer/Cargo.toml b/write_buffer/Cargo.toml index 61e2a20a43..fee3cd539e 100644 --- a/write_buffer/Cargo.toml +++ b/write_buffer/Cargo.toml @@ -5,7 +5,6 @@ edition = "2021" [dependencies] async-trait = "0.1" -data_types = { path = "../data_types" } data_types2 = { path = "../data_types2" } dml = { path = "../dml" } dotenv = "0.15.0" diff --git a/write_summary/Cargo.toml b/write_summary/Cargo.toml index f6e660933e..4fdcb66bbf 100644 --- a/write_summary/Cargo.toml +++ b/write_summary/Cargo.toml @@ -3,27 +3,15 @@ name = "write_summary" version = "0.1.0" edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - [dependencies] -# Workspace dependencies, in alphabetical order +base64 = "0.13" data_types2 = { path = "../data_types2" } dml = { path = "../dml" } generated_types = { path = "../generated_types" } observability_deps = { path = "../observability_deps" } -workspace-hack = { path = "../workspace-hack"} - - -# Crates.io dependencies, in alphabetical order -base64 = "0.13" serde_json = "1.0.81" snafu = "0.7" - +workspace-hack = { path = "../workspace-hack"} [dev-dependencies] -# Workspace dependencies, in alphabetical order -data_types = { path = "../data_types" } iox_time = { path = "../iox_time" } - - -# Crates.io dependencies, in alphabetical order From e1bef1c218cd4fdc173e479d21b148c3e98cb46a Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 5 May 2022 14:03:01 -0400 Subject: [PATCH 23/31] fix: Remove OG data_types crate --- Cargo.lock | 18 - Cargo.toml | 1 - data_types/Cargo.toml | 23 - data_types/README.md | 5 - data_types/src/chunk_metadata.rs | 382 -------- data_types/src/consistent_hasher.rs | 176 ---- data_types/src/job.rs | 162 ---- data_types/src/lib.rs | 18 - data_types/src/partition_metadata.rs | 1289 -------------------------- data_types/src/timestamp.rs | 196 ---- data_types/src/write_summary.rs | 20 - 11 files changed, 2290 deletions(-) delete mode 100644 data_types/Cargo.toml delete mode 100644 data_types/README.md delete mode 100644 data_types/src/chunk_metadata.rs delete mode 100644 data_types/src/consistent_hasher.rs delete mode 100644 data_types/src/job.rs delete mode 100644 data_types/src/lib.rs delete mode 100644 data_types/src/partition_metadata.rs delete mode 100644 data_types/src/timestamp.rs delete mode 100644 data_types/src/write_summary.rs diff --git a/Cargo.lock b/Cargo.lock index aa02a7fb58..df75bee0f6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1148,24 +1148,6 @@ dependencies = [ "syn", ] -[[package]] -name = "data_types" -version = "0.1.0" -dependencies = [ - "bytes", - "iox_time", - "num_cpus", - "observability_deps", - "ordered-float 3.0.0", - "percent-encoding", - "regex", - "siphasher", - "snafu", - "test_helpers", - "uuid 0.8.2", - "workspace-hack", -] - [[package]] name = "data_types2" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 63435bd1ee..f216cd9c09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -6,7 +6,6 @@ members = [ "clap_blocks", "client_util", "compactor", - "data_types", "data_types2", "datafusion", "datafusion_util", diff --git a/data_types/Cargo.toml b/data_types/Cargo.toml deleted file mode 100644 index 61bf4a7158..0000000000 --- a/data_types/Cargo.toml +++ /dev/null @@ -1,23 +0,0 @@ -[package] -name = "data_types" -version = "0.1.0" -authors = ["pauldix "] -description = "InfluxDB IOx data_types, shared between IOx instances and IOx clients" -edition = "2021" -readme = "README.md" - -[dependencies] # In alphabetical order -bytes = "1.0" -num_cpus = "1.13.0" -observability_deps = { path = "../observability_deps" } -ordered-float = "3" -percent-encoding = "2.1.0" -regex = "1" -siphasher = "0.3" -snafu = "0.7" -iox_time = { path = "../iox_time" } -uuid = { version = "0.8", features = ["v4"] } -workspace-hack = { path = "../workspace-hack"} - -[dev-dependencies] # In alphabetical order -test_helpers = { path = "../test_helpers" } diff --git a/data_types/README.md b/data_types/README.md deleted file mode 100644 index 2911ec8088..0000000000 --- a/data_types/README.md +++ /dev/null @@ -1,5 +0,0 @@ -# Data Types - -This crate contains types that are designed for external consumption (in `influxdb_iox_client` and other "client" facing uses). - -*Client facing* in this case means exposed via management API or CLI and where changing the structs may require additional coordination / organization with clients. diff --git a/data_types/src/chunk_metadata.rs b/data_types/src/chunk_metadata.rs deleted file mode 100644 index a70fcd64e0..0000000000 --- a/data_types/src/chunk_metadata.rs +++ /dev/null @@ -1,382 +0,0 @@ -//! Module contains a representation of chunk metadata -use std::{convert::TryFrom, num::NonZeroU32, str::FromStr, sync::Arc}; - -use bytes::Bytes; -use iox_time::Time; -use snafu::{ResultExt, Snafu}; -use uuid::Uuid; - -use crate::partition_metadata::PartitionAddr; - -/// Address of the chunk within the catalog -#[derive(Debug, Clone, Eq, PartialEq, PartialOrd, Ord, Hash)] -pub struct ChunkAddr { - /// Database name - pub db_name: Arc, - - /// What table does the chunk belong to? - pub table_name: Arc, - - /// What partition does the chunk belong to? - pub partition_key: Arc, - - /// The ID of the chunk - pub chunk_id: ChunkId, -} - -impl ChunkAddr { - pub fn new(partition: &PartitionAddr, chunk_id: ChunkId) -> Self { - Self { - db_name: Arc::clone(&partition.db_name), - table_name: Arc::clone(&partition.table_name), - partition_key: Arc::clone(&partition.partition_key), - chunk_id, - } - } - - pub fn into_partition(self) -> PartitionAddr { - PartitionAddr { - db_name: self.db_name, - table_name: self.table_name, - partition_key: self.partition_key, - } - } -} - -impl std::fmt::Display for ChunkAddr { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - "Chunk('{}':'{}':'{}':{})", - self.db_name, - self.table_name, - self.partition_key, - self.chunk_id.get() - ) - } -} - -/// Which storage system is a chunk located in? -#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone, Copy)] -pub enum ChunkStorage { - /// The chunk is still open for new writes, in the Mutable Buffer - OpenMutableBuffer, - - /// The chunk is no longer open for writes, in the Mutable Buffer - ClosedMutableBuffer, - - /// The chunk is in the Read Buffer (where it can not be mutated) - ReadBuffer, - - /// The chunk is both in ReadBuffer and Object Store - ReadBufferAndObjectStore, - - /// The chunk is stored in Object Storage (where it can not be mutated) - ObjectStoreOnly, -} - -impl ChunkStorage { - /// Return a str representation of this storage state - pub fn as_str(&self) -> &'static str { - match self { - Self::OpenMutableBuffer => "OpenMutableBuffer", - Self::ClosedMutableBuffer => "ClosedMutableBuffer", - Self::ReadBuffer => "ReadBuffer", - Self::ReadBufferAndObjectStore => "ReadBufferAndObjectStore", - Self::ObjectStoreOnly => "ObjectStoreOnly", - } - } - - /// Returns true if has data in the mutable buffer - pub fn has_mutable_buffer(&self) -> bool { - matches!(self, Self::OpenMutableBuffer | Self::ClosedMutableBuffer) - } - - /// Returns true if has data in the read buffer - pub fn has_read_buffer(&self) -> bool { - matches!(self, Self::ReadBuffer | Self::ReadBufferAndObjectStore) - } - - /// Returns true if has data in object storage - pub fn has_object_store(&self) -> bool { - matches!(self, Self::ReadBufferAndObjectStore | Self::ObjectStoreOnly) - } -} - -/// Any lifecycle action currently in progress for this chunk -#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord)] -pub enum ChunkLifecycleAction { - /// Chunk is in the process of being written to object storage - Persisting, - - /// Chunk is in the process of being compacted - Compacting, - - /// Object Store Chunk is in the process of being compacted - CompactingObjectStore, - - /// Chunk is about to be dropped from memory and (if persisted) from object store - Dropping, - - /// Chunk is in the process of being loaded back into the RUB - LoadingReadBuffer, -} - -impl std::fmt::Display for ChunkLifecycleAction { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.name()) - } -} - -impl ChunkLifecycleAction { - pub fn name(&self) -> &'static str { - match self { - Self::Persisting => "Persisting to Object Storage", - Self::Compacting => "Compacting", - Self::CompactingObjectStore => "Compacting Object Store", - Self::Dropping => "Dropping", - Self::LoadingReadBuffer => "Loading to Read Buffer", - } - } -} - -/// Represents metadata about the physical storage of a chunk in a -/// database. -#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone)] -pub struct ChunkSummary { - /// The partition key of this chunk - pub partition_key: Arc, - - /// The table of this chunk - pub table_name: Arc, - - /// Order of this chunk relative to other overlapping chunks. - pub order: ChunkOrder, - - /// The id of this chunk - pub id: ChunkId, - - /// How is this chunk stored? - pub storage: ChunkStorage, - - /// Is there any outstanding lifecycle action for this chunk? - pub lifecycle_action: Option, - - /// The number of bytes used to store this chunk in memory - pub memory_bytes: usize, - - /// The number of bytes used to store this chunk in object storage - pub object_store_bytes: usize, - - /// The total number of rows in this chunk - pub row_count: usize, - - /// The time at which the chunk data was accessed, by a query or a write - pub time_of_last_access: Option