From ce155ac1c1dd1f738252f0caf9af1771d99b12da Mon Sep 17 00:00:00 2001 From: Benjamin Naecker Date: Tue, 17 Sep 2024 06:07:01 -0700 Subject: [PATCH] Add a query planner for OxQL - Add types to represent the plan tree and the currently-supported plan tree nodes. These mostly correspond to the existing query AST nodes, but include information about the expected schema for the input and output tables, along with the query AST nodes that "implement" that transformation. - Add an explicit node for computing deltas from a cumulative timeseries, automatically after the node for fetching its data from the DB. This is currently implicitly done after fetching the data, but will be part of an explicit plan step going forward. The ultimate goal is to push that into the database itself where possible. - Adds methods to optimize a query plan, which currently includes the predicate-pushdown and limit-pushdown tricks we already do to limit the amount of data we get from the database. Adds some tests to verify behavior of these optimizations, in particular that they don't change the _planned_ output of the query itself. - Add pretty-printing of the plan tree, and include a way to show that in the OxQL shell. - Add detection of full table scans. Use the planner in OxQL queries, _only to verify them_ and check that there are no scans. The queries themselves are executed in the original method today. --- Cargo.lock | 9 +- Cargo.toml | 1 + nexus/tests/integration_tests/metrics.rs | 14 +- oximeter/db/Cargo.toml | 1 + oximeter/db/src/client/oxql.rs | 54 +- oximeter/db/src/native/mod.rs | 5 + oximeter/db/src/oxql/ast/literal.rs | 31 + oximeter/db/src/oxql/ast/mod.rs | 49 + oximeter/db/src/oxql/ast/table_ops/align.rs | 24 +- .../table_ops/{filter.rs => filter/mod.rs} | 150 +- .../db/src/oxql/ast/table_ops/filter/visit.rs | 494 ++++++ .../db/src/oxql/ast/table_ops/group_by.rs | 24 + oximeter/db/src/oxql/ast/table_ops/limit.rs | 13 +- oximeter/db/src/oxql/ast/table_ops/mod.rs | 40 + oximeter/db/src/oxql/mod.rs | 4 +- oximeter/db/src/oxql/plan/align.rs | 90 ++ oximeter/db/src/oxql/plan/delta.rs | 49 + oximeter/db/src/oxql/plan/filter.rs | 383 +++++ oximeter/db/src/oxql/plan/get.rs | 85 + oximeter/db/src/oxql/plan/group_by.rs | 134 ++ oximeter/db/src/oxql/plan/join.rs | 101 ++ oximeter/db/src/oxql/plan/limit.rs | 17 + oximeter/db/src/oxql/plan/mod.rs | 20 + oximeter/db/src/oxql/plan/node.rs | 113 ++ oximeter/db/src/oxql/plan/plan.rs | 1366 +++++++++++++++++ oximeter/db/src/oxql/plan/predicates.rs | 649 ++++++++ oximeter/db/src/oxql/query/mod.rs | 20 +- oximeter/db/src/oxql/schema.rs | 121 ++ oximeter/db/src/shells/oxql.rs | 59 +- oximeter/db/tests/integration_test.rs | 6 +- oximeter/db/tests/timeseries-schema.json | 83 + oximeter/oxql-types/src/point.rs | 16 +- 32 files changed, 4195 insertions(+), 30 deletions(-) rename oximeter/db/src/oxql/ast/table_ops/{filter.rs => filter/mod.rs} (91%) create mode 100644 oximeter/db/src/oxql/ast/table_ops/filter/visit.rs create mode 100644 oximeter/db/src/oxql/plan/align.rs create mode 100644 oximeter/db/src/oxql/plan/delta.rs create mode 100644 oximeter/db/src/oxql/plan/filter.rs create mode 100644 oximeter/db/src/oxql/plan/get.rs create mode 100644 oximeter/db/src/oxql/plan/group_by.rs create mode 100644 oximeter/db/src/oxql/plan/join.rs create mode 100644 oximeter/db/src/oxql/plan/limit.rs create mode 100644 oximeter/db/src/oxql/plan/mod.rs create mode 100644 oximeter/db/src/oxql/plan/node.rs create mode 100644 oximeter/db/src/oxql/plan/plan.rs create mode 100644 oximeter/db/src/oxql/plan/predicates.rs create mode 100644 oximeter/db/src/oxql/schema.rs create mode 100644 oximeter/db/tests/timeseries-schema.json diff --git a/Cargo.lock b/Cargo.lock index f5e44318bb..0af4e739c2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7907,6 +7907,7 @@ dependencies = [ "strum", "tabled", "tempfile", + "termtree 0.5.1", "thiserror 1.0.69", "tokio", "tokio-util", @@ -8795,7 +8796,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "41b740d195ed3166cd147c8047ec98db0e22ec019eb8eeb76d343b795304fb13" dependencies = [ "predicates-core", - "termtree", + "termtree 0.4.1", ] [[package]] @@ -11595,6 +11596,12 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" +[[package]] +name = "termtree" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f50febec83f5ee1df3015341d8bd429f2d1cc62bcba7ea2076759d315084683" + [[package]] name = "test-strategy" version = "0.3.1" diff --git a/Cargo.toml b/Cargo.toml index b65617f082..844ff1c7aa 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -629,6 +629,7 @@ tar = "0.4" tempfile = "3.10" term = "0.7" termios = "0.3" +termtree = "0.5.1" textwrap = "0.16.1" test-strategy = "0.3.1" thiserror = "1.0" diff --git a/nexus/tests/integration_tests/metrics.rs b/nexus/tests/integration_tests/metrics.rs index 7e5441c16a..7131fe4de9 100644 --- a/nexus/tests/integration_tests/metrics.rs +++ b/nexus/tests/integration_tests/metrics.rs @@ -450,7 +450,8 @@ async fn test_instance_watcher_metrics( const STATE_STARTING: &str = "starting"; const STATE_RUNNING: &str = "running"; const STATE_STOPPING: &str = "stopping"; - const OXQL_QUERY: &str = "get virtual_machine:check"; + const OXQL_QUERY: &str = "get virtual_machine:check | \ + filter timestamp > @2000-01-01"; let client = &cptestctx.external_client; let internal_client = &cptestctx.internal_client; @@ -707,7 +708,13 @@ async fn test_project_timeseries_query( // fields are. This is helpful generally, but here it would be better if // we could say something more like "you can't query this timeseries from // this endpoint" - assert_eq!(result.message, "The filter expression contains identifiers that are not valid for its input timeseries. Invalid identifiers: [\"project_id\", \"silo_id\"], timeseries fields: {\"datum\", \"metric_name\", \"target_name\", \"timestamp\"}"); + const EXPECTED_ERROR_MESSAGE: &str = "\ + The filter expression refers to \ + identifiers that are not valid for its input \ + table \"integration_target:integration_metric\". \ + Invalid identifiers: [\"silo_id\", \"project_id\"], \ + valid identifiers: [\"datum\", \"metric_name\", \"target_name\", \"timestamp\"]"; + assert!(result.message.ends_with(EXPECTED_ERROR_MESSAGE)); // nonexistent project let url = "/v1/timeseries/query?project=nonexistent"; @@ -872,7 +879,8 @@ async fn test_mgs_metrics( return; } - let query = format!("get {metric_name}"); + let query = + format!("get {metric_name} | filter timestamp > @2000-01-01"); // MGS polls SP sensor data once every second. It's possible that, when // we triggered Oximeter to collect samples from MGS, it may not have diff --git a/oximeter/db/Cargo.toml b/oximeter/db/Cargo.toml index 180f9738d2..df341ceeb0 100644 --- a/oximeter/db/Cargo.toml +++ b/oximeter/db/Cargo.toml @@ -43,6 +43,7 @@ slog-async.workspace = true slog-dtrace.workspace = true slog-term.workspace = true strum.workspace = true +termtree.workspace = true thiserror.workspace = true tokio-util.workspace = true usdt.workspace = true diff --git a/oximeter/db/src/client/oxql.rs b/oximeter/db/src/client/oxql.rs index 52be9ae81c..fe10f1e961 100644 --- a/oximeter/db/src/client/oxql.rs +++ b/oximeter/db/src/client/oxql.rs @@ -15,6 +15,7 @@ use crate::oxql::ast::table_ops::filter; use crate::oxql::ast::table_ops::filter::Filter; use crate::oxql::ast::table_ops::limit::Limit; use crate::oxql::ast::table_ops::limit::LimitKind; +use crate::oxql::Query; use crate::query::field_table_name; use crate::Error; use crate::Metric; @@ -113,6 +114,34 @@ struct ConsistentKeyGroup { } impl Client { + /// Build a query plan for the OxQL query. + pub async fn plan_oxql_query( + &self, + query: impl AsRef, + ) -> Result { + let query = query.as_ref(); + let parsed_query = oxql::Query::new(query)?; + self.build_query_plan(&parsed_query).await + } + + /// Build a query plan for the OxQL query. + async fn build_query_plan( + &self, + query: &Query, + ) -> Result { + let referenced_timeseries = query.all_timeseries_names(); + let mut schema = BTreeMap::new(); + for name in referenced_timeseries.into_iter() { + let Some(sch) = self.schema_for_timeseries(name).await? else { + return Err(Error::TimeseriesNotFound(name.to_string())); + }; + schema.insert(name.clone(), sch); + } + let plan = + oxql::plan::Plan::new(query.parsed_query().clone(), &schema)?; + Ok(plan) + } + /// Run a OxQL query. pub async fn oxql_query( &self, @@ -132,6 +161,15 @@ impl Client { // See https://github.com/oxidecomputer/omicron/issues/5298. let query = query.as_ref(); let parsed_query = oxql::Query::new(query)?; + let plan = self.build_query_plan(&parsed_query).await?; + if plan.requires_full_table_scan() { + return Err(Error::Oxql(anyhow::anyhow!( + "This query requires at least one full table scan. \ + Please rewrite the query to filter either the fields \ + or timestamps, in order to reduce the amount of data \ + fetched from the database." + ))); + } let query_id = Uuid::new_v4(); let query_log = self.log.new(slog::o!("query_id" => query_id.to_string())); @@ -837,12 +875,12 @@ impl Client { // return. // // This is used to ensure that we never go above the limit in - // `MAX_RESULT_SIZE`. That restricts the _total_ number of rows we want - // to retch from the database. So we set our limit to be one more than - // the remainder on our allotment. If we get exactly as many as we set - // in the limit, then we fail the query because there are more rows that - // _would_ be returned. We don't know how many more, but there is at - // least 1 that pushes us over the limit. This prevents tricky + // `MAX_DATABASE_ROWS`. That restricts the _total_ number of rows we + // want to retch from the database. So we set our limit to be one more + // than the remainder on our allotment. If we get exactly as many as we + // set in the limit, then we fail the query because there are more row + // that _would_ be returned. We don't know how many more, but there is + // at least 1 that pushes us over the limit. This prevents tricky // TOCTOU-like bugs where we need to check the limit twice, and improves // performance, since we don't return much more than we could possibly // handle. @@ -1293,7 +1331,9 @@ mod tests { #[tokio::test] async fn test_get_entire_table() { let ctx = setup_oxql_test("test_get_entire_table").await; - let query = "get some_target:some_metric"; + // We need _some_ filter here to avoid a provable full-table scan. + let query = + "get some_target:some_metric | filter timestamp > @2020-01-01"; let result = ctx .client .oxql_query(query) diff --git a/oximeter/db/src/native/mod.rs b/oximeter/db/src/native/mod.rs index 4861b4e044..1f0e76cb21 100644 --- a/oximeter/db/src/native/mod.rs +++ b/oximeter/db/src/native/mod.rs @@ -239,6 +239,11 @@ pub enum Error { #[error("Expected an empty data block")] ExpectedEmptyDataBlock, + + #[error( + "A query unexpectedly resulted in an empty data block; query: {query}" + )] + UnexpectedEmptyBlock { query: String }, } impl Error { diff --git a/oximeter/db/src/oxql/ast/literal.rs b/oximeter/db/src/oxql/ast/literal.rs index d80977fe49..4420a09fa6 100644 --- a/oximeter/db/src/oxql/ast/literal.rs +++ b/oximeter/db/src/oxql/ast/literal.rs @@ -13,6 +13,7 @@ use chrono::DateTime; use chrono::Utc; use oximeter::FieldType; use oximeter::FieldValue; +use oxql_types::point::DataType; use regex::Regex; use std::borrow::Borrow; use std::fmt; @@ -35,6 +36,20 @@ pub enum Literal { } impl Literal { + // Return the name of this literal's type as a string. + pub(crate) fn type_name(&self) -> &'static str { + match self { + Literal::Integer(_) => "Integer", + Literal::Double(_) => "Double", + Literal::String(_) => "String", + Literal::Boolean(_) => "Boolean", + Literal::Uuid(_) => "Uuid", + Literal::Duration(_) => "Duration", + Literal::Timestamp(_) => "Timestamp", + Literal::IpAddr(_) => "IpAddr", + } + } + // Format the literal as a safe, typed string for ClickHouse. pub(crate) fn as_db_safe_string(&self) -> String { match self { @@ -93,6 +108,22 @@ impl Literal { } } + // Return true if this literal can be compared to a datum of the provided + // type. + pub(crate) fn is_compatible_with_datum(&self, data_type: DataType) -> bool { + match (self, data_type) { + (Literal::Integer(_), DataType::Integer) + | (Literal::Double(_), DataType::Double) + | (Literal::String(_), DataType::String) + | (Literal::Boolean(_), DataType::Boolean) + | (Literal::Duration(_), DataType::Integer) + | (Literal::Duration(_), DataType::Double) + | (Literal::Timestamp(_), DataType::Integer) + | (Literal::Timestamp(_), DataType::Double) => true, + (_, _) => false, + } + } + /// Apply the comparison op between self and the provided field. /// /// Return None if the comparison cannot be applied, either because the type diff --git a/oximeter/db/src/oxql/ast/mod.rs b/oximeter/db/src/oxql/ast/mod.rs index 21fe5b0387..8d09bb4096 100644 --- a/oximeter/db/src/oxql/ast/mod.rs +++ b/oximeter/db/src/oxql/ast/mod.rs @@ -6,6 +6,9 @@ // Copyright 2024 Oxide Computer Company +use std::collections::BTreeSet; +use std::fmt; + use chrono::DateTime; use chrono::Utc; use oximeter::TimeseriesName; @@ -26,12 +29,32 @@ pub struct Query { ops: Vec, } +impl fmt::Display for Query { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let n_ops = self.ops.len(); + for (i, op) in self.ops.iter().enumerate() { + write!(f, "{op}")?; + if i < n_ops - 1 { + write!(f, " | ")?; + } + } + Ok(()) + } +} + impl Query { // Return the first operation in the query, which is always a form of `get`. fn first_op(&self) -> &TableOp { self.ops.first().expect("Should have parsed at least 1 operation") } + /// Iterate over the table operations. + pub(crate) fn table_ops( + &self, + ) -> impl ExactSizeIterator + '_ { + self.ops.iter() + } + pub(crate) fn timeseries_name(&self) -> &TimeseriesName { match self.first_op() { TableOp::Basic(BasicTableOp::Get(n)) => n, @@ -42,6 +65,32 @@ impl Query { } } + /// Return _all_ timeseries names referred to by get table operations. + pub(crate) fn all_timeseries_names(&self) -> BTreeSet<&TimeseriesName> { + let mut set = BTreeSet::new(); + self.all_timeseries_names_impl(&mut set); + set + } + + fn all_timeseries_names_impl<'a>( + &'a self, + set: &mut BTreeSet<&'a TimeseriesName>, + ) { + for op in self.ops.iter() { + match op { + TableOp::Basic(BasicTableOp::Get(name)) => { + set.insert(name); + } + TableOp::Basic(_) => {} + TableOp::Grouped(GroupedTableOp { ops }) => { + for query in ops.iter() { + query.all_timeseries_names_impl(set); + } + } + } + } + } + // Check that this query (and any subqueries) start with a get table op, and // that there are no following get operations. I.e., we have: // diff --git a/oximeter/db/src/oxql/ast/table_ops/align.rs b/oximeter/db/src/oxql/ast/table_ops/align.rs index b0cd7d80f1..3606af689c 100644 --- a/oximeter/db/src/oxql/ast/table_ops/align.rs +++ b/oximeter/db/src/oxql/ast/table_ops/align.rs @@ -19,6 +19,7 @@ use oxql_types::point::Values; use oxql_types::Alignment; use oxql_types::Table; use oxql_types::Timeseries; +use std::fmt; use std::time::Duration; // The maximum factor by which an alignment operation may upsample data. @@ -68,7 +69,7 @@ fn verify_max_upsampling_ratio( /// /// Alignment is used to produce data at the defined timestamps, so that samples /// from multiple timeseries may be combined or correlated in meaningful ways. -#[derive(Clone, Debug, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub struct Align { /// The alignment method, used to describe how data over the input period /// is used to generate an output sample. @@ -87,6 +88,16 @@ pub struct Align { pub period: Duration, } +impl std::fmt::Display for Align { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let name = match self.method { + AlignmentMethod::Interpolate => "interpolate", + AlignmentMethod::MeanWithin => "mean_within", + }; + write!(f, "{}({:?})", name, self.period) + } +} + impl Align { // Apply the alignment function to the set of tables. pub(crate) fn apply( @@ -108,7 +119,7 @@ impl Align { } /// An alignment method. -#[derive(Clone, Debug, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum AlignmentMethod { /// Alignment is done by interpolating the output data at the specified /// period. @@ -118,6 +129,15 @@ pub enum AlignmentMethod { MeanWithin, } +impl fmt::Display for AlignmentMethod { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + AlignmentMethod::Interpolate => write!(f, "interpolate"), + AlignmentMethod::MeanWithin => write!(f, "mean_within"), + } + } +} + // Align the timeseries in a table by computing the average within each output // period. fn align_mean_within( diff --git a/oximeter/db/src/oxql/ast/table_ops/filter.rs b/oximeter/db/src/oxql/ast/table_ops/filter/mod.rs similarity index 91% rename from oximeter/db/src/oxql/ast/table_ops/filter.rs rename to oximeter/db/src/oxql/ast/table_ops/filter/mod.rs index ad398da983..e125e0505c 100644 --- a/oximeter/db/src/oxql/ast/table_ops/filter.rs +++ b/oximeter/db/src/oxql/ast/table_ops/filter/mod.rs @@ -12,6 +12,7 @@ use crate::oxql::ast::literal::Literal; use crate::oxql::ast::logical_op::LogicalOp; use crate::oxql::ast::table_ops::limit::Limit; use crate::oxql::ast::table_ops::limit::LimitKind; +use crate::oxql::schema::TableSchema; use crate::oxql::Error; use crate::shells::special_idents; use chrono::DateTime; @@ -27,6 +28,9 @@ use oxql_types::Timeseries; use regex::Regex; use std::collections::BTreeSet; use std::fmt; +use std::time::Duration; + +pub(crate) mod visit; /// An AST node for the `filter` table operation. /// @@ -80,6 +84,105 @@ impl Filter { out } + // Recursively apply the visitor to all the filter nodes in self. + fn visit_fields(&self, visitor: &V) -> Result, Error> + where + V: visit::Visit, + { + match &self.expr { + FilterExpr::Simple(simple) => { + visitor.visit_simple(self.negated, &simple) + } + FilterExpr::Compound(compound) => { + let left = compound.left.visit_fields(visitor)?; + let right = compound.right.visit_fields(visitor)?; + let out = match (left, right) { + (None, None) => None, + (None, Some(single)) | (Some(single), None) => Some(single), + (Some(left), Some(right)) => Some(visitor.combine( + self.negated, + left, + right, + compound.op, + )), + }; + Ok(out) + } + } + } + + /// Remove any predicates on the datum of a table. + pub fn remove_datum( + &self, + schema: &TableSchema, + ) -> Result, Error> { + let visitor = visit::RemoveDatum { schema }; + self.visit_fields(&visitor) + } + + /// Remove any predicates _not_ on the datum of a table. + pub fn only_datum( + &self, + schema: &TableSchema, + ) -> Result, Error> { + let visitor = visit::OnlyDatum { schema }; + self.visit_fields(&visitor) + } + + /// Restrict predicates so that they apply only to the schema's fields. + pub fn restrict_to_fields( + &self, + schema: &TableSchema, + ) -> Result, Error> { + let visitor = visit::RestrictToFields { schema }; + self.visit_fields(&visitor) + } + + /// Restrict predicates so that they apply only to the schema's + /// measurements. + pub fn restrict_to_measurements( + &self, + schema: &TableSchema, + ) -> Result, Error> { + anyhow::ensure!( + schema.data_types.len() == 1, + "Rewriting measurement filters is only valid \ + for 1-dimensional tables" + ); + let visitor = visit::RestrictToMeasurements { schema }; + self.visit_fields(&visitor) + } + + /// Rewrite predicates so that they apply only to the schema's field tables. + /// + /// This returns the _database_ filters that can be applied in a `WHERE` + /// clause to implement the corresponding filter. + pub fn rewrite_for_field_tables( + &self, + schema: &TableSchema, + ) -> Result, Error> { + let visitor = visit::RewriteForFieldTables { schema }; + self.visit_fields(&visitor) + } + + /// Rewrite predicates so that they apply only to the schema's measurement + /// table. + /// + /// This returns the _database_ filters that can be applied in a `WHERE` + /// clause to implement the corresponding filter. + pub fn rewrite_for_measurement_table( + &self, + schema: &TableSchema, + ) -> Result, Error> { + anyhow::ensure!( + schema.data_types.len() == 1, + "Rewriting measurement filters is only valid \ + for 1-dimensional tables" + ); + let visitor = visit::RewriteForMeasurementTable { schema }; + self.visit_fields(&visitor) + } + fn flatten_disjunctions_inner(&self, dis: &mut Vec) { // Recursion is only needed if this is an OR expression. In that case, // we split the left and push it, and then recurse on the right. @@ -231,6 +334,11 @@ impl Filter { Ok(Self { negated: self.negated, ..new }) } + // Helper method to combine this filter with another, using an AND operator. + pub(crate) fn and(&self, other: &Filter) -> Self { + self.merge(other, LogicalOp::And) + } + // Merge this filter with another one, using the provided operator. pub(crate) fn merge(&self, other: &Filter, op: LogicalOp) -> Self { Self { @@ -301,7 +409,7 @@ impl Filter { // There are extra, implied names that depend on the data type of the // timeseries itself, check those as well. - let extras = implicit_field_names(first_timeseries); + let extras = implicit_field_names_for_timeseries(first_timeseries); let ident_names = self.ident_names(); let not_valid = ident_names .iter() @@ -483,21 +591,51 @@ impl Filter { } } } + + /// Shift timestamps in any filters forward or backwards by this period, + /// depending on the direction implied by their comparison. + /// + /// E.g., `timestamp > t0` is shifted _backwards_ by `period`, so that it + /// ensures it captures `t0 - period`. + pub(crate) fn shift_timestamp_by(&self, period: Duration) -> Self { + let visitor = visit::ShiftTimestamps { period }; + self.visit_fields(&visitor).unwrap().unwrap() + } } /// Return the names of the implicit fields / columns that a filter can apply /// to, based on the metric types of the contained data points. -fn implicit_field_names( +fn implicit_field_names_for_timeseries( first_timeseries: &Timeseries, ) -> BTreeSet<&'static str> { - let mut out = BTreeSet::new(); - - // Everything has a timestamp! - out.insert(special_idents::TIMESTAMP); let type_info = first_timeseries .points .metric_types() .zip(first_timeseries.points.data_types()); + implicit_field_names_for_types(type_info) +} + +/// Return the implicit field names a filter can apply to, from the given table +/// schema. +pub fn implicit_field_names_for_table_schema( + schema: &TableSchema, +) -> BTreeSet<&'static str> { + implicit_field_names_for_types( + schema + .metric_types + .iter() + .copied() + .zip(schema.data_types.iter().copied()), + ) +} + +fn implicit_field_names_for_types( + type_info: impl Iterator, +) -> BTreeSet<&'static str> { + let mut out = BTreeSet::new(); + + // Everything has a timestamp! + out.insert(special_idents::TIMESTAMP); for (metric_type, data_type) in type_info { match (metric_type, data_type) { // Scalar gauges. diff --git a/oximeter/db/src/oxql/ast/table_ops/filter/visit.rs b/oximeter/db/src/oxql/ast/table_ops/filter/visit.rs new file mode 100644 index 0000000000..f68198c77c --- /dev/null +++ b/oximeter/db/src/oxql/ast/table_ops/filter/visit.rs @@ -0,0 +1,494 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Methods for walking a tree of filter nodes and applying a function to them. + +// Copyright 2024 Oxide Computer Company + +use std::time::Duration; + +use super::CompoundFilter; +use super::Filter; +use super::FilterExpr; +use super::SimpleFilter; +use crate::oxql::ast::cmp::Comparison; +use crate::oxql::ast::literal::Literal; +use crate::oxql::ast::logical_op::LogicalOp; +use crate::oxql::schema::TableSchema; +use crate::oxql::Error; +use crate::shells::special_idents; + +/// A trait for visiting a tree of filter nodes. +/// +/// This is used to walk a tree of filter nodes and produce some oputput based +/// on them. For example, this can be used to convert the tree into a string +/// representation for use in the database. +pub trait Visit { + /// The result of applying the visitor to a simple filter node. + type Output; + + /// Visit one simple filter node, returning the output. + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error>; + + /// Combine the output of previously-visited filter nodes. + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output; +} + +/// A visitor that shifts timestamps. +pub struct ShiftTimestamps { + pub period: Duration, +} + +impl Visit for ShiftTimestamps { + type Output = Filter; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + let SimpleFilter { ident, cmp, value } = filter; + if ident.as_str() == special_idents::TIMESTAMP { + let Literal::Timestamp(timestamp) = value else { + anyhow::bail!("Filter expression is not a timestamp"); + }; + let new_timestamp = match cmp { + Comparison::Eq + | Comparison::Ne + | Comparison::Lt + | Comparison::Le => *timestamp, + Comparison::Gt | Comparison::Ge => *timestamp - self.period, + Comparison::Like => unreachable!(), + }; + Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(SimpleFilter { + value: Literal::Timestamp(new_timestamp), + ..filter.clone() + }), + })) + } else { + Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })) + } + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + Filter { + negated, + expr: FilterExpr::Compound(CompoundFilter { + left: Box::new(left), + op, + right: Box::new(right), + }), + } + } +} + +/// A visitor that _keeps_ filter expressions that refer to the datum. +pub struct OnlyDatum<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for OnlyDatum<'a> { + type Output = Filter; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + // It should be a datum. Check and remove it. + if filter.ident.as_str() == special_idents::DATUM { + for ty in self.schema.data_types.iter().copied() { + anyhow::ensure!( + filter.value.is_compatible_with_datum(ty), + "Expression for datum on table {} is not \ + compatible with its type {}", + self.schema.name, + ty, + ); + } + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + + // Anything else is outta here. + Ok(None) + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + Filter { + negated, + expr: FilterExpr::Compound(CompoundFilter { + left: Box::new(left), + op, + right: Box::new(right), + }), + } + } +} + +/// A visitor that removes filter expressions that refer to the datum. +pub struct RemoveDatum<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for RemoveDatum<'a> { + type Output = Filter; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + if let Some(field_type) = self.schema.field_type(filter.ident.as_str()) + { + if !filter.value_type_is_compatible_with_field(*field_type) { + return Err(anyhow::anyhow!( + "Expression for field {} is not compatible with \ + its type {}", + filter.ident, + field_type, + )); + } + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + + // The relevant columns on which we filter depend on the datum + // type of the table. All tables support "timestamp". + let ident = filter.ident.as_str(); + if ident == special_idents::TIMESTAMP { + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // Check for comparison against the start time, which only works + // for cumulative tables. + if ident == special_idents::START_TIME { + if !self.schema.metric_types[0].is_cumulative() { + return Err(anyhow::anyhow!( + "Start time can only be compared if the metric \ + is cumulative, but table '{}' has metric type {}", + self.schema.name, + &self.schema.metric_types[0], + )); + } + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // It should be a datum. Check and remove it. + if ident == special_idents::DATUM { + return Ok(None); + } + + // Anything else is a bug. + unreachable!("Filter identifier '{}' is not valid", filter.ident,); + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + Filter { + negated, + expr: FilterExpr::Compound(CompoundFilter { + left: Box::new(left), + op, + right: Box::new(right), + }), + } + } +} + +/// A visitor that restricts filter expressions to those that apply to the +/// _fields_ of a table schema. +pub struct RestrictToFields<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for RestrictToFields<'a> { + type Output = Filter; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + let Some(field_type) = self.schema.field_type(filter.ident.as_str()) + else { + return Ok(None); + }; + if !filter.value_type_is_compatible_with_field(*field_type) { + return Err(anyhow::anyhow!( + "Expression for field {} is not compatible with \ + its type {}", + filter.ident, + field_type, + )); + } + Ok(Some(Filter { negated, expr: FilterExpr::Simple(filter.clone()) })) + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + Filter { + negated, + expr: FilterExpr::Compound(CompoundFilter { + left: Box::new(left), + op, + right: Box::new(right), + }), + } + } +} + +/// A visitor that restricts filter expressions to those that apply to the +/// _measurements_ of a table schema. +pub struct RestrictToMeasurements<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for RestrictToMeasurements<'a> { + type Output = Filter; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + // The relevant columns on which we filter depend on the datum + // type of the table. All tables support "timestamp". + let ident = filter.ident.as_str(); + if ident == special_idents::TIMESTAMP { + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // Check for comparison against the start time, which only works + // for cumulative tables. + if ident == special_idents::START_TIME { + if !self.schema.metric_types[0].is_cumulative() { + return Err(anyhow::anyhow!( + "Start time can only be compared if the metric \ + is cumulative, but table '{}' has metric type {}", + self.schema.name, + &self.schema.metric_types[0], + )); + } + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(Filter { + negated, + expr: FilterExpr::Simple(filter.clone()), + })); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // We'll delegate to the actual table op to filter on any of the + // data columns. + // + // TODO-completeness: We should allow filtering here once we + // push deltas into the database. + Ok(None) + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + Filter { + negated, + expr: FilterExpr::Compound(CompoundFilter { + left: Box::new(left), + op, + right: Box::new(right), + }), + } + } +} + +/// Rewrite the filters so that they apply to the database field tables only. +pub struct RewriteForFieldTables<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for RewriteForFieldTables<'a> { + type Output = String; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + // If the predicate names a field in this table schema, + // return that predicate printed as a string. If not, we return + // None. + let Some(field_type) = self.schema.field_type(filter.ident.as_str()) + else { + return Ok(None); + }; + if !filter.value_type_is_compatible_with_field(*field_type) { + return Err(anyhow::anyhow!( + "Expression for field {} is not compatible with \ + its type {}", + filter.ident, + field_type, + )); + } + let maybe_not = if negated { "NOT " } else { "" }; + Ok(Some(format!("{}{}", maybe_not, filter.as_db_safe_string()))) + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + let maybe_not = if negated { "NOT " } else { "" }; + format!("{}{}({left}, {right})", maybe_not, op.as_db_function_name()) + } +} + +/// Rewrite filters so that they apply only to the database measurement table. +pub struct RewriteForMeasurementTable<'a> { + pub schema: &'a TableSchema, +} + +impl<'a> Visit for RewriteForMeasurementTable<'a> { + type Output = String; + + fn visit_simple( + &self, + negated: bool, + filter: &SimpleFilter, + ) -> Result, Error> { + let maybe_not = if negated { "NOT " } else { "" }; + // The relevant columns on which we filter depend on the datum + // type of the table. All tables support "timestamp". + let ident = filter.ident.as_str(); + if ident == special_idents::TIMESTAMP { + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(format!( + "{}{}", + maybe_not, + filter.as_db_safe_string() + ))); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // Check for comparison against the start time, which only works + // for cumulative tables. + if ident == special_idents::START_TIME { + if !self.schema.metric_types[0].is_cumulative() { + return Err(anyhow::anyhow!( + "Start time can only be compared if the metric \ + is cumulative, but table '{}' has metric type {}", + self.schema.name, + &self.schema.metric_types[0], + )); + } + if matches!(filter.value, Literal::Timestamp(_)) { + return Ok(Some(format!( + "{}{}", + maybe_not, + filter.as_db_safe_string() + ))); + } + return Err(anyhow::anyhow!( + "Literal cannot be compared with a timestamp" + )); + } + + // We'll delegate to the actual table op to filter on any of the data + // columns. + // + // TODO-completeness: We should allow pushing the filters here once we + // push deltas into the database. + Ok(None) + } + + fn combine( + &self, + negated: bool, + left: Self::Output, + right: Self::Output, + op: LogicalOp, + ) -> Self::Output { + let maybe_not = if negated { "NOT " } else { "" }; + format!("{}{}({left}, {right})", maybe_not, op.as_db_function_name()) + } +} + +#[cfg(test)] +mod tests {} diff --git a/oximeter/db/src/oxql/ast/table_ops/group_by.rs b/oximeter/db/src/oxql/ast/table_ops/group_by.rs index c48804a788..aeadc828ea 100644 --- a/oximeter/db/src/oxql/ast/table_ops/group_by.rs +++ b/oximeter/db/src/oxql/ast/table_ops/group_by.rs @@ -19,6 +19,7 @@ use oxql_types::Table; use oxql_types::Timeseries; use std::collections::btree_map::Entry; use std::collections::BTreeMap; +use std::fmt; /// A table operation for grouping data by fields, apply a reducer to the /// remaining. @@ -28,6 +29,20 @@ pub struct GroupBy { pub reducer: Reducer, } +impl fmt::Display for GroupBy { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "group_by [")?; + let n_idents = self.identifiers.len(); + for (i, ident) in self.identifiers.iter().enumerate() { + write!(f, "{}", ident.as_str())?; + if i < n_idents - 1 { + write!(f, ", ")?; + } + } + write!(f, "], {}", self.reducer) + } +} + impl GroupBy { // Apply the group_by table operation. pub(crate) fn apply(&self, tables: &[Table]) -> Result, Error> { @@ -423,6 +438,15 @@ pub enum Reducer { Sum, } +impl fmt::Display for Reducer { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Reducer::Mean => write!(f, "mean"), + Reducer::Sum => write!(f, "sum"), + } + } +} + #[cfg(test)] mod tests { use super::{GroupBy, Reducer}; diff --git a/oximeter/db/src/oxql/ast/table_ops/limit.rs b/oximeter/db/src/oxql/ast/table_ops/limit.rs index 89afb31a7c..e093246d68 100644 --- a/oximeter/db/src/oxql/ast/table_ops/limit.rs +++ b/oximeter/db/src/oxql/ast/table_ops/limit.rs @@ -8,7 +8,7 @@ use anyhow::Error; use oxql_types::Table; -use std::num::NonZeroUsize; +use std::{fmt, num::NonZeroUsize}; /// The kind of limiting operation #[derive(Clone, Copy, Debug, PartialEq)] @@ -27,6 +27,17 @@ pub struct Limit { /// The number of points the timeseries is limited to. pub count: NonZeroUsize, } + +impl fmt::Display for Limit { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let kind = match self.kind { + LimitKind::First => "first", + LimitKind::Last => "last", + }; + write!(f, "{kind} {}", self.count) + } +} + impl Limit { /// Apply the limit operation to the input tables. pub(crate) fn apply(&self, tables: &[Table]) -> Result, Error> { diff --git a/oximeter/db/src/oxql/ast/table_ops/mod.rs b/oximeter/db/src/oxql/ast/table_ops/mod.rs index 8b8d4cbe1b..5222bb4b19 100644 --- a/oximeter/db/src/oxql/ast/table_ops/mod.rs +++ b/oximeter/db/src/oxql/ast/table_ops/mod.rs @@ -13,6 +13,8 @@ pub mod group_by; pub mod join; pub mod limit; +use std::fmt; + use self::align::Align; use self::filter::Filter; use self::group_by::GroupBy; @@ -36,6 +38,19 @@ pub enum BasicTableOp { Limit(Limit), } +impl fmt::Display for BasicTableOp { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + BasicTableOp::Get(name) => write!(f, "get {name}"), + BasicTableOp::Filter(filter) => write!(f, "filter {filter}"), + BasicTableOp::GroupBy(group_by) => write!(f, "{group_by}"), + BasicTableOp::Join(_) => write!(f, "join"), + BasicTableOp::Align(align) => write!(f, "align {align}"), + BasicTableOp::Limit(limit) => write!(f, "{limit}"), + } + } +} + impl BasicTableOp { pub(crate) fn apply( &self, @@ -59,6 +74,22 @@ pub struct GroupedTableOp { pub ops: Vec, } +impl fmt::Display for GroupedTableOp { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{{ ")?; + let n_queries = self.ops.len(); + for (i, query) in self.ops.iter().enumerate() { + write!(f, "{query}")?; + if i < n_queries - 1 { + write!(f, "; ")?; + } else { + write!(f, "")?; + } + } + write!(f, " }}") + } +} + /// Any kind of OxQL table operation. #[derive(Clone, Debug, PartialEq)] pub enum TableOp { @@ -66,6 +97,15 @@ pub enum TableOp { Grouped(GroupedTableOp), } +impl fmt::Display for TableOp { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + TableOp::Basic(basic) => write!(f, "{basic}"), + TableOp::Grouped(grouped) => write!(f, "{grouped}"), + } + } +} + impl TableOp { pub(crate) fn apply( &self, diff --git a/oximeter/db/src/oxql/mod.rs b/oximeter/db/src/oxql/mod.rs index fcdfb783c5..3e57e03821 100644 --- a/oximeter/db/src/oxql/mod.rs +++ b/oximeter/db/src/oxql/mod.rs @@ -10,13 +10,15 @@ use peg::error::ParseError as PegError; use peg::str::LineCol; pub mod ast; +pub mod plan; pub mod query; +pub mod schema; pub use self::query::Query; pub use anyhow::Error; /// Format a PEG parsing error into a nice anyhow error. -fn fmt_parse_error(source: &str, err: PegError) -> Error { +pub fn fmt_parse_error(source: &str, err: PegError) -> Error { use std::fmt::Write; let mut out = format!("Error at {}:{}", err.location.line, err.location.column); diff --git a/oximeter/db/src/oxql/plan/align.rs b/oximeter/db/src/oxql/plan/align.rs new file mode 100644 index 0000000000..8221728142 --- /dev/null +++ b/oximeter/db/src/oxql/plan/align.rs @@ -0,0 +1,90 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! OxQL query plan node for aligning tables. + +// Copyright 2024 Oxide Computer Company + +use oxql_types::point::DataType; +use oxql_types::point::MetricType; + +use crate::oxql::ast::table_ops::align; +use crate::oxql::plan::plan::TableOpData; +use crate::oxql::plan::plan::TableOpInput; +use crate::oxql::plan::plan::TableOpOutput; +use crate::oxql::schema::TableSchema; + +/// A node that aligns its input tables. +#[derive(Clone, Debug, PartialEq)] +pub struct Align { + pub output: TableOpOutput, + pub alignment: align::Align, +} + +impl Align { + /// Plan the application of the alignment operation to the input tables + pub fn new( + alignment: align::Align, + input: TableOpInput, + ) -> anyhow::Result { + let tables = input + .tables + .into_iter() + .map(|TableOpData { schema, .. }| { + align_input_schema(schema, alignment.method).map(|schema| { + TableOpData { schema, alignment: Some(alignment) } + }) + }) + .collect::, _>>()?; + let output = TableOpOutput { tables }; + Ok(Align { output, alignment }) + } + + /// Print this plan node as a plan tree entry. + pub fn plan_tree_entry(&self) -> termtree::Tree { + termtree::Tree::new(format!( + "align: method={}, period={:?}", + self.alignment.method, self.alignment.period + )) + } +} + +// Align the input schema, returning the output schema it will produce. +fn align_input_schema( + schema: TableSchema, + method: align::AlignmentMethod, +) -> anyhow::Result { + for metric_type in schema.metric_types.iter() { + anyhow::ensure!( + metric_type != &MetricType::Cumulative, + "Only gauge or delta metric types may be aligned, \ + but table '{}' has cumulative metric type", + schema.name, + ); + } + let mut data_types = Vec::with_capacity(schema.data_types.len()); + for data_type in schema.data_types.iter() { + match (data_type, method) { + ( + DataType::Integer | DataType::Double, + align::AlignmentMethod::MeanWithin, + ) => { + data_types.push(DataType::Double); + } + ( + DataType::Integer | DataType::Double, + align::AlignmentMethod::Interpolate, + ) => { + anyhow::bail!( + "Alignment via interpolation is not yet implemented" + ); + } + (_, _) => anyhow::bail!( + "Tables with '{}' data types cannot be aligned", + data_type, + ), + } + } + Ok(TableSchema { data_types, ..schema }) +} diff --git a/oximeter/db/src/oxql/plan/delta.rs b/oximeter/db/src/oxql/plan/delta.rs new file mode 100644 index 0000000000..f3c22b8aca --- /dev/null +++ b/oximeter/db/src/oxql/plan/delta.rs @@ -0,0 +1,49 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Query plan node for constructing a delta from a cumulative timeseries. + +// Copyright 2024 Oxide Computer Company + +use oxql_types::point::MetricType; + +use crate::oxql::schema::TableSchema; + +/// A plan node for computing deltas from a cumulative timeseries. +#[derive(Clone, Debug, PartialEq)] +pub struct Delta { + pub output: TableSchema, +} + +impl Delta { + pub fn new(schema: &TableSchema) -> anyhow::Result { + anyhow::ensure!( + schema.metric_types.len() == 1, + "Deltas can only be applied to 1-dimensional tables", + ); + anyhow::ensure!( + schema.metric_types[0] == MetricType::Cumulative, + "Deltas can only be applied to cumulative tables", + ); + anyhow::ensure!( + schema.data_types[0].is_numeric() + || schema.data_types[0].is_distribution(), + "Deltas can only be applied to numeric or distribution \ + data types, not {}", + schema.data_types[0], + ); + let output = TableSchema { + metric_types: vec![MetricType::Delta], + ..schema.clone() + }; + Ok(Self { output }) + } + + pub fn plan_tree_entry(&self) -> termtree::Tree { + termtree::Tree::new(format!( + "delta: cumulative -> delta ({})", + self.output.data_types[0] + )) + } +} diff --git a/oximeter/db/src/oxql/plan/filter.rs b/oximeter/db/src/oxql/plan/filter.rs new file mode 100644 index 0000000000..db3c5ff350 --- /dev/null +++ b/oximeter/db/src/oxql/plan/filter.rs @@ -0,0 +1,383 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! OxQL query plan node for representing a filtering operation. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::literal::Literal; +use crate::oxql::ast::table_ops::filter; +use crate::oxql::ast::table_ops::filter::implicit_field_names_for_table_schema; +use crate::oxql::ast::table_ops::limit; +use crate::oxql::plan::plan::TableOpInput; +use crate::oxql::plan::predicates::Predicates; +use crate::oxql::schema::TableSchema; +use crate::shells::special_idents; +use std::collections::BTreeSet; +use std::fmt; + +/// A node that filters its input tables. +#[derive(Clone, Debug, PartialEq)] +pub struct Filter { + /// The input tables to the filter. + /// + /// Note that these are also the output tables, since a filter does not + /// change the schema of its inputs. + pub input: TableOpInput, + + /// The predicates associated with this filter plan node. + /// + /// See the documentation of `Predicates` for details. + pub predicates: Predicates, +} + +impl Filter { + /// Plan the application of the filter to the input tables. + pub fn new( + filter: &filter::Filter, + input: TableOpInput, + ) -> anyhow::Result { + Self::new_owned(filter.clone(), input) + } + + /// Same a `Self::new`, but from an owned filter + pub fn new_owned( + filter: filter::Filter, + input: TableOpInput, + ) -> anyhow::Result { + for table in input.tables.iter() { + Self::ensure_filter_expr_application_is_valid( + &filter.expr, + &table.schema, + )?; + } + Ok(Filter { input, predicates: Predicates::Single(filter) }) + } + + pub fn plan_tree_entry(&self) -> termtree::Tree { + let mut out = + termtree::Tree::new(String::from("filter")).with_multiline(true); + out.extend(self.predicates.plan_tree_entries()); + out + } + + pub fn from_predicates( + predicates: Predicates, + input: TableOpInput, + ) -> anyhow::Result { + for table in input.tables.iter() { + match &predicates { + Predicates::Single(filter) => { + Self::ensure_filter_expr_application_is_valid( + &filter.expr, + &table.schema, + )?; + } + Predicates::Disjunctions(disjunctions) => { + for filter in + disjunctions.iter().filter_map(|dis| dis.as_ref()) + { + Self::ensure_filter_expr_application_is_valid( + &filter.expr, + &table.schema, + )?; + } + } + } + } + Ok(Filter { input, predicates }) + } + + // Check that the provided filter expression is valid for a schema, trying + // to catch as many errors as possible by pushing them onto the input error + // list. + fn ensure_filter_expr_application_is_valid_impl( + expr: &filter::FilterExpr, + schema: &TableSchema, + errors: &mut Vec, + ) { + match expr { + filter::FilterExpr::Simple(simple) => { + let implicit_fields = + implicit_field_names_for_table_schema(schema); + // Check that the identifier in the filter is in the table. + if let Some(type_) = schema.fields.get(simple.ident.as_str()) { + if !simple.value.is_compatible_with_field(*type_) { + let err = FilterErrorKind::IncompatibleExpression { + identifier: simple.ident.to_string(), + expression_type: simple.value.type_name(), + expected_type: type_.to_string(), + }; + errors.push(err); + } + } else if implicit_fields.contains(simple.ident.as_str()) { + // And check that the filter expression is compatible with the + // provided identifier type. + if let Err(e) = Self::ensure_special_ident_is_compatible( + simple.ident.as_str(), + &simple.value, + schema, + ) { + errors.push(e); + } + } else { + let err = FilterErrorKind::InvalidIdentifier { + identifier: simple.ident.to_string(), + table: schema.name.clone(), + valid_identifiers: schema + .fields + .keys() + .map(String::as_str) + .collect::>() + .union(&implicit_fields) + .map(|s| s.to_string()) + .collect::>(), + }; + errors.push(err); + } + } + filter::FilterExpr::Compound(compound) => { + Self::ensure_filter_expr_application_is_valid_impl( + &compound.left.expr, + &schema, + errors, + ); + Self::ensure_filter_expr_application_is_valid_impl( + &compound.right.expr, + &schema, + errors, + ); + } + } + } + + /// Check that the provided filter is valid for a schema. + pub fn ensure_filter_expr_application_is_valid( + expr: &filter::FilterExpr, + schema: &TableSchema, + ) -> anyhow::Result<()> { + let mut errors = Vec::new(); + Self::ensure_filter_expr_application_is_valid_impl( + expr, + schema, + &mut errors, + ); + anyhow::ensure!( + errors.is_empty(), + "The filter expression \"{}\" is not valid, \ + the following errors were encountered\n{}", + expr, + FilterErrorKind::format_list(errors) + ); + Ok(()) + } + + // Check that the provided lteral is valid for a special identifier. + fn ensure_special_ident_is_compatible( + ident: &str, + value: &Literal, + schema: &TableSchema, + ) -> Result<(), FilterErrorKind> { + if ident == special_idents::TIMESTAMP + || ident == special_idents::START_TIME + { + if matches!(value, Literal::Timestamp(_)) { + Ok(()) + } else { + Err(FilterErrorKind::IncompatibleExpression { + identifier: ident.to_string(), + expression_type: value.type_name(), + expected_type: String::from("timestamp"), + }) + } + } else if ident == special_idents::DATUM { + // TODO-completeness: Support writing comparisons on the datum's value + // by _naming_ the metric, e.g., for `hardware_component:temperature`, + // writing `filter temperature > 30.0`. That's a bit tricky, since we + // need to alias the datum column in the SQL queries, or rewrite the + // filter to refer to the `datum` automatically. + if schema.data_types.len() != 1 { + return Err(FilterErrorKind::IdentifierInvalidForNDTables { + identifier: ident.to_string(), + table: schema.name.clone(), + n_dims: schema.data_types.len(), + }); + } + let data_type = schema.data_types[0]; + if value.is_compatible_with_datum(data_type) { + Ok(()) + } else { + Err(FilterErrorKind::IncompatibleExpression { + identifier: ident.to_string(), + expected_type: data_type.to_string(), + expression_type: value.type_name(), + }) + } + } else { + // TODO-completeness: Support comparison for special distribution + // identifiers: min, max, mean, p50, p90, p99, and possibly bins and + // counts. + Err(FilterErrorKind::FilteringNotSupported { + identifier: ident.to_string(), + }) + } + } + + // Return true if we can reorder a limit around the provided filter. + pub fn can_reorder_around(&self, limit: &limit::Limit) -> bool { + match &self.predicates { + Predicates::Single(f) => f.can_reorder_around(limit), + Predicates::Disjunctions(disjunctions) => { + disjunctions.iter().all(|maybe_disjunction| { + maybe_disjunction + .as_ref() + .map(|filter| filter.can_reorder_around(limit)) + .unwrap_or(true) + }) + } + } + } +} + +// Helper error enum to catch all errors applying a filter expression to a +// table. +#[derive(Clone, Debug)] +enum FilterErrorKind { + // The filter expression itself isn't compatible with the type of the field + // the filter applies to. + IncompatibleExpression { + identifier: String, + expected_type: String, + expression_type: &'static str, + }, + // The identifier isn't one of the input tables' identifiers. + InvalidIdentifier { + identifier: String, + table: String, + valid_identifiers: BTreeSet, + }, + // Special identifier only valid for 1-D tables + IdentifierInvalidForNDTables { + identifier: String, + table: String, + n_dims: usize, + }, + // Filter is not yet supported on this identifier. + FilteringNotSupported { + identifier: String, + }, +} + +impl FilterErrorKind { + fn format_list(errors: Vec) -> String { + assert!(!errors.is_empty()); + // Pull together the errors about invalid idents, since those all share + // the same _valid_ identifiers. + let (invalid_idents, others): (Vec<_>, Vec<_>) = errors + .into_iter() + .partition(|err| matches!(err, Self::InvalidIdentifier { .. })); + let maybe_invalid_ident_message = + Self::invalid_identifier_message(invalid_idents); + maybe_invalid_ident_message + .into_iter() + .chain(others.into_iter().map(|msg| msg.to_string())) + .map(|msg| format!(" > {msg}")) + .collect::>() + .join("\n") + } + + fn invalid_identifier_message( + errors: Vec, + ) -> Option { + let Some(FilterErrorKind::InvalidIdentifier { + table, + valid_identifiers, + .. + }) = errors.first() + else { + return None; + }; + let all_invalid_idents = errors + .iter() + .map(|err| { + let Self::InvalidIdentifier { identifier, .. } = err else { + unreachable!(); + }; + format!("\"{}\"", identifier) + }) + .collect::>() + .join(", "); + let e = format!( + "The filter expression refers to identifiers \ + that are not valid for its input table \"{}\". Invalid \ + identifiers: [{}], valid identifiers: [{}]", + table, + all_invalid_idents, + valid_identifiers + .iter() + .map(|ident| format!("\"{}\"", ident)) + .collect::>() + .join(", ") + ); + Some(e) + } +} + +impl fmt::Display for FilterErrorKind { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + FilterErrorKind::IncompatibleExpression { + identifier: field, + expected_type, + expression_type, + } => { + write!( + f, + "Filter expression for identifier \"{}\" has type \"{}\" \ + which is not compatible with the expected type \"{}\"", + field, expression_type, expected_type, + ) + } + FilterErrorKind::InvalidIdentifier { + identifier, + table, + valid_identifiers, + } => { + write!( + f, + "The filter expression refers to an identifier \ + that is not valid for its input table \"{}\". \ + Invalid identifier: \"{}\", valid identifiers: [{}]", + identifier, + table, + valid_identifiers + .iter() + .map(|ident| format!("\"{}\"", ident)) + .collect::>() + .join(", ") + ) + } + FilterErrorKind::IdentifierInvalidForNDTables { + identifier, + table, + n_dims, + } => { + write!( + f, + "The special identifier \"{}\" may only be used \ + for 1-dimensional tables, but table \"{}\" has {} dimensions", + identifier, table, n_dims, + ) + } + FilterErrorKind::FilteringNotSupported { identifier } => { + write!( + f, + "Filtering on the special identifier \"{}\" is \ + not yet supported", + identifier, + ) + } + } + } +} diff --git a/oximeter/db/src/oxql/plan/get.rs b/oximeter/db/src/oxql/plan/get.rs new file mode 100644 index 0000000000..3e56f65fdf --- /dev/null +++ b/oximeter/db/src/oxql/plan/get.rs @@ -0,0 +1,85 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! The `get` plan node, for fetching data from the ClickHouse database. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::table_ops::filter; +use crate::oxql::ast::table_ops::limit; +use crate::oxql::schema::TableSchema; + +/// A node for fetching data from the named timeseries in the database. +#[derive(Clone, Debug, PartialEq)] +pub struct Get { + /// The table schema we're fetching from. + /// + /// This is really a _timeseries_ schema, selecting the named timeseries + /// from the database. These two are synonymous at this point in a plan + /// tree, but we use the table schema for consistency with other plan nodes. + pub table_schema: TableSchema, + + /// The filters applied to the database table for this schema. + /// + /// There is one entry here for every disjunction in the filters that we + /// ultimately push down into the get operation. E.g., for a filter like + /// `filter (x == 0 || x == 1)`, there will be two entries here, `filter (x + /// == 0)` and `filter x == 1`. These are used to construct "consistent key + /// groups", sets of timeseries keys that can all be fetched in one + /// combination of (field SQL query, measurements SQL query). + pub filters: Vec, + + /// An optional limit to the number of samples selected from each + /// timeseries. + /// + /// This is both optional, and only of one kind ("first" or "last"). While a + /// query can express more than one, we currently only push one into the + /// database, which ever appears closer to the front of the query. + pub limit: Option, +} + +impl Get { + /// Convert this node into an entry in a plan tree. + pub fn plan_tree_entry(&self) -> termtree::Tree { + // Push each consistent key group as a child tree. + let mut subtrees = Vec::with_capacity(self.filters.len()); + let mut any_is_full_scan = false; + for (i, filter) in self.filters.iter().enumerate() { + let mut subtree = termtree::Tree::new(format!("key group {i}")) + .with_multiline(true); + let mut is_full_scan = true; + subtree.push(format!( + "field filters={}", + filter + .rewrite_for_field_tables(&self.table_schema) + .unwrap() + .inspect(|_| is_full_scan = false) + .unwrap_or_else(|| String::from("[]")) + )); + subtree.push(format!( + "measurement filters={}", + filter + .rewrite_for_measurement_table(&self.table_schema) + .unwrap() + .inspect(|_| is_full_scan = false) + .unwrap_or_else(|| String::from("[]")) + )); + subtree.push(format!( + "full scan: {}", + if is_full_scan { "YES" } else { "no" } + )); + any_is_full_scan |= is_full_scan; + subtrees.push(subtree); + } + if subtrees.is_empty() || any_is_full_scan { + subtrees.push(termtree::Tree::new(String::from("full scan: YES"))); + } + if let Some(limit) = &self.limit { + subtrees.push(termtree::Tree::new(format!("limit: {limit}"))); + } + termtree::Tree::new(format!("get: \"{}\"", self.table_schema.name)) + .with_multiline(true) + .with_leaves(subtrees) + } +} diff --git a/oximeter/db/src/oxql/plan/group_by.rs b/oximeter/db/src/oxql/plan/group_by.rs new file mode 100644 index 0000000000..a6b1c3560e --- /dev/null +++ b/oximeter/db/src/oxql/plan/group_by.rs @@ -0,0 +1,134 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! OxQL query plan node for grouping data by fields. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::table_ops::group_by; +use crate::oxql::plan::plan::TableOpData; +use crate::oxql::plan::plan::TableOpInput; +use crate::oxql::schema::TableSchema; +use oxql_types::point::DataType; +use oxql_types::point::MetricType; +use std::collections::BTreeMap; +use std::fmt::Write as _; + +/// A node that groups input tables with the same values for the listed field. +#[derive(Clone, Debug, PartialEq)] +pub struct GroupBy { + pub output: TableOpData, + pub group_by: group_by::GroupBy, +} + +impl GroupBy { + /// Plan the application of a group_by operation to the input tables. + pub fn new( + group_by: &group_by::GroupBy, + input: TableOpInput, + ) -> anyhow::Result { + anyhow::ensure!( + input.tables.len() == 1, + "`group_by` table operations require exactly one input \ + table, but found {}", + input.tables.len(), + ); + let table = input + .tables + .into_iter() + .next() + .expect("table operations must have at least one schema"); + anyhow::ensure!( + table.alignment.is_some(), + "Input table to a `group_by` table operation must be aligned, \ + but table '{}' is not aligned", + table.schema.name, + ); + anyhow::ensure!( + table.schema.metric_types.len() == 1, + "`group_by` table operations require 1-dimensional tables, but \ + table '{}' has {} dimensions", + table.schema.name, + table.schema.metric_types.len(), + ); + anyhow::ensure!( + table.schema.metric_types[0] != MetricType::Cumulative, + "`group_by` table operations require delta or gauge metric \ + types, but table '{}' is cumulative", + table.schema.name, + ); + anyhow::ensure!( + table.schema.data_types[0].is_numeric(), + "`group_by` table operations require numeric data types, \ + but table '{}' has data type '{}'", + table.schema.name, + table.schema.data_types[0], + ); + let output_data_type = + match (&table.schema.data_types[0], group_by.reducer) { + (DataType::Integer, group_by::Reducer::Sum) => { + DataType::Integer + } + ( + DataType::Double, + group_by::Reducer::Mean | group_by::Reducer::Sum, + ) + | (DataType::Integer, group_by::Reducer::Mean) => { + DataType::Double + } + (ty, _) => anyhow::bail!( + "`group_by` table operations require numeric data types, \ + but table '{}' has data type '{}'", + table.schema.name, + ty, + ), + }; + let mut output_fields = BTreeMap::new(); + for ident in group_by.identifiers.iter() { + let Some(type_) = table.schema.fields.get(ident.as_str()) else { + anyhow::bail!( + "Cannot group by field '{}', which does not appear in the \ + input tables. Valid fields are: {:?}", + ident.as_str(), + table + .schema + .fields + .keys() + .map(String::as_str) + .collect::>(), + ); + }; + output_fields.insert(ident.to_string(), *type_); + } + let output_schema = TableSchema { + data_types: vec![output_data_type], + fields: output_fields, + ..table.schema + }; + let output = + TableOpData { schema: output_schema, alignment: table.alignment }; + Ok(GroupBy { output, group_by: group_by.clone() }) + } + + fn output_data_type(&self) -> DataType { + self.output.schema.data_types[0] + } + + /// Print this node as a plan tree entry. + pub fn plan_tree_entry(&self) -> termtree::Tree { + let mut out = String::from("group_by: fields=["); + let n_fields = self.group_by.identifiers.len(); + for (i, field) in self.group_by.identifiers.iter().enumerate() { + out.push_str(field.as_str()); + if i < n_fields - 1 { + out.push(','); + } + } + out.push_str("], reducer="); + write!(out, "{}, ", self.group_by.reducer).unwrap(); + out.push_str("output type="); + write!(out, "{}", self.output_data_type()).unwrap(); + termtree::Tree::new(out) + } +} diff --git a/oximeter/db/src/oxql/plan/join.rs b/oximeter/db/src/oxql/plan/join.rs new file mode 100644 index 0000000000..527b00baa0 --- /dev/null +++ b/oximeter/db/src/oxql/plan/join.rs @@ -0,0 +1,101 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! OxQL query plan node for joining tables. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::plan::plan::TableOpData; +use crate::oxql::plan::plan::TableOpInput; +use crate::oxql::schema::TableSchema; + +/// A node that joins input tables with the same field values. +#[derive(Clone, Debug, PartialEq)] +pub struct Join { + pub output: TableOpData, +} + +impl Join { + /// Plan the application of a join node. + pub fn new(input: TableOpInput) -> anyhow::Result { + anyhow::ensure!( + input.tables.len() >= 2, + "`join` table operations require at least 2 tables", + ); + let first = input + .tables + .first() + .expect("table operations must have at least one schema"); + let Some(alignment) = first.alignment else { + anyhow::bail!( + "All input tables to a `join` operation must \ + be aligned, but table '{}' is not aligned", + first.schema.name, + ); + }; + let fields = &first.schema.fields; + let mut names = Vec::with_capacity(input.tables.len()); + let mut metric_types = Vec::with_capacity(input.tables.len()); + let mut data_types = Vec::with_capacity(input.tables.len()); + for table in input.tables.iter() { + let Some(this_alignment) = table.alignment else { + anyhow::bail!( + "All input tables to a `join` operation must \ + be aligned, but table '{}' is not aligned", + table.schema.name, + ); + }; + anyhow::ensure!( + this_alignment == alignment, + "All input tables to a `join` operation must have the \ + same alignment, table '{}' was expected to be aligned \ + with {}, but found {}", + table.schema.name, + alignment, + this_alignment, + ); + anyhow::ensure!( + table.schema.metric_types.len() == 1, + "All input tables to `join` operation must be \ + 1-dimensional, but table '{}' has {} dimensions", + table.schema.name, + table.schema.metric_types.len(), + ); + anyhow::ensure!( + &table.schema.fields == fields, + "All input tables to `join` operation must have \ + the same field names and types, but table '{}' \ + has fields [{}] and table '{}' has fields [{}]", + table.schema.name, + table + .schema + .fields + .iter() + .map(|(name, typ)| format!("\"{name}\" ({typ})")) + .collect::>() + .join(", "), + first.schema.name, + fields + .iter() + .map(|(name, typ)| format!("\"{name}\" ({typ})")) + .collect::>() + .join(", "), + ); + names.push(table.schema.name.as_str()); + metric_types.push(table.schema.metric_types[0]); + data_types.push(table.schema.data_types[0]); + } + let name = names.join(","); + let output = TableOpData { + schema: TableSchema { + name, + fields: fields.clone(), + metric_types, + data_types, + }, + alignment: Some(alignment), + }; + Ok(Self { output }) + } +} diff --git a/oximeter/db/src/oxql/plan/limit.rs b/oximeter/db/src/oxql/plan/limit.rs new file mode 100644 index 0000000000..730e3dc82f --- /dev/null +++ b/oximeter/db/src/oxql/plan/limit.rs @@ -0,0 +1,17 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! OxQL query plan node for limiting data. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::table_ops::limit; +use crate::oxql::plan::plan::TableOpOutput; + +/// A plan node for limiting the number of samples per-timeseries. +#[derive(Clone, Debug, PartialEq)] +pub struct Limit { + pub output: TableOpOutput, + pub limit: limit::Limit, +} diff --git a/oximeter/db/src/oxql/plan/mod.rs b/oximeter/db/src/oxql/plan/mod.rs new file mode 100644 index 0000000000..f5dfdda8ae --- /dev/null +++ b/oximeter/db/src/oxql/plan/mod.rs @@ -0,0 +1,20 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Query plans for the Oximeter Query Language. + +// Copyright 2024 Oxide Computer Company + +mod align; +mod delta; +mod filter; +mod get; +mod group_by; +mod join; +mod limit; +mod node; +mod plan; +mod predicates; + +pub use plan::Plan; diff --git a/oximeter/db/src/oxql/plan/node.rs b/oximeter/db/src/oxql/plan/node.rs new file mode 100644 index 0000000000..d762df6da8 --- /dev/null +++ b/oximeter/db/src/oxql/plan/node.rs @@ -0,0 +1,113 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Nodes in an OxQL query plan. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::plan::align::Align; +use crate::oxql::plan::delta::Delta; +use crate::oxql::plan::filter::Filter; +use crate::oxql::plan::get::Get; +use crate::oxql::plan::group_by::GroupBy; +use crate::oxql::plan::join::Join; +use crate::oxql::plan::limit::Limit; +use crate::oxql::plan::plan::TableOpData; +use crate::oxql::plan::plan::TableOpOutput; +use crate::oxql::plan::Plan; + +/// A node in the query plan. +/// +/// This is roughly synonymous with the table operation, but includes more +/// metadata about inputs and outputs for the operation. In addition, some query +/// plan nodes do not have direct analogs in a query itself. For example, a +/// `Delta` node is inserted automatically after fetching data from a cumulative +/// table. +#[derive(Clone, Debug, PartialEq)] +pub enum Node { + /// A node representing the plan of a subquery. + Subquery(Vec), + /// A node for fetching data from the database. + Get(Get), + /// A node for computing deltas between adjacent samples for a cumulative + /// table. + /// + /// This node is always inserted implicitly after a `Get`, if the table + /// being selected is cumulative. + Delta(Delta), + /// A node for filtering data from its inputs. + Filter(Filter), + /// A node for aligning its input data to regular time intervals. + Align(Align), + /// A node for grouping timeseries within a table with the same field + /// values. + GroupBy(GroupBy), + /// A node for joining timeseries with the same field values from two or + /// more tables. + Join(Join), + /// A node that limits the number of points the timeseries of its input + /// tables. + Limit(Limit), +} + +impl Node { + /// Return the output of a query plan node. + pub fn output(&self) -> TableOpOutput { + match self { + Node::Subquery(subplans) => { + // The output of a subquery is the last output from every + // subplan in the subquery. + let tables = subplans + .iter() + .flat_map(|plan| plan.output().tables) + .collect(); + TableOpOutput { tables } + } + Node::Get(Get { table_schema, .. }) => TableOpOutput { + tables: vec![TableOpData { + schema: table_schema.clone(), + alignment: None, + }], + }, + Node::Delta(Delta { output }) => TableOpOutput { + tables: vec![TableOpData { + schema: output.clone(), + alignment: None, + }], + }, + Node::Filter(Filter { input, .. }) => { + TableOpOutput { tables: input.tables.clone() } + } + Node::Align(Align { output, .. }) => output.clone(), + Node::GroupBy(GroupBy { output, .. }) => { + TableOpOutput { tables: vec![output.clone()] } + } + Node::Join(Join { output, .. }) => { + TableOpOutput { tables: vec![output.clone()] } + } + Node::Limit(Limit { output, .. }) => output.clone(), + } + } + + /// Return a string summarizing a node as a plan tree entry. + /// + /// # Panics + /// + /// This panics for subquery plans, that should be printed separately using + /// the recurisve `to_plan_tree_impl()` method. + pub fn plan_tree_entry(&self) -> termtree::Tree { + match self { + Node::Subquery(_) => unreachable!(), + Node::Get(get) => get.plan_tree_entry(), + Node::Delta(delta) => delta.plan_tree_entry(), + Node::Filter(filter) => filter.plan_tree_entry(), + Node::Align(align) => align.plan_tree_entry(), + Node::GroupBy(group_by) => group_by.plan_tree_entry(), + Node::Join(_) => termtree::Tree::new(String::from("join")), + Node::Limit(limit) => { + termtree::Tree::new(format!("{}", limit.limit)) + } + } + } +} diff --git a/oximeter/db/src/oxql/plan/plan.rs b/oximeter/db/src/oxql/plan/plan.rs new file mode 100644 index 0000000000..e8048a0127 --- /dev/null +++ b/oximeter/db/src/oxql/plan/plan.rs @@ -0,0 +1,1366 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! The top-level query plan itself. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::table_ops::align; +use crate::oxql::ast::table_ops::BasicTableOp; +use crate::oxql::ast::table_ops::GroupedTableOp; +use crate::oxql::ast::table_ops::TableOp; +use crate::oxql::ast::Query; +use crate::oxql::plan::align::Align; +use crate::oxql::plan::delta::Delta; +use crate::oxql::plan::filter::Filter; +use crate::oxql::plan::get::Get; +use crate::oxql::plan::group_by::GroupBy; +use crate::oxql::plan::join::Join; +use crate::oxql::plan::limit::Limit; +use crate::oxql::plan::node::Node; +use crate::oxql::plan::predicates::Predicates; +use crate::oxql::plan::predicates::SplitPredicates; +use crate::oxql::schema::TableSchema; +use anyhow::Context as _; +use oximeter::TimeseriesName; +use oximeter::TimeseriesSchema; +use oxql_types::point::MetricType; +use std::collections::BTreeMap; +use std::collections::VecDeque; +use std::fmt; +use std::time::Duration; +use std::time::Instant; + +/// The data for a table operation, either input or output. +#[derive(Clone, Debug, PartialEq)] +pub struct TableOpData { + /// The table schema. + pub schema: TableSchema, + /// The alignment method of the table, if any. + pub alignment: Option, +} + +impl fmt::Display for TableOpData { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + writeln!(f, "Table: {}", self.schema.name)?; + writeln!( + f, + " Metric types: [{}]", + self.schema + .metric_types + .iter() + .map(|ty| ty.to_string()) + .collect::>() + .join(",") + )?; + writeln!( + f, + " Data types: [{}]", + self.schema + .data_types + .iter() + .map(|ty| ty.to_string()) + .collect::>() + .join(",") + )?; + writeln!(f, " Fields:")?; + for (name, ty) in self.schema.fields.iter() { + writeln!(f, " {name}: {ty}")?; + } + if let Some(alignment) = &self.alignment { + write!(f, " Alignment: {:?}", alignment.period) + } else { + write!(f, " Alignment: none") + } + } +} + +/// The input tables to a table operation. +#[derive(Clone, Debug, PartialEq)] +pub struct TableOpInput { + pub tables: Vec, +} + +impl From for TableOpOutput { + fn from(input: TableOpInput) -> Self { + Self { tables: input.tables } + } +} + +/// The output tables from a table operation. +#[derive(Clone, Debug, PartialEq)] +pub struct TableOpOutput { + pub tables: Vec, +} + +impl fmt::Display for TableOpOutput { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + for table in self.tables.iter() { + writeln!(f, "{table}")?; + } + Ok(()) + } +} + +impl TableOpOutput { + /// Convert this output into an input. + pub fn into_input(self) -> TableOpInput { + TableOpInput::from(self) + } + + /// Return the number of tables a table operation emits. + pub fn n_tables(&self) -> usize { + self.tables.len() + } +} + +impl From for TableOpInput { + fn from(output: TableOpOutput) -> Self { + Self { tables: output.tables } + } +} + +// Helper to distinguish when a plan has changed during optimized. +#[derive(Clone, Debug, PartialEq)] +enum OptimizedPlan { + Unchanged(Vec), + Optimized(Vec), +} + +impl OptimizedPlan { + // Print this optimized plan as a tree of plan nodes + fn to_plan_tree(&self) -> termtree::Tree { + let nodes = match self { + OptimizedPlan::Unchanged(nodes) => nodes, + OptimizedPlan::Optimized(nodes) => nodes, + }; + let mut root = + termtree::Tree::new(String::from("•")).with_multiline(true); + Plan::to_plan_tree_impl(&mut root, nodes, /* optimized = */ true); + root + } + + // Return the nodes in `self`. + fn nodes(&self) -> &[Node] { + match self { + OptimizedPlan::Unchanged(nodes) => nodes, + OptimizedPlan::Optimized(nodes) => nodes, + } + } + + // Check if the plan has any full table scans. + fn requires_full_table_scan(&self) -> bool { + let nodes = match self { + OptimizedPlan::Unchanged(nodes) => nodes, + OptimizedPlan::Optimized(nodes) => nodes, + }; + Self::requires_full_table_scan_impl(nodes) + } + + // Check if the plan has any full table scans. + fn requires_full_table_scan_impl(nodes: &[Node]) -> bool { + nodes.iter().any(|node| match node { + Node::Subquery(subplans) => { + subplans.iter().any(Plan::requires_full_table_scan) + } + Node::Get(get) => get.filters.is_empty(), + Node::Delta(_) + | Node::Filter(_) + | Node::Align(_) + | Node::GroupBy(_) + | Node::Join(_) + | Node::Limit(_) => false, + }) + } +} + +/// An OxQL query plan. +#[derive(Clone, Debug)] +pub struct Plan { + /// The original parsed query. + pub query: Query, + // The original query plan. + nodes: Vec, + // The possibly optimized, rewritten query plan. + optimized: OptimizedPlan, + /// Time spent generating the query plan. + pub duration: Duration, +} + +impl PartialEq for Plan { + fn eq(&self, other: &Self) -> bool { + self.query.eq(&other.query) + && self.nodes.eq(&other.nodes) + && self.optimized.eq(&other.optimized) + } +} + +impl Plan { + /// Generate a plan for the provided query. + /// + /// The provided `schema` must include the timeseries schema for any + /// timeseries referred to in the orginal query. I.e., it lists the schema + /// for the raw data to be selected from the database. + pub fn new( + query: Query, + schema: &BTreeMap, + ) -> anyhow::Result { + let start = Instant::now(); + let mut nodes = Vec::with_capacity(query.table_ops().len()); + Self::plan_query(&query, &schema, &mut nodes)?; + let optimized = Self::optimize_plan(&nodes)?; + let duration = start.elapsed(); + if let OptimizedPlan::Optimized(optimized_plan) = &optimized { + let original_output = + nodes.last().expect("plan cannot be empty").output(); + let optimized_output = + optimized_plan.last().expect("plan cannot be empty").output(); + assert_eq!( + original_output, optimized_output, + "Query optimization resulted in different outputs!\n\ + original = {:?}\n\ + optimized = {:?}", + original_output, optimized_output, + ); + } + Ok(Self { query, nodes, optimized, duration }) + } + + /// Pretty-print the plan as a tree of plan nodes. + pub fn to_plan_tree(&self) -> (String, String) { + let mut root = + termtree::Tree::new(String::from("•")).with_multiline(true); + Self::to_plan_tree_impl( + &mut root, + &self.nodes, + /* optimized = */ false, + ); + let original_tree = root.to_string(); + let optimized_tree = self.optimized.to_plan_tree(); + (original_tree, optimized_tree.to_string()) + } + + /// Return the table schema for the output of a plan. + pub fn output(&self) -> TableOpOutput { + self.nodes.last().expect("plan cannot be empty").output() + } + + // Return the new, possibly optimized plan nodes. + #[cfg(test)] + pub fn optimized_nodes(&self) -> &[Node] { + self.optimized.nodes() + } + + // Return the original, unoptimized plan nodes. + #[cfg(test)] + pub fn nodes(&self) -> &[Node] { + &self.nodes + } + + // Push nodes of the plan onto the tree, recursively. + fn to_plan_tree_impl( + root: &mut termtree::Tree, + mut nodes: &[Node], + optimized: bool, + ) { + if let Node::Subquery(plans) = &nodes[0] { + let subplans = plans.iter().map(|plan| { + let mut root = + termtree::Tree::new(format!("• {}", plan.query)) + .with_multiline(true); + let nodes = if optimized { + plan.optimized.nodes() + } else { + &plan.nodes + }; + Self::to_plan_tree_impl(&mut root, nodes, optimized); + root + }); + root.extend(subplans); + nodes = &nodes[1..]; + } + for node in nodes.iter() { + root.push(node.plan_tree_entry()); + } + } + + // Plan the provided query, inserting the plan nodes into the list. + fn plan_query( + query: &Query, + schema: &BTreeMap, + nodes: &mut Vec, + ) -> anyhow::Result<()> { + for table_op in query.table_ops() { + match table_op { + TableOp::Basic(basic) => { + Self::plan_basic_table_op(&schema, basic, nodes)? + } + TableOp::Grouped(grouped) => { + Self::plan_subquery(&schema, grouped, nodes)? + } + } + } + Ok(()) + } + + // Generate plan nodes for a basic table op. + fn plan_basic_table_op( + schema: &BTreeMap, + op: &BasicTableOp, + nodes: &mut Vec, + ) -> anyhow::Result<()> { + match op { + BasicTableOp::Get(name) => { + let db_schema = schema.get(name).with_context(|| { + format!("Schema for timeseries '{}' not found", name) + })?; + let table_schema = TableSchema::new(db_schema); + let node = Node::Get(Get { + table_schema: table_schema.clone(), + filters: vec![], + limit: None, + }); + nodes.push(node); + + // Insert the plan node for computing deltas for a cumulative + // timeseries, if needed. + if table_schema.metric_types[0] == MetricType::Cumulative { + let node = Node::Delta(Delta::new(&table_schema)?); + nodes.push(node); + } + } + BasicTableOp::Filter(filter) => { + // A filter doesn't change the schema, but we do have to check + // that it applies to every input table. + let input = nodes + .last() + .expect("Must have a previous node") + .output() + .into_input(); + let node = Node::Filter(Filter::new(filter, input)?); + nodes.push(node); + } + BasicTableOp::GroupBy(group_by) => { + // A group_by just drops the fields that are not named, and + // aggregates the others in a specific way. + let input = nodes + .last() + .expect("Must have a previous node") + .output() + .into_input(); + let node = Node::GroupBy(GroupBy::new(group_by, input)?); + nodes.push(node); + } + BasicTableOp::Join(_) => { + // A join concatenates all the data from the input tables that + // have the same values for their fields. All tables have to + // have the same field names / types. + let inputs = nodes + .last() + .expect("Must have a previous node") + .output() + .into_input(); + let node = Node::Join(Join::new(inputs)?); + nodes.push(node); + } + BasicTableOp::Align(align) => { + let input = nodes + .last() + .expect("Must have a previous node") + .output() + .into_input(); + let node = Node::Align(Align::new(*align, input)?); + nodes.push(node); + } + BasicTableOp::Limit(limit) => { + // Limit operations do not modify anything about the data at + // all, so just return the input schema unchanged. + let output = + nodes.last().expect("Must have a previous node").output(); + nodes.push(Node::Limit(Limit { limit: *limit, output })); + } + } + Ok(()) + } + + // Generate a plan node for a grouped table op, i.e., a subquery. + // + // Note that subqueries are not optimized themselves. The entire query is + // optimized as a whole. That means `optimized` is unchanged. + fn plan_subquery( + schema: &BTreeMap, + grouped: &GroupedTableOp, + nodes: &mut Vec, + ) -> anyhow::Result<()> { + let mut subplans = Vec::with_capacity(grouped.ops.len()); + for query in grouped.ops.iter().cloned() { + let start = Instant::now(); + let mut new_nodes = Vec::new(); + Self::plan_query(&query, schema, &mut new_nodes)?; + subplans.push(Plan { + query, + nodes: new_nodes.clone(), + optimized: OptimizedPlan::Unchanged(new_nodes), + duration: start.elapsed(), + }); + } + nodes.push(Node::Subquery(subplans)); + Ok(()) + } + + // Optimize a plan, if possible. + // + // This attempts to apply a number of plan-rewriting steps, to make the + // query as efficient as possible. Today these steps include: + // + // - Predicate pushdown: moving filtering predicates as close to the data as + // possible + // - Limit pushdown: Moving `first` or `last` table operations as close to + // the data as possible. + // + // There is a lot of room for new steps here. The most obvious next + // candidates are: + // + // - Deltas: push the computation of deltas from cumulative timeseries into + // the database + // - Alignment: push alignment operations into the database + // - Constant evaluation: As we support richer expressions, such as writing + // filters like `filter x > 1 / 2` or `filter x > log(5)`, we can evaluate + // those expressions at query-plan time. + fn optimize_plan(nodes: &[Node]) -> anyhow::Result { + let optimized = Self::pushdown_predicates(nodes)?; + Self::pushdown_limit(optimized.nodes()) + } + + // Push down limit operations in the list of plan nodes. + fn pushdown_limit(nodes: &[Node]) -> anyhow::Result { + anyhow::ensure!( + !nodes.is_empty(), + "Planning error: plan nodes cannot be empty" + ); + let mut modified = false; + + // Collect nodes in the plan. + let mut remaining_nodes = + nodes.iter().cloned().collect::>(); + let mut processed_nodes = VecDeque::with_capacity(nodes.len()); + + while let Some(current_node) = remaining_nodes.pop_back() { + // What we do with the limit node depends on what's in front of it. + let Some(next_node) = remaining_nodes.pop_back() else { + // If there _isn't_ one, then the current node must be the start + // of the plan, and so push it and break out. + processed_nodes.push_front(current_node); + break; + }; + + // If this isn't a limit node, just push it and continue + let Node::Limit(limit) = current_node else { + processed_nodes.push_front(current_node); + processed_nodes.push_front(next_node); + continue; + }; + + match next_node { + Node::Subquery(subplans) => { + // Push the limit onto the subquery plans and recurse. + let new_subplans = subplans + .into_iter() + .map(|mut plan| { + let start = Instant::now(); + let nodes = [ + plan.optimized.nodes(), + &[Node::Limit(limit.clone())], + ] + .concat(); + plan.optimized = Self::pushdown_limit(&nodes)?; + plan.duration += start.elapsed(); + Ok(plan) + }) + .collect::>>()?; + processed_nodes.push_front(Node::Subquery(new_subplans)); + modified = true; + } + Node::Get(mut get) => { + // If we've gotten here, we _may_ be able to push this into + // the databse, but only if there isn't one already in the + // get node. + match get.limit.as_mut() { + Some(existing) => { + // There's already a limiting operation here. We may + // be able to coalesce them, but only if they're the + // same kind. If not, it's not correct to reorder + // them, so we have to push the current node and + // then the next (get) node onto the processed list. + if existing.kind == limit.limit.kind { + // Take the smaller of the two! + existing.count = + existing.count.min(limit.limit.count); + modified = true; + } else { + // These are different kinds, push them in + // order, starting with the limit. + processed_nodes.push_front(Node::Limit(limit)); + } + } + None => { + let old = get.limit.replace(limit.limit); + assert!(old.is_none()); + modified = true; + } + } + + // We always push the get node last. + processed_nodes.push_front(Node::Get(get)); + } + Node::Delta(_) + | Node::Align(_) + | Node::GroupBy(_) + | Node::Join(_) => { + remaining_nodes.push_back(Node::Limit(limit)); + processed_nodes.push_front(next_node); + modified = true; + } + Node::Filter(filter) => { + // We might be able to reorder the limit through the filter, + // but it depends on whether and how the filter references + // timestamps. They need to point in the same "direction" -- + // see `can_reorder_around()` for details. + if filter.can_reorder_around(&limit.limit) { + processed_nodes.push_front(Node::Filter(filter)); + remaining_nodes.push_back(Node::Limit(limit)); + modified = true; + } else { + processed_nodes.push_front(Node::Limit(limit)); + processed_nodes.push_front(Node::Filter(filter)); + } + } + Node::Limit(mut other_limit) => { + // We might be able to coalesce these, if they're of the + // same kind. If they are not, push the current one, and + // then start carrying through the next one. + if limit.limit.kind == other_limit.limit.kind { + other_limit.limit.count = + other_limit.limit.count.min(limit.limit.count); + remaining_nodes.push_back(Node::Limit(other_limit)); + modified = true; + } else { + processed_nodes.push_front(Node::Limit(limit)); + remaining_nodes.push_back(Node::Limit(other_limit)); + } + } + } + } + + let out = processed_nodes.make_contiguous().to_vec(); + if modified { + Ok(OptimizedPlan::Optimized(out)) + } else { + Ok(OptimizedPlan::Unchanged(out)) + } + } + + // Push down predicates in the list of plan nodes. + fn pushdown_predicates(nodes: &[Node]) -> anyhow::Result { + Self::pushdown_predicates_impl(nodes, None) + } + + // Recursive implementation of predicate pushdown. + fn pushdown_predicates_impl( + nodes: &[Node], + outer_predicates: Option, + ) -> anyhow::Result { + anyhow::ensure!( + !nodes.is_empty(), + "Planning error: plan nodes cannot be empty" + ); + + // Used to return correct variant of `OptimizedPlan`. + let mut modified = false; + + // Collect the nodes in the plan. + // + // We'll process the query plan from back to front, pushing nodes onto + // the output plan as we consider them. This is so we can "push" the + // filters from the back of the plan towards the front, as we process + // nodes. + let mut remaining_nodes = + nodes.iter().cloned().collect::>(); + let mut processed_nodes = VecDeque::with_capacity(nodes.len()); + + // If we were provided with outer predicates, let's just add on a filter + // plan node to the provided plan. It will be processed like any other + // node. + if let Some(predicates) = outer_predicates { + let filt = Filter::from_predicates( + predicates, + nodes + .last() + .expect("length verified above") + .output() + .into_input(), + )?; + remaining_nodes.push_back(Node::Filter(filt)); + }; + + // Process nodes in the query plan, in reverse. + while let Some(current_node) = remaining_nodes.pop_back() { + // What we do with the node depends on what's in front of it, so + // take the next node in the plan. + let Some(next_node) = remaining_nodes.pop_back() else { + // If there _isn't_ one, then the current node must be the start + // of the plan, and so push it and break out. + processed_nodes.push_front(current_node); + break; + }; + + // If the current node isn't a filter, then this pass doesn't modify + // it at all. Instead, just push it and continue. But! We have to + // push back the next node we just popped above. + let Node::Filter(current_filter) = current_node else { + processed_nodes.push_front(current_node); + remaining_nodes.push_back(next_node); + continue; + }; + + // At this point we are looking at a filter node, and the next one + // exists. We might be able to modify them in a few ways. + match next_node { + Node::Subquery(subplans) => { + // We can push the filter into each one of the subquery + // plans, as its outer predicates. + let new_subplans = subplans + .into_iter() + .map(|mut plan| { + let start = Instant::now(); + plan.optimized = Self::pushdown_predicates_impl( + &plan.nodes, + Some(current_filter.predicates.clone()), + )?; + plan.duration += start.elapsed(); + Ok(plan) + }) + .collect::>>()?; + processed_nodes.push_front(Node::Subquery(new_subplans)); + modified = true; + } + Node::Get(mut get) => { + // Push the filters into the get plan node. + // + // At this point, we're at the front of the query plan, so + // the filters won't be pushed any farther. It's required + // that every filter we push through be _non-empty_. See + // `Filter` for details. + get.filters = current_filter.predicates.to_required()?; + processed_nodes.push_front(Node::Get(get)); + modified = true; + } + Node::Delta(ref delta) => { + // We can _sometimes_ push a filter around a delta, but not + // the entire filter expression in general. + // + // Delta nodes change the datum values for a cumulative + // timeseries, by computing adjacent differences. In a + // logical, unoptimized query plan, we always fetch the + // timeseries and immediately (and implicitly) construct the + // deltas. So to match the semantics of the literal query, + // any filter on the datum must necessarily refer to those + // differences, not the original, cumulative values. + // + // That means we can push through filter expressions, except + // those portions which refer to the datum. Field filters + // and any filters on the timestamps are valid as well. Note + // that the latter are valid because computing differences + // doesn't change the timestamps themselves, only the start + // times. + let SplitPredicates { pushed, not_pushed } = + current_filter.predicates.split_around_delta(delta)?; + if let Some(after) = not_pushed { + let new_input = next_node.output().into_input(); + let filter = Filter::from_predicates(after, new_input)?; + processed_nodes.push_front(Node::Filter(filter)); + } + processed_nodes.push_front(next_node); + + // The filters we did push through need to go back on the + // remaining node queue, with the next node's output as its + // input. + if let Some(before) = pushed { + let new_input = remaining_nodes + .back() + .as_ref() + .expect("Align cannot start a query") + .output() + .into_input(); + let filter = + Filter::from_predicates(before, new_input.clone())?; + remaining_nodes.push_back(Node::Filter(filter)); + } + modified = true; + } + Node::Filter(mut next_filter) => { + // If the next node is also a filter, we can just merge the + // current one into it, and push it back onto the remaining + // nodes. We'll process it again on the next pass through + // the loop. + let new_predicates = next_filter + .predicates + .and(¤t_filter.predicates)?; + next_filter.predicates = new_predicates; + remaining_nodes.push_back(Node::Filter(next_filter)); + modified = true; + } + Node::Align(ref align) => { + // We can also push a filter around an alignment operation, + // but we may need to split it up. + // + // Filters that apply to the _fields_ can always be pushed, + // but the filters that apply to the measurements cannot. + // Separate these into a new set of filter nodes that come + // before and after the alignment, respectively. + let SplitPredicates { pushed, not_pushed } = + current_filter.predicates.split_around_align(align)?; + if let Some(after) = not_pushed { + let new_input = next_node.output().into_input(); + let filter = Filter::from_predicates(after, new_input)?; + processed_nodes.push_front(Node::Filter(filter)); + } + processed_nodes.push_front(next_node); + + // The filters we did push through need to go back on the + // remaining node queue, with the next node's output as its + // input. + if let Some(before) = pushed { + let new_input = remaining_nodes + .back() + .as_ref() + .expect("Align cannot start a query") + .output() + .into_input(); + let filter = + Filter::from_predicates(before, new_input.clone())?; + remaining_nodes.push_back(Node::Filter(filter)); + } + modified = true; + } + Node::GroupBy(_) => { + // We can always push a filter around a group by, but it's a + // bit subtle. + // + // At this point in the plan creation, we have checked that + // the filter node we're operating on only refers to fields + // that are valid at this point in the query. That can only + // be fields that are named in the group-by table operation! + // Any of those can be pushed through without changing the + // contents of the group. We do need to change the input + // schema the filter considers, however, since the groupby + // operation probably has a different one. + processed_nodes.push_front(next_node); + let input = remaining_nodes + .back() + .expect("group_by cannot start a query") + .output() + .into_input(); + let new_filter = Node::Filter( + Filter::from_predicates( + current_filter.predicates.clone(), + input, + ) + .context("planning error")?, + ); + remaining_nodes.push_back(new_filter); + modified = true; + } + Node::Join(_) => { + // We can also always push a filter around a join operation. + // + // TODO-completeness: It would be very nice to figure out + // how to refer to `datum`s in joins. Right now, we would + // apply the filter to both, which is probably nonsensical. + // We should consider letting folks name the metric portion + // of the timeseries name / table name, in addition to the + // special identifier `datum`. + // + // See https://github.com/oxidecomputer/omicron/issues/6761. + processed_nodes.push_front(next_node); + let input = remaining_nodes + .back() + .expect("join cannot start a query") + .output() + .into_input(); + let new_filter = Node::Filter( + Filter::from_predicates( + current_filter.predicates.clone(), + input, + ) + .context("planning error")?, + ); + remaining_nodes.push_back(new_filter); + modified = true; + } + Node::Limit(limit) => { + // We _might_ be able to reorder the filter around the + // limit, in a few cases. See `can_reorder_around()` for + // details. + if current_filter + .predicates + .can_reorder_around(&limit.limit) + { + // Push the limit node onto the output plan, and then + // push the filter back on the remaining nodes. + processed_nodes.push_front(Node::Limit(limit)); + remaining_nodes.push_back(Node::Filter(current_filter)); + modified = true; + } else { + // We can't reorder these -- push the filter, and then + // put the limit back so we can process it on the next + // pass. + processed_nodes.push_back(Node::Filter(current_filter)); + remaining_nodes.push_front(Node::Limit(limit)); + } + } + } + } + + // If this pass modified anything, return the new plan, else return + // nothing. + let output_nodes = processed_nodes.make_contiguous().to_vec(); + if modified { + Ok(OptimizedPlan::Optimized(output_nodes)) + } else { + Ok(OptimizedPlan::Unchanged(output_nodes)) + } + } + + /// Return true if this plan requires at least 1 full table scan. + pub fn requires_full_table_scan(&self) -> bool { + self.optimized.requires_full_table_scan() + } +} + +#[cfg(test)] +pub(super) mod test_utils { + use chrono::NaiveDateTime; + use oximeter::FieldSchema; + use oximeter::TimeseriesName; + use oximeter::TimeseriesSchema; + use std::collections::BTreeMap; + use tokio::sync::OnceCell; + + const FILE: &str = + concat!(env!("CARGO_MANIFEST_DIR"), "/tests/timeseries-schema.json"); + + /// Type representing the direct JSON output of the timeseries_schema table. + #[derive(Clone, Debug, serde::Deserialize)] + struct DbTimeseriesSchema { + timeseries_name: TimeseriesName, + #[serde(rename = "fields.name")] + field_names: Vec, + #[serde(rename = "fields.type")] + field_types: Vec, + #[serde(rename = "fields.source")] + field_sources: Vec, + datum_type: String, + created: String, + } + + impl From for TimeseriesSchema { + fn from(value: DbTimeseriesSchema) -> Self { + assert_eq!(value.field_names.len(), value.field_types.len()); + assert_eq!(value.field_names.len(), value.field_sources.len()); + let field_schema = value + .field_names + .into_iter() + .zip(value.field_types) + .zip(value.field_sources) + .map(|((name, field_type), source)| FieldSchema { + name, + field_type: field_type.parse().unwrap(), + source: source.parse().unwrap(), + description: String::new(), + }) + .collect(); + Self { + timeseries_name: value.timeseries_name, + description: Default::default(), + field_schema, + datum_type: value.datum_type.parse().unwrap(), + version: std::num::NonZeroU8::new(1).unwrap(), + authz_scope: oximeter::AuthzScope::Fleet, + units: oximeter::Units::None, + created: NaiveDateTime::parse_from_str( + &value.created, + "%Y-%m-%d %H:%M:%S%.9f", + ) + .unwrap() + .and_utc(), + } + } + } + + async fn load_schema() -> BTreeMap { + let contents = tokio::fs::read_to_string(&FILE).await.unwrap(); + contents + .lines() + .map(|line| { + let schema: DbTimeseriesSchema = + serde_json::from_str(&line).unwrap(); + ( + TimeseriesName::try_from(schema.timeseries_name.as_str()) + .unwrap(), + schema.into(), + ) + }) + .collect() + } + + static ALL_SCHEMA: OnceCell> = + OnceCell::const_new(); + + pub async fn all_schema( + ) -> &'static BTreeMap { + ALL_SCHEMA.get_or_init(load_schema).await + } +} + +#[cfg(test)] +mod tests { + use crate::oxql::ast::grammar::query_parser; + use crate::oxql::plan::node::Node; + use crate::oxql::plan::plan::test_utils::all_schema; + use crate::oxql::plan::Plan; + use oxql_types::point::DataType; + use oxql_types::point::MetricType; + + #[tokio::test] + async fn get_gauge_plan_emits_one_node() { + const TIMESERIES_NAME: &str = "collection_target:cpus_provisioned"; + let query = + query_parser::query(&format!("get {TIMESERIES_NAME}")).unwrap(); + let all_schema = all_schema().await; + let plan = Plan::new(query, &all_schema).unwrap(); + + assert_eq!( + plan.nodes.len(), + 1, + "getting a single table by name with no filters and \ + a gauge metric type should result in one table operation", + ); + let Node::Get(get) = &plan.nodes[0] else { + panic!("expected just a get plan node, found {:?}", plan.nodes[0]); + }; + let original_schema = + all_schema.get(&TIMESERIES_NAME.parse().unwrap()).unwrap(); + assert_eq!(get.table_schema.name, TIMESERIES_NAME); + assert_eq!(get.table_schema.metric_types, &[MetricType::Gauge]); + assert_eq!(get.table_schema.data_types, &[DataType::Integer]); + let mut n_seen = 0; + for field_schema in original_schema.field_schema.iter() { + let ty = get + .table_schema + .fields + .get(&field_schema.name) + .expect("Field should be found"); + assert_eq!(ty, &field_schema.field_type); + n_seen += 1; + } + assert_eq!(n_seen, get.table_schema.fields.len()); + } + + #[tokio::test] + async fn get_cumulative_plan_emits_two_nodes() { + const TIMESERIES_NAME: &str = "physical_data_link:bytes_sent"; + let query = + query_parser::query(&format!("get {TIMESERIES_NAME}")).unwrap(); + let all_schema = all_schema().await; + let plan = Plan::new(query, &all_schema).unwrap(); + + assert_eq!( + plan.nodes.len(), + 2, + "getting a single table by name with no filters and \ + a cumulative metric type should result in two table operations", + ); + let Node::Get(get) = &plan.nodes[0] else { + panic!("expected a get plan node, found {:?}", plan.nodes[0]); + }; + let original_schema = + all_schema.get(&TIMESERIES_NAME.parse().unwrap()).unwrap(); + assert_eq!(get.table_schema.name, TIMESERIES_NAME); + assert_eq!(get.table_schema.metric_types, &[MetricType::Cumulative]); + assert_eq!(get.table_schema.data_types, &[DataType::Integer]); + let mut n_seen = 0; + for field_schema in original_schema.field_schema.iter() { + let ty = get + .table_schema + .fields + .get(&field_schema.name) + .expect("Field should be found"); + assert_eq!(ty, &field_schema.field_type); + n_seen += 1; + } + assert_eq!(n_seen, get.table_schema.fields.len()); + + let Node::Delta(delta) = &plan.nodes[1] else { + panic!("expected a delta plan node, found {:?}", plan.nodes[0]); + }; + assert_eq!(delta.output.name, get.table_schema.name); + assert_eq!(delta.output.fields, get.table_schema.fields); + assert_eq!(delta.output.data_types, get.table_schema.data_types); + assert_eq!(delta.output.metric_types[0], MetricType::Delta); + } + + #[tokio::test] + async fn subquery_plan_returns_multiple_tables() { + let query = query_parser::query( + "{ \ + get physical_data_link:bytes_received; \ + get physical_data_link:bytes_received \ + }", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + assert_eq!( + plan.nodes.len(), + 1, + "Outer query plan should have one node" + ); + let Node::Subquery(plans) = &plan.nodes[0] else { + panic!( + "Plan should have generated a subquery node, found {:?}", + plan.nodes[0], + ); + }; + assert_eq!(plans.len(), 2, "Should have generated two subquery plans",); + assert_eq!(plans[0], plans[1], "Both subquery plans should be equal"); + let nodes = &plans[0].nodes; + assert_eq!(nodes.len(), 2, "Each subquery should have 2 nodes"); + assert!(matches!(nodes[0], Node::Get(_))); + assert!(matches!(nodes[1], Node::Delta(_))); + + let output = plan.output(); + assert_eq!( + output.tables.len(), + 2, + "Query plan with 2 suqueries should emit 2 tables" + ); + assert_eq!(output.tables[0], output.tables[1]); + } + + #[tokio::test] + async fn group_by_plan_leaves_only_grouped_fields() { + let query = query_parser::query( + "get physical_data_link:bytes_received \ + | align mean_within(20s) \ + | group_by [sled_id, serial]", + ) + .unwrap(); + let all_schema = all_schema().await; + let plan = Plan::new(query, all_schema).unwrap(); + assert_eq!( + plan.nodes.len(), + 4, + "Should have four nodes (+1 is for implicit delta)" + ); + let Node::GroupBy(group_by) = plan.nodes.last().unwrap() else { + panic!( + "expected a group_by as the last node, found {:?}", + plan.nodes.last().unwrap(), + ); + }; + let output = &group_by.output; + let plan_output = plan.output(); + assert_eq!(plan_output.tables.len(), 1); + assert_eq!( + &plan_output.tables[0], output, + "plan output should be the output of the last node" + ); + assert_eq!( + output.schema.fields.len(), + 2, + "group_by should have only resulted in two fields, \ + those listed in the table operation" + ); + let timeseries_name = + "physical_data_link:bytes_received".parse().unwrap(); + for field in ["sled_id", "serial"] { + let output_type = output.schema.fields[field]; + let original_type = all_schema + .get(×eries_name) + .unwrap() + .field_schema + .iter() + .find_map(|s| { + if s.name == field { + Some(s.field_type) + } else { + None + } + }) + .unwrap(); + assert_eq!( + output_type, original_type, + "group_by operation should not change field types" + ); + } + } + + #[tokio::test] + async fn filter_plan_node_does_not_change_table_schema() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | filter serial == 'foo'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + assert_eq!(plan.nodes.len(), 3); + let input = plan.nodes[1].output(); + let output = plan.output(); + assert_eq!(input, output, "Filter should not change the table schema"); + } + + #[tokio::test] + async fn limit_plan_node_does_not_change_table_schema() { + let query = + query_parser::query("get physical_data_link:bytes_sent | last 1") + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + assert_eq!(plan.nodes.len(), 3); + let input = plan.nodes[1].output(); + let output = plan.output(); + assert_eq!(input, output, "Last should not change the table schema"); + } + + #[tokio::test] + async fn cannot_group_multiple_tables() { + let query = query_parser::query( + "{ \ + get physical_data_link:bytes_sent; \ + get physical_data_link:bytes_received \ + } \ + | align mean_within(10s) \ + | group_by [sled_id]", + ) + .unwrap(); + let err = Plan::new(query, all_schema().await).expect_err( + "Should fail to plan query that groups multiple tables", + ); + assert!( + err.to_string().contains("require exactly one input table"), + "Error message should complain about multiple tables, \ + but the error message is: {:#?}", + err + ); + } + + #[tokio::test] + async fn cannot_join_one_table() { + let query = query_parser::query( + "get physical_data_link:bytes_sent \ + | align mean_within(10s) \ + | join", + ) + .unwrap(); + let err = Plan::new(query, all_schema().await) + .expect_err("Should fail to plan query that joins one tables"); + assert!( + err.to_string().contains("require at least 2 tables"), + "Error message should complain about only having one \ + table, but the error message is: {:#?}", + err + ); + } + + #[tokio::test] + async fn cannot_join_unaligned_tables() { + let query = query_parser::query( + "{ \ + get physical_data_link:bytes_sent; \ + get physical_data_link:bytes_received \ + } | join", + ) + .unwrap(); + let err = Plan::new(query, all_schema().await).expect_err( + "Should fail to plan query that joins unaligned tables", + ); + assert!( + err.to_string().contains("is not aligned"), + "Error message should complain that the input tables are \ + not aligned, but the error message is: {:#?}", + err + ); + } + + #[tokio::test] + async fn cannot_group_unaligned_tables() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | group_by [sled_id]", + ) + .unwrap(); + let err = Plan::new(query, all_schema().await).expect_err( + "Should fail to plan query that groups unaligned tables", + ); + assert!( + err.to_string().contains("is not aligned"), + "Error message should complain that the input tables are \ + not aligned, but the error message is: {:#?}", + err + ); + } + + #[tokio::test] + async fn cannot_align_non_numeric_tables() { + let query = query_parser::query( + "get http_service:request_latency_histogram | align mean_within(10s)" + ).unwrap(); + let err = Plan::new(query, all_schema().await) + .expect_err("Should fail to plan query that aligns histograms"); + assert!( + err.to_string().contains("cannot be aligned"), + "Error message should complain that histogram tables cannot \ + be aligned, but the error message is: {:#?}", + err, + ); + } + + #[tokio::test] + async fn cannot_filter_with_incomparable_types() { + let query = query_parser::query( + "get http_service:request_latency_histogram | filter name == 0", + ) + .unwrap(); + let err = Plan::new(query, all_schema().await).expect_err( + "Should fail to plan query with an incomparable filter", + ); + assert!( + err.to_string().contains("is not compatible with the field"), + "Error message should complain that a filter cannot compare \ + a field against an incompatible type, but the error message is: {:#?}", + err, + ); + } + + #[tokio::test] + async fn predicate_pushdown_merges_neighboring_filter_nodes() { + let query = query_parser::query( + "get physical_data_link:bytes_sent \ + | filter serial == 'foo' \ + | filter link_name == 'bar'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let optimized_nodes = plan.optimized_nodes(); + assert_eq!(optimized_nodes.len(), 2); + let Node::Get(get) = &optimized_nodes[0] else { + panic!("Expected a get node, found {:?}", &optimized_nodes[0]); + }; + assert_eq!(get.filters.len(), 1); + assert!(matches!(&optimized_nodes[1], Node::Delta(_))); + assert_eq!( + plan.nodes.last().unwrap().output(), + optimized_nodes.last().unwrap().output() + ); + } + + #[tokio::test] + async fn predicate_pushdown_pushes_filter_nodes_through_group_by() { + let query = query_parser::query( + "get physical_data_link:bytes_sent \ + | align mean_within(1m) \ + | group_by [serial] \ + | filter serial == 'foo'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let optimized_nodes = plan.optimized_nodes(); + assert_eq!(optimized_nodes.len(), 4); + let Node::Get(get) = &optimized_nodes[0] else { + panic!("Expected a get node, found {:?}", &optimized_nodes[0]); + }; + assert_eq!(get.filters.len(), 1); + assert_eq!( + plan.nodes.last().unwrap().output(), + optimized_nodes.last().unwrap().output() + ); + } + + #[tokio::test] + async fn predicate_pushdown_pushes_filter_nodes_through_join() { + let query = query_parser::query( + "{ \ + get physical_data_link:bytes_sent; \ + get physical_data_link:bytes_received \ + } \ + | align mean_within(1m) \ + | join + | filter serial == 'foo'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let optimized_nodes = plan.optimized_nodes(); + assert_eq!(optimized_nodes.len(), 3); + let Node::Subquery(subqueries) = &optimized_nodes[0] else { + panic!("Expected a subquery node, found {:?}", &optimized_nodes[0]); + }; + for subq in subqueries.iter() { + let nodes = subq.optimized_nodes(); + assert_eq!(nodes.len(), 2); + let Node::Get(get) = &nodes[0] else { + panic!("Expected a get node, found {:?}", &nodes[0]); + }; + assert_eq!(get.filters.len(), 1); + assert!(matches!(&nodes[1], Node::Delta(_))); + } + assert!(matches!(&optimized_nodes[1], Node::Align(_))); + assert!(matches!(&optimized_nodes[2], Node::Join(_))); + assert_eq!( + plan.nodes.last().unwrap().output(), + optimized_nodes.last().unwrap().output() + ); + } + + #[tokio::test] + async fn predicate_pushdown_pushes_filter_nodes_into_subqueries() { + let query = query_parser::query( + "{ \ + get physical_data_link:bytes_sent; \ + get physical_data_link:bytes_received \ + } | filter serial == 'foo'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let optimized_nodes = plan.optimized_nodes(); + assert_eq!(optimized_nodes.len(), 1); + let Node::Subquery(subqueries) = &optimized_nodes[0] else { + panic!("Expected a subquery node, found {:?}", &optimized_nodes[0]); + }; + for subq in subqueries.iter() { + let nodes = subq.optimized_nodes(); + assert_eq!(nodes.len(), 2); + let Node::Get(get) = &nodes[0] else { + panic!("Expected a get node, found {:?}", &nodes[0]); + }; + assert_eq!(get.filters.len(), 1); + assert!(matches!(&nodes[1], Node::Delta(_))); + } + assert_eq!( + plan.nodes.last().unwrap().output(), + optimized_nodes.last().unwrap().output() + ); + } + + #[tokio::test] + async fn requires_full_table_scan() { + let query = + query_parser::query("get physical_data_link:bytes_sent").unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + assert!(plan.requires_full_table_scan()); + + let query = query_parser::query( + "get physical_data_link:bytes_sent | filter link_name == 'foo'", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + assert!(!plan.requires_full_table_scan()); + } +} diff --git a/oximeter/db/src/oxql/plan/predicates.rs b/oximeter/db/src/oxql/plan/predicates.rs new file mode 100644 index 0000000000..bbf08a66b1 --- /dev/null +++ b/oximeter/db/src/oxql/plan/predicates.rs @@ -0,0 +1,649 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Operate on predicates in a query plan, including spliting, merging and +//! reordering. + +// Copyright 2024 Oxide Computer Company + +use crate::oxql::ast::table_ops::filter; +use crate::oxql::ast::table_ops::limit::Limit; +use crate::oxql::plan::align::Align; +use crate::oxql::plan::delta::Delta; +use crate::oxql::plan::filter::Filter; + +/// Predicates in an OxQL plan node that filters data. +/// +/// # Overview +/// +/// During query planning, we try to manipulate the predicates to push as much +/// as possible close to the data. This is called "predicate pushdown". For a +/// variety of reasons, it's not always straightforward to do this. For example, +/// some filters only apply to data that we process in Rust, not the database. +/// This includes queries like: +/// +/// align mean_within(1m) | filter field == 0 && datum > 100 +/// +/// In this case, the field predicate can be pushed into the database, but it's +/// not correct to push the filter on the datum. That only applies to the +/// _aligned_ data, which is only computed in Rust, after fetching the data from +/// the database. +/// +/// In this particular case, it's fairly straightforward to split these two. +/// Because they are joined by a logical AND, this is equivalent to: +/// +/// filter field == 0 | align mean_within(1m) | filter datum > 100 +/// +/// And that's exactly what we do: push the field filters (and timestamps, +/// though it's less obvious that that's correct), and keep the datum filters on +/// the "outside" of the alignment. +/// +/// # Disjunctions +/// +/// This all becomes vastly more complicated in the presence of disjunctions, +/// statements joined by logical ORs. Consider this: +/// +/// align mean_within(1m) | filter field == 0 || datum > 100 +/// +/// In this case, pushing the field filter through is not correct! We actually +/// want _all the data__ to make its way through the alignment operation. +/// However, for scalability, we disallow that kind of "full table scan". But we +/// can still show the complexities of disjunctions with a filter like: +/// +/// filter (field == 0) || (field == 1 && datum > 100) +/// +/// In this case, we want to push these two disjunctions _separately_ through +/// the alignment. However, we now need to match up the predicates we can push +/// through with those we can't, so we don't accidentally end up apply the +/// `datum > 100` predicate to the `field == 0` data! So we now need to maintain +/// a _list_ of predicates, with those pieces we've pushed through and those we +/// haven't, so we can match them up. +/// +/// This enum is used to keep track of the predicates in a query plan, as we +/// perform these kinds of operations. Continuing with the above example, the +/// query plan would look something like this: +/// +/// 1. First, we construct predicates from the single query AST node +/// representing the filter. That is, we take `filter (field == 0) || (field +/// == 1 && datum > 100)`, and we construct: +/// +/// ```rust,ignore +/// Predicates::Single("(field == 0) || (field == 1 && datum > 100)") +/// ``` +/// +/// 2. When we push this through an alignment operation, we will split them into +/// two other variants. Those moving through the aligment are: +/// +/// ```rust,ignore +/// Predicates::Disjunction(vec![Some("field == 0"), Some("field == 1")]) +/// ``` +/// +/// and those left outside are: +/// +/// ```rust,ignore +/// Predicates::Disjunctions(vec![None, Some("datum > 100")]) +/// ``` +/// +/// It's important to note that we _do_ allow pushing through None in this +/// particular piece of code. That's because we cannot tell locally whether +/// we'll end up with a full table scan. The overall plan constructor checks +/// that as one of its invariants at the end of optimization, but we don't know +/// that just when we push predicates around individual nodes. +/// +/// # Merging filters +/// +/// Another query optimization is to merge neighboring filters. That is, if you +/// write `filter field == 0 | filter other_field == 1`, we merge that into the +/// logically-equivalent `filter field == 0 || other_field == 1`. (Note that we +/// do not handle the case where you happen to write an unsatisfiable filter, +/// that's an extremely hard problem!) +/// +/// Query optimization always works "back-to-front", starting from the end of +/// the query and processing / optimizing nodes. We make only one pass rewriting +/// predicates, but we still don't know that all the predicates are "next to" +/// each other in the original query. For example: +/// +/// filter field == 0 +/// | align mean_within(1m) +/// | filter other_field == 0 || datum > 100 +/// +/// We should be able to push through the field filter, leave behind the +/// predicate on `datum`, and then merge the two field filters. +#[derive(Clone, Debug, PartialEq)] +pub enum Predicates { + /// A single filter, derived from an AST node, or known to contain 1 + /// disjunction. + /// + /// NOTE: There is no "single + optional" variant in this enum. If we push + /// the entire predicate through, we drop the outer plan node entirely, so + /// there is no "piece" of it that could not be pushed through. + Single(filter::Filter), + + /// A list of at least 1 possibly-empty disjunctions. + /// + /// As we push disjunctions through plan nodes, we _may_ leave behind the + /// parts that can't be pushed. Those are represented here. If an entire + /// disjunct can be pushed through, then the corresponding entry in this + /// variant is `None`. That is a placeholder, so that we can correctly match + /// up predicates we push through with those we cannot push through. + Disjunctions(OptionalDisjunctions), +} + +impl From for Predicates { + fn from(v: filter::Filter) -> Self { + Self::Single(v) + } +} + +impl From for Predicates { + fn from(v: OptionalDisjunctions) -> Self { + Self::Disjunctions(v) + } +} + +/// The return type from `Predicates` functions that split it around table +/// operations. +#[derive(Debug)] +pub struct SplitPredicates { + /// The predicates, if any, we pushed through an operation. + pub pushed: Option, + /// The predicates, if any, we did not push through an operation. + pub not_pushed: Option, +} + +impl Predicates { + /// Split self around an alignment plan node. + /// + /// If pushing through any of the predicates in self results in an "empty" + /// filter being pushed through, that returns None. This is to signal that + /// _none_ of the filter could be pushed through. This may indicate that the + /// query will result in a full table scan, which isn't allowed. But we + /// cannot prevent that here entirely, since there may actually be another + /// filtering node in the query prior to the alignment. I.e., we can't fail + /// the whole query until we get to the end of our optimization passes. + /// + /// For example, the filter `datum > 100` will return an error, as will + /// `field == 0 || datum > 100`. The filter `(field == 0 || field == 1 && + /// datum > 100)` is fine, and will result in a split predicate. + /// + /// An error is returned if `self` isn't `Self::Single`. + pub fn split_around_align( + &self, + align: &Align, + ) -> anyhow::Result { + let Self::Single(filter) = self else { + anyhow::bail!( + "Cannot split a filter with disjunctions around an \ + alignment table operation" + ); + }; + // Sanity check that the filter, before being split into disjunctions, + // applies to the schema. + let schema = &align.output.tables.first().unwrap().schema; + Filter::ensure_filter_expr_application_is_valid(&filter.expr, schema)?; + let disjunctions = filter.simplify_to_dnf()?.flatten_disjunctions(); + + // If we have a single disjunction, we are going to return an optional + // single variant. + if disjunctions.len() == 1 { + let disjunct = disjunctions.into_iter().next().unwrap(); + let pushed = disjunct + .remove_datum(schema) + .map(|maybe_removed| { + maybe_removed + .map(|f| f.shift_timestamp_by(align.alignment.period)) + })? + .map(Predicates::Single); + let not_pushed = + disjunct.only_datum(schema)?.map(Predicates::Single); + return Ok(SplitPredicates { pushed, not_pushed }); + } + + // We have multiple predicates, so let's separate out them all. + let mut pushed = Vec::with_capacity(disjunctions.len()); + let mut not_pushed = Vec::with_capacity(disjunctions.len()); + for disjunct in disjunctions { + pushed.push(disjunct.remove_datum(schema)?); + not_pushed.push(disjunct.only_datum(schema)?); + } + + // We need to "compress" either side if it's an array of all Nones. That + // means none of disjuncts could be reordered, and so we'll need to + // elide the plan node entirely. + let pushed = if pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(pushed))) + }; + let not_pushed = if not_pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(not_pushed))) + }; + Ok(SplitPredicates { pushed, not_pushed }) + } + + /// Split the predicates in self around a delta node. + pub fn split_around_delta( + &self, + delta: &Delta, + ) -> anyhow::Result { + match self { + Predicates::Single(single) => { + Self::split_single_predicates_around_delta(single, delta) + } + Predicates::Disjunctions(disjunctions) => { + // Even though we're pushing disjunctions, we only get this by + // splitting around another op. That means each element cannot + // _contain_ disjunctions -- they are joined by OR, though. So + // that means each element itself is single filter expr. So + // calling `flatten_disjunctions()` should return self. That + // means we know we'll always get a single predicate (or none). + // Either way, we push it onto our list, and then compress if + // needed. + let mut pushed = Vec::with_capacity(disjunctions.len()); + let mut not_pushed = Vec::with_capacity(disjunctions.len()); + for maybe_filter in disjunctions.iter() { + let Some(filter) = maybe_filter else { + continue; + }; + let this = Self::split_single_predicates_around_delta( + filter, delta, + )?; + match this.pushed { + Some(Predicates::Single(filter)) => { + pushed.push(Some(filter)) + } + Some(_) => unreachable!(), + None => pushed.push(None), + } + match this.not_pushed { + Some(Predicates::Single(filter)) => { + not_pushed.push(Some(filter)) + } + Some(_) => unreachable!(), + None => not_pushed.push(None), + } + } + + // Compress again. + let pushed = if pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(pushed))) + }; + let not_pushed = if not_pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(not_pushed))) + }; + Ok(SplitPredicates { pushed, not_pushed }) + } + } + } + + fn split_single_predicates_around_delta( + filter: &filter::Filter, + delta: &Delta, + ) -> anyhow::Result { + let schema = &delta.output; + let disjunctions = filter.simplify_to_dnf()?.flatten_disjunctions(); + + // If we have a single disjunction, we are going to return an optional + // single variant. + if disjunctions.len() == 1 { + let disjunct = disjunctions.into_iter().next().unwrap(); + let pushed = disjunct.remove_datum(schema)?.map(Predicates::Single); + let not_pushed = + disjunct.only_datum(schema)?.map(Predicates::Single); + return Ok(SplitPredicates { pushed, not_pushed }); + } + + // We have multiple predicates, so let's separate out them all. + let mut pushed = Vec::with_capacity(disjunctions.len()); + let mut not_pushed = Vec::with_capacity(disjunctions.len()); + for disjunct in disjunctions { + pushed.push(disjunct.remove_datum(schema)?); + not_pushed.push(disjunct.only_datum(schema)?); + } + + // We need to "compress" either side if it's an array of all Nones. That + // means none of disjuncts could be reordered, and so we'll need to + // elide the plan node entirely. + let pushed = if pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(pushed))) + }; + let not_pushed = if not_pushed.iter().all(Option::is_none) { + None + } else { + Some(Predicates::from(OptionalDisjunctions(not_pushed))) + }; + Ok(SplitPredicates { pushed, not_pushed }) + } + + pub(crate) fn to_required(&self) -> anyhow::Result> { + match self { + Predicates::Single(single) => { + single.simplify_to_dnf().map(|f| f.flatten_disjunctions()) + } + Predicates::Disjunctions(disjuncts) => { + let mut out = Vec::with_capacity(disjuncts.len()); + for disjunct in disjuncts.iter() { + out.extend( + disjunct + .as_ref() + .expect("Must be Some(_) here") + .simplify_to_dnf()? + .flatten_disjunctions(), + ); + } + Ok(out) + } + } + } + + /// And this predicate with another. + /// + /// This will fail if there are different numbers of predicates in each + /// argument, since we won't know how to match up the predicates in that + /// case. + pub(crate) fn and(&self, other: &Self) -> anyhow::Result { + match (self, other) { + (Predicates::Single(left), Predicates::Single(right)) => { + Ok(Self::Single(left.and(right))) + } + ( + Predicates::Single(single), + Predicates::Disjunctions(disjunctions), + ) + | ( + Predicates::Disjunctions(disjunctions), + Predicates::Single(single), + ) => { + anyhow::ensure!( + disjunctions.len() == 1, + "Can only merge together a list of predicates with a \ + single predicate if the list has length 1" + ); + if let Some(right) = &disjunctions[0] { + Ok(Self::Single(single.and(right))) + } else { + Ok(Self::Single(single.clone())) + } + } + ( + Predicates::Disjunctions(left), + Predicates::Disjunctions(right), + ) => { + anyhow::ensure!( + left.len() == 1 && right.len() == 1, + "Can only merge together lists of predicates with length 1" + ); + match (&left[0], &right[0]) { + (None, None) => { + anyhow::bail!("Both predicates cannot be empty") + } + (None, Some(single)) | (Some(single), None) => { + Ok(Self::Single(single.clone())) + } + (Some(left), Some(right)) => { + Ok(Self::Single(left.and(right))) + } + } + } + } + } + + pub fn plan_tree_entries(&self) -> Vec { + let mut out = Vec::with_capacity(1); + match self { + Predicates::Single(single) => { + out.push(format!("key group 0: {}", single)); + } + Predicates::Disjunctions(disjuncts) => { + out.reserve(disjuncts.len().saturating_sub(1)); + for (i, disjunct) in disjuncts.iter().enumerate() { + let s = match disjunct { + Some(d) => d.to_string(), + None => String::from("-"), + }; + out.push(format!("key group {i}: {s}")); + } + } + } + out + } + + /// Return `true` if we can reorder predicates around the limit. + pub(crate) fn can_reorder_around(&self, limit: &Limit) -> bool { + match self { + Predicates::Single(single) => single.can_reorder_around(limit), + Predicates::Disjunctions(disjunctions) => { + disjunctions.iter().all(|maybe_filter| { + maybe_filter + .as_ref() + .map(|filter| filter.can_reorder_around(limit)) + .unwrap_or(true) + }) + } + } + } +} + +/// A list of optional, disjunctive elements from a filter predicate. +/// +/// See [`Predicates`] for details. +#[derive(Clone, Debug, PartialEq)] +pub struct OptionalDisjunctions(Vec>); + +impl std::ops::Deref for OptionalDisjunctions { + type Target = [Option]; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[cfg(test)] +mod tests { + use crate::oxql::ast::grammar::query_parser; + use crate::oxql::plan::node::Node; + use crate::oxql::plan::plan::test_utils::all_schema; + use crate::oxql::plan::predicates::Predicates; + use crate::oxql::plan::predicates::SplitPredicates; + use crate::oxql::plan::Plan; + + #[tokio::test] + async fn push_single_predicate_through_alignment() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | align mean_within(1s)", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let Node::Align(align) = plan.nodes().last().unwrap() else { + panic!( + "Plan should end with an align node, found {:?}", + plan.nodes().last().unwrap() + ); + }; + let filter = query_parser::filter("filter link_name == 'foo'").unwrap(); + let predicates = Predicates::Single(filter.clone()); + let SplitPredicates { pushed, not_pushed } = + predicates.split_around_align(&align).unwrap(); + let Predicates::Single(single) = pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Single, found {:?}", pushed); + }; + assert_eq!( + &filter, single, + "Should have pushed entire prediate through" + ); + assert!(not_pushed.is_none(), "Should have pushed entire predicate"); + } + + #[tokio::test] + async fn push_single_predicate_partway_through_alignment() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | align mean_within(1s)", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let Node::Align(align) = plan.nodes().last().unwrap() else { + panic!( + "Plan should end with an align node, found {:?}", + plan.nodes().last().unwrap() + ); + }; + let filter = + query_parser::filter("filter link_name == 'foo' && datum > 100.0") + .unwrap(); + let predicates = Predicates::Single(filter.clone()); + let SplitPredicates { pushed, not_pushed } = + predicates.split_around_align(&align).unwrap(); + let Predicates::Single(single) = pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Single, found {:?}", pushed); + }; + let expected = + query_parser::filter("filter link_name == 'foo'").unwrap(); + assert_eq!( + single, &expected, + "Incorrect filter pushed through alignment" + ); + let Predicates::Single(left) = not_pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Single, found {:?}", not_pushed,); + }; + let expected = query_parser::filter("filter datum > 100.0").unwrap(); + assert_eq!( + left, &expected, + "The remaining predicate on datum should have been left behind" + ); + } + + #[tokio::test] + async fn push_none_of_single_predicate_through_alignment() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | align mean_within(1s)", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let Node::Align(align) = plan.nodes().last().unwrap() else { + panic!( + "Plan should end with an align node, found {:?}", + plan.nodes().last().unwrap() + ); + }; + let filter = query_parser::filter("filter datum > 100.0").unwrap(); + let predicates = Predicates::Single(filter.clone()); + let SplitPredicates { pushed, not_pushed } = + predicates.split_around_align(&align).unwrap(); + assert!( + pushed.is_none(), + "Should have pushed nothing through alignment" + ); + let Predicates::Single(left) = not_pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Single, found {:?}", not_pushed,); + }; + let expected = query_parser::filter("filter datum > 100.0").unwrap(); + assert_eq!( + left, &expected, + "The entire predicate on datum should have been left behind" + ); + } + + #[tokio::test] + async fn push_multiple_predicates_partway_through_alignment() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | align mean_within(1s)", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let Node::Align(align) = plan.nodes().last().unwrap() else { + panic!( + "Plan should end with an align node, found {:?}", + plan.nodes().last().unwrap() + ); + }; + let filter = + query_parser::filter("filter (link_name == 'foo' || link_name == 'foo') && datum > 100.0") + .unwrap(); + let predicates = Predicates::Single(filter.clone()); + let SplitPredicates { pushed, not_pushed } = + predicates.split_around_align(&align).unwrap(); + let Predicates::Disjunctions(list) = pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Disjunctions, found {:?}", pushed); + }; + let expected = + query_parser::filter("filter link_name == 'foo'").unwrap(); + assert_eq!(list.len(), 2); + for each in list.iter() { + let each = each.as_ref().expect("Should have pushed through"); + assert_eq!( + each, &expected, + "Incorrect filter pushed through alignment" + ); + } + + let Predicates::Disjunctions(list) = not_pushed.as_ref().unwrap() + else { + panic!("Expected Predicates::Disjunctions, found {:?}", not_pushed,); + }; + let expected = query_parser::filter("filter datum > 100.0").unwrap(); + assert_eq!(list.len(), 2); + for each in list.iter() { + let each = each.as_ref().expect("Should have pushed through"); + assert_eq!( + each, &expected, + "Incorrect filter pushed through alignment" + ); + } + } + + #[tokio::test] + async fn push_predicates_partway_through_alignment_differently() { + let query = query_parser::query( + "get physical_data_link:bytes_sent | align mean_within(1s)", + ) + .unwrap(); + let plan = Plan::new(query, all_schema().await).unwrap(); + let Node::Align(align) = plan.nodes().last().unwrap() else { + panic!( + "Plan should end with an align node, found {:?}", + plan.nodes().last().unwrap() + ); + }; + let filter = + query_parser::filter("filter link_name == 'foo' || (link_name == 'foo' && datum > 100.0)") + .unwrap(); + let predicates = Predicates::Single(filter.clone()); + let SplitPredicates { pushed, not_pushed } = + predicates.split_around_align(&align).unwrap(); + let Predicates::Disjunctions(list) = pushed.as_ref().unwrap() else { + panic!("Expected Predicates::Disjunctions, found {:?}", pushed); + }; + let expected = + query_parser::filter("filter link_name == 'foo'").unwrap(); + assert_eq!(list.len(), 2); + for each in list.iter() { + let each = each.as_ref().expect("Should have pushed through"); + assert_eq!( + each, &expected, + "Incorrect filter pushed through alignment" + ); + } + + let Predicates::Disjunctions(list) = not_pushed.as_ref().unwrap() + else { + panic!("Expected Predicates::Disjunctions, found {:?}", not_pushed,); + }; + assert_eq!(list.len(), 2); + assert!(list[0].is_none(), "There is no disjunct to push through"); + let expected = query_parser::filter("filter datum > 100.0").unwrap(); + let each = list[1].as_ref().expect("Should have pushed through"); + assert_eq!( + each, &expected, + "Incorrect filter pushed through alignment" + ); + } +} diff --git a/oximeter/db/src/oxql/query/mod.rs b/oximeter/db/src/oxql/query/mod.rs index 46c9bbc92c..ec5b578ffd 100644 --- a/oximeter/db/src/oxql/query/mod.rs +++ b/oximeter/db/src/oxql/query/mod.rs @@ -6,6 +6,8 @@ // Copyright 2024 Oxide Computer Company +use std::collections::BTreeSet; + use super::ast::ident::Ident; use super::ast::logical_op::LogicalOp; use super::ast::table_ops::filter::CompoundFilter; @@ -34,15 +36,15 @@ pub struct Query { impl Query { /// Construct a query written in OxQL. pub fn new(query: impl AsRef) -> Result { - let raw = query.as_ref().trim(); + let query = query.as_ref().trim(); const MAX_LEN: usize = 4096; anyhow::ensure!( - raw.len() <= MAX_LEN, + query.len() <= MAX_LEN, "Queries must be <= {} characters", MAX_LEN, ); - let parsed = grammar::query_parser::query(raw) - .map_err(|e| fmt_parse_error(raw, e))?; + let parsed = grammar::query_parser::query(query) + .map_err(|e| fmt_parse_error(query, e))?; // Fetch the latest query end time referred to in the parsed query, or // use now if there isn't one. @@ -350,6 +352,16 @@ impl Query { pub(crate) fn split(&self) -> SplitQuery { self.parsed.split(self.end_time) } + + /// Return the set of all timeseries names referred to by the query. + pub(crate) fn all_timeseries_names(&self) -> BTreeSet<&TimeseriesName> { + self.parsed.all_timeseries_names() + } + + /// Return the parsed query AST node. + pub(crate) fn parsed_query(&self) -> &QueryNode { + &self.parsed + } } // Return a new filter containing only parts that refer to either: diff --git a/oximeter/db/src/oxql/schema.rs b/oximeter/db/src/oxql/schema.rs new file mode 100644 index 0000000000..269e311bfb --- /dev/null +++ b/oximeter/db/src/oxql/schema.rs @@ -0,0 +1,121 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Schema for tables in OxQL queries. + +// Copyright 2024 Oxide Computer Company + +use std::collections::BTreeMap; + +use oximeter::DatumType; +use oximeter::FieldType; +use oximeter::TimeseriesSchema; +use oxql_types::point::DataType; +use oxql_types::point::MetricType; +use serde::Deserialize; +use serde::Serialize; + +// TODO(ben) Move MetricType and DataType enums here, or probably this type +// there. +// TODO(ben) Consider biting the bullet and supporting every data type now. Why +// not? Lots more code, since we need to deal with type promotion rules and +// such. + +#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)] +pub struct TableSchema { + /// The name of the table. + pub name: String, + /// The mapping from field names to their types. + pub fields: BTreeMap, + /// The metric types of the contained data, one for each dimension. + /// + /// There can be more than one in the case of joins. + pub metric_types: Vec, + /// The types of the raw data, one for each dimension. + /// + /// There can be more than one in the case of joins. + pub data_types: Vec, +} + +impl TableSchema { + /// Construct a new table schema from a schema in the database. + /// + /// This builds an initial table schema for an OxQL query. Queries always + /// start by referring to one or more schema in the database itself. They + /// can change as they are processed by the query. + pub fn new(timeseries_schema: &TimeseriesSchema) -> Self { + let (metric_type, data_type) = match timeseries_schema.datum_type { + DatumType::Bool => (MetricType::Gauge, DataType::Boolean), + DatumType::I8 => (MetricType::Gauge, DataType::Integer), + DatumType::U8 => (MetricType::Gauge, DataType::Integer), + DatumType::I16 => (MetricType::Gauge, DataType::Integer), + DatumType::U16 => (MetricType::Gauge, DataType::Integer), + DatumType::I32 => (MetricType::Gauge, DataType::Integer), + DatumType::U32 => (MetricType::Gauge, DataType::Integer), + DatumType::I64 => (MetricType::Gauge, DataType::Integer), + DatumType::U64 => (MetricType::Gauge, DataType::Integer), + DatumType::F32 => (MetricType::Gauge, DataType::Double), + DatumType::F64 => (MetricType::Gauge, DataType::Double), + DatumType::String => (MetricType::Gauge, DataType::String), + DatumType::Bytes => (MetricType::Gauge, DataType::String), + DatumType::CumulativeI64 => { + (MetricType::Cumulative, DataType::Integer) + } + DatumType::CumulativeU64 => { + (MetricType::Cumulative, DataType::Integer) + } + DatumType::CumulativeF32 => { + (MetricType::Cumulative, DataType::Double) + } + DatumType::CumulativeF64 => { + (MetricType::Cumulative, DataType::Double) + } + DatumType::HistogramI8 => { + (MetricType::Cumulative, DataType::IntegerDistribution) + } + DatumType::HistogramU8 => { + (MetricType::Cumulative, DataType::IntegerDistribution) + } + DatumType::HistogramI16 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramU16 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramI32 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramU32 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramI64 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramU64 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramF32 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + DatumType::HistogramF64 => { + (MetricType::Cumulative, DataType::DoubleDistribution) + } + }; + Self { + name: timeseries_schema.timeseries_name.to_string(), + fields: timeseries_schema + .field_schema + .iter() + .map(|field| (field.name.clone(), field.field_type)) + .collect(), + metric_types: vec![metric_type], + data_types: vec![data_type], + } + } + + /// Return the type of the named field, if it is part of the schema. + pub(crate) fn field_type(&self, name: &str) -> Option<&FieldType> { + self.fields.get(name) + } +} diff --git a/oximeter/db/src/shells/oxql.rs b/oximeter/db/src/shells/oxql.rs index f46d08c0cf..62d7f1bedc 100644 --- a/oximeter/db/src/shells/oxql.rs +++ b/oximeter/db/src/shells/oxql.rs @@ -83,6 +83,47 @@ pub async fn shell( } else { print_oxql_operation_help(stmt); } + } else if let Some(stmt) = cmd.strip_prefix("plan") { + match client + .plan_oxql_query( + stmt.trim().trim_end_matches(';'), + ) + .await + { + Ok(plan) => { + println!( + "{}\n{}\n", + "OxQL query plan".underlined(), + plan.query, + ); + let (original, optimized_plan) = + plan.to_plan_tree(); + println!("{}", "Original".underlined()); + println!("{}", original); + println!("{}", "Optimized".underlined()); + println!("{}", optimized_plan); + let output = plan.output(); + println!( + "{} ({} table{})", + "Output".underlined(), + output.n_tables(), + if output.n_tables() == 1 { + "" + } else { + "s" + }, + ); + println!("{}", output); + println!( + "Planning time: {:?}", + plan.duration + ); + } + Err(e) => { + eprintln!("{}", "Error".underlined().red()); + eprintln!("{e}"); + } + } } else { match client .oxql_query(cmd.trim().trim_end_matches(';')) @@ -151,7 +192,7 @@ Get instances of a timeseries by name"#; println!("{HELP}"); } "filter" => { - const HELP: &str = r#"filter "); + const HELP: &str = r#"filter "; Filter timeseries based on their attributes. can be a logical combination of filtering @@ -190,6 +231,18 @@ account the timestamps, and does not align the outputs directly."#; println!("{HELP}"); } + "align" => { + const HELP: &str = r#"align + +Align the timepoints in each timeseries, so that they +occur on exactly regular intervals. Nearby timepoints +may be combined with the specified alignment method. +Alignment is required to combine multiple timeseries +together, either by joining them across tables, or grouping +them within tables. +"#; + println!("{HELP}"); + } _ => eprintln!("unrecognized OxQL operation: '{op}'"), } } @@ -202,6 +255,7 @@ fn print_basic_commands() { println!(" \\l - List timeseries"); println!(" \\d - Describe a timeseries"); println!(" \\ql [] - Get OxQL help about an operation"); + println!(" plan - Emit the query plan for an OxQL query"); println!(); println!("Or try entering an OxQL `get` query"); } @@ -219,12 +273,13 @@ operator, "|". All queries start with a `get` operation, which selects a timeseries from the database, by name. For example: -`get physical_data_link:bytes_received` +`get sled_data_link:bytes_received` The supported timeseries operations are: - get: Select a timeseries by name - filter: Filter timeseries by field or sample values +- align: Temporally align timeseries, combining nearby points. - group_by: Group timeseries by fields, applying a reducer. - join: Join two or more timeseries together diff --git a/oximeter/db/tests/integration_test.rs b/oximeter/db/tests/integration_test.rs index eca0adf7d5..eedc7b0862 100644 --- a/oximeter/db/tests/integration_test.rs +++ b/oximeter/db/tests/integration_test.rs @@ -211,7 +211,9 @@ async fn test_cluster() -> anyhow::Result<()> { // Get all the samples from the replica where the data was inserted let start = tokio::time::Instant::now(); let oxql_res1 = client1 - .oxql_query("get virtual_machine:cpu_busy") + .oxql_query( + "get virtual_machine:cpu_busy | filter timestamp > @2000-01-01", + ) .await .expect("failed to get all samples"); info!(log, "query samples from client1 time = {:?}", start.elapsed()); @@ -425,7 +427,7 @@ async fn wait_for_num_points( poll::wait_for_condition( || async { let oxql_res = client - .oxql_query("get virtual_machine:cpu_busy") + .oxql_query("get virtual_machine:cpu_busy | filter timestamp > @2000-01-01") .await .map_err(|_| { poll::CondCheckError::::NotYet diff --git a/oximeter/db/tests/timeseries-schema.json b/oximeter/db/tests/timeseries-schema.json new file mode 100644 index 0000000000..43be9dd80f --- /dev/null +++ b/oximeter/db/tests/timeseries-schema.json @@ -0,0 +1,83 @@ +{"timeseries_name":"collection_target:cpus_provisioned","fields.name":["id"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"I64","created":"2023-10-25 23:05:04.909050887"} +{"timeseries_name":"collection_target:ram_provisioned","fields.name":["id"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"I64","created":"2023-10-25 23:05:04.915652918"} +{"timeseries_name":"collection_target:virtual_disk_space_provisioned","fields.name":["id"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"I64","created":"2023-10-25 23:13:04.927950665"} +{"timeseries_name":"crucible_upstairs:activated","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.899663499"} +{"timeseries_name":"crucible_upstairs:extent_no_op","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.930943640"} +{"timeseries_name":"crucible_upstairs:extent_reopen","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.934827868"} +{"timeseries_name":"crucible_upstairs:extent_repair","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.927207024"} +{"timeseries_name":"crucible_upstairs:flush","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.903985541"} +{"timeseries_name":"crucible_upstairs:flush_close","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.923454748"} +{"timeseries_name":"crucible_upstairs:read","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.915831652"} +{"timeseries_name":"crucible_upstairs:read_bytes","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.919609014"} +{"timeseries_name":"crucible_upstairs:write","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.908085310"} +{"timeseries_name":"crucible_upstairs:write_bytes","fields.name":["upstairs_uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.911931515"} +{"timeseries_name":"data_link:abort","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.798993111"} +{"timeseries_name":"data_link:b_e_r_check_done","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.783030992"} +{"timeseries_name":"data_link:b_e_r_check_start","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.780161540"} +{"timeseries_name":"data_link:bad_sync_headers","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.740936507"} +{"timeseries_name":"data_link:disabled","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.801743186"} +{"timeseries_name":"data_link:enabled","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"Bool","created":"2023-10-25 23:03:45.612915119"} +{"timeseries_name":"data_link:end","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.785599421"} +{"timeseries_name":"data_link:errored_blocks","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.743619233"} +{"timeseries_name":"data_link:fec_align","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"Bool","created":"2023-10-25 23:03:45.714770919"} +{"timeseries_name":"data_link:fec_corr_cnt","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.717128696"} +{"timeseries_name":"data_link:fec_hi_ser","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"Bool","created":"2023-10-25 23:03:45.712431057"} +{"timeseries_name":"data_link:fec_ser_lane0","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.721850240"} +{"timeseries_name":"data_link:fec_ser_lane1","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.724264062"} +{"timeseries_name":"data_link:fec_ser_lane2","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.726673517"} +{"timeseries_name":"data_link:fec_ser_lane3","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.729043757"} +{"timeseries_name":"data_link:fec_ser_lane4","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.731511643"} +{"timeseries_name":"data_link:fec_ser_lane5","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.733885410"} +{"timeseries_name":"data_link:fec_ser_lane6","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.736336104"} +{"timeseries_name":"data_link:fec_ser_lane7","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.738594962"} +{"timeseries_name":"data_link:fec_uncorr_cnt","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.719451726"} +{"timeseries_name":"data_link:idle","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.761751371"} +{"timeseries_name":"data_link:link_down","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.774724311"} +{"timeseries_name":"data_link:link_up","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"Bool","created":"2023-10-25 23:03:45.660176566"} +{"timeseries_name":"data_link:monitor_p_r_b_s_errors","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.793275938"} +{"timeseries_name":"data_link:pci_hi_ber","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.751291996"} +{"timeseries_name":"data_link:pcs_block_lock_loss","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.748686727"} +{"timeseries_name":"data_link:pcs_invalid_errors","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.759320906"} +{"timeseries_name":"data_link:pcs_sync_loss","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.746113450"} +{"timeseries_name":"data_link:pcs_unknown_errors","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.756839686"} +{"timeseries_name":"data_link:pcs_valid_errors","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.754036510"} +{"timeseries_name":"data_link:remote_fault","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.772173897"} +{"timeseries_name":"data_link:rx_buf_full","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.700321323"} +{"timeseries_name":"data_link:rx_bytes","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.695469190"} +{"timeseries_name":"data_link:rx_crc_errs","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.697937216"} +{"timeseries_name":"data_link:rx_errs","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.702700701"} +{"timeseries_name":"data_link:rx_pkts","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.689905229"} +{"timeseries_name":"data_link:tofino3_states","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.796116455"} +{"timeseries_name":"data_link:tx_bytes","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.707581108"} +{"timeseries_name":"data_link:tx_errs","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.709933314"} +{"timeseries_name":"data_link:tx_pkts","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.705180840"} +{"timeseries_name":"data_link:wait_auto_neg_done","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.788057428"} +{"timeseries_name":"data_link:wait_auto_neg_link_training_done","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.790645334"} +{"timeseries_name":"data_link:wait_d_f_e_done","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.769677587"} +{"timeseries_name":"data_link:wait_p_l_l_ready","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.764520692"} +{"timeseries_name":"data_link:wait_signal_o_k","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.767257141"} +{"timeseries_name":"data_link:wait_test_done","fields.name":["link_id","port_id","rack_id","sidecar_id","sled_id"],"fields.type":["I64","String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.777584285"} +{"timeseries_name":"database_transaction:retry_data","fields.name":["attempt","name"],"fields.type":["U32","String"],"fields.source":["Metric","Target"],"datum_type":"F64","created":"2023-12-09 05:05:12.777325080"} +{"timeseries_name":"http_service:request_latency_histogram","fields.name":["id","method","name","route","status_code"],"fields.type":["Uuid","String","String","String","I64"],"fields.source":["Target","Metric","Target","Metric","Metric"],"datum_type":"HistogramF64","created":"2023-10-25 23:04:04.891225842"} +{"timeseries_name":"instance_uuid:pv_panic_guest_handled","fields.name":["uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2024-02-01 06:42:49.623791045"} +{"timeseries_name":"instance_uuid:pv_panic_host_handled","fields.name":["uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2024-02-01 06:42:49.632352775"} +{"timeseries_name":"instance_uuid:reset","fields.name":["uuid"],"fields.type":["Uuid"],"fields.source":["Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:04:34.894527394"} +{"timeseries_name":"kstat_sampler:samples_dropped","fields.name":["hostname","target_id","target_name"],"fields.type":["String","U64","String"],"fields.source":["Target","Metric","Metric"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.642824724"} +{"timeseries_name":"oximeter_collector:collections","fields.name":["base_route","collector_id","collector_ip","collector_port","producer_id","producer_ip","producer_port"],"fields.type":["String","Uuid","IpAddr","U16","Uuid","IpAddr","U16"],"fields.source":["Metric","Target","Target","Target","Metric","Metric","Metric"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:57.255983666"} +{"timeseries_name":"oximeter_collector:failed_collections","fields.name":["base_route","collector_id","collector_ip","collector_port","producer_id","producer_ip","producer_port","reason"],"fields.type":["String","Uuid","IpAddr","U16","Uuid","IpAddr","U16","String"],"fields.source":["Metric","Target","Target","Target","Metric","Metric","Metric","Metric"],"datum_type":"CumulativeU64","created":"2023-11-17 02:56:39.525743256"} +{"timeseries_name":"physical_data_link:bytes_received","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.484386388"} +{"timeseries_name":"physical_data_link:bytes_sent","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.138101022"} +{"timeseries_name":"physical_data_link:errors_received","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.449876345"} +{"timeseries_name":"physical_data_link:errors_sent","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.477064098"} +{"timeseries_name":"physical_data_link:packets_received","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.488575473"} +{"timeseries_name":"physical_data_link:packets_sent","fields.name":["hostname","link_name","rack_id","serial","sled_id"],"fields.type":["String","String","Uuid","String","Uuid"],"fields.source":["Target","Target","Target","Target","Target"],"datum_type":"CumulativeU64","created":"2023-11-17 02:53:34.439549378"} +{"timeseries_name":"sidecar:sample_time","fields.name":["board_rev","rack_id","sidecar_id","sled_id"],"fields.type":["String","Uuid","Uuid","Uuid"],"fields.source":["Target","Target","Target","Target"],"datum_type":"I64","created":"2023-10-25 23:03:45.947714957"} +{"timeseries_name":"switch_table:capacity","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"I64","created":"2023-10-25 23:03:45.916472267"} +{"timeseries_name":"switch_table:collisions","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.930654934"} +{"timeseries_name":"switch_table:delete_misses","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.935986861"} +{"timeseries_name":"switch_table:deletes","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.925327996"} +{"timeseries_name":"switch_table:exhaustion","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.938635382"} +{"timeseries_name":"switch_table:inserts","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.922203276"} +{"timeseries_name":"switch_table:occupancy","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"I64","created":"2023-10-25 23:03:45.919412144"} +{"timeseries_name":"switch_table:update_misses","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.933217752"} +{"timeseries_name":"switch_table:updates","fields.name":["rack_id","sidecar_id","sled_id","table"],"fields.type":["Uuid","Uuid","Uuid","String"],"fields.source":["Target","Target","Target","Target"],"datum_type":"CumulativeI64","created":"2023-10-25 23:03:45.927951288"} diff --git a/oximeter/oxql-types/src/point.rs b/oximeter/oxql-types/src/point.rs index 6e3c7143dc..98338cfd76 100644 --- a/oximeter/oxql-types/src/point.rs +++ b/oximeter/oxql-types/src/point.rs @@ -43,9 +43,17 @@ pub enum DataType { impl DataType { /// True if this is a numeric scalar type. - pub fn is_numeric(&self) -> bool { + pub const fn is_numeric(&self) -> bool { matches!(self, DataType::Integer | DataType::Double) } + + /// Return true if this is a distribution data type. + pub const fn is_distribution(&self) -> bool { + matches!( + self, + DataType::IntegerDistribution | DataType::DoubleDistribution + ) + } } impl TryFrom for DataType { @@ -107,6 +115,12 @@ pub enum MetricType { /// The value represents an accumulation between two points in time. Cumulative, } +impl MetricType { + /// Return true if this is cumulative. + pub const fn is_cumulative(&self) -> bool { + matches!(self, MetricType::Cumulative) + } +} impl fmt::Display for MetricType { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {