Merge branch 'main' into dom/object-store-metrics

pull/24376/head
kodiakhq[bot] 2022-03-15 14:27:19 +00:00 committed by GitHub
commit f4cd5c9ba8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 3569 additions and 581 deletions

View File

@ -16,6 +16,7 @@
# Will generate the deployment JSON file and output it to stdout. If
# IMAGE_PROMOTION_COMMAND is set in the environment, the deployment JSON file is
# piped to it at the end of execution.
#
set -euo pipefail

83
Cargo.lock generated
View File

@ -738,23 +738,31 @@ dependencies = [
name = "compactor"
version = "0.1.0"
dependencies = [
"arrow",
"arrow_util",
"async-trait",
"backoff 0.1.0",
"data_types2",
"datafusion 0.1.0",
"futures",
"iox_catalog",
"iox_object_store",
"iox_tests",
"metric",
"object_store",
"observability_deps",
"parking_lot 0.12.0",
"parquet_file",
"predicate",
"query",
"schema",
"snafu",
"test_helpers",
"thiserror",
"time 0.1.0",
"tokio",
"tokio-util 0.7.0",
"uuid",
"workspace-hack",
]
@ -1084,7 +1092,7 @@ dependencies = [
[[package]]
name = "datafusion"
version = "7.0.0"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2"
dependencies = [
"ahash",
"arrow",
@ -1114,7 +1122,7 @@ dependencies = [
[[package]]
name = "datafusion-common"
version = "7.0.0"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2"
dependencies = [
"arrow",
"ordered-float 2.10.0",
@ -1125,7 +1133,7 @@ dependencies = [
[[package]]
name = "datafusion-expr"
version = "7.0.0"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2"
dependencies = [
"ahash",
"arrow",
@ -1136,7 +1144,7 @@ dependencies = [
[[package]]
name = "datafusion-physical-expr"
version = "7.0.0"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=0e440eaa5ca52c0261e88590b846808539198241#0e440eaa5ca52c0261e88590b846808539198241"
source = "git+https://github.com/apache/arrow-datafusion.git?rev=ca952bd33402816dbb1550debb9b8cac3b13e8f2#ca952bd33402816dbb1550debb9b8cac3b13e8f2"
dependencies = [
"ahash",
"arrow",
@ -2712,7 +2720,7 @@ dependencies = [
"pem 1.0.2",
"pin-project",
"rustls 0.20.4",
"rustls-pemfile",
"rustls-pemfile 0.2.1",
"secrecy",
"serde",
"serde_json",
@ -2858,9 +2866,9 @@ dependencies = [
[[package]]
name = "libc"
version = "0.2.119"
version = "0.2.120"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1bf2e165bb3457c8e098ea76f3e3bc9db55f87aa90d52d0e6be741470916aaa4"
checksum = "ad5c14e80759d0939d013e6ca49930e59fc53dd8e5009132f76240c179380c09"
[[package]]
name = "libloading"
@ -3241,13 +3249,12 @@ checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb"
[[package]]
name = "nom"
version = "7.1.0"
version = "7.1.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1b1d11e1ef389c76fe5b81bcaf2ea32cf88b62bc494e19f493d0b30e7a930109"
checksum = "a8903e5a29a317527874d0402f867152a3d21c908bb0b933e416c65e301d4c36"
dependencies = [
"memchr",
"minimal-lexical",
"version_check",
]
[[package]]
@ -4016,9 +4023,9 @@ dependencies = [
[[package]]
name = "pretty_assertions"
version = "1.1.0"
version = "1.2.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "76d5b548b725018ab5496482b45cb8bef21e9fed1858a6d674e3a8a0f0bb5d50"
checksum = "57c038cb5319b9c704bf9c227c261d275bfec0ad438118a2787ce47944fb228b"
dependencies = [
"ansi_term",
"ctor",
@ -4073,6 +4080,23 @@ dependencies = [
"thiserror",
]
[[package]]
name = "proptest"
version = "1.0.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1e0d9cc07f18492d879586c92b485def06bc850da3118075cd45d50e9c95b0e5"
dependencies = [
"bitflags",
"byteorder",
"lazy_static",
"num-traits",
"quick-error",
"rand",
"rand_chacha",
"rand_xorshift",
"regex-syntax",
]
[[package]]
name = "prost"
version = "0.9.0"
@ -4155,6 +4179,7 @@ dependencies = [
"parquet_file",
"pin-project",
"predicate",
"proptest",
"prost",
"query",
"rand",
@ -4307,6 +4332,15 @@ dependencies = [
"rand_core",
]
[[package]]
name = "rand_xorshift"
version = "0.3.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "d25bf25ec5ae4a3f1b92f929810509a2f53d7dca2f50b794ff57e3face536c8f"
dependencies = [
"rand_core",
]
[[package]]
name = "rayon"
version = "1.5.1"
@ -4414,9 +4448,9 @@ dependencies = [
[[package]]
name = "reqwest"
version = "0.11.9"
version = "0.11.10"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "87f242f1488a539a79bac6dbe7c8609ae43b7914b7736210f239a37cccb32525"
checksum = "46a1f7aa4f35e5e8b4160449f51afc758f0ce6454315a9fa7d0d113e958c41eb"
dependencies = [
"base64 0.13.0",
"bytes",
@ -4436,7 +4470,7 @@ dependencies = [
"percent-encoding",
"pin-project-lite",
"rustls 0.20.4",
"rustls-pemfile",
"rustls-pemfile 0.3.0",
"serde",
"serde_json",
"serde_urlencoded",
@ -4722,7 +4756,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "5ca9ebdfa27d3fc180e42879037b5338ab1c040c06affd00d8338598e7800943"
dependencies = [
"openssl-probe",
"rustls-pemfile",
"rustls-pemfile 0.2.1",
"schannel",
"security-framework",
]
@ -4736,6 +4770,15 @@ dependencies = [
"base64 0.13.0",
]
[[package]]
name = "rustls-pemfile"
version = "0.3.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1ee86d63972a7c661d1536fefe8c3c8407321c3df668891286de28abcd087360"
dependencies = [
"base64 0.13.0",
]
[[package]]
name = "rustversion"
version = "1.0.6"
@ -5456,9 +5499,9 @@ dependencies = [
[[package]]
name = "syn"
version = "1.0.86"
version = "1.0.88"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8a65b3f4ffa0092e9887669db0eae07941f023991ab58ea44da8fe8e2d511c6b"
checksum = "ebd69e719f31e88618baa1eaa6ee2de5c9a1c004f1e9ecdb58e8352a13f20a01"
dependencies = [
"proc-macro2",
"quote",
@ -6463,9 +6506,9 @@ checksum = "504a2476202769977a040c6364301a3f65d0cc9e3fb08600b2bda150a0488316"
[[package]]
name = "winreg"
version = "0.7.0"
version = "0.10.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "0120db82e8a1e0b9fb3345a539c478767c0048d842860994d96113d5b667bd69"
checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d"
dependencies = [
"winapi",
]

View File

@ -0,0 +1,65 @@
/// CLI config for catalog ingest lifecycle
#[derive(Debug, Clone, clap::Parser)]
pub struct IngesterConfig {
/// Write buffer partition number to start (inclusive) range with
#[clap(
long = "--write-buffer-partition-range-start",
env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_START"
)]
pub write_buffer_partition_range_start: i32,
/// Write buffer partition number to end (inclusive) range with
#[clap(
long = "--write-buffer-partition-range-end",
env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_END"
)]
pub write_buffer_partition_range_end: i32,
/// The ingester will continue to pull data and buffer it from Kafka
/// as long as it is below this size. If it hits this size it will pause
/// ingest from Kafka until persistence goes below this threshold.
#[clap(
long = "--pause-ingest-size-bytes",
env = "INFLUXDB_IOX_PAUSE_INGEST_SIZE_BYTES"
)]
pub pause_ingest_size_bytes: usize,
/// Once the ingester crosses this threshold of data buffered across
/// all sequencers, it will pick the largest partitions and persist
/// them until it falls below this threshold. An ingester running in
/// a steady state is expected to take up this much memory.
#[clap(
long = "--persist-memory-threshold-bytes",
env = "INFLUXDB_IOX_PERSIST_MEMORY_THRESHOLD_BYTES"
)]
pub persist_memory_threshold_bytes: usize,
/// If an individual partition crosses this size threshold, it will be persisted.
/// The default value is 300MB (in bytes).
#[clap(
long = "--persist-partition-size-threshold-bytes",
env = "INFLUXDB_IOX_PERSIST_PARTITION_SIZE_THRESHOLD_BYTES",
default_value = "314572800"
)]
pub persist_partition_size_threshold_bytes: usize,
/// If a partition has had data buffered for longer than this period of time
/// it will be persisted. This puts an upper bound on how far back the
/// ingester may need to read in Kafka on restart or recovery. The default value
/// is 30 minutes (in seconds).
#[clap(
long = "--persist-partition-age-threshold-seconds",
env = "INFLUXDB_IOX_PERSIST_PARTITION_AGE_THRESHOLD_SECONDS",
default_value = "1800"
)]
pub persist_partition_age_threshold_seconds: u64,
/// If a partition has had data buffered and hasn't received a write for this
/// period of time, it will be persisted. The default value is 300 seconds (5 minutes).
#[clap(
long = "--persist-partition-cold-threshold-seconds",
env = "INFLUXDB_IOX_PERSIST_PARTITION_COLD_THRESHOLD_SECONDS",
default_value = "300"
)]
pub persist_partition_cold_threshold_seconds: u64,
}

View File

@ -2,6 +2,7 @@
//!
//! They can easily be re-used using `#[clap(flatten)]`.
pub mod catalog_dsn;
pub mod ingester;
pub mod object_store;
pub mod run_config;
pub mod server_id;

View File

@ -5,23 +5,31 @@ authors = ["Luke Bond <luke.n.bond@gmail.com>"]
edition = "2021"
[dependencies]
arrow = { version = "10.0.0", features = ["prettyprint"] }
async-trait = "0.1.42"
backoff = { path = "../backoff" }
data_types2 = { path = "../data_types2" }
datafusion = { path = "../datafusion" }
futures = "0.3"
iox_catalog = { path = "../iox_catalog" }
iox_object_store = { path = "../iox_object_store" }
metric = { path = "../metric" }
object_store = { path = "../object_store" }
observability_deps = { path = "../observability_deps" }
parking_lot = "0.12"
parquet_file = { path = "../parquet_file" }
predicate = { path = "../predicate" }
query = { path = "../query" }
schema = { path = "../schema" }
snafu = "0.7"
thiserror = "1.0"
time = { path = "../time" }
tokio = { version = "1.17", features = ["macros", "parking_lot", "rt-multi-thread", "sync", "time"] }
tokio-util = { version = "0.7.0" }
uuid = { version = "0.8", features = ["v4"] }
workspace-hack = { path = "../workspace-hack"}
[dev-dependencies]
arrow_util = { path = "../arrow_util" }
iox_tests = { path = "../iox_tests" }
test_helpers = { path = "../test_helpers" }

View File

@ -1,18 +1,33 @@
//! Data Points for the lifecycle of the Compactor
use crate::{
query::QueryableParquetChunk,
utils::{CompactedData, ParquetFileWithTombstone},
};
use backoff::BackoffConfig;
use data_types2::{
ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, TablePartition, Timestamp,
TombstoneId,
};
use datafusion::error::DataFusionError;
use iox_catalog::interface::Catalog;
use object_store::ObjectStore;
use observability_deps::tracing::warn;
use parquet_file::metadata::IoxMetadata;
use query::{
compute_sort_key_for_chunks,
exec::{Executor, ExecutorType},
frontend::reorg::ReorgPlanner,
util::compute_timenanosecond_min_max,
};
use snafu::{ensure, ResultExt, Snafu};
use std::{
cmp::{max, min},
collections::{BTreeMap, HashSet},
sync::Arc,
};
use backoff::BackoffConfig;
use data_types2::{
ParquetFile, ParquetFileId, PartitionId, SequencerId, TableId, Tombstone, TombstoneId,
};
use iox_catalog::interface::Catalog;
use object_store::ObjectStore;
use query::exec::Executor;
use snafu::Snafu;
use time::{Time, TimeProvider};
use uuid::Uuid;
/// 24 hours in nanoseconds
// TODO: make this a config parameter
@ -26,6 +41,75 @@ pub enum Error {
sequencer_id
))]
SequencerNotFound { sequencer_id: SequencerId },
#[snafu(display(
"The given parquet files are not in the same partition ({}, {}, {}), ({}, {}, {})",
sequencer_id_1,
table_id_1,
partition_id_1,
sequencer_id_2,
table_id_2,
partition_id_2
))]
NotSamePartition {
sequencer_id_1: SequencerId,
table_id_1: TableId,
partition_id_1: PartitionId,
sequencer_id_2: SequencerId,
table_id_2: TableId,
partition_id_2: PartitionId,
},
#[snafu(display(
"Cannot compact parquet files for table ID {} due to an internal error: {}",
table_id,
source
))]
TableNotFound {
source: iox_catalog::interface::Error,
table_id: TableId,
},
#[snafu(display(
"Cannot compact parquet files for an non-existing table ID {}",
table_id
))]
TableNotExist { table_id: TableId },
#[snafu(display("Error building compact logical plan {}", source))]
CompactLogicalPlan {
source: query::frontend::reorg::Error,
},
#[snafu(display("Error building compact physical plan {}", source))]
CompactPhysicalPlan { source: DataFusionError },
#[snafu(display("Error executing compact plan {}", source))]
ExecuteCompactPlan { source: DataFusionError },
#[snafu(display("Error collecting stream yto record batches {}", source))]
CollectStream { source: DataFusionError },
#[snafu(display("Could not convert row count to i64"))]
RowCountTypeConversion { source: std::num::TryFromIntError },
#[snafu(display("Error computing min and max for record batches: {}", source))]
MinMax { source: query::util::Error },
#[snafu(display("Error while requesting level 0 parquet files {}", source))]
Level0 {
source: iox_catalog::interface::Error,
},
#[snafu(display("Error while requesting level 1 parquet files {}", source))]
Level1 {
source: iox_catalog::interface::Error,
},
#[snafu(display("Error updating catalog {}", source))]
Update {
source: iox_catalog::interface::Error,
},
}
/// A specialized `Error` for Compactor Data errors
@ -42,7 +126,9 @@ pub struct Compactor {
catalog: Arc<dyn Catalog>,
/// Executor for running queries and compacting and persisting
exec: Executor,
exec: Arc<Executor>,
/// Time provider for all activities in this compactor
pub time_provider: Arc<dyn TimeProvider>,
/// Backoff config
backoff_config: BackoffConfig,
@ -54,7 +140,8 @@ impl Compactor {
sequencers: Vec<SequencerId>,
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
exec: Executor,
exec: Arc<Executor>,
time_provider: Arc<dyn TimeProvider>,
backoff_config: BackoffConfig,
) -> Self {
Self {
@ -62,10 +149,59 @@ impl Compactor {
catalog,
object_store,
exec,
time_provider,
backoff_config,
}
}
async fn level_0_parquet_files(&self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>> {
let mut repos = self.catalog.repositories().await;
repos
.parquet_files()
.level_0(sequencer_id)
.await
.context(Level0Snafu)
}
async fn level_1_parquet_files(
&self,
table_partition: TablePartition,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<Vec<ParquetFile>> {
let mut repos = self.catalog.repositories().await;
repos
.parquet_files()
.level_1(table_partition, min_time, max_time)
.await
.context(Level1Snafu)
}
async fn update_to_level_1(&self, parquet_file_ids: &[ParquetFileId]) -> Result<()> {
let mut repos = self.catalog.repositories().await;
let updated = repos
.parquet_files()
.update_to_level_1(parquet_file_ids)
.await
.context(UpdateSnafu)?;
if updated.len() < parquet_file_ids.len() {
let parquet_file_ids: HashSet<_> = parquet_file_ids.iter().collect();
let updated: HashSet<_> = updated.iter().collect();
let not_updated = parquet_file_ids.difference(&updated);
warn!(
"Unable to update to level 1 parquet files with IDs: {:?}",
not_updated
);
}
Ok(())
}
// TODO: this function should be invoked in a backround loop
/// Find and compact parquet files for a given sequencer
pub async fn find_and_compact(&self, sequencer_id: SequencerId) -> Result<()> {
@ -74,14 +210,31 @@ impl Compactor {
}
// Read level-0 parquet files
let level_0_files: Vec<Arc<ParquetFile>> = vec![]; // TODO: #3946
let level_0_files = self.level_0_parquet_files(sequencer_id).await?;
// If there are no level-0 parquet files, return because there's nothing to do
if level_0_files.is_empty() {
return Ok(());
}
// Group files into table partition
let mut partitions = Self::group_parquet_files_into_partition(level_0_files);
// Get level-1 files overlapped with level-0
for (_key, val) in &mut partitions.iter_mut() {
let level_1_files: Vec<Arc<ParquetFile>> = vec![]; // TODO: #3946
// Get level-1 files overlapped in time with level-0
for (key, val) in &mut partitions.iter_mut() {
let overall_min_time = val
.iter()
.map(|pf| pf.min_time)
.min()
.expect("The list of files was checked for emptiness above");
let overall_max_time = val
.iter()
.map(|pf| pf.max_time)
.max()
.expect("The list of files was checked for emptiness above");
let level_1_files = self
.level_1_parquet_files(*key, overall_min_time, overall_max_time)
.await?;
val.extend(level_1_files);
}
@ -89,7 +242,7 @@ impl Compactor {
// groups overlapped files in each partition
let mut overlapped_file_groups = vec![];
for _val in partitions.values_mut() {
let overlapped_files: Vec<Vec<Arc<ParquetFile>>> = vec![]; // TODO: #3949
let overlapped_files: Vec<Vec<ParquetFile>> = vec![]; // TODO: #3949
overlapped_file_groups.extend(overlapped_files);
}
@ -107,7 +260,7 @@ impl Compactor {
// keep tombstone ids
tombstones = Self::union_tombstone_ids(tombstones, &overlapped_files);
// Only one file, no need to compact
// Only one file without tombstones, no need to compact
if overlapped_files.len() == 1 && overlapped_files[0].no_tombstones() {
// If the file is old enough, it would not have any overlaps. Add it
// to the list to be upgraded to level 1
@ -118,7 +271,10 @@ impl Compactor {
}
// compact
let output_parquet_files: Vec<ParquetFile> = vec![]; // TODO: #3907
let _compacted_data = self.compact(overlapped_files).await;
// split the compacted data into 2 files 90/10
let output_parquet_files: Vec<ParquetFile> = vec![]; // TODO: #3999
for _file in output_parquet_files {
// persist the file
@ -133,16 +289,16 @@ impl Compactor {
// TODO: #3953 - remove_fully_processed_tombstones(tombstones)
// Upgrade old level-0 to level 1
// TODO: #3950 - update_to_level_1(upgrade_level_list)
self.update_to_level_1(&upgrade_level_list).await?;
Ok(())
}
// Group given parquet files into parittion of the same (sequencer_id, table_id, partition_id)
// Group given parquet files into partition of the same (sequencer_id, table_id, partition_id)
fn group_parquet_files_into_partition(
parquet_files: Vec<Arc<ParquetFile>>,
) -> BTreeMap<TablePartition, Vec<Arc<ParquetFile>>> {
let mut groups: BTreeMap<TablePartition, Vec<Arc<ParquetFile>>> = BTreeMap::default();
parquet_files: Vec<ParquetFile>,
) -> BTreeMap<TablePartition, Vec<ParquetFile>> {
let mut groups: BTreeMap<TablePartition, Vec<ParquetFile>> = BTreeMap::default();
for file in parquet_files {
let key = TablePartition::new(file.sequencer_id, file.table_id, file.partition_id);
if let Some(val) = groups.get_mut(&key) {
@ -167,47 +323,412 @@ impl Compactor {
}
tombstones
}
// Compact given files. Assume the given files are overlaped in time.
// If the assumption does not meet, we will spend time not to compact anything but put data
// together
async fn compact(
&self,
overlapped_files: Vec<ParquetFileWithTombstone>,
) -> Result<Option<CompactedData>> {
// Nothing to compact
if overlapped_files.is_empty() {
return Ok(None);
}
#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)]
struct TablePartition {
sequencer_id: SequencerId,
table_id: TableId,
partition_id: PartitionId,
// One file without tombstone, no need to compact
if overlapped_files.len() == 1 && overlapped_files[0].tombstones.is_empty() {
return Ok(None);
}
impl TablePartition {
fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self {
Self {
sequencer_id,
table_id,
partition_id,
// Keep the fist IoxMetadata to reuse same IDs and names
let iox_metadata = overlapped_files[0].iox_metadata();
// Verify if the given files belong to the same partition
// Note: we can ignore this verification if we assume this is a must-have condition
if let Some((head, tail)) = overlapped_files.split_first() {
for file in tail {
let is_same = file.data.sequencer_id == head.data.sequencer_id
&& file.data.table_id == head.data.table_id
&& file.data.partition_id == head.data.partition_id;
ensure!(
is_same,
NotSamePartitionSnafu {
sequencer_id_1: head.data.sequencer_id,
table_id_1: head.data.table_id,
partition_id_1: head.data.partition_id,
sequencer_id_2: file.data.sequencer_id,
table_id_2: file.data.table_id,
partition_id_2: file.data.partition_id
}
)
}
}
struct ParquetFileWithTombstone {
data: Arc<ParquetFile>,
tombstones: Vec<Tombstone>,
// Convert the input files into QueryableParquetChunk for making query plan
let query_chunks: Vec<_> = overlapped_files
.iter()
.map(|f| {
f.to_queryable_parquet_chunk(
Arc::clone(&self.object_store),
iox_metadata.table_name.to_string(),
iox_metadata.partition_key.to_string(),
)
})
.collect();
// Compute min & max sequence numbers
// unwrap here will work becasue the len of the query_chunks already >= 1
let (head, tail) = query_chunks.split_first().unwrap();
let mut min_sequence_number = head.min_sequence_number();
let mut max_sequence_number = head.max_sequence_number();
for c in tail {
min_sequence_number = min(min_sequence_number, c.min_sequence_number());
max_sequence_number = max(max_sequence_number, c.max_sequence_number());
}
impl ParquetFileWithTombstone {
fn tombstone_ids(&self) -> HashSet<TombstoneId> {
self.tombstones.iter().map(|t| t.id).collect::<HashSet<_>>()
// Merge schema of the compacting chunks
let merged_schema = QueryableParquetChunk::merge_schemas(&query_chunks);
// Compute the sorted output of the compacting result
let sort_key = compute_sort_key_for_chunks(&merged_schema, &query_chunks);
// Build compact query plan
let plan = ReorgPlanner::new()
.compact_plan(
Arc::clone(&merged_schema),
query_chunks.into_iter().map(Arc::new),
sort_key.clone(),
)
.context(CompactLogicalPlanSnafu)?;
let ctx = self.exec.new_context(ExecutorType::Reorg);
let physical_plan = ctx
.prepare_plan(&plan)
.await
.context(CompactPhysicalPlanSnafu)?;
// Run the plan
let stream = ctx
.execute_stream(physical_plan)
.await
.context(ExecuteCompactPlanSnafu)?;
// Collect compacted data into record batches for computing statistics
let output_batches = datafusion::physical_plan::common::collect(stream)
.await
.context(CollectStreamSnafu)?;
// Filter empty record batches
let output_batches: Vec<_> = output_batches
.into_iter()
.filter(|b| b.num_rows() != 0)
.collect();
let row_count: usize = output_batches.iter().map(|b| b.num_rows()).sum();
let row_count = row_count.try_into().context(RowCountTypeConversionSnafu)?;
if row_count == 0 {
return Ok(None);
}
fn no_tombstones(&self) -> bool {
self.tombstones.is_empty()
// Compute min and max of the `time` column
let (min_time, max_time) =
compute_timenanosecond_min_max(&output_batches).context(MinMaxSnafu)?;
let meta = IoxMetadata {
object_store_id: Uuid::new_v4(),
creation_timestamp: self.time_provider.now(),
sequencer_id: iox_metadata.sequencer_id,
namespace_id: iox_metadata.namespace_id,
namespace_name: iox_metadata.namespace_name,
table_id: iox_metadata.table_id,
table_name: iox_metadata.table_name,
partition_id: iox_metadata.partition_id,
partition_key: iox_metadata.partition_key,
time_of_first_write: Time::from_timestamp_nanos(min_time),
time_of_last_write: Time::from_timestamp_nanos(max_time),
min_sequence_number,
max_sequence_number,
row_count,
};
let compacted_data = CompactedData::new(output_batches, meta);
Ok(Some(compacted_data))
}
}
// Check if the parquet file is old enough to upgarde its level
fn level_upgradable(&self) -> bool {
// TODO: need to wait for creation_time added
// if time_provider.now() - self.data.creation_time > LEVEL_UPGRADE_THRESHOLD_NANO
true
#[cfg(test)]
mod tests {
use super::*;
use arrow_util::assert_batches_sorted_eq;
use iox_tests::util::TestCatalog;
use time::SystemProvider;
#[tokio::test]
async fn test_compact_one_file() {
let catalog = TestCatalog::new();
let lp = vec![
"table,tag1=WA field_int=1000 8000",
"table,tag1=VT field_int=10 10000",
"table,tag1=UT field_int=70 20000",
]
.join("\n");
let ns = catalog.create_namespace("ns").await;
let sequencer = ns.create_sequencer(1).await;
let table = ns.create_table("table").await;
let parquet_file = table
.with_sequencer(&sequencer)
.create_partition("part")
.await
.create_parquet_file(&lp)
.await
.parquet_file
.clone();
let compactor = Compactor {
sequencers: vec![sequencer.sequencer.id],
object_store: Arc::clone(&catalog.object_store),
catalog: Arc::clone(&catalog.catalog),
exec: Arc::new(Executor::new(1)),
time_provider: Arc::new(SystemProvider::new()),
backoff_config: BackoffConfig::default(),
};
// ------------------------------------------------
// no files provided
let result = compactor.compact(vec![]).await.unwrap();
assert!(result.is_none());
// ------------------------------------------------
// File without tombstones
let mut pf = ParquetFileWithTombstone {
data: Arc::new(parquet_file),
tombstones: vec![],
};
// Nothing compacted for one file without tombstones
let result = compactor.compact(vec![pf.clone()]).await.unwrap();
assert!(result.is_none());
// ------------------------------------------------
// Let add a tombstone
let tombstone = table
.with_sequencer(&sequencer)
.create_tombstone(20, 6000, 12000, "tag1=VT")
.await;
pf.add_tombstones(vec![tombstone.tombstone.clone()]);
// should have compacted data
let batches = compactor.compact(vec![pf]).await.unwrap().unwrap().data;
// one row tag1=VT was removed
assert_batches_sorted_eq!(
&[
"+-----------+------+-----------------------------+",
"| field_int | tag1 | time |",
"+-----------+------+-----------------------------+",
"| 70 | UT | 1970-01-01T00:00:00.000020Z |",
"| 1000 | WA | 1970-01-01T00:00:00.000008Z |",
"+-----------+------+-----------------------------+",
],
&batches
);
}
fn parquet_file_id(&self) -> ParquetFileId {
self.data.id
#[tokio::test]
async fn test_compact_two_files() {
let catalog = TestCatalog::new();
let lp1 = vec![
"table,tag1=WA field_int=1000 8000", // will be eliminated due to duplicate
"table,tag1=VT field_int=10 10000", // will be deleted
"table,tag1=UT field_int=70 20000",
]
.join("\n");
let lp2 = vec![
"table,tag1=WA field_int=1500 8000", // latest duplicate and kept
"table,tag1=VT field_int=10 6000",
"table,tag1=UT field_int=270 25000",
]
.join("\n");
let ns = catalog.create_namespace("ns").await;
let sequencer = ns.create_sequencer(1).await;
let table = ns.create_table("table").await;
let partition = table
.with_sequencer(&sequencer)
.create_partition("part")
.await;
let parquet_file1 = partition
.create_parquet_file_with_sequence_numbers(&lp1, 1, 5)
.await
.parquet_file
.clone();
let parquet_file2 = partition
.create_parquet_file_with_sequence_numbers(&lp2, 10, 15)
.await
.parquet_file
.clone();
let compactor = Compactor {
sequencers: vec![sequencer.sequencer.id],
object_store: Arc::clone(&catalog.object_store),
catalog: Arc::clone(&catalog.catalog),
exec: Arc::new(Executor::new(1)),
time_provider: Arc::new(SystemProvider::new()),
backoff_config: BackoffConfig::default(),
};
// File 1 with tombstone
let tombstone = table
.with_sequencer(&sequencer)
.create_tombstone(6, 6000, 12000, "tag1=VT")
.await;
let pf1 = ParquetFileWithTombstone {
data: Arc::new(parquet_file1),
tombstones: vec![tombstone.tombstone.clone()],
};
// File 2 without tombstones
let pf2 = ParquetFileWithTombstone {
data: Arc::new(parquet_file2),
tombstones: vec![],
};
// Compact them
let batches = compactor
.compact(vec![pf1, pf2])
.await
.unwrap()
.unwrap()
.data;
// Should have 4 rows left
assert_batches_sorted_eq!(
&[
"+-----------+------+-----------------------------+",
"| field_int | tag1 | time |",
"+-----------+------+-----------------------------+",
"| 10 | VT | 1970-01-01T00:00:00.000006Z |",
"| 1500 | WA | 1970-01-01T00:00:00.000008Z |",
"| 270 | UT | 1970-01-01T00:00:00.000025Z |",
"| 70 | UT | 1970-01-01T00:00:00.000020Z |",
"+-----------+------+-----------------------------+",
],
&batches
);
}
#[tokio::test]
async fn test_compact_three_files_different_cols() {
let catalog = TestCatalog::new();
let lp1 = vec![
"table,tag1=WA field_int=1000 8000", // will be eliminated due to duplicate
"table,tag1=VT field_int=10 10000", // will be deleted
"table,tag1=UT field_int=70 20000",
]
.join("\n");
let lp2 = vec![
"table,tag1=WA field_int=1500 8000", // latest duplicate and kept
"table,tag1=VT field_int=10 6000",
"table,tag1=UT field_int=270 25000",
]
.join("\n");
let lp3 = vec![
"table,tag2=WA,tag3=10 field_int=1500 8000",
"table,tag2=VT,tag3=20 field_int=10 6000",
]
.join("\n");
let ns = catalog.create_namespace("ns").await;
let sequencer = ns.create_sequencer(1).await;
let table = ns.create_table("table").await;
let partition = table
.with_sequencer(&sequencer)
.create_partition("part")
.await;
// Sequence numbers are important here.
// Time/sequence order from small to large: parquet_file_1, parquet_file_2, parquet_file_3
let parquet_file1 = partition
.create_parquet_file_with_sequence_numbers(&lp1, 1, 5)
.await
.parquet_file
.clone();
let parquet_file2 = partition
.create_parquet_file_with_sequence_numbers(&lp2, 10, 15)
.await
.parquet_file
.clone();
let parquet_file3 = partition
.create_parquet_file_with_sequence_numbers(&lp3, 20, 25)
.await
.parquet_file
.clone();
let compactor = Compactor {
sequencers: vec![sequencer.sequencer.id],
object_store: Arc::clone(&catalog.object_store),
catalog: Arc::clone(&catalog.catalog),
exec: Arc::new(Executor::new(1)),
time_provider: Arc::new(SystemProvider::new()),
backoff_config: BackoffConfig::default(),
};
// File 1 with tombstone
let tombstone = table
.with_sequencer(&sequencer)
.create_tombstone(6, 6000, 12000, "tag1=VT")
.await;
let pf1 = ParquetFileWithTombstone {
data: Arc::new(parquet_file1),
tombstones: vec![tombstone.tombstone.clone()],
};
// File 2 without tombstones
let pf2 = ParquetFileWithTombstone {
data: Arc::new(parquet_file2),
tombstones: vec![],
};
// File 3 without tombstones
let pf3 = ParquetFileWithTombstone {
data: Arc::new(parquet_file3),
tombstones: vec![],
};
// Compact them
let batches = compactor
.compact(vec![pf1.clone(), pf2.clone(), pf3.clone()])
.await
.unwrap()
.unwrap()
.data;
// Should have 6 rows
let expected = vec![
"+-----------+------+------+------+-----------------------------+",
"| field_int | tag1 | tag2 | tag3 | time |",
"+-----------+------+------+------+-----------------------------+",
"| 10 | | VT | 20 | 1970-01-01T00:00:00.000006Z |",
"| 10 | VT | | | 1970-01-01T00:00:00.000006Z |",
"| 1500 | | WA | 10 | 1970-01-01T00:00:00.000008Z |",
"| 1500 | WA | | | 1970-01-01T00:00:00.000008Z |",
"| 270 | UT | | | 1970-01-01T00:00:00.000025Z |",
"| 70 | UT | | | 1970-01-01T00:00:00.000020Z |",
"+-----------+------+------+------+-----------------------------+",
];
assert_batches_sorted_eq!(&expected, &batches);
// Make a vector of different file order but the result is still the same
// becasue the actual order for deduplication does not rely on their order in the vector
// Compact them
let batches = compactor
.compact(vec![pf2, pf3, pf1]) // different order in the vector
.await
.unwrap()
.unwrap()
.data;
// Should have 6 rows
assert_batches_sorted_eq!(&expected, &batches);
}
}

View File

@ -14,6 +14,7 @@ use observability_deps::tracing::warn;
use query::exec::Executor;
use std::sync::Arc;
use thiserror::Error;
use time::TimeProvider;
use tokio::task::{JoinError, JoinHandle};
use tokio_util::sync::CancellationToken;
@ -62,7 +63,8 @@ impl CompactorHandlerImpl {
sequencers: Vec<SequencerId>,
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
exec: Executor,
exec: Arc<Executor>,
time_provider: Arc<dyn TimeProvider>,
_registry: &metric::Registry,
) -> Self {
let shutdown = CancellationToken::new();
@ -74,6 +76,7 @@ impl CompactorHandlerImpl {
catalog,
object_store,
exec,
time_provider,
BackoffConfig::default(),
));

View File

@ -14,4 +14,6 @@
pub mod compact;
pub mod handler;
pub mod query;
pub mod server;
pub mod utils;

216
compactor/src/query.rs Normal file
View File

@ -0,0 +1,216 @@
//! Queryable Compactor Data
use std::sync::Arc;
use data_types2::{
tombstones_to_delete_predicates, ChunkAddr, ChunkId, ChunkOrder, DeletePredicate,
SequenceNumber, TableSummary, Tombstone,
};
use datafusion::physical_plan::SendableRecordBatchStream;
use observability_deps::tracing::trace;
use parquet_file::{chunk::ParquetChunk, metadata::IoxMetadata};
use predicate::{Predicate, PredicateMatch};
use query::{
exec::{stringset::StringSet, IOxExecutionContext},
QueryChunk, QueryChunkMeta,
};
use schema::{merge::SchemaMerger, selection::Selection, sort::SortKey, Schema};
use snafu::{ResultExt, Snafu};
#[derive(Debug, Snafu)]
#[allow(missing_copy_implementations, missing_docs)]
pub enum Error {
#[snafu(display("Failed to read parquet: {}", source))]
ReadParquet { source: parquet_file::chunk::Error },
#[snafu(display(
"Error reading IOx Metadata from Parquet IoxParquetMetadata: {}",
source
))]
ReadParquetMeta {
source: parquet_file::metadata::Error,
},
}
/// A specialized `Error` for Compactor's query errors
pub type Result<T, E = Error> = std::result::Result<T, E>;
/// QueryableParquetChunk that implements QueryChunk and QueryMetaChunk for building query plan
#[derive(Debug, Clone)]
pub struct QueryableParquetChunk {
data: Arc<ParquetChunk>, // data of the parquet file
iox_metadata: Arc<IoxMetadata>, // metadata of the parquet file
delete_predicates: Vec<Arc<DeletePredicate>>, // converted from tombstones
table_name: String, // needed to build query plan
}
impl QueryableParquetChunk {
/// Initialize a QueryableParquetChunk
pub fn new(
table_name: impl Into<String>,
data: Arc<ParquetChunk>,
iox_metadata: Arc<IoxMetadata>,
deletes: &[Tombstone],
) -> Self {
let delete_predicates = tombstones_to_delete_predicates(deletes);
Self {
data,
iox_metadata,
delete_predicates,
table_name: table_name.into(),
}
}
/// Merge schema of the given chunks
pub fn merge_schemas(chunks: &[Self]) -> Arc<Schema> {
let mut merger = SchemaMerger::new();
for chunk in chunks {
merger = merger
.merge(&chunk.data.schema())
.expect("schemas compatible");
}
Arc::new(merger.build())
}
/// Return min sequence number
pub fn min_sequence_number(&self) -> SequenceNumber {
self.iox_metadata.min_sequence_number
}
/// Return max sequence number
pub fn max_sequence_number(&self) -> SequenceNumber {
self.iox_metadata.max_sequence_number
}
}
impl QueryChunkMeta for QueryableParquetChunk {
fn summary(&self) -> Option<&TableSummary> {
None
}
fn schema(&self) -> Arc<Schema> {
self.data.schema()
}
fn sort_key(&self) -> Option<&SortKey> {
None // TODO: return the sortkey when it is available in the parquet file #3968
}
fn delete_predicates(&self) -> &[Arc<DeletePredicate>] {
self.delete_predicates.as_ref()
}
}
impl QueryChunk for QueryableParquetChunk {
type Error = Error;
// Todo: This function should not be used in this NG chunk context
// For now, since we also use scan for both OG and NG, the chunk id
// is used as second key in build_deduplicate_plan_for_overlapped_chunks
// to sort the chunk to deduplicate them correctly.
// Since we make the first key, order, always different, it is fine
// to have the second key the sames and always 0
fn id(&self) -> ChunkId {
// always return id 0 for debugging mode and with reason above
ChunkId::new_test(0)
}
// This function should not be used in this context
fn addr(&self) -> ChunkAddr {
unimplemented!()
}
/// Returns the name of the table stored in this chunk
fn table_name(&self) -> &str {
&self.table_name
}
/// Returns true if the chunk may contain a duplicate "primary
/// key" within itself
fn may_contain_pk_duplicates(&self) -> bool {
// data within this parquet chunk was deduplicated
false
}
/// Returns the result of applying the `predicate` to the chunk
/// using an efficient, but inexact method, based on metadata.
///
/// NOTE: This method is suitable for calling during planning, and
/// may return PredicateMatch::Unknown for certain types of
/// predicates.
fn apply_predicate_to_metadata(
&self,
_predicate: &Predicate,
) -> Result<PredicateMatch, Self::Error> {
Ok(PredicateMatch::Unknown)
}
/// Returns a set of Strings with column names from the specified
/// table that have at least one row that matches `predicate`, if
/// the predicate can be evaluated entirely on the metadata of
/// this Chunk. Returns `None` otherwise
fn column_names(
&self,
_ctx: IOxExecutionContext,
_predicate: &Predicate,
_columns: Selection<'_>,
) -> Result<Option<StringSet>, Self::Error> {
Ok(None)
}
/// Return a set of Strings containing the distinct values in the
/// specified columns. If the predicate can be evaluated entirely
/// on the metadata of this Chunk. Returns `None` otherwise
///
/// The requested columns must all have String type.
fn column_values(
&self,
_ctx: IOxExecutionContext,
_column_name: &str,
_predicate: &Predicate,
) -> Result<Option<StringSet>, Self::Error> {
Ok(None)
}
/// Provides access to raw `QueryChunk` data as an
/// asynchronous stream of `RecordBatch`es filtered by a *required*
/// predicate. Note that not all chunks can evaluate all types of
/// predicates and this function will return an error
/// if requested to evaluate with a predicate that is not supported
///
/// This is the analog of the `TableProvider` in DataFusion
///
/// The reason we can't simply use the `TableProvider` trait
/// directly is that the data for a particular Table lives in
/// several chunks within a partition, so there needs to be an
/// implementation of `TableProvider` that stitches together the
/// streams from several different `QueryChunk`s.
fn read_filter(
&self,
mut ctx: IOxExecutionContext,
predicate: &Predicate,
selection: Selection<'_>,
) -> Result<SendableRecordBatchStream, Self::Error> {
ctx.set_metadata("storage", "compactor");
ctx.set_metadata("projection", format!("{}", selection));
trace!(?selection, "selection");
self.data
.read_filter(predicate, selection)
.context(ReadParquetSnafu)
}
/// Returns chunk type
fn chunk_type(&self) -> &str {
"QueryableParquetChunk"
}
// Order of the chunk so they can be deduplicate correctly
fn order(&self) -> ChunkOrder {
let seq_num = self.iox_metadata.min_sequence_number.get();
let seq_num = u32::try_from(seq_num)
.expect("Sequence number should have been converted to chunk order successfully");
ChunkOrder::new(seq_num)
.expect("Sequence number should have been converted to chunk order successfully")
}
}

94
compactor/src/utils.rs Normal file
View File

@ -0,0 +1,94 @@
//! Helpers of the Compactor
use crate::query::QueryableParquetChunk;
use arrow::record_batch::RecordBatch;
use data_types2::{ParquetFile, ParquetFileId, Tombstone, TombstoneId};
use iox_object_store::IoxObjectStore;
use object_store::ObjectStore;
use parquet_file::{
chunk::{new_parquet_chunk, ChunkMetrics, DecodedParquetFile},
metadata::IoxMetadata,
};
use std::{collections::HashSet, sync::Arc};
/// Wrapper of a parquet file and its tombstones
#[allow(missing_docs)]
#[derive(Debug, Clone)]
pub struct ParquetFileWithTombstone {
pub(crate) data: Arc<ParquetFile>,
pub(crate) tombstones: Vec<Tombstone>,
}
impl ParquetFileWithTombstone {
/// Return all tombstone ids
pub fn tombstone_ids(&self) -> HashSet<TombstoneId> {
self.tombstones.iter().map(|t| t.id).collect()
}
/// Return true if there is no tombstone
pub fn no_tombstones(&self) -> bool {
self.tombstones.is_empty()
}
/// Check if the parquet file is old enough to upgarde its level
pub fn level_upgradable(&self) -> bool {
// TODO: need to wait for creation_time added
// if time_provider.now() - self.data.creation_time > LEVEL_UPGRADE_THRESHOLD_NANO
true
}
/// Return id of this parquet file
pub fn parquet_file_id(&self) -> ParquetFileId {
self.data.id
}
/// Add more tombstones
pub fn add_tombstones(&mut self, tombstones: Vec<Tombstone>) {
self.tombstones.extend(tombstones);
}
/// Convert to a QueryableParquetChunk
pub fn to_queryable_parquet_chunk(
&self,
object_store: Arc<ObjectStore>,
table_name: String,
partition_key: String,
) -> QueryableParquetChunk {
let decoded_parquet_file = DecodedParquetFile::new((*self.data).clone());
let root_path = IoxObjectStore::root_path_for(&object_store, self.data.object_store_id);
let iox_object_store = IoxObjectStore::existing(object_store, root_path);
let parquet_chunk = new_parquet_chunk(
&decoded_parquet_file,
Arc::from(table_name.clone()),
Arc::from(partition_key),
ChunkMetrics::new_unregistered(), // TODO: need to add metrics
Arc::new(iox_object_store),
);
QueryableParquetChunk::new(
table_name,
Arc::new(parquet_chunk),
Arc::new(decoded_parquet_file.iox_metadata),
&self.tombstones,
)
}
/// Return iox metadata of the parquet file
pub fn iox_metadata(&self) -> IoxMetadata {
let decoded_parquet_file = DecodedParquetFile::new((*self.data).clone());
decoded_parquet_file.iox_metadata
}
}
/// Struct holding output of a compacted stream
pub struct CompactedData {
pub(crate) data: Vec<RecordBatch>,
pub(crate) meta: IoxMetadata,
}
impl CompactedData {
/// Initialize compacted data
pub fn new(data: Vec<RecordBatch>, meta: IoxMetadata) -> Self {
Self { data, meta }
}
}

View File

@ -11,12 +11,14 @@
)]
use influxdb_line_protocol::FieldValue;
use predicate::Predicate;
use predicate::{delete_predicate::parse_delete_predicate, Predicate};
use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType, Schema};
use std::{
collections::BTreeMap,
convert::TryFrom,
fmt::{Debug, Formatter},
ops::{Add, Sub},
sync::Arc,
};
use uuid::Uuid;
@ -188,6 +190,29 @@ impl std::fmt::Display for PartitionId {
}
}
/// Combination of Sequencer ID, Table ID, and Partition ID useful for identifying groups of
/// Parquet files to be compacted together.
#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)]
pub struct TablePartition {
/// The sequencer ID
pub sequencer_id: SequencerId,
/// The table ID
pub table_id: TableId,
/// The partition ID
pub partition_id: PartitionId,
}
impl TablePartition {
/// Combine the relevant parts
pub fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self {
Self {
sequencer_id,
table_id,
partition_id,
}
}
}
/// Unique ID for a `Tombstone`
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)]
#[sqlx(transparent)]
@ -233,6 +258,22 @@ impl Timestamp {
}
}
impl Add<i64> for Timestamp {
type Output = Self;
fn add(self, other: i64) -> Self {
Self(self.0 + other)
}
}
impl Sub<i64> for Timestamp {
type Output = Self;
fn sub(self, other: i64) -> Self {
Self(self.0 - other)
}
}
/// Unique ID for a `ParquetFile`
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)]
#[sqlx(transparent)]
@ -619,6 +660,27 @@ pub struct Tombstone {
pub serialized_predicate: String,
}
/// Convert tombstones to delete predicates
pub fn tombstones_to_delete_predicates(tombstones: &[Tombstone]) -> Vec<Arc<DeletePredicate>> {
tombstones_to_delete_predicates_iter(tombstones).collect()
}
/// Return Iterator of delete predicates
pub fn tombstones_to_delete_predicates_iter(
tombstones: &[Tombstone],
) -> impl Iterator<Item = Arc<DeletePredicate>> + '_ {
tombstones.iter().map(|tombstone| {
Arc::new(
parse_delete_predicate(
&tombstone.min_time.get().to_string(),
&tombstone.max_time.get().to_string(),
&tombstone.serialized_predicate,
)
.expect("Error building delete predicate"),
)
})
}
/// Data for a parquet file reference that has been inserted in the catalog.
#[derive(Debug, Clone, PartialEq, sqlx::FromRow)]
pub struct ParquetFile {

View File

@ -9,5 +9,5 @@ description = "Re-exports datafusion at a specific version"
# Rename to workaround doctest bug
# Turn off optional datafusion features (e.g. don't get support for crypo functions or avro)
upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="0e440eaa5ca52c0261e88590b846808539198241", default-features = false, package = "datafusion" }
upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="ca952bd33402816dbb1550debb9b8cac3b13e8f2", default-features = false, package = "datafusion" }
workspace-hack = { path = "../workspace-hack"}

View File

@ -639,7 +639,7 @@ fn from_expr(expr: Expr) -> Result<proto::LogicalExprNode, FieldViolation> {
expr_type: Some(ExprType::AggregateExpr(aggregate_expr)),
})
}
Expr::ScalarVariable(_) => unimplemented!(),
Expr::ScalarVariable(_, _) => unimplemented!(),
Expr::ScalarFunction { fun, args } => {
let fun = from_scalar_function(fun)?;
let args: Vec<proto::LogicalExprNode> = args

View File

@ -1,18 +1,20 @@
//! Implementation of command line option for running the compactor
use compactor::{handler::CompactorHandlerImpl, server::CompactorServer};
use data_types2::SequencerId;
use object_store::ObjectStore;
use observability_deps::tracing::*;
use query::exec::Executor;
use std::sync::Arc;
use thiserror::Error;
use time::SystemProvider;
use clap_blocks::{catalog_dsn::CatalogDsnConfig, run_config::RunConfig};
use influxdb_ioxd::{
self,
server_type::common_state::{CommonServerState, CommonServerStateError},
server_type::compactor::CompactorServerType,
server_type::{
common_state::{CommonServerState, CommonServerStateError},
compactor::create_compactor_server_type,
},
};
#[derive(Debug, Error)]
@ -85,19 +87,22 @@ pub async fn command(config: Config) -> Result<(), Error> {
.map_err(Error::ObjectStoreParsing)?,
);
let exec = Arc::new(Executor::new(config.query_exect_thread_count));
let time_provider = Arc::new(SystemProvider::new());
// TODO: modify config to let us get assigned sequence numbers
let sequencers: Vec<SequencerId> = vec![];
let compactor_handler = Arc::new(CompactorHandlerImpl::new(
sequencers,
let server_type = create_compactor_server_type(
&common_state,
metric_registry,
catalog,
object_store,
Executor::new(config.query_exect_thread_count),
&metric_registry,
));
let compactor = CompactorServer::new(metric_registry, compactor_handler);
let server_type = Arc::new(CompactorServerType::new(compactor, &common_state));
exec,
time_provider,
sequencers,
)
.await;
info!("starting compactor");

View File

@ -1,25 +1,20 @@
//! Implementation of command line option for running ingester
use clap_blocks::{
catalog_dsn::CatalogDsnConfig, run_config::RunConfig, write_buffer::WriteBufferConfig,
catalog_dsn::CatalogDsnConfig, ingester::IngesterConfig, run_config::RunConfig,
write_buffer::WriteBufferConfig,
};
use data_types2::KafkaPartition;
use influxdb_ioxd::{
self,
server_type::{
common_state::{CommonServerState, CommonServerStateError},
ingester::IngesterServerType,
ingester::create_ingester_server_type,
},
};
use ingester::{
handler::IngestHandlerImpl,
lifecycle::LifecycleConfig,
server::{grpc::GrpcDelegate, http::HttpDelegate, IngesterServer},
};
use object_store::ObjectStore;
use observability_deps::tracing::*;
use query::exec::Executor;
use std::{collections::BTreeMap, convert::TryFrom, sync::Arc, time::Duration};
use std::{convert::TryFrom, sync::Arc};
use thiserror::Error;
#[derive(Debug, Error)]
@ -30,26 +25,11 @@ pub enum Error {
#[error("Invalid config: {0}")]
InvalidConfig(#[from] CommonServerStateError),
#[error("Catalog error: {0}")]
Catalog(#[from] iox_catalog::interface::Error),
#[error("Kafka topic {0} not found in the catalog")]
KafkaTopicNotFound(String),
#[error("Cannot parse object store config: {0}")]
ObjectStoreParsing(#[from] clap_blocks::object_store::ParseError),
#[error("kafka_partition_range_start must be <= kafka_partition_range_end")]
KafkaRange,
#[error("error initializing ingester: {0}")]
Ingester(#[from] ingester::handler::Error),
#[error("error initializing write buffer {0}")]
WriteBuffer(#[from] write_buffer::core::WriteBufferError),
#[error("Invalid number of sequencers: {0}")]
NumSequencers(#[from] std::num::TryFromIntError),
Ingester(#[from] influxdb_ioxd::server_type::ingester::Error),
#[error("Catalog DSN error: {0}")]
CatalogDsn(#[from] clap_blocks::catalog_dsn::Error),
@ -81,67 +61,8 @@ pub struct Config {
#[clap(flatten)]
pub(crate) write_buffer_config: WriteBufferConfig,
/// Write buffer partition number to start (inclusive) range with
#[clap(
long = "--write-buffer-partition-range-start",
env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_START"
)]
pub write_buffer_partition_range_start: i32,
/// Write buffer partition number to end (inclusive) range with
#[clap(
long = "--write-buffer-partition-range-end",
env = "INFLUXDB_IOX_WRITE_BUFFER_PARTITION_RANGE_END"
)]
pub write_buffer_partition_range_end: i32,
/// The ingester will continue to pull data and buffer it from Kafka
/// as long as it is below this size. If it hits this size it will pause
/// ingest from Kafka until persistence goes below this threshold.
#[clap(
long = "--pause-ingest-size-bytes",
env = "INFLUXDB_IOX_PAUSE_INGEST_SIZE_BYTES"
)]
pub pause_ingest_size_bytes: usize,
/// Once the ingester crosses this threshold of data buffered across
/// all sequencers, it will pick the largest partitions and persist
/// them until it falls below this threshold. An ingester running in
/// a steady state is expected to take up this much memory.
#[clap(
long = "--persist-memory-threshold-bytes",
env = "INFLUXDB_IOX_PERSIST_MEMORY_THRESHOLD_BYTES"
)]
pub persist_memory_threshold_bytes: usize,
/// If an individual partition crosses this size threshold, it will be persisted.
/// The default value is 300MB (in bytes).
#[clap(
long = "--persist-partition-size-threshold-bytes",
env = "INFLUXDB_IOX_PERSIST_PARTITION_SIZE_THRESHOLD_BYTES",
default_value = "314572800"
)]
pub persist_partition_size_threshold_bytes: usize,
/// If a partition has had data buffered for longer than this period of time
/// it will be persisted. This puts an upper bound on how far back the
/// ingester may need to read in Kafka on restart or recovery. The default value
/// is 30 minutes (in seconds).
#[clap(
long = "--persist-partition-age-threshold-seconds",
env = "INFLUXDB_IOX_PERSIST_PARTITION_AGE_THRESHOLD_SECONDS",
default_value = "1800"
)]
pub persist_partition_age_threshold_seconds: u64,
/// If a partition has had data buffered and hasn't received a write for this
/// period of time, it will be persisted. The default value is 300 seconds (5 minutes).
#[clap(
long = "--persist-partition-cold-threshold-seconds",
env = "INFLUXDB_IOX_PERSIST_PARTITION_COLD_THRESHOLD_SECONDS",
default_value = "300"
)]
pub persist_partition_cold_threshold_seconds: u64,
#[clap(flatten)]
pub(crate) ingester_config: IngesterConfig,
/// Number of threads to use for the ingester query execution, compaction and persistence.
#[clap(
@ -149,7 +70,7 @@ pub struct Config {
env = "INFLUXDB_IOX_QUERY_EXEC_THREAD_COUNT",
default_value = "4"
)]
pub query_exect_thread_count: usize,
pub query_exec_thread_count: usize,
}
pub async fn command(config: Config) -> Result<()> {
@ -161,66 +82,22 @@ pub async fn command(config: Config) -> Result<()> {
.get_catalog("ingester", Arc::clone(&metric_registry))
.await?;
let mut txn = catalog.start_transaction().await?;
let kafka_topic = txn
.kafka_topics()
.get_by_name(config.write_buffer_config.topic())
.await?
.ok_or_else(|| Error::KafkaTopicNotFound(config.write_buffer_config.topic().to_string()))?;
if config.write_buffer_partition_range_start > config.write_buffer_partition_range_end {
return Err(Error::KafkaRange);
}
let kafka_partitions: Vec<_> = (config.write_buffer_partition_range_start
..=config.write_buffer_partition_range_end)
.map(KafkaPartition::new)
.collect();
let object_store = Arc::new(
ObjectStore::try_from(config.run_config.object_store_config())
.map_err(Error::ObjectStoreParsing)?,
);
let mut sequencers = BTreeMap::new();
for k in kafka_partitions {
let s = txn.sequencers().create_or_get(&kafka_topic, k).await?;
sequencers.insert(k, s);
}
txn.commit().await?;
let trace_collector = common_state.trace_collector();
let write_buffer = config
.write_buffer_config
.reading(Arc::clone(&metric_registry), trace_collector.clone())
.await?;
let lifecycle_config = LifecycleConfig::new(
config.pause_ingest_size_bytes,
config.persist_memory_threshold_bytes,
config.persist_partition_size_threshold_bytes,
Duration::from_secs(config.persist_partition_age_threshold_seconds),
Duration::from_secs(config.persist_partition_cold_threshold_seconds),
);
let ingest_handler = Arc::new(
IngestHandlerImpl::new(
lifecycle_config,
kafka_topic,
sequencers,
let exec = Arc::new(Executor::new(config.query_exec_thread_count));
let server_type = create_ingester_server_type(
&common_state,
metric_registry,
catalog,
object_store,
write_buffer,
Executor::new(config.query_exect_thread_count),
Arc::clone(&metric_registry),
exec,
&config.write_buffer_config,
config.ingester_config,
)
.await?,
);
let http = HttpDelegate::new(Arc::clone(&ingest_handler));
let grpc = GrpcDelegate::new(Arc::clone(&ingest_handler));
let ingester = IngesterServer::new(metric_registry, http, grpc, ingest_handler);
let server_type = Arc::new(IngesterServerType::new(ingester, &common_state));
.await?;
info!("starting ingester");

View File

@ -2,7 +2,6 @@
use object_store::ObjectStore;
use observability_deps::tracing::*;
use querier::{database::QuerierDatabase, handler::QuerierHandlerImpl, server::QuerierServer};
use query::exec::Executor;
use std::sync::Arc;
use thiserror::Error;
@ -11,8 +10,10 @@ use time::SystemProvider;
use clap_blocks::{catalog_dsn::CatalogDsnConfig, run_config::RunConfig};
use influxdb_ioxd::{
self,
server_type::common_state::{CommonServerState, CommonServerStateError},
server_type::querier::QuerierServerType,
server_type::{
common_state::{CommonServerState, CommonServerStateError},
querier::create_querier_server_type,
},
};
#[derive(Debug, Error)]
@ -82,17 +83,15 @@ pub async fn command(config: Config) -> Result<(), Error> {
info!(%num_threads, "using specified number of threads per thread pool");
let exec = Arc::new(Executor::new(num_threads));
let database = Arc::new(QuerierDatabase::new(
let server_type = create_querier_server_type(
&common_state,
metric_registry,
catalog,
Arc::clone(&metric_registry),
object_store,
time_provider,
exec,
));
let querier_handler = Arc::new(QuerierHandlerImpl::new(Arc::clone(&database)));
let querier = QuerierServer::new(metric_registry, querier_handler);
let server_type = Arc::new(QuerierServerType::new(querier, database, &common_state));
)
.await;
info!("starting querier");

View File

@ -4,9 +4,17 @@ use std::{
};
use async_trait::async_trait;
use compactor::{handler::CompactorHandler, server::CompactorServer};
use compactor::{
handler::{CompactorHandler, CompactorHandlerImpl},
server::CompactorServer,
};
use data_types2::SequencerId;
use hyper::{Body, Request, Response};
use iox_catalog::interface::Catalog;
use metric::Registry;
use object_store::ObjectStore;
use query::exec::Executor;
use time::TimeProvider;
use tokio_util::sync::CancellationToken;
use trace::TraceCollector;
@ -98,3 +106,26 @@ impl HttpApiErrorSource for IoxHttpError {
HttpApiError::new(self.status_code(), self.to_string())
}
}
/// Instantiate a compactor server
pub async fn create_compactor_server_type(
common_state: &CommonServerState,
metric_registry: Arc<metric::Registry>,
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
exec: Arc<Executor>,
time_provider: Arc<dyn TimeProvider>,
sequencers: Vec<SequencerId>,
) -> Arc<dyn ServerType> {
let compactor_handler = Arc::new(CompactorHandlerImpl::new(
sequencers,
catalog,
object_store,
exec,
time_provider,
&metric_registry,
));
let compactor = CompactorServer::new(metric_registry, compactor_handler);
Arc::new(CompactorServerType::new(compactor, common_state))
}

View File

@ -1,12 +1,23 @@
use std::{
collections::BTreeMap,
fmt::{Debug, Display},
sync::Arc,
time::Duration,
};
use async_trait::async_trait;
use clap_blocks::{ingester::IngesterConfig, write_buffer::WriteBufferConfig};
use data_types2::KafkaPartition;
use hyper::{Body, Request, Response};
use ingester::server::IngesterServer;
use ingester::{
handler::IngestHandlerImpl,
lifecycle::LifecycleConfig,
server::{grpc::GrpcDelegate, http::HttpDelegate, IngesterServer},
};
use iox_catalog::interface::Catalog;
use metric::Registry;
use object_store::ObjectStore;
use query::exec::Executor;
use trace::TraceCollector;
use crate::{
@ -15,6 +26,27 @@ use crate::{
server_type::{common_state::CommonServerState, RpcError, ServerType},
};
use ingester::handler::IngestHandler;
use thiserror::Error;
#[derive(Debug, Error)]
pub enum Error {
#[error("Catalog error: {0}")]
Catalog(#[from] iox_catalog::interface::Error),
#[error("Kafka topic {0} not found in the catalog")]
KafkaTopicNotFound(String),
#[error("kafka_partition_range_start must be <= kafka_partition_range_end")]
KafkaRange,
#[error("error initializing ingester: {0}")]
Ingester(#[from] ingester::handler::Error),
#[error("error initializing write buffer {0}")]
WriteBuffer(#[from] write_buffer::core::WriteBufferError),
}
pub type Result<T, E = Error> = std::result::Result<T, E>;
#[derive(Debug)]
pub struct IngesterServerType<I: IngestHandler> {
@ -96,3 +128,73 @@ impl HttpApiErrorSource for IoxHttpError {
HttpApiError::new(self.status_code(), self.to_string())
}
}
/// Instantiate an ingester server type
pub async fn create_ingester_server_type(
common_state: &CommonServerState,
metric_registry: Arc<metric::Registry>,
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
exec: Arc<Executor>,
write_buffer_config: &WriteBufferConfig,
ingester_config: IngesterConfig,
) -> Result<Arc<dyn ServerType>> {
let mut txn = catalog.start_transaction().await?;
let kafka_topic = txn
.kafka_topics()
.get_by_name(write_buffer_config.topic())
.await?
.ok_or_else(|| Error::KafkaTopicNotFound(write_buffer_config.topic().to_string()))?;
if ingester_config.write_buffer_partition_range_start
> ingester_config.write_buffer_partition_range_end
{
return Err(Error::KafkaRange);
}
let kafka_partitions: Vec<_> = (ingester_config.write_buffer_partition_range_start
..=ingester_config.write_buffer_partition_range_end)
.map(KafkaPartition::new)
.collect();
let mut sequencers = BTreeMap::new();
for k in kafka_partitions {
let s = txn.sequencers().create_or_get(&kafka_topic, k).await?;
sequencers.insert(k, s);
}
txn.commit().await?;
let trace_collector = common_state.trace_collector();
let write_buffer = write_buffer_config
.reading(Arc::clone(&metric_registry), trace_collector.clone())
.await?;
let lifecycle_config = LifecycleConfig::new(
ingester_config.pause_ingest_size_bytes,
ingester_config.persist_memory_threshold_bytes,
ingester_config.persist_partition_size_threshold_bytes,
Duration::from_secs(ingester_config.persist_partition_age_threshold_seconds),
Duration::from_secs(ingester_config.persist_partition_cold_threshold_seconds),
);
let ingest_handler = Arc::new(
IngestHandlerImpl::new(
lifecycle_config,
kafka_topic,
sequencers,
catalog,
object_store,
write_buffer,
exec,
Arc::clone(&metric_registry),
)
.await?,
);
let http = HttpDelegate::new(Arc::clone(&ingest_handler));
let grpc = GrpcDelegate::new(Arc::clone(&ingest_handler));
let ingester = IngesterServer::new(metric_registry, http, grpc, ingest_handler);
let server_type = Arc::new(IngesterServerType::new(ingester, common_state));
Ok(server_type)
}

View File

@ -5,8 +5,16 @@ use std::{
use async_trait::async_trait;
use hyper::{Body, Request, Response};
use iox_catalog::interface::Catalog;
use metric::Registry;
use querier::{database::QuerierDatabase, handler::QuerierHandler, server::QuerierServer};
use object_store::ObjectStore;
use querier::{
database::QuerierDatabase,
handler::{QuerierHandler, QuerierHandlerImpl},
server::QuerierServer,
};
use query::exec::Executor;
use time::TimeProvider;
use tokio_util::sync::CancellationToken;
use trace::TraceCollector;
@ -113,3 +121,25 @@ impl HttpApiErrorSource for IoxHttpError {
HttpApiError::new(self.status_code(), self.to_string())
}
}
/// Instantiate a querier server
pub async fn create_querier_server_type(
common_state: &CommonServerState,
metric_registry: Arc<metric::Registry>,
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
time_provider: Arc<dyn TimeProvider>,
exec: Arc<Executor>,
) -> Arc<dyn ServerType> {
let database = Arc::new(QuerierDatabase::new(
catalog,
Arc::clone(&metric_registry),
object_store,
time_provider,
exec,
));
let querier_handler = Arc::new(QuerierHandlerImpl::new(Arc::clone(&database)));
let querier = QuerierServer::new(metric_registry, querier_handler);
Arc::new(QuerierServerType::new(querier, database, common_state))
}

View File

@ -1,17 +1,17 @@
//! This module is responsible for compacting Ingester's data
use crate::data::{PersistingBatch, QueryableBatch};
use arrow::{array::TimestampNanosecondArray, record_batch::RecordBatch};
use arrow::record_batch::RecordBatch;
use data_types2::NamespaceId;
use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream};
use parquet_file::metadata::IoxMetadata;
use query::{
exec::{Executor, ExecutorType},
frontend::reorg::ReorgPlanner,
util::compute_timenanosecond_min_max,
QueryChunkMeta,
};
use schema::TIME_COLUMN_NAME;
use snafu::{OptionExt, ResultExt, Snafu};
use snafu::{ResultExt, Snafu};
use std::sync::Arc;
use time::{Time, TimeProvider};
@ -40,71 +40,16 @@ pub enum Error {
predicate: String,
},
#[snafu(display("The Record batch is empty"))]
EmptyBatch,
#[snafu(display("Error while searching Time column in a Record Batch"))]
TimeColumn { source: arrow::error::ArrowError },
#[snafu(display("Error while casting Timenanosecond on Time column"))]
TimeCasting,
#[snafu(display("Could not convert row count to i64"))]
RowCountTypeConversion { source: std::num::TryFromIntError },
#[snafu(display("Error computing min and max for record batches: {}", source))]
MinMax { source: query::util::Error },
}
/// A specialized `Error` for Ingester's Compact errors
pub type Result<T, E = Error> = std::result::Result<T, E>;
/// Return min and max for column `time` of the given set of record batches
pub fn compute_timenanosecond_min_max(batches: &[RecordBatch]) -> Result<(i64, i64)> {
let mut min_time = i64::MAX;
let mut max_time = i64::MIN;
for batch in batches {
let (min, max) = compute_timenanosecond_min_max_for_one_record_bacth(batch)?;
if min_time > min {
min_time = min;
}
if max_time < max {
max_time = max;
}
}
Ok((min_time, max_time))
}
/// Return min and max for column `time` in the given record batch
pub fn compute_timenanosecond_min_max_for_one_record_bacth(
batch: &RecordBatch,
) -> Result<(i64, i64)> {
if batch.num_columns() == 0 {
return Err(Error::EmptyBatch);
}
let index = batch
.schema()
.index_of(TIME_COLUMN_NAME)
.context(TimeColumnSnafu {})?;
let time_col = batch
.column(index)
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.context(TimeCastingSnafu {})?;
let min = time_col
.iter()
.min()
.expect("Time column must have values")
.expect("Time column cannot be NULL");
let max = time_col
.iter()
.max()
.expect("Time column must have values")
.expect("Time column cannot be NULL");
Ok((min, max))
}
/// Compact a given persisting batch
/// Return compacted data with its metadata
pub async fn compact_persisting_batch(
@ -138,7 +83,8 @@ pub async fn compact_persisting_batch(
let row_count = row_count.try_into().context(RowCountTypeConversionSnafu)?;
// Compute min and max of the `time` column
let (min_time, max_time) = compute_timenanosecond_min_max(&output_batches)?;
let (min_time, max_time) =
compute_timenanosecond_min_max(&output_batches).context(MinMaxSnafu)?;
// Compute min and max sequence numbers
let (min_seq, max_seq) = batch.data.min_max_sequence_numbers();

View File

@ -99,8 +99,7 @@ pub struct IngesterData {
/// get ingested.
pub(crate) sequencers: BTreeMap<SequencerId, SequencerData>,
/// Executor for running queries and compacting and persisting
pub(crate) exec: Executor,
pub(crate) exec: Arc<Executor>,
/// Backoff config
pub(crate) backoff_config: BackoffConfig,
}
@ -1427,7 +1426,7 @@ mod tests {
object_store: Arc::clone(&object_store),
catalog: Arc::clone(&catalog),
sequencers,
exec: Executor::new(1),
exec: Arc::new(Executor::new(1)),
backoff_config: BackoffConfig::default(),
});
@ -1508,7 +1507,7 @@ mod tests {
object_store: Arc::clone(&object_store),
catalog: Arc::clone(&catalog),
sequencers,
exec: Executor::new(1),
exec: Arc::new(Executor::new(1)),
backoff_config: BackoffConfig::default(),
});

View File

@ -116,7 +116,7 @@ impl IngestHandlerImpl {
catalog: Arc<dyn Catalog>,
object_store: Arc<ObjectStore>,
write_buffer: Arc<dyn WriteBufferReading>,
exec: Executor,
exec: Arc<Executor>,
metric_registry: Arc<metric::Registry>,
) -> Result<Self> {
// build the initial ingester data state
@ -729,7 +729,7 @@ mod tests {
Arc::clone(&catalog),
object_store,
reading,
Executor::new(1),
Arc::new(Executor::new(1)),
Arc::clone(&metrics),
)
.await
@ -829,7 +829,7 @@ mod tests {
Arc::clone(&catalog),
object_store,
reading,
Executor::new(1),
Arc::new(Executor::new(1)),
Arc::clone(&metrics),
)
.await

View File

@ -65,11 +65,5 @@ impl JobRegistry {
}
fn f_attributes(job: &Job) -> metric::Attributes {
let mut attributes = metric::Attributes::from(&[("name", job.name())]);
if let Some(partition_id) = job.partition_id() {
attributes.insert("partition_id", partition_id.get().to_string());
}
attributes
metric::Attributes::from(&[("name", job.name())])
}

View File

@ -7,7 +7,8 @@ use arrow::{
};
use arrow_util::util::merge_record_batches;
use data_types2::{
ChunkAddr, ChunkId, ChunkOrder, DeletePredicate, SequenceNumber, TableSummary, Tombstone,
tombstones_to_delete_predicates, tombstones_to_delete_predicates_iter, ChunkAddr, ChunkId,
ChunkOrder, DeletePredicate, SequenceNumber, TableSummary, Tombstone,
};
use datafusion::{
error::DataFusionError,
@ -19,7 +20,7 @@ use datafusion::{
},
};
use observability_deps::tracing::{debug, trace};
use predicate::{delete_predicate::parse_delete_predicate, Predicate, PredicateMatch};
use predicate::{Predicate, PredicateMatch};
use query::{
exec::{stringset::StringSet, IOxExecutionContext},
util::{df_physical_expr_from_schema_and_expr, MissingColumnsToNull},
@ -62,7 +63,7 @@ pub type Result<T, E = Error> = std::result::Result<T, E>;
impl QueryableBatch {
/// Initilaize a QueryableBatch
pub fn new(table_name: &str, data: Vec<Arc<SnapshotBatch>>, deletes: Vec<Tombstone>) -> Self {
let delete_predicates = Self::tombstones_to_delete_predicates(&deletes);
let delete_predicates = tombstones_to_delete_predicates(&deletes);
Self {
data,
delete_predicates,
@ -72,30 +73,10 @@ impl QueryableBatch {
/// Add more tombstones
pub fn add_tombstones(&mut self, deletes: &[Tombstone]) {
let delete_predicates = Self::tombstones_to_delete_predicates_iter(deletes);
let delete_predicates = tombstones_to_delete_predicates_iter(deletes);
self.delete_predicates.extend(delete_predicates);
}
/// Convert tombstones to delete predicates
pub fn tombstones_to_delete_predicates(tombstones: &[Tombstone]) -> Vec<Arc<DeletePredicate>> {
Self::tombstones_to_delete_predicates_iter(tombstones).collect()
}
fn tombstones_to_delete_predicates_iter(
tombstones: &[Tombstone],
) -> impl Iterator<Item = Arc<DeletePredicate>> + '_ {
tombstones.iter().map(|tombstone| {
Arc::new(
parse_delete_predicate(
&tombstone.min_time.get().to_string(),
&tombstone.max_time.get().to_string(),
&tombstone.serialized_predicate,
)
.expect("Error building delete predicate"),
)
})
}
/// return min and max of all the snapshots
pub fn min_max_sequence_numbers(&self) -> (SequenceNumber, SequenceNumber) {
let min = self
@ -128,7 +109,7 @@ impl QueryChunkMeta for QueryableBatch {
fn schema(&self) -> Arc<Schema> {
// TODO: may want store this schema as a field of QueryableBatch and
// only do this schema merge the first time it is call
// only do this schema merge the first time it is called
// Merge schema of all RecordBatches of the PerstingBatch
let batches: Vec<Arc<RecordBatch>> =

View File

@ -608,7 +608,7 @@ pub fn make_ingester_data(two_partitions: bool, loc: DataLocation) -> IngesterDa
let metrics: Arc<metric::Registry> = Default::default();
let catalog: Arc<dyn Catalog> = Arc::new(MemCatalog::new(metrics));
let object_store = Arc::new(object_store::ObjectStore::new_in_memory());
let exec = query::exec::Executor::new(1);
let exec = Arc::new(query::exec::Executor::new(1));
// Make data for one sequencer/shard and two tables
let seq_id = SequencerId::new(1);
@ -661,7 +661,7 @@ pub async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterDa
let metrics: Arc<metric::Registry> = Default::default();
let catalog: Arc<dyn Catalog> = Arc::new(MemCatalog::new(metrics));
let object_store = Arc::new(object_store::ObjectStore::new_in_memory());
let exec = query::exec::Executor::new(1);
let exec = Arc::new(query::exec::Executor::new(1));
// Make data for one sequencer/shard and two tables
let seq_id = SequencerId::new(1);

View File

@ -24,7 +24,7 @@ workspace-hack = { path = "../workspace-hack"}
dotenv = "0.15.0"
mutable_batch_lp = { path = "../mutable_batch_lp" }
paste = "1.0.6"
pretty_assertions = "1.0.0"
pretty_assertions = "1.2.0"
rand = "0.8"
tempfile = "3"
test_helpers = { path = "../test_helpers" }

View File

@ -5,7 +5,8 @@ use data_types2::{
Column, ColumnSchema, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace,
NamespaceId, NamespaceSchema, ParquetFile, ParquetFileId, ParquetFileParams, Partition,
PartitionId, PartitionInfo, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber,
Sequencer, SequencerId, Table, TableId, TableSchema, Timestamp, Tombstone, TombstoneId,
Sequencer, SequencerId, Table, TableId, TablePartition, TableSchema, Timestamp, Tombstone,
TombstoneId,
};
use snafu::{OptionExt, Snafu};
use std::{collections::BTreeMap, convert::TryFrom, fmt::Debug, sync::Arc};
@ -434,6 +435,27 @@ pub trait ParquetFileRepo: Send + Sync {
namespace_id: NamespaceId,
) -> Result<Vec<ParquetFile>>;
/// List parquet files for a given sequencer with compaction level 0 and other criteria that
/// define a file as a candidate for compaction
async fn level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>>;
/// List parquet files for a given table partition, in a given time range, with compaction
/// level 1, and other criteria that define a file as a candidate for compaction with a level 0
/// file
async fn level_1(
&mut self,
table_partition: TablePartition,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<Vec<ParquetFile>>;
/// Update the compaction level of the specified parquet files to level 1. Returns the IDs
/// of the files that were successfully updated.
async fn update_to_level_1(
&mut self,
parquet_file_ids: &[ParquetFileId],
) -> Result<Vec<ParquetFileId>>;
/// Verify if the parquet file exists by selecting its id
async fn exist(&mut self, id: ParquetFileId) -> Result<bool>;
@ -536,6 +558,9 @@ pub(crate) mod test_helpers {
test_partition(Arc::clone(&catalog)).await;
test_tombstone(Arc::clone(&catalog)).await;
test_parquet_file(Arc::clone(&catalog)).await;
test_parquet_file_compaction_level_0(Arc::clone(&catalog)).await;
test_parquet_file_compaction_level_1(Arc::clone(&catalog)).await;
test_update_to_compaction_level_1(Arc::clone(&catalog)).await;
test_add_parquet_file_with_tombstones(Arc::clone(&catalog)).await;
test_txn_isolation(Arc::clone(&catalog)).await;
test_txn_drop(Arc::clone(&catalog)).await;
@ -1358,7 +1383,7 @@ pub(crate) mod test_helpers {
.list_by_namespace_not_to_delete(namespace2.id)
.await
.unwrap();
assert_eq!(Vec::<ParquetFile>::new(), files);
assert!(files.is_empty());
let f1_params = ParquetFileParams {
table_id: partition2.table_id,
@ -1383,19 +1408,472 @@ pub(crate) mod test_helpers {
.await
.unwrap();
assert_eq!(vec![f1.clone(), f2.clone()], files);
repos.parquet_files().flag_for_delete(f2.id).await.unwrap();
let files = repos
.parquet_files()
.list_by_namespace_not_to_delete(namespace2.id)
.await
.unwrap();
assert_eq!(vec![f1], files);
assert_eq!(vec![f1.clone()], files);
let files = repos
.parquet_files()
.list_by_namespace_not_to_delete(NamespaceId::new(i32::MAX))
.await
.unwrap();
assert_eq!(Vec::<ParquetFile>::new(), files);
assert!(files.is_empty());
}
async fn test_parquet_file_compaction_level_0(catalog: Arc<dyn Catalog>) {
let mut repos = catalog.repositories().await;
let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap();
let pool = repos.query_pools().create_or_get("foo").await.unwrap();
let namespace = repos
.namespaces()
.create(
"namespace_parquet_file_compaction_level_0_test",
"inf",
kafka.id,
pool.id,
)
.await
.unwrap();
let table = repos
.tables()
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = repos
.sequencers()
.create_or_get(&kafka, KafkaPartition::new(100))
.await
.unwrap();
let other_sequencer = repos
.sequencers()
.create_or_get(&kafka, KafkaPartition::new(101))
.await
.unwrap();
let partition = repos
.partitions()
.create_or_get("one", sequencer.id, table.id)
.await
.unwrap();
let min_time = Timestamp::new(1);
let max_time = Timestamp::new(10);
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
table_id: partition.table_id,
partition_id: partition.id,
object_store_id: Uuid::new_v4(),
min_sequence_number: SequenceNumber::new(10),
max_sequence_number: SequenceNumber::new(140),
min_time,
max_time,
file_size_bytes: 1337,
parquet_metadata: b"md1".to_vec(),
row_count: 0,
created_at: Timestamp::new(1),
};
let parquet_file = repos
.parquet_files()
.create(parquet_file_params.clone())
.await
.unwrap();
// Create a compaction level 0 file for some other sequencer
let other_sequencer_params = ParquetFileParams {
sequencer_id: other_sequencer.id,
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let _other_sequencer_file = repos
.parquet_files()
.create(other_sequencer_params)
.await
.unwrap();
// Create a compaction level 0 file marked to delete
let to_delete_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let to_delete_file = repos
.parquet_files()
.create(to_delete_params)
.await
.unwrap();
repos
.parquet_files()
.flag_for_delete(to_delete_file.id)
.await
.unwrap();
// Create a compaction level 1 file
let level_1_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let level_1_file = repos.parquet_files().create(level_1_params).await.unwrap();
repos
.parquet_files()
.update_to_level_1(&[level_1_file.id])
.await
.unwrap();
// Level 0 parquet files for a sequencer should contain only those that match the right
// criteria
let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap();
let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect();
level_0_ids.sort();
let expected = vec![parquet_file];
let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect();
expected_ids.sort();
assert_eq!(
level_0_ids, expected_ids,
"\nlevel 0: {:#?}\nexpected: {:#?}",
level_0, expected,
);
}
async fn test_parquet_file_compaction_level_1(catalog: Arc<dyn Catalog>) {
let mut repos = catalog.repositories().await;
let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap();
let pool = repos.query_pools().create_or_get("foo").await.unwrap();
let namespace = repos
.namespaces()
.create(
"namespace_parquet_file_compaction_level_1_test",
"inf",
kafka.id,
pool.id,
)
.await
.unwrap();
let table = repos
.tables()
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let other_table = repos
.tables()
.create_or_get("test_table2", namespace.id)
.await
.unwrap();
let sequencer = repos
.sequencers()
.create_or_get(&kafka, KafkaPartition::new(100))
.await
.unwrap();
let other_sequencer = repos
.sequencers()
.create_or_get(&kafka, KafkaPartition::new(101))
.await
.unwrap();
let partition = repos
.partitions()
.create_or_get("one", sequencer.id, table.id)
.await
.unwrap();
let other_partition = repos
.partitions()
.create_or_get("two", sequencer.id, table.id)
.await
.unwrap();
// Set up the window of times we're interested in level 1 files for
let query_min_time = Timestamp::new(5);
let query_max_time = Timestamp::new(10);
// Create a file with times entirely within the window
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
table_id: partition.table_id,
partition_id: partition.id,
object_store_id: Uuid::new_v4(),
min_sequence_number: SequenceNumber::new(10),
max_sequence_number: SequenceNumber::new(140),
min_time: query_min_time + 1,
max_time: query_max_time - 1,
file_size_bytes: 1337,
parquet_metadata: b"md1".to_vec(),
row_count: 0,
created_at: Timestamp::new(1),
};
let parquet_file = repos
.parquet_files()
.create(parquet_file_params.clone())
.await
.unwrap();
// Create a file that will remain as level 0
let level_0_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let _level_0_file = repos.parquet_files().create(level_0_params).await.unwrap();
// Create a file completely before the window
let too_early_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
min_time: query_min_time - 2,
max_time: query_min_time - 1,
..parquet_file_params.clone()
};
let too_early_file = repos
.parquet_files()
.create(too_early_params)
.await
.unwrap();
// Create a file overlapping the window on the lower end
let overlap_lower_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
min_time: query_min_time - 1,
max_time: query_min_time + 1,
..parquet_file_params.clone()
};
let overlap_lower_file = repos
.parquet_files()
.create(overlap_lower_params)
.await
.unwrap();
// Create a file overlapping the window on the upper end
let overlap_upper_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
min_time: query_max_time - 1,
max_time: query_max_time + 1,
..parquet_file_params.clone()
};
let overlap_upper_file = repos
.parquet_files()
.create(overlap_upper_params)
.await
.unwrap();
// Create a file completely after the window
let too_late_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
min_time: query_max_time + 1,
max_time: query_max_time + 2,
..parquet_file_params.clone()
};
let too_late_file = repos.parquet_files().create(too_late_params).await.unwrap();
// Create a file for some other sequencer
let other_sequencer_params = ParquetFileParams {
sequencer_id: other_sequencer.id,
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let other_sequencer_file = repos
.parquet_files()
.create(other_sequencer_params)
.await
.unwrap();
// Create a file for the same sequencer but a different table
let other_table_params = ParquetFileParams {
table_id: other_table.id,
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let other_table_file = repos
.parquet_files()
.create(other_table_params)
.await
.unwrap();
// Create a file for the same sequencer and table but a different partition
let other_partition_params = ParquetFileParams {
partition_id: other_partition.id,
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let other_partition_file = repos
.parquet_files()
.create(other_partition_params)
.await
.unwrap();
// Create a file marked to be deleted
let to_delete_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let to_delete_file = repos
.parquet_files()
.create(to_delete_params)
.await
.unwrap();
repos
.parquet_files()
.flag_for_delete(to_delete_file.id)
.await
.unwrap();
// Make all but _level_0_file compaction level 1
repos
.parquet_files()
.update_to_level_1(&[
parquet_file.id,
too_early_file.id,
too_late_file.id,
overlap_lower_file.id,
overlap_upper_file.id,
other_sequencer_file.id,
other_table_file.id,
other_partition_file.id,
to_delete_file.id,
])
.await
.unwrap();
// Level 1 parquet files for a sequencer should contain only those that match the right
// criteria
let table_partition = TablePartition::new(sequencer.id, table.id, partition.id);
let level_1 = repos
.parquet_files()
.level_1(table_partition, query_min_time, query_max_time)
.await
.unwrap();
let mut level_1_ids: Vec<_> = level_1.iter().map(|pf| pf.id).collect();
level_1_ids.sort();
let expected = vec![parquet_file, overlap_lower_file, overlap_upper_file];
let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect();
expected_ids.sort();
assert_eq!(
level_1_ids, expected_ids,
"\nlevel 1: {:#?}\nexpected: {:#?}",
level_1, expected,
);
}
async fn test_update_to_compaction_level_1(catalog: Arc<dyn Catalog>) {
let mut repos = catalog.repositories().await;
let kafka = repos.kafka_topics().create_or_get("foo").await.unwrap();
let pool = repos.query_pools().create_or_get("foo").await.unwrap();
let namespace = repos
.namespaces()
.create(
"namespace_update_to_compaction_level_1_test",
"inf",
kafka.id,
pool.id,
)
.await
.unwrap();
let table = repos
.tables()
.create_or_get("update_table", namespace.id)
.await
.unwrap();
let sequencer = repos
.sequencers()
.create_or_get(&kafka, KafkaPartition::new(1000))
.await
.unwrap();
let partition = repos
.partitions()
.create_or_get("one", sequencer.id, table.id)
.await
.unwrap();
// Set up the window of times we're interested in level 1 files for
let query_min_time = Timestamp::new(5);
let query_max_time = Timestamp::new(10);
// Create a file with times entirely within the window
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
table_id: partition.table_id,
partition_id: partition.id,
object_store_id: Uuid::new_v4(),
min_sequence_number: SequenceNumber::new(10),
max_sequence_number: SequenceNumber::new(140),
min_time: query_min_time + 1,
max_time: query_max_time - 1,
file_size_bytes: 1337,
parquet_metadata: b"md1".to_vec(),
row_count: 0,
created_at: Timestamp::new(1),
};
let parquet_file = repos
.parquet_files()
.create(parquet_file_params.clone())
.await
.unwrap();
// Create a file that will remain as level 0
let level_0_params = ParquetFileParams {
object_store_id: Uuid::new_v4(),
..parquet_file_params.clone()
};
let level_0_file = repos.parquet_files().create(level_0_params).await.unwrap();
// Create a ParquetFileId that doesn't actually exist in the catalog
let nonexistent_parquet_file_id = ParquetFileId::new(level_0_file.id.get() + 1);
// Level 0 parquet files should contain both existing files at this point
let expected = vec![parquet_file.clone(), level_0_file.clone()];
let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap();
let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect();
level_0_ids.sort();
let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect();
expected_ids.sort();
assert_eq!(
level_0_ids, expected_ids,
"\nlevel 0: {:#?}\nexpected: {:#?}",
level_0, expected,
);
// Make parquet_file compaction level 1, attempt to mark the nonexistent file; operation
// should succeed
let updated = repos
.parquet_files()
.update_to_level_1(&[parquet_file.id, nonexistent_parquet_file_id])
.await
.unwrap();
assert_eq!(updated, vec![parquet_file.id]);
// Level 0 parquet files should only contain level_0_file
let expected = vec![level_0_file];
let level_0 = repos.parquet_files().level_0(sequencer.id).await.unwrap();
let mut level_0_ids: Vec<_> = level_0.iter().map(|pf| pf.id).collect();
level_0_ids.sort();
let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect();
expected_ids.sort();
assert_eq!(
level_0_ids, expected_ids,
"\nlevel 0: {:#?}\nexpected: {:#?}",
level_0, expected,
);
// Level 1 parquet files for a sequencer should only contain parquet_file
let expected = vec![parquet_file];
let table_partition = TablePartition::new(sequencer.id, table.id, partition.id);
let level_1 = repos
.parquet_files()
.level_1(table_partition, query_min_time, query_max_time)
.await
.unwrap();
let mut level_1_ids: Vec<_> = level_1.iter().map(|pf| pf.id).collect();
level_1_ids.sort();
let mut expected_ids: Vec<_> = expected.iter().map(|pf| pf.id).collect();
expected_ids.sort();
assert_eq!(
level_1_ids, expected_ids,
"\nlevel 1: {:#?}\nexpected: {:#?}",
level_1, expected,
);
}
async fn test_add_parquet_file_with_tombstones(catalog: Arc<dyn Catalog>) {

View File

@ -15,7 +15,7 @@ use data_types2::{
Column, ColumnId, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId,
ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo,
ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table,
TableId, Timestamp, Tombstone, TombstoneId,
TableId, TablePartition, Timestamp, Tombstone, TombstoneId,
};
use observability_deps::tracing::warn;
use std::fmt::Formatter;
@ -843,6 +843,61 @@ impl ParquetFileRepo for MemTxn {
Ok(parquet_files)
}
async fn level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>> {
let stage = self.stage();
Ok(stage
.parquet_files
.iter()
.filter(|f| f.sequencer_id == sequencer_id && f.compaction_level == 0 && !f.to_delete)
.cloned()
.collect())
}
async fn level_1(
&mut self,
table_partition: TablePartition,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<Vec<ParquetFile>> {
let stage = self.stage();
Ok(stage
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == table_partition.sequencer_id
&& f.table_id == table_partition.table_id
&& f.partition_id == table_partition.partition_id
&& f.compaction_level == 1
&& !f.to_delete
&& ((f.min_time <= min_time && f.max_time >= min_time)
|| (f.min_time > min_time && f.min_time <= max_time))
})
.cloned()
.collect())
}
async fn update_to_level_1(
&mut self,
parquet_file_ids: &[ParquetFileId],
) -> Result<Vec<ParquetFileId>> {
let stage = self.stage();
let mut updated = Vec::with_capacity(parquet_file_ids.len());
for f in stage
.parquet_files
.iter_mut()
.filter(|p| parquet_file_ids.contains(&p.id))
{
f.compaction_level = 1;
updated.push(f.id);
}
Ok(updated)
}
async fn exist(&mut self, id: ParquetFileId) -> Result<bool> {
let stage = self.stage();

View File

@ -10,7 +10,7 @@ use data_types2::{
Column, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId,
ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo,
ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table,
TableId, Timestamp, Tombstone, TombstoneId,
TableId, TablePartition, Timestamp, Tombstone, TombstoneId,
};
use metric::{Metric, U64Histogram, U64HistogramOptions};
use std::{fmt::Debug, sync::Arc};
@ -262,6 +262,9 @@ decorate!(
"parquet_flag_for_delete" = flag_for_delete(&mut self, id: ParquetFileId) -> Result<()>;
"parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result<Vec<ParquetFile>>;
"parquet_list_by_namespace_not_to_delete" = list_by_namespace_not_to_delete(&mut self, namespace_id: NamespaceId) -> Result<Vec<ParquetFile>>;
"parquet_level_0" = level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>>;
"parquet_level_1" = level_1(&mut self, table_partition: TablePartition, min_time: Timestamp, max_time: Timestamp) -> Result<Vec<ParquetFile>>;
"parquet_update_to_level_1" = update_to_level_1(&mut self, parquet_file_ids: &[ParquetFileId]) -> Result<Vec<ParquetFileId>>;
"parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result<bool>;
"parquet_count" = count(&mut self) -> Result<i64>;
]

View File

@ -14,7 +14,7 @@ use data_types2::{
Column, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId,
ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo,
ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber, Sequencer, SequencerId, Table,
TableId, Timestamp, Tombstone, TombstoneId,
TableId, TablePartition, Timestamp, Tombstone, TombstoneId,
};
use observability_deps::tracing::{info, warn};
use sqlx::{migrate::Migrator, postgres::PgPoolOptions, Acquire, Executor, Postgres, Row};
@ -174,8 +174,8 @@ impl Drop for PostgresTxn {
if let PostgresTxnInner::Txn(Some(_)) = self.inner {
warn!("Dropping PostgresTxn w/o finalizing (commit or abort)");
// SQLx ensures that the inner transaction enqueues a rollback when it is dropped, so we don't need to spawn
// a task here to call `rollback` manually.
// SQLx ensures that the inner transaction enqueues a rollback when it is dropped, so
// we don't need to spawn a task here to call `rollback` manually.
}
}
}
@ -214,15 +214,15 @@ impl TransactionFinalize for PostgresTxn {
#[async_trait]
impl Catalog for PostgresCatalog {
async fn setup(&self) -> Result<(), Error> {
// We need to create the schema if we're going to set it as the first item of the search_path
// otherwise when we run the sqlx migration scripts for the first time, sqlx will create the
// `_sqlx_migrations` table in the public namespace (the only namespace that exists), but the second
// time it will create it in the `<schema_name>` namespace and re-run all the migrations without
// skipping the ones already applied (see #3893).
// We need to create the schema if we're going to set it as the first item of the
// search_path otherwise when we run the sqlx migration scripts for the first time, sqlx
// will create the `_sqlx_migrations` table in the public namespace (the only namespace
// that exists), but the second time it will create it in the `<schema_name>` namespace and
// re-run all the migrations without skipping the ones already applied (see #3893).
//
// This makes the migrations/20210217134322_create_schema.sql step unnecessary; we need to keep that
// file because migration files are immutable.
let create_schema_query = format!("CREATE SCHEMA IF NOT EXISTS {}", &self.schema_name);
// This makes the migrations/20210217134322_create_schema.sql step unnecessary; we need to
// keep that file because migration files are immutable.
let create_schema_query = format!("CREATE SCHEMA IF NOT EXISTS {};", &self.schema_name);
self.pool
.execute(sqlx::query(&create_schema_query))
.await
@ -293,16 +293,16 @@ async fn new_raw_pool(
// set as part of the DSN, and we can set it explictly.
// Recall that this block is running on connection, not when creating the pool!
let current_application_name: String =
sqlx::query_scalar("SELECT current_setting('application_name')")
sqlx::query_scalar("SELECT current_setting('application_name');")
.fetch_one(&mut *c)
.await?;
if current_application_name.is_empty() {
sqlx::query("SELECT set_config('application_name', $1, false)")
sqlx::query("SELECT set_config('application_name', $1, false);")
.bind(&*app_name)
.execute(&mut *c)
.await?;
}
let search_path_query = format!("SET search_path TO {},public", schema_name);
let search_path_query = format!("SET search_path TO {},public;", schema_name);
c.execute(sqlx::query(&search_path_query)).await?;
Ok(())
})
@ -395,7 +395,12 @@ async fn new_pool(
current_dsn = new_dsn;
}
Err(e) => {
warn!(error=%e, filename=%dsn_file, "not replacing hotswap pool because of an error connecting to the new DSN");
warn!(
error=%e,
filename=%dsn_file,
"not replacing hotswap pool because of an error \
connecting to the new DSN"
);
}
}
}
@ -467,7 +472,8 @@ impl KafkaTopicRepo for PostgresTxn {
INSERT INTO kafka_topic ( name )
VALUES ( $1 )
ON CONFLICT ON CONSTRAINT kafka_topic_name_unique
DO UPDATE SET name = kafka_topic.name RETURNING *;
DO UPDATE SET name = kafka_topic.name
RETURNING *;
"#,
)
.bind(&name) // $1
@ -481,7 +487,9 @@ DO UPDATE SET name = kafka_topic.name RETURNING *;
async fn get_by_name(&mut self, name: &str) -> Result<Option<KafkaTopic>> {
let rec = sqlx::query_as::<_, KafkaTopic>(
r#"
SELECT * FROM kafka_topic WHERE name = $1;
SELECT *
FROM kafka_topic
WHERE name = $1;
"#,
)
.bind(&name) // $1
@ -506,7 +514,8 @@ impl QueryPoolRepo for PostgresTxn {
INSERT INTO query_pool ( name )
VALUES ( $1 )
ON CONFLICT ON CONSTRAINT query_pool_name_unique
DO UPDATE SET name = query_pool.name RETURNING *;
DO UPDATE SET name = query_pool.name
RETURNING *;
"#,
)
.bind(&name) // $1
@ -531,7 +540,7 @@ impl NamespaceRepo for PostgresTxn {
r#"
INSERT INTO namespace ( name, retention_duration, kafka_topic_id, query_pool_id )
VALUES ( $1, $2, $3, $4 )
RETURNING *
RETURNING *;
"#,
)
.bind(&name) // $1
@ -558,7 +567,8 @@ RETURNING *
async fn list(&mut self) -> Result<Vec<Namespace>> {
let rec = sqlx::query_as::<_, Namespace>(
r#"
SELECT * FROM namespace;
SELECT *
FROM namespace;
"#,
)
.fetch_all(&mut self.inner)
@ -571,7 +581,9 @@ SELECT * FROM namespace;
async fn get_by_id(&mut self, id: NamespaceId) -> Result<Option<Namespace>> {
let rec = sqlx::query_as::<_, Namespace>(
r#"
SELECT * FROM namespace WHERE id = $1;
SELECT *
FROM namespace
WHERE id = $1;
"#,
)
.bind(&id) // $1
@ -590,7 +602,9 @@ SELECT * FROM namespace WHERE id = $1;
async fn get_by_name(&mut self, name: &str) -> Result<Option<Namespace>> {
let rec = sqlx::query_as::<_, Namespace>(
r#"
SELECT * FROM namespace WHERE name = $1;
SELECT *
FROM namespace
WHERE name = $1;
"#,
)
.bind(&name) // $1
@ -609,7 +623,10 @@ SELECT * FROM namespace WHERE name = $1;
async fn update_table_limit(&mut self, name: &str, new_max: i32) -> Result<Namespace> {
let rec = sqlx::query_as::<_, Namespace>(
r#"
UPDATE namespace SET max_tables = $1 WHERE name = $2 RETURNING *;
UPDATE namespace
SET max_tables = $1
WHERE name = $2
RETURNING *;
"#,
)
.bind(&new_max)
@ -630,7 +647,10 @@ UPDATE namespace SET max_tables = $1 WHERE name = $2 RETURNING *;
async fn update_column_limit(&mut self, name: &str, new_max: i32) -> Result<Namespace> {
let rec = sqlx::query_as::<_, Namespace>(
r#"
UPDATE namespace SET max_columns_per_table = $1 WHERE name = $2 RETURNING *;
UPDATE namespace
SET max_columns_per_table = $1
WHERE name = $2
RETURNING *;
"#,
)
.bind(&new_max)
@ -657,7 +677,8 @@ impl TableRepo for PostgresTxn {
INSERT INTO table_name ( name, namespace_id )
VALUES ( $1, $2 )
ON CONFLICT ON CONSTRAINT table_name_unique
DO UPDATE SET name = table_name.name RETURNING *;
DO UPDATE SET name = table_name.name
RETURNING *;
"#,
)
.bind(&name) // $1
@ -676,7 +697,13 @@ DO UPDATE SET name = table_name.name RETURNING *;
}
async fn get_by_id(&mut self, table_id: TableId) -> Result<Option<Table>> {
let rec = sqlx::query_as::<_, Table>(r#"SELECT * FROM table_name WHERE id = $1;"#)
let rec = sqlx::query_as::<_, Table>(
r#"
SELECT *
FROM table_name
WHERE id = $1;
"#,
)
.bind(&table_id) // $1
.fetch_one(&mut self.inner)
.await;
@ -693,7 +720,8 @@ DO UPDATE SET name = table_name.name RETURNING *;
async fn list_by_namespace_id(&mut self, namespace_id: NamespaceId) -> Result<Vec<Table>> {
let rec = sqlx::query_as::<_, Table>(
r#"
SELECT * FROM table_name
SELECT *
FROM table_name
WHERE namespace_id = $1;
"#,
)
@ -714,7 +742,8 @@ WHERE namespace_id = $1;
let rec = sqlx::query_as::<_, TablePersistInfo>(
r#"
WITH tid as (SELECT id FROM table_name WHERE name = $2 AND namespace_id = $3)
SELECT $1 as sequencer_id, id as table_id, parquet_file.max_sequence_number AS parquet_max_sequence_number,
SELECT $1 as sequencer_id, id as table_id,
parquet_file.max_sequence_number AS parquet_max_sequence_number,
tombstone.sequence_number as tombstone_max_sequence_number
FROM tid
LEFT JOIN (
@ -731,7 +760,8 @@ WHERE namespace_id = $1;
ORDER BY max_sequence_number DESC
LIMIT 1
) parquet_file ON parquet_file.table_id = tid.id;
"#)
"#,
)
.bind(&sequencer_id) // $1
.bind(&table_name) // $2
.bind(&namespace_id) // $3
@ -763,7 +793,8 @@ impl ColumnRepo for PostgresTxn {
INSERT INTO column_name ( name, table_id, column_type )
VALUES ( $1, $2, $3 )
ON CONFLICT ON CONSTRAINT column_name_unique
DO UPDATE SET name = column_name.name RETURNING *;
DO UPDATE SET name = column_name.name
RETURNING *;
"#,
)
.bind(&name) // $1
@ -825,14 +856,15 @@ WHERE table_name.namespace_id = $1;
SELECT name, table_id, column_type FROM UNNEST($1, $2, $3) as a(name, table_id, column_type)
ON CONFLICT ON CONSTRAINT column_name_unique
DO UPDATE SET name = column_name.name
RETURNING *
RETURNING *;
"#,
)
.bind(&v_name)
.bind(&v_table_id)
.bind(&v_column_type)
.fetch_all(&mut self.inner)
.await.map_err(|e| {
.await
.map_err(|e| {
if is_fk_violation(&e) {
Error::ForeignKeyViolation { source: e }
} else {
@ -874,7 +906,8 @@ impl SequencerRepo for PostgresTxn {
VALUES
( $1, $2, 0 )
ON CONFLICT ON CONSTRAINT sequencer_unique
DO UPDATE SET kafka_topic_id = sequencer.kafka_topic_id RETURNING *;
DO UPDATE SET kafka_topic_id = sequencer.kafka_topic_id
RETURNING *;;
"#,
)
.bind(&topic.id) // $1
@ -897,7 +930,10 @@ impl SequencerRepo for PostgresTxn {
) -> Result<Option<Sequencer>> {
let rec = sqlx::query_as::<_, Sequencer>(
r#"
SELECT * FROM sequencer WHERE kafka_topic_id = $1 AND kafka_partition = $2;
SELECT *
FROM sequencer
WHERE kafka_topic_id = $1
AND kafka_partition = $2;
"#,
)
.bind(topic_id) // $1
@ -962,7 +998,8 @@ impl PartitionRepo for PostgresTxn {
VALUES
( $1, $2, $3 )
ON CONFLICT ON CONSTRAINT partition_key_unique
DO UPDATE SET partition_key = partition.partition_key RETURNING *;
DO UPDATE SET partition_key = partition.partition_key
RETURNING *;
"#,
)
.bind(key) // $1
@ -1014,12 +1051,17 @@ impl PartitionRepo for PostgresTxn {
}
async fn list_by_namespace(&mut self, namespace_id: NamespaceId) -> Result<Vec<Partition>> {
sqlx::query_as::<_, Partition>(r#"
SELECT partition.id as id, partition.sequencer_id as sequencer_id, partition.table_id as table_id, partition.partition_key as partition_key
sqlx::query_as::<_, Partition>(
r#"
SELECT partition.id as id,
partition.sequencer_id as sequencer_id,
partition.table_id as table_id,
partition.partition_key as partition_key
FROM table_name
INNER JOIN partition on partition.table_id = table_name.id
WHERE table_name.namespace_id = $1;
"#)
"#,
)
.bind(&namespace_id) // $1
.fetch_all(&mut self.inner)
.await
@ -1037,7 +1079,8 @@ WHERE table_name.namespace_id = $1;
FROM partition
INNER JOIN table_name on table_name.id = partition.table_id
INNER JOIN namespace on namespace.id = table_name.namespace_id
WHERE partition.id = $1;"#,
WHERE partition.id = $1;
"#,
)
.bind(&partition_id) // $1
.fetch_one(&mut self.inner)
@ -1079,7 +1122,8 @@ impl TombstoneRepo for PostgresTxn {
VALUES
( $1, $2, $3, $4, $5, $6 )
ON CONFLICT ON CONSTRAINT tombstone_unique
DO UPDATE SET table_id = tombstone.table_id RETURNING *;
DO UPDATE SET table_id = tombstone.table_id
RETURNING *;
"#,
)
.bind(&table_id) // $1
@ -1147,7 +1191,15 @@ WHERE table_name.namespace_id = $1;
sequencer_id: SequencerId,
sequence_number: SequenceNumber,
) -> Result<Vec<Tombstone>> {
sqlx::query_as::<_, Tombstone>(r#"SELECT * FROM tombstone WHERE sequencer_id = $1 AND sequence_number > $2 ORDER BY id;"#)
sqlx::query_as::<_, Tombstone>(
r#"
SELECT *
FROM tombstone
WHERE sequencer_id = $1
AND sequence_number > $2
ORDER BY id;
"#,
)
.bind(&sequencer_id) // $1
.bind(&sequence_number) // $2
.fetch_all(&mut self.inner)
@ -1176,9 +1228,12 @@ impl ParquetFileRepo for PostgresTxn {
let rec = sqlx::query_as::<_, ParquetFile>(
r#"
INSERT INTO parquet_file ( sequencer_id, table_id, partition_id, object_store_id, min_sequence_number, max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata, row_count, compaction_level, created_at )
INSERT INTO parquet_file (
sequencer_id, table_id, partition_id, object_store_id, min_sequence_number,
max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata,
row_count, compaction_level, created_at )
VALUES ( $1, $2, $3, $4, $5, $6, $7, $8, false, $9, $10, $11, $12, $13 )
RETURNING *
RETURNING *;
"#,
)
.bind(sequencer_id) // $1
@ -1198,9 +1253,7 @@ RETURNING *
.await
.map_err(|e| {
if is_unique_violation(&e) {
Error::FileExists {
object_store_id,
}
Error::FileExists { object_store_id }
} else if is_fk_violation(&e) {
Error::ForeignKeyViolation { source: e }
} else {
@ -1226,7 +1279,15 @@ RETURNING *
sequencer_id: SequencerId,
sequence_number: SequenceNumber,
) -> Result<Vec<ParquetFile>> {
sqlx::query_as::<_, ParquetFile>(r#"SELECT * FROM parquet_file WHERE sequencer_id = $1 AND max_sequence_number > $2 ORDER BY id;"#)
sqlx::query_as::<_, ParquetFile>(
r#"
SELECT *
FROM parquet_file
WHERE sequencer_id = $1
AND max_sequence_number > $2
ORDER BY id;
"#,
)
.bind(&sequencer_id) // $1
.bind(&sequence_number) // $2
.fetch_all(&mut self.inner)
@ -1240,25 +1301,11 @@ RETURNING *
) -> Result<Vec<ParquetFile>> {
sqlx::query_as::<_, ParquetFile>(
r#"
SELECT
parquet_file.id as id,
parquet_file.sequencer_id as sequencer_id,
parquet_file.table_id as table_id,
parquet_file.partition_id as partition_id,
parquet_file.object_store_id as object_store_id,
parquet_file.min_sequence_number as min_sequence_number,
parquet_file.max_sequence_number as max_sequence_number,
parquet_file.min_time as min_time,
parquet_file.max_time as max_time,
parquet_file.to_delete as to_delete,
parquet_file.file_size_bytes as file_size_bytes,
parquet_file.parquet_metadata as parquet_metadata,
parquet_file.row_count as row_count,
parquet_file.compaction_level as compaction_level,
parquet_file.created_at as created_at
SELECT parquet_file.*
FROM parquet_file
INNER JOIN table_name on table_name.id = parquet_file.table_id
WHERE table_name.namespace_id = $1 AND parquet_file.to_delete = false;
WHERE table_name.namespace_id = $1
AND parquet_file.to_delete = false;
"#,
)
.bind(&namespace_id) // $1
@ -1267,6 +1314,75 @@ WHERE table_name.namespace_id = $1 AND parquet_file.to_delete = false;
.map_err(|e| Error::SqlxError { source: e })
}
async fn level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>> {
sqlx::query_as::<_, ParquetFile>(
r#"
SELECT *
FROM parquet_file
WHERE parquet_file.sequencer_id = $1
AND parquet_file.compaction_level = 0
AND parquet_file.to_delete = false;
"#,
)
.bind(&sequencer_id) // $1
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })
}
async fn level_1(
&mut self,
table_partition: TablePartition,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<Vec<ParquetFile>> {
sqlx::query_as::<_, ParquetFile>(
r#"
SELECT *
FROM parquet_file
WHERE parquet_file.sequencer_id = $1
AND parquet_file.table_id = $2
AND parquet_file.partition_id = $3
AND parquet_file.compaction_level = 1
AND parquet_file.to_delete = false
AND ((parquet_file.min_time <= $5 AND parquet_file.max_time >= $4)
OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $5));
"#,
)
.bind(&table_partition.sequencer_id) // $1
.bind(&table_partition.table_id) // $2
.bind(&table_partition.partition_id) // $3
.bind(min_time) // $4
.bind(max_time) // $5
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })
}
async fn update_to_level_1(
&mut self,
parquet_file_ids: &[ParquetFileId],
) -> Result<Vec<ParquetFileId>> {
// If I try to do `.bind(parquet_file_ids)` directly, I get a compile error from sqlx.
// See https://github.com/launchbadge/sqlx/issues/1744
let ids: Vec<_> = parquet_file_ids.iter().map(|p| p.get()).collect();
let updated = sqlx::query(
r#"
UPDATE parquet_file
SET compaction_level = 1
WHERE id = ANY($1)
RETURNING id;
"#,
)
.bind(&ids[..])
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })?;
let updated = updated.into_iter().map(|row| row.get("id")).collect();
Ok(updated)
}
async fn exist(&mut self, id: ParquetFileId) -> Result<bool> {
let read_result = sqlx::query_as::<_, Count>(
r#"SELECT count(*) as count FROM parquet_file WHERE id = $1;"#,
@ -1301,7 +1417,7 @@ impl ProcessedTombstoneRepo for PostgresTxn {
r#"
INSERT INTO processed_tombstone ( tombstone_id, parquet_file_id )
VALUES ( $1, $2 )
RETURNING *
RETURNING *;
"#,
)
.bind(tombstone_id) // $1
@ -1328,7 +1444,13 @@ impl ProcessedTombstoneRepo for PostgresTxn {
tombstone_id: TombstoneId,
) -> Result<bool> {
let read_result = sqlx::query_as::<_, Count>(
r#"SELECT count(*) as count FROM processed_tombstone WHERE parquet_file_id = $1 AND tombstone_id = $2;"#)
r#"
SELECT count(*) as count
FROM processed_tombstone
WHERE parquet_file_id = $1
AND tombstone_id = $2;
"#,
)
.bind(&parquet_file_id) // $1
.bind(&tombstone_id) // $2
.fetch_one(&mut self.inner)
@ -1892,9 +2014,9 @@ mod tests {
paste::paste! {
#[tokio::test]
async fn [<test_column_create_or_get_many_ $name>]() {
// If running an integration test on your laptop, this requires that you have Postgres
// running and that you've done the sqlx migrations. See the README in this crate for
// info to set it up.
// If running an integration test on your laptop, this requires that you have
// Postgres running and that you've done the sqlx migrations. See the README in
// this crate for info to set it up.
maybe_skip_integration!();
let postgres = setup_db().await;

View File

@ -24,7 +24,7 @@ pub struct TestCatalog {
pub catalog: Arc<dyn Catalog>,
pub metric_registry: Arc<metric::Registry>,
pub object_store: Arc<ObjectStore>,
pub time_provider: Arc<dyn TimeProvider>,
pub time_provider: Arc<MockProvider>,
pub exec: Arc<Executor>,
}
@ -61,9 +61,18 @@ impl TestCatalog {
Arc::clone(&self.object_store)
}
/// Return the mockable version of the catalog's time provider.
///
/// If you need a generic time provider, use [`time_provider`](Self::time_provider) instead.
pub fn mock_time_provider(&self) -> &MockProvider {
self.time_provider.as_ref()
}
/// Return the catalog's time provider
///
/// If you need to mock the time, use [`mock_time_provider`](Self::mock_time_provider) instead.
pub fn time_provider(&self) -> Arc<dyn TimeProvider> {
Arc::clone(&self.time_provider)
Arc::clone(&self.time_provider) as _
}
/// Return the catalog's executor
@ -158,7 +167,7 @@ pub struct TestTable {
}
impl TestTable {
/// Attach a sequncer to the table
/// Attach a sequencer to the table
pub fn with_sequencer(
self: &Arc<Self>,
sequencer: &Arc<TestSequencer>,
@ -257,8 +266,19 @@ pub struct TestPartition {
}
impl TestPartition {
/// CReate a parquet for the partition
/// Create a parquet for the partition
pub async fn create_parquet_file(self: &Arc<Self>, lp: &str) -> Arc<TestParquetFile> {
self.create_parquet_file_with_sequence_numbers(lp, 1, 100)
.await
}
/// Create a parquet for the partition
pub async fn create_parquet_file_with_sequence_numbers(
self: &Arc<Self>,
lp: &str,
min_seq: i64,
max_seq: i64,
) -> Arc<TestParquetFile> {
let mut repos = self.catalog.catalog.repositories().await;
let (table, batch) = lp_to_mutable_batch(lp);
@ -267,8 +287,8 @@ impl TestPartition {
let record_batch = batch.to_arrow(Selection::All).unwrap();
let object_store_id = Uuid::new_v4();
let min_sequence_number = SequenceNumber::new(1);
let max_sequence_number = SequenceNumber::new(100);
let min_sequence_number = SequenceNumber::new(min_seq);
let max_sequence_number = SequenceNumber::new(max_seq);
let metadata = IoxMetadata {
object_store_id,
creation_timestamp: now(),

View File

@ -44,3 +44,4 @@ arrow_util = { path = "../arrow_util" }
bytes = "1.0"
iox_tests = { path = "../iox_tests" }
mutable_batch_lp = { path = "../mutable_batch_lp" }
proptest = { version = "1", default_features = false, features = ["std"] }

View File

@ -1,11 +1,16 @@
use backoff::BackoffConfig;
use iox_catalog::interface::Catalog;
use std::sync::Arc;
use time::TimeProvider;
use self::{namespace::NamespaceCache, partition::PartitionCache, table::TableCache};
use self::{
namespace::NamespaceCache, partition::PartitionCache,
processed_tombstones::ProcessedTombstonesCache, table::TableCache,
};
pub mod namespace;
pub mod partition;
pub mod processed_tombstones;
pub mod table;
#[cfg(test)]
@ -25,22 +30,28 @@ pub struct CatalogCache {
/// Namespace cache.
namespace_cache: NamespaceCache,
/// Processed tombstone cache.
processed_tombstones: ProcessedTombstonesCache,
}
impl CatalogCache {
/// Create empty cache.
pub fn new(catalog: Arc<dyn Catalog>) -> Self {
pub fn new(catalog: Arc<dyn Catalog>, time_provider: Arc<dyn TimeProvider>) -> Self {
let backoff_config = BackoffConfig::default();
let namespace_cache = NamespaceCache::new(Arc::clone(&catalog), backoff_config.clone());
let table_cache = TableCache::new(Arc::clone(&catalog), backoff_config.clone());
let partition_cache = PartitionCache::new(Arc::clone(&catalog), backoff_config);
let partition_cache = PartitionCache::new(Arc::clone(&catalog), backoff_config.clone());
let processed_tombstones =
ProcessedTombstonesCache::new(Arc::clone(&catalog), backoff_config, time_provider);
Self {
catalog,
partition_cache,
table_cache,
namespace_cache,
processed_tombstones,
}
}
@ -63,4 +74,9 @@ impl CatalogCache {
pub fn partition(&self) -> &PartitionCache {
&self.partition_cache
}
/// Processed tombstone cache.
pub fn processed_tombstones(&self) -> &ProcessedTombstonesCache {
&self.processed_tombstones
}
}

View File

@ -0,0 +1,162 @@
use std::{collections::HashMap, sync::Arc, time::Duration};
use backoff::{Backoff, BackoffConfig};
use data_types2::{ParquetFileId, TombstoneId};
use iox_catalog::interface::Catalog;
use time::TimeProvider;
use crate::cache_system::{
backend::ttl::{TtlBackend, TtlProvider},
driver::Cache,
loader::FunctionLoader,
};
/// Duration to keep "tombstone is NOT processed yet".
///
/// Marking tombstones as processed is a mere optimization, so we can keep this cache entry for a while.
pub const TTL_NOT_PROCESSED: Duration = Duration::from_secs(100);
/// Cache for processed tombstones.
#[derive(Debug)]
pub struct ProcessedTombstonesCache {
cache: Cache<(ParquetFileId, TombstoneId), bool>,
}
impl ProcessedTombstonesCache {
/// Create new empty cache.
pub fn new(
catalog: Arc<dyn Catalog>,
backoff_config: BackoffConfig,
time_provider: Arc<dyn TimeProvider>,
) -> Self {
let loader = Arc::new(FunctionLoader::new(
move |(parquet_file_id, tombstone_id)| {
let catalog = Arc::clone(&catalog);
let backoff_config = backoff_config.clone();
async move {
Backoff::new(&backoff_config)
.retry_all_errors("processed tombstone exists", || async {
catalog
.repositories()
.await
.processed_tombstones()
.exist(parquet_file_id, tombstone_id)
.await
})
.await
.expect("retry forever")
}
},
));
let backend = Box::new(HashMap::new());
let backend = Box::new(TtlBackend::new(
backend,
Arc::new(KeepExistsForever {}),
time_provider,
));
Self {
cache: Cache::new(loader, backend),
}
}
/// Check if the specified tombstone is mark as "processed" for the given parquet file.
pub async fn exists(&self, parquet_file_id: ParquetFileId, tombstone_id: TombstoneId) -> bool {
self.cache.get((parquet_file_id, tombstone_id)).await
}
}
#[derive(Debug)]
struct KeepExistsForever;
impl TtlProvider for KeepExistsForever {
type K = (ParquetFileId, TombstoneId);
type V = bool;
fn expires_in(&self, _k: &Self::K, v: &Self::V) -> Option<Duration> {
if *v {
// keep forever
None
} else {
// marking tombstones as processed is a mere optimization, so we can keep this cache entry for a while
Some(TTL_NOT_PROCESSED)
}
}
}
#[cfg(test)]
mod tests {
use iox_tests::util::TestCatalog;
use time::{MockProvider, Time};
use crate::cache::test_util::assert_histogram_metric_count;
use super::*;
#[tokio::test]
async fn test() {
let catalog = TestCatalog::new();
let ns = catalog.create_namespace("ns").await;
let table = ns.create_table("table").await;
let sequencer = ns.create_sequencer(1).await;
let partition = table.with_sequencer(&sequencer).create_partition("k").await;
let file1 = partition.create_parquet_file("table foo=1 11").await;
let file2 = partition.create_parquet_file("table foo=1 11").await;
let ts1 = table
.with_sequencer(&sequencer)
.create_tombstone(1, 1, 10, "foo=1")
.await;
let ts2 = table
.with_sequencer(&sequencer)
.create_tombstone(2, 1, 10, "foo=1")
.await;
ts1.mark_processed(&file1).await;
let time_provider = Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let cache = ProcessedTombstonesCache::new(
catalog.catalog(),
BackoffConfig::default(),
Arc::clone(&time_provider) as _,
);
assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await);
assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await);
assert!(!cache.exists(file2.parquet_file.id, ts1.tombstone.id).await);
assert!(!cache.exists(file2.parquet_file.id, ts2.tombstone.id).await);
assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 4);
ts2.mark_processed(&file2).await;
// values are cached for a while
assert!(TTL_NOT_PROCESSED > Duration::from_millis(1));
time_provider.inc(TTL_NOT_PROCESSED - Duration::from_millis(1));
assert!(!cache.exists(file2.parquet_file.id, ts2.tombstone.id).await);
assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 4);
time_provider.inc(Duration::from_millis(1));
assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await);
assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 5);
// "true" results are cached forever
assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await);
assert_histogram_metric_count(&catalog.metric_registry, "processed_tombstone_exist", 5);
// cache key has two dimensions
assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await);
assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await);
assert!(!cache.exists(file2.parquet_file.id, ts1.tombstone.id).await);
assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await);
ts1.mark_processed(&file2).await;
time_provider.inc(TTL_NOT_PROCESSED);
assert!(cache.exists(file1.parquet_file.id, ts1.tombstone.id).await);
assert!(!cache.exists(file1.parquet_file.id, ts2.tombstone.id).await);
assert!(cache.exists(file2.parquet_file.id, ts1.tombstone.id).await);
assert!(cache.exists(file2.parquet_file.id, ts2.tombstone.id).await);
}
}

View File

@ -0,0 +1,499 @@
use std::{
collections::{HashMap, VecDeque},
hash::Hash,
};
/// Addressable heap.
///
/// Stores a value `V` together with a key `K` and an order `O`. Elements are sorted by `O` and the smallest element can
/// be peeked/popped. At the same time elements can be addressed via `K`.
///
/// Note that `K` requires [`Ord`] to implement the inner data structure as a tie breaker.
/// structure.
#[derive(Debug, Clone)]
pub struct AddressableHeap<K, V, O>
where
K: Clone + Eq + Hash + Ord,
O: Clone + Ord,
{
/// Key to order and value.
///
/// The order is required to lookup data within the queue.
///
/// The value is stored here instead of the queue since HashMap entries are copied around less often than queue elements.
key_to_order_and_value: HashMap<K, (V, O)>,
/// Queue that handles the priorities.
///
/// The order goes first, the key goes second.
///
/// Note: This is not really a heap, but it fullfills the interface that we need.
queue: VecDeque<(O, K)>,
}
impl<K, V, O> AddressableHeap<K, V, O>
where
K: Clone + Eq + Hash + Ord,
O: Clone + Ord,
{
/// Create new, empty heap.
pub fn new() -> Self {
Self {
key_to_order_and_value: HashMap::new(),
queue: VecDeque::new(),
}
}
/// Insert element.
///
/// If the element (compared by `K`) already exists, it will be returned.
pub fn insert(&mut self, k: K, v: V, o: O) -> Option<(V, O)> {
// always remove the entry first so we have a clean queue
let result = self.remove(&k);
assert!(
self.key_to_order_and_value
.insert(k.clone(), (v, o.clone()))
.is_none(),
"entry should have been removed by now"
);
match self.queue.binary_search_by_key(&(&o, &k), project_tuple) {
Ok(_) => unreachable!("entry should have been removed by now"),
Err(index) => {
self.queue.insert(index, (o, k));
}
}
result
}
/// Peek first element (by smallest `O`).
pub fn peek(&self) -> Option<(&K, &V, &O)> {
if let Some((o, k)) = self.queue.front() {
let (v, o2) = self
.key_to_order_and_value
.get(k)
.expect("value is in queue");
assert!(o == o2);
Some((k, v, o))
} else {
None
}
}
/// Pop first element (by smallest `O`) from heap.
pub fn pop(&mut self) -> Option<(K, V, O)> {
if let Some((o, k)) = self.queue.pop_front() {
let (v, o2) = self
.key_to_order_and_value
.remove(&k)
.expect("value is in queue");
assert!(o == o2);
Some((k, v, o))
} else {
None
}
}
/// Get element by key.
pub fn get(&self, k: &K) -> Option<(&V, &O)> {
self.key_to_order_and_value.get(k).map(project_tuple)
}
/// Remove element by key.
///
/// If the element exists within the heap (addressed via `K`), the value and order will be returned.
pub fn remove(&mut self, k: &K) -> Option<(V, O)> {
if let Some((v, o)) = self.key_to_order_and_value.remove(k) {
let index = self
.queue
.binary_search_by_key(&(&o, k), project_tuple)
.expect("key was in key_to_order");
self.queue.remove(index);
Some((v, o))
} else {
None
}
}
}
impl<K, V, O> Default for AddressableHeap<K, V, O>
where
K: Clone + Eq + Hash + Ord,
O: Clone + Ord,
{
fn default() -> Self {
Self::new()
}
}
/// Project tuple references.
fn project_tuple<A, B>(t: &(A, B)) -> (&A, &B) {
(&t.0, &t.1)
}
#[cfg(test)]
mod tests {
use proptest::prelude::*;
use super::*;
#[test]
fn test_peek_empty() {
let heap = AddressableHeap::<i32, &str, i32>::new();
assert_eq!(heap.peek(), None);
}
#[test]
fn test_peek_some() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(3, "c", 5);
assert_eq!(heap.peek(), Some((&2, &"b", &3)));
}
#[test]
fn test_peek_tie() {
let mut heap = AddressableHeap::new();
heap.insert(3, "a", 1);
heap.insert(1, "b", 1);
heap.insert(2, "c", 1);
assert_eq!(heap.peek(), Some((&1, &"b", &1)));
}
#[test]
fn test_peek_after_remove() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(3, "c", 5);
assert_eq!(heap.peek(), Some((&2, &"b", &3)));
heap.remove(&3);
assert_eq!(heap.peek(), Some((&2, &"b", &3)));
heap.remove(&2);
assert_eq!(heap.peek(), Some((&1, &"a", &4)));
heap.remove(&1);
assert_eq!(heap.peek(), None);
}
#[test]
fn test_peek_after_override() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(1, "c", 2);
assert_eq!(heap.peek(), Some((&1, &"c", &2)));
}
#[test]
fn test_pop_empty() {
let mut heap = AddressableHeap::<i32, &str, i32>::new();
assert_eq!(heap.pop(), None);
}
#[test]
fn test_pop_all() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(3, "c", 5);
assert_eq!(heap.pop(), Some((2, "b", 3)));
assert_eq!(heap.pop(), Some((1, "a", 4)));
assert_eq!(heap.pop(), Some((3, "c", 5)));
assert_eq!(heap.pop(), None);
}
#[test]
fn test_pop_tie() {
let mut heap = AddressableHeap::new();
heap.insert(3, "a", 1);
heap.insert(1, "b", 1);
heap.insert(2, "c", 1);
assert_eq!(heap.pop(), Some((1, "b", 1)));
assert_eq!(heap.pop(), Some((2, "c", 1)));
assert_eq!(heap.pop(), Some((3, "a", 1)));
assert_eq!(heap.pop(), None);
}
#[test]
fn test_pop_after_insert() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(3, "c", 5);
assert_eq!(heap.pop(), Some((2, "b", 3)));
heap.insert(4, "d", 2);
assert_eq!(heap.pop(), Some((4, "d", 2)));
assert_eq!(heap.pop(), Some((1, "a", 4)));
}
#[test]
fn test_pop_after_remove() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(3, "c", 5);
heap.remove(&2);
assert_eq!(heap.pop(), Some((1, "a", 4)));
}
#[test]
fn test_pop_after_override() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.insert(1, "c", 2);
assert_eq!(heap.pop(), Some((1, "c", 2)));
assert_eq!(heap.pop(), Some((2, "b", 3)));
assert_eq!(heap.pop(), None);
}
#[test]
fn test_get_empty() {
let heap = AddressableHeap::<i32, &str, i32>::new();
assert_eq!(heap.get(&1), None);
}
#[test]
fn test_get_multiple() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
assert_eq!(heap.get(&1), Some((&"a", &4)));
assert_eq!(heap.get(&2), Some((&"b", &3)));
}
#[test]
fn test_get_after_remove() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.remove(&1);
assert_eq!(heap.get(&1), None);
assert_eq!(heap.get(&2), Some((&"b", &3)));
}
#[test]
fn test_get_after_pop() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.pop();
assert_eq!(heap.get(&1), Some((&"a", &4)));
assert_eq!(heap.get(&2), None);
}
#[test]
fn test_get_after_override() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(1, "b", 3);
assert_eq!(heap.get(&1), Some((&"b", &3)));
}
#[test]
fn test_remove_empty() {
let mut heap = AddressableHeap::<i32, &str, i32>::new();
assert_eq!(heap.remove(&1), None);
}
#[test]
fn test_remove_some() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
assert_eq!(heap.remove(&1), Some(("a", 4)));
assert_eq!(heap.remove(&2), Some(("b", 3)));
}
#[test]
fn test_remove_twice() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
assert_eq!(heap.remove(&1), Some(("a", 4)));
assert_eq!(heap.remove(&1), None);
}
#[test]
fn test_remove_after_pop() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(2, "b", 3);
heap.pop();
assert_eq!(heap.remove(&1), Some(("a", 4)));
assert_eq!(heap.remove(&2), None);
}
#[test]
fn test_remove_after_override() {
let mut heap = AddressableHeap::new();
heap.insert(1, "a", 4);
heap.insert(1, "b", 3);
assert_eq!(heap.remove(&1), Some(("b", 3)));
assert_eq!(heap.remove(&1), None);
}
#[test]
fn test_override() {
let mut heap = AddressableHeap::new();
assert_eq!(heap.insert(1, "a", 4), None);
assert_eq!(heap.insert(2, "b", 3), None);
assert_eq!(heap.insert(1, "c", 5), Some(("a", 4)));
}
/// Simple version of [`AddressableHeap`] for testing.
struct SimpleAddressableHeap {
inner: Vec<(u8, String, i8)>,
}
impl SimpleAddressableHeap {
fn new() -> Self {
Self { inner: Vec::new() }
}
fn insert(&mut self, k: u8, v: String, o: i8) -> Option<(String, i8)> {
let res = self.remove(&k);
self.inner.push((k, v, o));
res
}
fn peek(&self) -> Option<(&u8, &String, &i8)> {
self.inner
.iter()
.min_by_key(|(k, _v, o)| (o, k))
.map(|(k, v, o)| (k, v, o))
}
fn pop(&mut self) -> Option<(u8, String, i8)> {
self.inner
.iter()
.enumerate()
.min_by_key(|(_idx, (k, _v, o))| (o, k))
.map(|(idx, _)| idx)
.map(|idx| self.inner.remove(idx))
}
fn get(&self, k: &u8) -> Option<(&String, &i8)> {
self.inner
.iter()
.find(|(k2, _v, _o)| k2 == k)
.map(|(_k, v, o)| (v, o))
}
fn remove(&mut self, k: &u8) -> Option<(String, i8)> {
self.inner
.iter()
.enumerate()
.find(|(_idx, (k2, _v, _o))| k2 == k)
.map(|(idx, _)| idx)
.map(|idx| {
let (_k, v, o) = self.inner.remove(idx);
(v, o)
})
}
}
#[derive(Debug, Clone)]
enum Action {
Insert { k: u8, v: String, o: i8 },
Peek,
Pop,
Get { k: u8 },
Remove { k: u8 },
}
// Use a hand-rolled strategy instead of `proptest-derive`, because the latter one is quite a heavy dependency.
fn action() -> impl Strategy<Value = Action> {
prop_oneof![
(any::<u8>(), ".*", any::<i8>()).prop_map(|(k, v, o)| Action::Insert { k, v, o }),
Just(Action::Peek),
Just(Action::Pop),
any::<u8>().prop_map(|k| Action::Get { k }),
any::<u8>().prop_map(|k| Action::Remove { k }),
]
}
proptest! {
#[test]
fn test_proptest(actions in prop::collection::vec(action(), 0..100)) {
let mut heap = AddressableHeap::new();
let mut sim = SimpleAddressableHeap::new();
for action in actions {
match action {
Action::Insert{k, v, o} => {
let res1 = heap.insert(k, v.clone(), o);
let res2 = sim.insert(k, v, o);
assert_eq!(res1, res2);
}
Action::Peek => {
let res1 = heap.peek();
let res2 = sim.peek();
assert_eq!(res1, res2);
}
Action::Pop => {
let res1 = heap.pop();
let res2 = sim.pop();
assert_eq!(res1, res2);
}
Action::Get{k} => {
let res1 = heap.get(&k);
let res2 = sim.get(&k);
assert_eq!(res1, res2);
}
Action::Remove{k} => {
let res1 = heap.remove(&k);
let res2 = sim.remove(&k);
assert_eq!(res1, res2);
}
}
}
}
}
}

View File

@ -1,4 +1,5 @@
use std::{
any::Any,
collections::HashMap,
fmt::Debug,
hash::{BuildHasher, Hash},
@ -26,6 +27,10 @@ where
fn remove(&mut self, k: &Self::K) {
self.remove(k);
}
fn as_any(&self) -> &dyn Any {
self as &dyn Any
}
}
#[cfg(test)]

View File

@ -1,6 +1,8 @@
use std::{fmt::Debug, hash::Hash};
use std::{any::Any, fmt::Debug, hash::Hash};
mod addressable_heap;
pub mod hash_map;
pub mod ttl;
#[cfg(test)]
mod test_util;
@ -27,4 +29,7 @@ pub trait CacheBackend: Debug + Send + 'static {
///
/// It is OK to remove a key even when it does not exist.
fn remove(&mut self, k: &Self::K);
/// Return backend as [`Any`] which can be used to downcast to a specifc implementation.
fn as_any(&self) -> &dyn Any;
}

View File

@ -0,0 +1,450 @@
use std::{any::Any, fmt::Debug, hash::Hash, marker::PhantomData, sync::Arc, time::Duration};
use time::{Time, TimeProvider};
use super::{addressable_heap::AddressableHeap, CacheBackend};
/// Interface to provide TTL (time to live) data for a key-value pair.
pub trait TtlProvider: std::fmt::Debug + Send + Sync + 'static {
/// Cache key.
type K;
/// Cached value.
type V;
/// When should the given key-value pair expire?
///
/// Return `None` for "never".
///
/// The function is only called once for a newly cached key-value pair. This means:
/// - There is no need in remembering the time of a given pair (e.g. you can safely always return a constant).
/// - You cannot change the TTL after the data was cached.
///
/// Expiration is set to take place AT OR AFTER the provided duration.
fn expires_in(&self, k: &Self::K, v: &Self::V) -> Option<Duration>;
}
/// [`TtlProvider`] that never expires.
#[derive(Default)]
pub struct NeverTtlProvider<K, V>
where
K: 'static,
V: 'static,
{
// phantom data that is Send and Sync, see https://stackoverflow.com/a/50201389
_k: PhantomData<fn() -> K>,
_v: PhantomData<fn() -> V>,
}
impl<K, V> std::fmt::Debug for NeverTtlProvider<K, V> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("NeverTtlProvider").finish_non_exhaustive()
}
}
impl<K, V> TtlProvider for NeverTtlProvider<K, V> {
type K = K;
type V = V;
fn expires_in(&self, _k: &Self::K, _v: &Self::V) -> Option<Duration> {
None
}
}
/// Cache backend that implements Time To Life.
///
/// # Cache Eviction
/// Every method ([`get`](CacheBackend::get), [`set`](CacheBackend::set), [`remove`](CacheBackend::remove)) causes the
/// cache to check for expired keys. This may lead to certain delays, esp. when dropping the contained values takes a
/// long time.
#[derive(Debug)]
pub struct TtlBackend<K, V>
where
K: Clone + Eq + Debug + Hash + Ord + Send + 'static,
V: Clone + Debug + Send + 'static,
{
inner_backend: Box<dyn CacheBackend<K = K, V = V>>,
ttl_provider: Arc<dyn TtlProvider<K = K, V = V>>,
time_provider: Arc<dyn TimeProvider>,
expiration: AddressableHeap<K, (), Time>,
}
impl<K, V> TtlBackend<K, V>
where
K: Clone + Eq + Debug + Hash + Ord + Send + 'static,
V: Clone + Debug + Send + 'static,
{
/// Create new backend w/o any known keys.
///
/// The inner backend SHOULD NOT contain any data at this point, otherwise we will not track any TTLs for these entries.
pub fn new(
inner_backend: Box<dyn CacheBackend<K = K, V = V>>,
ttl_provider: Arc<dyn TtlProvider<K = K, V = V>>,
time_provider: Arc<dyn TimeProvider>,
) -> Self {
Self {
inner_backend,
ttl_provider,
time_provider,
expiration: Default::default(),
}
}
fn evict_expired(&mut self, now: Time) {
while self
.expiration
.peek()
.map(|(_k, _, t)| *t <= now)
.unwrap_or_default()
{
let (k, _, _t) = self.expiration.pop().unwrap();
self.inner_backend.remove(&k);
}
}
/// Reference to inner backend.
pub fn inner_backend(&self) -> &dyn CacheBackend<K = K, V = V> {
self.inner_backend.as_ref()
}
/// Reference to TTL provider.
pub fn ttl_provider(&self) -> &Arc<dyn TtlProvider<K = K, V = V>> {
&self.ttl_provider
}
}
impl<K, V> CacheBackend for TtlBackend<K, V>
where
K: Clone + Eq + Debug + Hash + Ord + Send + 'static,
V: Clone + Debug + Send + 'static,
{
type K = K;
type V = V;
fn get(&mut self, k: &Self::K) -> Option<Self::V> {
self.evict_expired(self.time_provider.now());
self.inner_backend.get(k)
}
fn set(&mut self, k: Self::K, v: Self::V) {
let now = self.time_provider.now();
self.evict_expired(now);
if let Some(ttl) = self
.ttl_provider
.expires_in(&k, &v)
.and_then(|d| now.checked_add(d))
{
self.expiration.insert(k.clone(), (), ttl);
} else {
// Still need to ensure that any current expiration is disabled
self.expiration.remove(&k);
}
self.inner_backend.set(k, v);
}
fn remove(&mut self, k: &Self::K) {
self.evict_expired(self.time_provider.now());
self.inner_backend.remove(k);
self.expiration.remove(k);
}
fn as_any(&self) -> &dyn Any {
self as &dyn Any
}
}
#[cfg(test)]
mod tests {
use std::{collections::HashMap, time::Duration};
use parking_lot::Mutex;
use time::MockProvider;
use super::*;
#[test]
fn test_expires_single() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
time_provider.inc(Duration::from_secs(1));
assert_eq!(backend.get(&1), None);
}
#[test]
fn test_overflow_expire() {
let ttl_provider = Arc::new(TestTtlProvider::new());
// init time provider at MAX!
let time_provider = Arc::new(MockProvider::new(Time::MAX));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::MAX));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_never_expire() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), None);
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
time_provider.inc(Duration::from_secs(1));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_expiration_uses_key_and_value() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
ttl_provider.set_expires_in(1, String::from("b"), Some(Duration::from_secs(4)));
ttl_provider.set_expires_in(2, String::from("a"), Some(Duration::from_secs(2)));
backend.set(1, String::from("b"));
time_provider.inc(Duration::from_secs(3));
assert_eq!(backend.get(&1), Some(String::from("b")));
}
#[test]
fn test_override_with_different_expiration() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(3)));
backend.set(1, String::from("a"));
time_provider.inc(Duration::from_secs(2));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_override_with_no_expiration() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
ttl_provider.set_expires_in(1, String::from("a"), None);
backend.set(1, String::from("a"));
time_provider.inc(Duration::from_secs(2));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_readd_with_different_expiration() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(3)));
backend.remove(&1);
backend.set(1, String::from("a"));
time_provider.inc(Duration::from_secs(2));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_readd_with_no_expiration() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
ttl_provider.set_expires_in(1, String::from("a"), None);
backend.remove(&1);
backend.set(1, String::from("a"));
time_provider.inc(Duration::from_secs(2));
assert_eq!(backend.get(&1), Some(String::from("a")));
}
#[test]
fn test_update_cleans_multiple_keys() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
ttl_provider.set_expires_in(2, String::from("b"), Some(Duration::from_secs(2)));
ttl_provider.set_expires_in(3, String::from("c"), Some(Duration::from_secs(2)));
ttl_provider.set_expires_in(4, String::from("d"), Some(Duration::from_secs(3)));
backend.set(1, String::from("a"));
backend.set(2, String::from("b"));
backend.set(3, String::from("c"));
backend.set(4, String::from("d"));
assert_eq!(backend.get(&1), Some(String::from("a")));
assert_eq!(backend.get(&2), Some(String::from("b")));
assert_eq!(backend.get(&3), Some(String::from("c")));
assert_eq!(backend.get(&4), Some(String::from("d")));
time_provider.inc(Duration::from_secs(2));
assert_eq!(backend.get(&1), None);
let inner_backend = backend
.inner_backend()
.as_any()
.downcast_ref::<HashMap<u8, String>>()
.unwrap();
assert!(!inner_backend.contains_key(&1));
assert!(!inner_backend.contains_key(&2));
assert!(!inner_backend.contains_key(&3));
assert!(inner_backend.contains_key(&4));
assert_eq!(backend.get(&2), None);
assert_eq!(backend.get(&3), None);
assert_eq!(backend.get(&4), Some(String::from("d")));
}
#[test]
fn test_remove_expired_key() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
backend.set(1, String::from("a"));
assert_eq!(backend.get(&1), Some(String::from("a")));
time_provider.inc(Duration::from_secs(1));
backend.remove(&1);
assert_eq!(backend.get(&1), None);
}
#[test]
fn test_expire_removed_key() {
let ttl_provider = Arc::new(TestTtlProvider::new());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
let mut backend = TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
Arc::clone(&ttl_provider) as _,
Arc::clone(&time_provider) as _,
);
ttl_provider.set_expires_in(1, String::from("a"), Some(Duration::from_secs(1)));
ttl_provider.set_expires_in(2, String::from("b"), Some(Duration::from_secs(2)));
backend.set(1, String::from("a"));
backend.remove(&1);
time_provider.inc(Duration::from_secs(1));
backend.set(2, String::from("b"));
assert_eq!(backend.get(&1), None);
assert_eq!(backend.get(&2), Some(String::from("b")));
}
#[test]
fn test_generic() {
use crate::cache_system::backend::test_util::test_generic;
test_generic(|| {
let ttl_provider = Arc::new(NeverTtlProvider::default());
let time_provider = Arc::new(MockProvider::new(Time::MIN));
TtlBackend::new(
Box::new(HashMap::<u8, String>::new()),
ttl_provider,
time_provider,
)
});
}
#[derive(Debug)]
struct TestTtlProvider {
expires_in: Mutex<HashMap<(u8, String), Option<Duration>>>,
}
impl TestTtlProvider {
fn new() -> Self {
Self {
expires_in: Mutex::new(HashMap::new()),
}
}
fn set_expires_in(&self, k: u8, v: String, d: Option<Duration>) {
self.expires_in.lock().insert((k, v), d);
}
}
impl TtlProvider for TestTtlProvider {
type K = u8;
type V = String;
fn expires_in(&self, k: &Self::K, v: &Self::V) -> Option<Duration> {
*self
.expires_in
.lock()
.get(&(*k, v.clone()))
.expect("expires_in value not mocked")
}
}
}

View File

@ -4,8 +4,12 @@ use futures::{
future::{BoxFuture, Shared},
FutureExt, TryFutureExt,
};
use observability_deps::tracing::debug;
use parking_lot::Mutex;
use tokio::{sync::oneshot::error::RecvError, task::JoinHandle};
use tokio::{
sync::oneshot::{error::RecvError, Sender},
task::JoinHandle,
};
use super::{backend::CacheBackend, loader::Loader};
@ -61,22 +65,24 @@ where
}
// check if there is already a query for this key running
if let Some((receiver, _handle)) = state.running_queries.get(&k) {
receiver.clone()
if let Some(running_query) = state.running_queries.get(&k) {
running_query.recv.clone()
} else {
// requires new query
let (tx, rx) = tokio::sync::oneshot::channel();
let receiver = rx
let (tx_main, rx_main) = tokio::sync::oneshot::channel();
let receiver = rx_main
.map_ok(|v| Arc::new(Mutex::new(v)))
.map_err(Arc::new)
.boxed()
.shared();
let (tx_set, rx_set) = tokio::sync::oneshot::channel();
// need to wrap the query into a tokio task so that it doesn't get cancelled when this very request is canceled
let state_captured = Arc::clone(&self.state);
let loader = Arc::clone(&self.loader);
let k_captured = k.clone();
let handle = tokio::spawn(async move {
let loader_fut = async move {
// need to clone K and bind it so rustc doesn't require `K: Sync`
let k_for_loader = k_captured.clone();
@ -84,10 +90,6 @@ where
// If we panic here then `tx` will be dropped and the receivers will be notified.
let v = loader.load(k_for_loader).await;
// broadcast result
// It's OK if the receiver side is gone. This might happen during shutdown
tx.send(v.clone()).ok();
// remove "running" state and store result
//
// Note: we need to manually drop the result of `.remove(...).expect(...)` here to convince rustc
@ -102,10 +104,44 @@ where
.remove(&k_captured)
.expect("query should be running"),
);
state.cached_entries.set(k_captured, v);
state.cached_entries.set(k_captured, v.clone());
v
};
let v = tokio::select! {
v = loader_fut => v,
maybe_v = rx_set => {
match maybe_v {
Ok(v) => {
// data get side-loaded via `Cache::set`. In this case, we do NOT modify the state
// because there would be a lock-gap. The `set` function will do that for us instead.
v
}
Err(_) => {
// sender side is gone, very likely the cache is shutting down
debug!(
"Sender for side-loading data into running query gone.",
);
return;
}
}
}
};
// broadcast result
// It's OK if the receiver side is gone. This might happen during shutdown
tx_main.send(v).ok();
});
state.running_queries.insert(k, (receiver.clone(), handle));
state.running_queries.insert(
k,
RunningQuery {
recv: receiver.clone(),
set: tx_set,
join_handle: handle,
},
);
receiver
}
};
@ -116,6 +152,24 @@ where
.lock()
.clone()
}
/// Side-load an entry into the cache.
///
/// This will also complete a currently running request for this key.
pub fn set(&self, k: K, v: V) {
let mut state = self.state.lock();
if let Some(running_query) = state.running_queries.remove(&k) {
// it's OK when the receiver side is gone (likely panicked)
running_query.set.send(v.clone()).ok();
// When we side-load data into the running task, the task does NOT modify the backend, so we have to do
// that. The reason for not letting the task feed the side-loaded data back into `cached_entries` is that we
// would need to drop the state lock here before the task could acquire it, leading to a lock gap.
}
state.cached_entries.set(k, v);
}
}
impl<K, V> Drop for Cache<K, V>
@ -124,11 +178,11 @@ where
V: Clone + std::fmt::Debug + Send + 'static,
{
fn drop(&mut self) {
for (_k, (_receiver, handle)) in self.state.lock().running_queries.drain() {
for (_k, running_query) in self.state.lock().running_queries.drain() {
// It's unlikely that anyone is still using the shared receiver at this point, because Cache::get borrow
// the self. If it is still in use, aborting the task will cancel the contained future which in turn will
// drop the sender of the oneshot channel. The receivers will be notified.
handle.abort();
running_query.join_handle.abort();
}
}
}
@ -144,6 +198,21 @@ where
/// - `Shared`: Allow the receiver to be cloned and be awaited from multiple places.
type SharedReceiver<V> = Shared<BoxFuture<'static, Result<Arc<Mutex<V>>, Arc<RecvError>>>>;
/// State for coordinating the execution of a single running query.
#[derive(Debug)]
struct RunningQuery<V> {
/// A receiver that can await the result as well.
recv: SharedReceiver<V>,
/// A sender that enables setting entries while the query is running.
set: Sender<V>,
/// A handle for the task that is currently executing the query.
///
/// The handle can be used to abort the running query, e.g. when dropping the cache.
join_handle: JoinHandle<()>,
}
/// Inner cache state that is usually guarded by a lock.
///
/// The state parts must be updated in a consistent manner, i.e. while using the same lock guard.
@ -153,10 +222,7 @@ struct CacheState<K, V> {
cached_entries: Box<dyn CacheBackend<K = K, V = V>>,
/// Currently running queries indexed by cache key.
///
/// For each query we have a receiver that can await the result as well as a handle for the task that is currently
/// executing the query. The handle can be used to abort the running query, e.g. when dropping the cache.
running_queries: HashMap<K, (SharedReceiver<V>, JoinHandle<()>)>,
running_queries: HashMap<K, RunningQuery<V>>,
}
#[cfg(test)]
@ -308,6 +374,50 @@ mod tests {
assert_eq!(Arc::strong_count(&loader), 1);
}
#[tokio::test]
async fn test_set_before_request() {
let (cache, loader) = setup();
loader.block();
cache.set(1, String::from("foo"));
// blocked loader is not used
let res = tokio::time::timeout(Duration::from_millis(10), cache.get(1))
.await
.unwrap();
assert_eq!(res, String::from("foo"));
assert_eq!(loader.loaded(), Vec::<u8>::new());
}
#[tokio::test]
async fn test_set_during_request() {
let (cache, loader) = setup();
loader.block();
let cache_captured = Arc::clone(&cache);
let handle = tokio::spawn(async move { cache_captured.get(1).await });
tokio::time::sleep(Duration::from_millis(10)).await;
cache.set(1, String::from("foo"));
// request succeeds even though the loader is blocked
let res = tokio::time::timeout(Duration::from_millis(10), handle)
.await
.unwrap()
.unwrap();
assert_eq!(res, String::from("foo"));
assert_eq!(loader.loaded(), vec![1]);
// still cached
let res = tokio::time::timeout(Duration::from_millis(10), cache.get(1))
.await
.unwrap();
assert_eq!(res, String::from("foo"));
assert_eq!(loader.loaded(), vec![1]);
}
fn setup() -> (Arc<Cache<u8, String>>, Arc<TestLoader>) {
let loader = Arc::new(TestLoader::default());
let cache = Arc::new(Cache::new(

View File

@ -165,7 +165,10 @@ mod tests {
let catalog = TestCatalog::new();
let adapter = ParquetChunkAdapter::new(
Arc::new(CatalogCache::new(catalog.catalog())),
Arc::new(CatalogCache::new(
catalog.catalog(),
catalog.time_provider(),
)),
catalog.object_store(),
catalog.metric_registry(),
catalog.time_provider(),

View File

@ -63,7 +63,10 @@ impl QuerierDatabase {
time_provider: Arc<dyn TimeProvider>,
exec: Arc<Executor>,
) -> Self {
let catalog_cache = Arc::new(CatalogCache::new(Arc::clone(&catalog)));
let catalog_cache = Arc::new(CatalogCache::new(
Arc::clone(&catalog),
Arc::clone(&time_provider),
));
Self {
backoff_config: BackoffConfig::default(),

View File

@ -575,17 +575,11 @@ impl QuerierNamespace {
let mut predicates_filtered = vec![];
for (tombstone_id, predicate) in predicates {
let is_processed = Backoff::new(&self.backoff_config)
.retry_all_errors("processed tombstone exists", || async {
self.catalog
.repositories()
.await
let is_processed = self
.catalog_cache
.processed_tombstones()
.exist(parquet_file_id, *tombstone_id)
.await
})
.await
.expect("retry forever");
.exists(parquet_file_id, *tombstone_id)
.await;
if !is_processed {
predicates_filtered.push(Arc::clone(predicate));
@ -606,7 +600,9 @@ impl QuerierNamespace {
#[cfg(test)]
mod tests {
use super::*;
use crate::namespace::test_util::querier_namespace;
use crate::{
cache::processed_tombstones::TTL_NOT_PROCESSED, namespace::test_util::querier_namespace,
};
use data_types2::{ChunkAddr, ChunkId, ColumnType, PartitionAddr};
use iox_tests::util::{TestCatalog, TestParquetFile};
use schema::{builder::SchemaBuilder, InfluxColumnType, InfluxFieldType};
@ -617,7 +613,10 @@ mod tests {
let catalog = TestCatalog::new();
let querier_namespace = QuerierNamespace::new(
Arc::new(CatalogCache::new(catalog.catalog())),
Arc::new(CatalogCache::new(
catalog.catalog(),
catalog.time_provider(),
)),
"ns".into(),
NamespaceId::new(1),
catalog.metric_registry(),
@ -935,6 +934,7 @@ mod tests {
.with_sequencer(&sequencer1)
.create_tombstone(4, 1, 10, "foo=4")
.await;
catalog.mock_time_provider().inc(TTL_NOT_PROCESSED); // cache timeout for processed tombstones
querier_namespace.sync().await;
assert_eq!(
delete_predicates(&querier_namespace),

View File

@ -7,7 +7,10 @@ use super::QuerierNamespace;
pub fn querier_namespace(catalog: &Arc<TestCatalog>, ns: &Arc<TestNamespace>) -> QuerierNamespace {
QuerierNamespace::new(
Arc::new(CatalogCache::new(catalog.catalog())),
Arc::new(CatalogCache::new(
catalog.catalog(),
catalog.time_provider(),
)),
ns.namespace.name.clone().into(),
ns.namespace.id,
catalog.metric_registry(),

View File

@ -25,6 +25,7 @@ datafusion_util = { path = "../datafusion_util" }
executor = { path = "../executor"}
futures = "0.3"
hashbrown = "0.12"
itertools = "0.10.2"
observability_deps = { path = "../observability_deps" }
parking_lot = "0.12"
pin-project = "1.0"

View File

@ -1,8 +1,13 @@
//! This module contains DataFusion utility functions and helpers
use std::{convert::TryInto, sync::Arc};
use std::{
cmp::{max, min},
convert::TryInto,
sync::Arc,
};
use arrow::{
array::TimestampNanosecondArray,
compute::SortOptions,
datatypes::{DataType, Schema as ArrowSchema},
record_batch::RecordBatch,
@ -22,9 +27,33 @@ use datafusion::{
scalar::ScalarValue,
};
use itertools::Itertools;
use observability_deps::tracing::trace;
use predicate::rpc_predicate::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME};
use schema::{sort::SortKey, Schema};
use schema::{sort::SortKey, Schema, TIME_COLUMN_NAME};
use snafu::{ensure, OptionExt, ResultExt, Snafu};
#[derive(Debug, Snafu)]
#[allow(missing_copy_implementations, missing_docs)]
pub enum Error {
#[snafu(display("The Record batch is empty"))]
EmptyBatch,
#[snafu(display("Error while searching Time column in a Record Batch"))]
TimeColumn { source: arrow::error::ArrowError },
#[snafu(display("Error while casting Timenanosecond on Time column"))]
TimeCasting,
#[snafu(display("Time column does not have value"))]
TimeValue,
#[snafu(display("Time column is null"))]
TimeNull,
}
/// A specialized `Error`
pub type Result<T, E = Error> = std::result::Result<T, E>;
/// Create a logical plan that produces the record batch
pub fn make_scan_plan(batch: RecordBatch) -> std::result::Result<LogicalPlan, DataFusionError> {
@ -198,6 +227,49 @@ impl<'a> ExprRewriter for MissingColumnsToNull<'a> {
}
}
/// Return min and max for column `time` of the given set of record batches
pub fn compute_timenanosecond_min_max(batches: &[RecordBatch]) -> Result<(i64, i64)> {
let mut min_time = i64::MAX;
let mut max_time = i64::MIN;
for batch in batches {
let (mi, ma) = compute_timenanosecond_min_max_for_one_record_batch(batch)?;
min_time = min(min_time, mi);
max_time = max(max_time, ma);
}
Ok((min_time, max_time))
}
/// Return min and max for column `time` in the given record batch
pub fn compute_timenanosecond_min_max_for_one_record_batch(
batch: &RecordBatch,
) -> Result<(i64, i64)> {
ensure!(batch.num_columns() > 0, EmptyBatchSnafu);
let index = batch
.schema()
.index_of(TIME_COLUMN_NAME)
.context(TimeColumnSnafu {})?;
let time_col = batch
.column(index)
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.context(TimeCastingSnafu {})?;
let (min, max) = match time_col.iter().minmax() {
itertools::MinMaxResult::NoElements => return Err(Error::TimeValue),
itertools::MinMaxResult::OneElement(val) => {
let val = val.context(TimeNullSnafu)?;
(val, val)
}
itertools::MinMaxResult::MinMax(min, max) => {
(min.context(TimeNullSnafu)?, max.context(TimeNullSnafu)?)
}
};
Ok((min, max))
}
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType;

View File

@ -41,7 +41,7 @@ assert_matches = "1.5"
criterion = { version = "0.3.4", features = ["async_tokio", "html_reports"] }
lazy_static = "1.4.0"
paste = "1.0.6"
pretty_assertions = "1.1.0"
pretty_assertions = "1.2.0"
rand = "0.8.3"
schema = { path = "../schema" }