chore: Merge branch 'ntran/compact_os_blocks' of https://github.com/influxdata/influxdb_iox into ntran/compact_os_blocks
commit
4e7e9f6fc9
|
@ -813,7 +813,7 @@ dependencies = [
|
|||
[[package]]
|
||||
name = "datafusion"
|
||||
version = "6.0.0"
|
||||
source = "git+https://github.com/apache/arrow-datafusion.git?rev=79f129d048667a4552e44ef740e1b1cf9de306a1#79f129d048667a4552e44ef740e1b1cf9de306a1"
|
||||
source = "git+https://github.com/apache/arrow-datafusion.git?rev=66f5d1906992dcf4f5b7ab787a39a4105c2f5b42#66f5d1906992dcf4f5b7ab787a39a4105c2f5b42"
|
||||
dependencies = [
|
||||
"ahash",
|
||||
"arrow",
|
||||
|
@ -3910,6 +3910,8 @@ dependencies = [
|
|||
"rand",
|
||||
"rand_distr",
|
||||
"read_buffer",
|
||||
"regex",
|
||||
"router",
|
||||
"schema",
|
||||
"serde",
|
||||
"serde_json",
|
||||
|
|
|
@ -34,7 +34,10 @@ our ability to support them effectively.
|
|||
|
||||
## Project Status
|
||||
|
||||
This project is very early and in active development. It isn't yet ready for testing, which is why we're not producing builds or documentation yet. If you're interested in following along with the project, drop into our community Slack channel #influxdb_iox. You can find [links to join here](https://community.influxdata.com/).
|
||||
This project is very early and in active development. It isn't yet ready for testing, which is why we're not producing builds or documentation yet.
|
||||
|
||||
If you would like contact the InfluxDB IOx developers,
|
||||
join the [InfluxData Community Slack](https://influxdata.com/slack) and look for the #influxdb_iox channel.
|
||||
|
||||
We're also hosting monthly tech talks and community office hours on the project on the 2nd Wednesday of the month at 8:30 AM Pacific Time.
|
||||
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
use crate::timestamp::TimestampRange;
|
||||
use std::{fmt::Write, num::FpCategory};
|
||||
|
||||
/// Represents a parsed delete predicate for evaluation by the InfluxDB IOx
|
||||
/// query engine.
|
||||
|
@ -16,6 +17,20 @@ pub struct DeletePredicate {
|
|||
pub exprs: Vec<DeleteExpr>,
|
||||
}
|
||||
|
||||
impl DeletePredicate {
|
||||
/// Format expr to SQL string.
|
||||
pub fn expr_sql_string(&self) -> String {
|
||||
let mut out = String::new();
|
||||
for expr in &self.exprs {
|
||||
if !out.is_empty() {
|
||||
write!(&mut out, " AND ").expect("writing to a string shouldn't fail");
|
||||
}
|
||||
write!(&mut out, "{}", expr).expect("writing to a string shouldn't fail");
|
||||
}
|
||||
out
|
||||
}
|
||||
}
|
||||
|
||||
/// Single expression to be used as parts of a predicate.
|
||||
///
|
||||
/// Only very simple expression of the type `<column> <op> <scalar>` are supported.
|
||||
|
@ -55,7 +70,15 @@ impl DeleteExpr {
|
|||
|
||||
impl std::fmt::Display for DeleteExpr {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{}{}{}", self.column(), self.op(), self.scalar())
|
||||
write!(
|
||||
f,
|
||||
r#""{}"{}{}"#,
|
||||
self.column()
|
||||
.replace(r#"\"#, r#"\\"#)
|
||||
.replace(r#"""#, r#"\""#),
|
||||
self.op(),
|
||||
self.scalar(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,8 +116,217 @@ impl std::fmt::Display for Scalar {
|
|||
match self {
|
||||
Scalar::Bool(value) => value.fmt(f),
|
||||
Scalar::I64(value) => value.fmt(f),
|
||||
Scalar::F64(value) => value.fmt(f),
|
||||
Scalar::String(value) => write!(f, "'{}'", value),
|
||||
Scalar::F64(value) => match value.classify() {
|
||||
FpCategory::Nan => write!(f, "'NaN'"),
|
||||
FpCategory::Infinite if *value.as_ref() < 0.0 => write!(f, "'-Infinity'"),
|
||||
FpCategory::Infinite => write!(f, "'Infinity'"),
|
||||
_ => write!(f, "{:?}", value.as_ref()),
|
||||
},
|
||||
Scalar::String(value) => {
|
||||
write!(
|
||||
f,
|
||||
"'{}'",
|
||||
value.replace(r#"\"#, r#"\\"#).replace(r#"'"#, r#"\'"#),
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use ordered_float::OrderedFloat;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_no_expressions() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![],
|
||||
};
|
||||
assert_eq!(&pred.expr_sql_string(), "");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_operators() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(1),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col2"),
|
||||
op: Op::Ne,
|
||||
scalar: Scalar::I64(2),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(&pred.expr_sql_string(), r#""col1"=1 AND "col2"!=2"#);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_column_escape() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col 1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(1),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from(r#"col\2"#),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(2),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from(r#"col"3"#),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(3),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(
|
||||
&pred.expr_sql_string(),
|
||||
r#""col 1"=1 AND "col\\2"=2 AND "col\"3"=3"#
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_bool() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::Bool(false),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col2"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::Bool(true),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(&pred.expr_sql_string(), r#""col1"=false AND "col2"=true"#);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_i64() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(0),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col2"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(-1),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col3"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(1),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col4"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(i64::MIN),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col5"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::I64(i64::MAX),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(
|
||||
&pred.expr_sql_string(),
|
||||
r#""col1"=0 AND "col2"=-1 AND "col3"=1 AND "col4"=-9223372036854775808 AND "col5"=9223372036854775807"#
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_f64() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(0.0)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col2"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(-0.0)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col3"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(1.0)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col4"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(f64::INFINITY)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col5"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(f64::NEG_INFINITY)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col6"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::F64(OrderedFloat::from(f64::NAN)),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(
|
||||
&pred.expr_sql_string(),
|
||||
r#""col1"=0.0 AND "col2"=-0.0 AND "col3"=1.0 AND "col4"='Infinity' AND "col5"='-Infinity' AND "col6"='NaN'"#
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_expr_to_sql_string() {
|
||||
let pred = DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![
|
||||
DeleteExpr {
|
||||
column: String::from("col1"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::String(String::from("")),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col2"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::String(String::from("foo")),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col3"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::String(String::from(r#"fo\o"#)),
|
||||
},
|
||||
DeleteExpr {
|
||||
column: String::from("col4"),
|
||||
op: Op::Eq,
|
||||
scalar: Scalar::String(String::from(r#"fo'o"#)),
|
||||
},
|
||||
],
|
||||
};
|
||||
assert_eq!(
|
||||
&pred.expr_sql_string(),
|
||||
r#""col1"='' AND "col2"='foo' AND "col3"='fo\\o' AND "col4"='fo\'o'"#
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -9,5 +9,5 @@ description = "Re-exports datafusion at a specific version"
|
|||
|
||||
# Rename to workaround doctest bug
|
||||
# Turn off optional datafusion features (e.g. don't get support for crypo functions or avro)
|
||||
upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="79f129d048667a4552e44ef740e1b1cf9de306a1", default-features = false, package = "datafusion" }
|
||||
upstream = { git = "https://github.com/apache/arrow-datafusion.git", rev="66f5d1906992dcf4f5b7ab787a39a4105c2f5b42", default-features = false, package = "datafusion" }
|
||||
workspace-hack = { path = "../workspace-hack"}
|
||||
|
|
|
@ -272,12 +272,12 @@ pub mod test_util {
|
|||
pub fn assert_op_eq(a: &DmlOperation, b: &DmlOperation) {
|
||||
match (a, b) {
|
||||
(DmlOperation::Write(a), DmlOperation::Write(b)) => assert_writes_eq(a, b),
|
||||
(DmlOperation::Delete(_), DmlOperation::Delete(_)) => unimplemented!(),
|
||||
(DmlOperation::Delete(a), DmlOperation::Delete(b)) => assert_deletes_eq(a, b),
|
||||
(a, b) => panic!("a != b, {:?} vs {:?}", a, b),
|
||||
}
|
||||
}
|
||||
|
||||
/// Asserts `a` contains a WriteOperation equal to `b`
|
||||
/// Asserts `a` contains a [`DmlWrite`] equal to `b`
|
||||
pub fn assert_write_op_eq(a: &DmlOperation, b: &DmlWrite) {
|
||||
match a {
|
||||
DmlOperation::Write(a) => assert_writes_eq(a, b),
|
||||
|
@ -287,8 +287,7 @@ pub mod test_util {
|
|||
|
||||
/// Asserts two writes are equal
|
||||
pub fn assert_writes_eq(a: &DmlWrite, b: &DmlWrite) {
|
||||
assert_eq!(a.meta().sequence(), b.meta().sequence());
|
||||
assert_eq!(a.meta().producer_ts(), b.meta().producer_ts());
|
||||
assert_meta_eq(a.meta(), b.meta());
|
||||
|
||||
assert_eq!(a.table_count(), b.table_count());
|
||||
|
||||
|
@ -303,4 +302,40 @@ pub mod test_util {
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
/// Asserts `a` contains a [`DmlDelete`] equal to `b`
|
||||
pub fn assert_delete_op_eq(a: &DmlOperation, b: &DmlDelete) {
|
||||
match a {
|
||||
DmlOperation::Delete(a) => assert_deletes_eq(a, b),
|
||||
_ => panic!("unexpected operation: {:?}", a),
|
||||
}
|
||||
}
|
||||
|
||||
/// Asserts two deletes are equal
|
||||
pub fn assert_deletes_eq(a: &DmlDelete, b: &DmlDelete) {
|
||||
assert_meta_eq(a.meta(), b.meta());
|
||||
|
||||
assert_eq!(a.table_name(), b.table_name());
|
||||
|
||||
assert_eq!(a.predicate(), b.predicate());
|
||||
}
|
||||
|
||||
/// Assert that two metadata objects are equal
|
||||
pub fn assert_meta_eq(a: &DmlMeta, b: &DmlMeta) {
|
||||
assert_eq!(a.sequence(), b.sequence());
|
||||
|
||||
assert_eq!(a.producer_ts(), b.producer_ts());
|
||||
|
||||
match (a.span_context(), b.span_context()) {
|
||||
(None, None) => (),
|
||||
(Some(a), Some(b)) => {
|
||||
assert_eq!(a, b);
|
||||
}
|
||||
(None, Some(_)) => panic!("rhs has span context but lhs has not"),
|
||||
(Some(_), None) => panic!("lhs has span context but rhs has not"),
|
||||
}
|
||||
|
||||
// TODO: https://github.com/influxdata/influxdb_iox/issues/3186
|
||||
// assert_eq!(a.bytes_read(), b.bytes_read());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@ fn main() -> Result<()> {
|
|||
///
|
||||
/// Creates:
|
||||
///
|
||||
/// - `com.github.influxdata.idpe.storage.read.rs`
|
||||
/// - `influxdata.iox.delete.v1.rs`
|
||||
/// - `influxdata.iox.deployment.v1.rs`
|
||||
/// - `influxdata.iox.management.v1.rs`
|
||||
|
@ -31,7 +30,6 @@ fn main() -> Result<()> {
|
|||
fn generate_grpc_types(root: &Path) -> Result<()> {
|
||||
let delete_path = root.join("influxdata/iox/delete/v1");
|
||||
let deployment_path = root.join("influxdata/iox/deployment/v1");
|
||||
let idpe_path = root.join("com/github/influxdata/idpe/storage/read");
|
||||
let management_path = root.join("influxdata/iox/management/v1");
|
||||
let predicate_path = root.join("influxdata/iox/predicate/v1");
|
||||
let preserved_catalog_path = root.join("influxdata/iox/preserved_catalog/v1");
|
||||
|
@ -43,7 +41,6 @@ fn generate_grpc_types(root: &Path) -> Result<()> {
|
|||
let proto_files = vec![
|
||||
delete_path.join("service.proto"),
|
||||
deployment_path.join("service.proto"),
|
||||
idpe_path.join("source.proto"),
|
||||
management_path.join("chunk.proto"),
|
||||
management_path.join("database_rules.proto"),
|
||||
management_path.join("jobs.proto"),
|
||||
|
@ -67,8 +64,8 @@ fn generate_grpc_types(root: &Path) -> Result<()> {
|
|||
router_path.join("shard.proto"),
|
||||
storage_path.join("predicate.proto"),
|
||||
storage_path.join("service.proto"),
|
||||
storage_path.join("source.proto"),
|
||||
storage_path.join("storage_common.proto"),
|
||||
storage_path.join("storage_common_idpe.proto"),
|
||||
storage_path.join("test.proto"),
|
||||
write_buffer_path.join("write_buffer.proto"),
|
||||
];
|
||||
|
@ -115,7 +112,12 @@ fn generate_grpc_types(root: &Path) -> Result<()> {
|
|||
|
||||
pbjson_build::Builder::new()
|
||||
.register_descriptors(&descriptor_set)?
|
||||
.build(&[".influxdata", ".google.longrunning", ".google.rpc"])?;
|
||||
.build(&[
|
||||
".influxdata.iox",
|
||||
".influxdata.pbdata",
|
||||
".google.longrunning",
|
||||
".google.rpc",
|
||||
])?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
|
|
@ -126,11 +126,6 @@ service ManagementService {
|
|||
|
||||
// Drop partition from memory and (if persisted) from object store.
|
||||
rpc DropPartition(DropPartitionRequest) returns (DropPartitionResponse);
|
||||
|
||||
// Delete data for a table on a specified predicate
|
||||
rpc Delete(DeleteRequest) returns (DeleteResponse) {
|
||||
option deprecated = true;
|
||||
};
|
||||
}
|
||||
|
||||
message GetServerIdRequest {}
|
||||
|
@ -522,25 +517,3 @@ message DropPartitionRequest {
|
|||
|
||||
message DropPartitionResponse {
|
||||
}
|
||||
|
||||
// Request to delete data from a table on a specified predicate
|
||||
message DeleteRequest {
|
||||
// name of the database
|
||||
string db_name = 1;
|
||||
|
||||
// table name
|
||||
string table_name = 2;
|
||||
|
||||
// start time range
|
||||
string start_time = 3;
|
||||
|
||||
// stop time range
|
||||
string stop_time = 4;
|
||||
|
||||
// predicate
|
||||
// conjunctive expressions of binary 'column_name = literal' or 'column_ame != literal'
|
||||
string predicate = 5;
|
||||
}
|
||||
|
||||
message DeleteResponse {
|
||||
}
|
||||
|
|
|
@ -9,7 +9,6 @@ package influxdata.platform.storage;
|
|||
|
||||
import "google/protobuf/empty.proto";
|
||||
import "influxdata/platform/storage/storage_common.proto";
|
||||
import "influxdata/platform/storage/storage_common_idpe.proto";
|
||||
|
||||
service Storage {
|
||||
// ReadFilter performs a filter operation at storage
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
syntax = "proto3";
|
||||
package com.github.influxdata.idpe.storage.read;
|
||||
package influxdata.platform.storage.read;
|
||||
|
||||
message ReadSource {
|
||||
// OrgID specifies the organization identifier for this request.
|
|
@ -9,59 +9,91 @@ package influxdata.platform.storage;
|
|||
|
||||
import "google/protobuf/any.proto";
|
||||
import "influxdata/platform/storage/predicate.proto";
|
||||
import "influxdata/platform/storage/source.proto";
|
||||
|
||||
|
||||
message OffsetsResponse {
|
||||
message PartitionOffsetResponse {
|
||||
int64 id = 1;
|
||||
int64 offset = 2;
|
||||
}
|
||||
repeated PartitionOffsetResponse partitions = 1;
|
||||
}
|
||||
|
||||
enum TagKeyMetaNames {
|
||||
// option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
||||
// TagKeyMetaNamesText means the tag keys for measurement and field will
|
||||
// be returned as _measurement and _field respectively.
|
||||
TagKeyMetaNamesText = 0;
|
||||
|
||||
// TagKeyMetaNames means the tag keys for measurement and field will
|
||||
// be returned as \x00 and \xff respectively.
|
||||
TagKeyMetaNamesBinary = 1;
|
||||
}
|
||||
|
||||
message ReadFilterRequest {
|
||||
google.protobuf.Any read_source = 1;
|
||||
TimestampRange range = 2;
|
||||
google.protobuf.Any ReadSource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
// KeySort determines the ordering of series keys from the server.
|
||||
KeySort key_sort = 4;
|
||||
|
||||
// TagKeyMetaNames determines the key format used for the measurement and field
|
||||
// tags.
|
||||
TagKeyMetaNames tag_key_meta_names = 5;
|
||||
|
||||
enum KeySort {
|
||||
// option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
||||
// KeySortUnspecified means the key order is unspecified.
|
||||
KeySortUnspecified = 0;
|
||||
|
||||
// KeySortAscending means the key order should be lexicographically ascending.
|
||||
//
|
||||
// NOTE: In order to preserve sort order, canonical tag keys are not
|
||||
// transformed from 0x00 → _measurement and 0xff → _field.
|
||||
KeySortAscending = 1;
|
||||
}
|
||||
}
|
||||
|
||||
message ReadGroupRequest {
|
||||
google.protobuf.Any read_source = 1;
|
||||
TimestampRange range = 2;
|
||||
google.protobuf.Any ReadSource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
|
||||
enum Group {
|
||||
// option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
||||
// GroupNone returns all series as a single group.
|
||||
// The single GroupFrame.TagKeys will be the union of all tag keys.
|
||||
GROUP_NONE = 0;
|
||||
GroupNone = 0;
|
||||
|
||||
// GroupBy returns a group for each unique value of the specified GroupKeys.
|
||||
GROUP_BY = 2;
|
||||
GroupBy = 2;
|
||||
}
|
||||
|
||||
// GroupKeys specifies a list of tag keys used to order the data.
|
||||
// It is dependent on the Group property to determine its behavior.
|
||||
repeated string group_keys = 4;
|
||||
repeated string GroupKeys = 4;
|
||||
|
||||
Group group = 5;
|
||||
Aggregate aggregate = 6;
|
||||
|
||||
// TODO(jlapacik): This field is only used in unit tests.
|
||||
// Specifically the two tests in group_resultset_test.go.
|
||||
// This field should be removed and the tests that depend
|
||||
// on it refactored.
|
||||
enum HintFlags {
|
||||
HINT_NONE = 0x00;
|
||||
HINT_NO_POINTS = 0x01;
|
||||
HINT_NO_SERIES = 0x02;
|
||||
// HintSchemaAllTime performs schema queries without using time ranges
|
||||
HINT_SCHEMA_ALL_TIME = 0x04;
|
||||
}
|
||||
fixed32 hints = 7;
|
||||
// Deprecated field only used in TSM storage-related tests.
|
||||
reserved "Hints";
|
||||
}
|
||||
|
||||
message Aggregate {
|
||||
enum AggregateType {
|
||||
NONE = 0;
|
||||
SUM = 1;
|
||||
COUNT = 2;
|
||||
MIN = 3;
|
||||
MAX = 4;
|
||||
FIRST = 5;
|
||||
LAST = 6;
|
||||
MEAN = 7;
|
||||
AggregateTypeNone = 0;
|
||||
AggregateTypeSum = 1;
|
||||
AggregateTypeCount = 2;
|
||||
AggregateTypeMin = 3;
|
||||
AggregateTypeMax = 4;
|
||||
AggregateTypeFirst = 5;
|
||||
AggregateTypeLast = 6;
|
||||
AggregateTypeMean = 7;
|
||||
}
|
||||
|
||||
AggregateType type = 1;
|
||||
|
@ -77,39 +109,39 @@ message Tag {
|
|||
// Response message for ReadFilter and ReadGroup
|
||||
message ReadResponse {
|
||||
enum FrameType {
|
||||
SERIES = 0;
|
||||
POINTS = 1;
|
||||
FrameTypeSeries = 0;
|
||||
FrameTypePoints = 1;
|
||||
}
|
||||
|
||||
enum DataType {
|
||||
FLOAT = 0;
|
||||
INTEGER = 1;
|
||||
UNSIGNED = 2;
|
||||
BOOLEAN = 3;
|
||||
STRING = 4;
|
||||
DataTypeFloat = 0;
|
||||
DataTypeInteger = 1;
|
||||
DataTypeUnsigned = 2;
|
||||
DataTypeBoolean = 3;
|
||||
DataTypeString = 4;
|
||||
}
|
||||
|
||||
message Frame {
|
||||
oneof data {
|
||||
GroupFrame group = 7;
|
||||
SeriesFrame series = 1;
|
||||
FloatPointsFrame float_points = 2;
|
||||
IntegerPointsFrame integer_points = 3;
|
||||
UnsignedPointsFrame unsigned_points = 4;
|
||||
BooleanPointsFrame boolean_points = 5;
|
||||
StringPointsFrame string_points = 6;
|
||||
FloatPointsFrame FloatPoints = 2;
|
||||
IntegerPointsFrame IntegerPoints = 3;
|
||||
UnsignedPointsFrame UnsignedPoints = 4;
|
||||
BooleanPointsFrame BooleanPoints = 5;
|
||||
StringPointsFrame StringPoints = 6;
|
||||
}
|
||||
}
|
||||
|
||||
message GroupFrame {
|
||||
// TagKeys
|
||||
repeated bytes tag_keys = 1;
|
||||
repeated bytes TagKeys = 1;
|
||||
// PartitionKeyVals is the values of the partition key for this group, order matching ReadGroupRequest.GroupKeys
|
||||
repeated bytes partition_key_vals = 2;
|
||||
repeated bytes PartitionKeyVals = 2;
|
||||
}
|
||||
|
||||
message SeriesFrame {
|
||||
repeated Tag tags = 1;
|
||||
repeated Tag tags = 1; // [(gogoproto.nullable) = false];
|
||||
DataType data_type = 2;
|
||||
}
|
||||
|
||||
|
@ -138,7 +170,7 @@ message ReadResponse {
|
|||
repeated string values = 2;
|
||||
}
|
||||
|
||||
repeated Frame frames = 1;
|
||||
repeated Frame frames = 1; // [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message Capability {
|
||||
|
@ -165,32 +197,47 @@ message TimestampRange {
|
|||
|
||||
// TagKeysRequest is the request message for Storage.TagKeys.
|
||||
message TagKeysRequest {
|
||||
google.protobuf.Any tags_source = 1;
|
||||
TimestampRange range = 2;
|
||||
google.protobuf.Any TagsSource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
}
|
||||
|
||||
// TagValuesRequest is the request message for Storage.TagValues.
|
||||
message TagValuesRequest {
|
||||
google.protobuf.Any tags_source = 1 ;
|
||||
google.protobuf.Any TagsSource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
|
||||
// string tag_key = 4;
|
||||
// AAL changed from string --> bytes to handle \xff literals in Rust which are not valid UTF-8
|
||||
bytes tag_key = 4;
|
||||
}
|
||||
|
||||
message ReadSeriesCardinalityRequest {
|
||||
google.protobuf.Any ReadSeriesCardinalitySource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
}
|
||||
|
||||
// Response message for Storage.TagKeys, Storage.TagValues Storage.MeasurementNames,
|
||||
// Storage.MeasurementTagKeys and Storage.MeasurementTagValues.
|
||||
message StringValuesResponse {
|
||||
repeated bytes values = 1;
|
||||
}
|
||||
|
||||
// Response message for Storage.TagValuesGroupedByMeasurementAndTagKey.
|
||||
message TagValuesResponse {
|
||||
string measurement = 1;
|
||||
string key = 2;
|
||||
repeated string values = 3;
|
||||
}
|
||||
|
||||
// Response message for Storage.SeriesCardinality
|
||||
message Int64ValuesResponse {
|
||||
repeated int64 values = 1;
|
||||
}
|
||||
|
||||
// MeasurementNamesRequest is the request message for Storage.MeasurementNames.
|
||||
message MeasurementNamesRequest {
|
||||
google.protobuf.Any source = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false]
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
}
|
||||
|
||||
|
@ -198,7 +245,7 @@ message MeasurementNamesRequest {
|
|||
message MeasurementTagKeysRequest {
|
||||
google.protobuf.Any source = 1;
|
||||
string measurement = 2;
|
||||
TimestampRange range = 3; // [(gogoproto.nullable) = false]
|
||||
TimestampRange range = 3; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 4;
|
||||
}
|
||||
|
||||
|
@ -222,12 +269,12 @@ message MeasurementFieldsRequest {
|
|||
// MeasurementFieldsResponse is the response message for Storage.MeasurementFields.
|
||||
message MeasurementFieldsResponse {
|
||||
enum FieldType {
|
||||
FLOAT = 0;
|
||||
INTEGER = 1;
|
||||
UNSIGNED = 2;
|
||||
STRING = 3;
|
||||
BOOLEAN = 4;
|
||||
UNDEFINED = 5;
|
||||
FieldTypeFloat = 0;
|
||||
FieldTypeInteger = 1;
|
||||
FieldTypeUnsigned = 2;
|
||||
FieldTypeString = 3;
|
||||
FieldTypeBoolean = 4;
|
||||
FieldTypeUndefined = 5;
|
||||
}
|
||||
|
||||
message MessageField {
|
||||
|
@ -236,11 +283,11 @@ message MeasurementFieldsResponse {
|
|||
sfixed64 timestamp = 3;
|
||||
}
|
||||
|
||||
repeated MessageField fields = 1;// [(gogoproto.nullable) = false];
|
||||
repeated MessageField fields = 1; // [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
message ReadWindowAggregateRequest {
|
||||
google.protobuf.Any read_source = 1;
|
||||
google.protobuf.Any ReadSource = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
int64 WindowEvery = 4;
|
||||
|
@ -249,6 +296,48 @@ message ReadWindowAggregateRequest {
|
|||
Window window = 7;
|
||||
}
|
||||
|
||||
message TagValuesGroupedByMeasurementAndTagKeyRequest {
|
||||
google.protobuf.Any source = 1;
|
||||
|
||||
// MeasurementPatterns holds the patterns to match the measurements
|
||||
// against (the "FROM" part of the SHOW TAG VALUES statement).
|
||||
repeated LiteralOrRegex MeasurementPatterns = 2;
|
||||
|
||||
// TagKeyPredicate holds a predicate for the tags to find values on.
|
||||
// (the "WITH KEY" part of the SHOW TAG VALUES statement.
|
||||
// It's in one of the forms:
|
||||
// OR(IDENT, OR(IDENT, ...))
|
||||
// EQ(IDENT)
|
||||
// NEQ(IDENT)
|
||||
// EQREGEX(REGEX)
|
||||
// NEQREGEX(REGEX)
|
||||
TagKeyPredicate TagKeyPredicate = 3;
|
||||
|
||||
// Condition holds any additional condition to evaluate on the results.
|
||||
Predicate Condition = 4;
|
||||
}
|
||||
|
||||
message TagKeyPredicate {
|
||||
oneof value {
|
||||
string Eq = 1;
|
||||
string Neq = 2;
|
||||
string EqRegex = 3;
|
||||
string NeqRegex = 4;
|
||||
StringList In = 5;
|
||||
}
|
||||
}
|
||||
|
||||
message StringList {
|
||||
repeated string Vals = 1;
|
||||
}
|
||||
|
||||
message LiteralOrRegex {
|
||||
oneof value {
|
||||
string literal_value = 1;
|
||||
string regex_value = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message Window {
|
||||
Duration every = 1;
|
||||
Duration offset = 2;
|
||||
|
|
|
@ -1,25 +0,0 @@
|
|||
// This file defines extensions to the InfluxDB storage gRPC common message types
|
||||
// that have not yet made it into influxdb.
|
||||
|
||||
// It is, effectively, the delta between these two files:
|
||||
// https://github.com/influxdata/influxdb/blob/master/storage/reads/datatypes/storage_common.proto
|
||||
// https://github.com/influxdata/idpe/blob/master/storage/storageproto/storage_common.proto
|
||||
|
||||
|
||||
syntax = "proto3";
|
||||
package influxdata.platform.storage;
|
||||
|
||||
import "google/protobuf/any.proto";
|
||||
import "influxdata/platform/storage/predicate.proto";
|
||||
import "influxdata/platform/storage/storage_common.proto";
|
||||
|
||||
message ReadSeriesCardinalityRequest {
|
||||
google.protobuf.Any read_series_cardinality_source = 1;
|
||||
TimestampRange range = 2; // [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
}
|
||||
|
||||
// Response message for Storage.SeriesCardinality
|
||||
message Int64ValuesResponse {
|
||||
repeated int64 values = 1;
|
||||
}
|
|
@ -9,11 +9,11 @@
|
|||
pub mod influxdata {
|
||||
pub mod platform {
|
||||
pub mod storage {
|
||||
include!(concat!(env!("OUT_DIR"), "/influxdata.platform.storage.rs"));
|
||||
include!(concat!(
|
||||
env!("OUT_DIR"),
|
||||
"/influxdata.platform.storage.serde.rs"
|
||||
"/influxdata.platform.storage.read.rs"
|
||||
));
|
||||
include!(concat!(env!("OUT_DIR"), "/influxdata.platform.storage.rs"));
|
||||
|
||||
// Can't implement `Default` because `prost::Message` implements `Default`
|
||||
impl TimestampRange {
|
||||
|
@ -127,23 +127,6 @@ pub mod influxdata {
|
|||
}
|
||||
}
|
||||
|
||||
pub mod com {
|
||||
pub mod github {
|
||||
pub mod influxdata {
|
||||
pub mod idpe {
|
||||
pub mod storage {
|
||||
pub mod read {
|
||||
include!(concat!(
|
||||
env!("OUT_DIR"),
|
||||
"/com.github.influxdata.idpe.storage.read.rs"
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Needed because of https://github.com/hyperium/tonic/issues/471
|
||||
pub mod grpc {
|
||||
pub mod health {
|
||||
|
@ -199,7 +182,6 @@ pub fn protobuf_type_url_eq(url: &str, protobuf_type: &str) -> bool {
|
|||
}
|
||||
|
||||
// TODO: Remove these (#2419)
|
||||
pub use com::github::influxdata::idpe::storage::read::*;
|
||||
pub use influxdata::platform::storage::*;
|
||||
|
||||
pub mod google;
|
||||
|
|
|
@ -3,7 +3,6 @@ use generated_types::{
|
|||
google::{AlreadyExists, FieldViolation, FieldViolationExt, NotFound},
|
||||
influxdata::iox::management::v1::{Error as ProtobufError, *},
|
||||
};
|
||||
use predicate::delete_predicate::parse_delete_predicate;
|
||||
use query::QueryDatabase;
|
||||
use server::{
|
||||
connection::ConnectionManager, rules::ProvidedDatabaseRules, ApplicationState, Error, Server,
|
||||
|
@ -601,44 +600,6 @@ where
|
|||
|
||||
Ok(Response::new(DropPartitionResponse {}))
|
||||
}
|
||||
|
||||
async fn delete(
|
||||
&self,
|
||||
request: tonic::Request<DeleteRequest>,
|
||||
) -> Result<tonic::Response<DeleteResponse>, tonic::Status> {
|
||||
let DeleteRequest {
|
||||
db_name,
|
||||
table_name,
|
||||
start_time,
|
||||
stop_time,
|
||||
predicate,
|
||||
} = request.into_inner();
|
||||
|
||||
// Validate that the database name is legit
|
||||
let db_name = DatabaseName::new(db_name).scope("db_name")?;
|
||||
let db = self
|
||||
.server
|
||||
.db(&db_name)
|
||||
.map_err(default_server_error_handler)?;
|
||||
|
||||
let del_predicate_result = parse_delete_predicate(&start_time, &stop_time, &predicate);
|
||||
match del_predicate_result {
|
||||
Err(_) => {
|
||||
return Err(default_server_error_handler(Error::DeleteExpression {
|
||||
start_time,
|
||||
stop_time,
|
||||
predicate,
|
||||
}))
|
||||
}
|
||||
Ok(del_predicate) => {
|
||||
// execute delete
|
||||
db.delete(&table_name, Arc::new(del_predicate))
|
||||
.map_err(default_db_error_handler)?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Response::new(DeleteResponse {}))
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns [`DatabaseRules`] formated according to the `omit_defaults` flag. If `omit_defaults` is
|
||||
|
|
|
@ -368,15 +368,15 @@ mod tests {
|
|||
let dumped_frames = dump_frames(&response.frames);
|
||||
|
||||
let expected_frames = vec![
|
||||
"SeriesFrame, tags: _field=string_field,_measurement=the_table,tag1=val1, type: 4",
|
||||
"SeriesFrame, tags: _measurement=the_table,tag1=val1,_field=string_field, type: 4",
|
||||
"StringPointsFrame, timestamps: [2000, 3000], values: bar,baz",
|
||||
"SeriesFrame, tags: _field=int_field,_measurement=the_table,tag1=val1, type: 1",
|
||||
"SeriesFrame, tags: _measurement=the_table,tag1=val1,_field=int_field, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000, 3000], values: \"2,3\"",
|
||||
"SeriesFrame, tags: _field=uint_field,_measurement=the_table,tag1=val1, type: 2",
|
||||
"SeriesFrame, tags: _measurement=the_table,tag1=val1,_field=uint_field, type: 2",
|
||||
"UnsignedPointsFrame, timestamps: [2000, 3000], values: \"22,33\"",
|
||||
"SeriesFrame, tags: _field=float_field,_measurement=the_table,tag1=val1, type: 0",
|
||||
"SeriesFrame, tags: _measurement=the_table,tag1=val1,_field=float_field, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000, 3000], values: \"20.1,30.1\"",
|
||||
"SeriesFrame, tags: _field=boolean_field,_measurement=the_table,tag1=val1, type: 3",
|
||||
"SeriesFrame, tags: _measurement=the_table,tag1=val1,_field=boolean_field, type: 3",
|
||||
"BooleanPointsFrame, timestamps: [2000, 3000], values: false,true",
|
||||
];
|
||||
|
||||
|
|
|
@ -220,6 +220,7 @@ where
|
|||
read_source: _read_source,
|
||||
range,
|
||||
predicate,
|
||||
..
|
||||
} = read_filter_request;
|
||||
|
||||
info!(%db_name, ?range, predicate=%predicate.loggable(),"read filter");
|
||||
|
@ -251,15 +252,10 @@ where
|
|||
group_keys,
|
||||
group,
|
||||
aggregate,
|
||||
hints,
|
||||
} = read_group_request;
|
||||
|
||||
info!(%db_name, ?range, ?group_keys, ?group, ?aggregate,predicate=%predicate.loggable(),"read_group");
|
||||
|
||||
if hints != 0 {
|
||||
InternalHintsFieldNotSupported { hints }.fail()?
|
||||
}
|
||||
|
||||
let aggregate_string = format!(
|
||||
"aggregate: {:?}, group: {:?}, group_keys: {:?}",
|
||||
aggregate, group, group_keys
|
||||
|
@ -1772,6 +1768,7 @@ mod tests {
|
|||
read_source: source.clone(),
|
||||
range: Some(make_timestamp_range(0, 10000)),
|
||||
predicate: Some(make_state_ma_predicate()),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let frames = fixture.storage_client.read_filter(request).await.unwrap();
|
||||
|
@ -1812,6 +1809,7 @@ mod tests {
|
|||
read_source: source.clone(),
|
||||
range: None,
|
||||
predicate: None,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
// Note we don't set the response on the test database, so we expect an error
|
||||
|
@ -1855,7 +1853,6 @@ mod tests {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: aggregate::AggregateType::Sum as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let frames = fixture.storage_client.read_group(request).await.unwrap();
|
||||
|
@ -1890,34 +1887,6 @@ mod tests {
|
|||
|
||||
let group = generated_types::read_group_request::Group::By as i32;
|
||||
|
||||
// ---
|
||||
// test error hit in request processing
|
||||
// ---
|
||||
let request = ReadGroupRequest {
|
||||
read_source: source.clone(),
|
||||
range: None,
|
||||
predicate: None,
|
||||
group_keys: vec!["tag1".into()],
|
||||
group,
|
||||
aggregate: Some(Aggregate {
|
||||
r#type: aggregate::AggregateType::Sum as i32,
|
||||
}),
|
||||
hints: 42,
|
||||
};
|
||||
|
||||
let response_string = fixture
|
||||
.storage_client
|
||||
.read_group(request)
|
||||
.await
|
||||
.unwrap_err()
|
||||
.to_string();
|
||||
assert_contains!(
|
||||
response_string,
|
||||
"Unexpected hint value on read_group request. Expected 0, got 42"
|
||||
);
|
||||
|
||||
grpc_request_metric_has_count(&fixture, "ReadGroup", "server_error", 1);
|
||||
|
||||
// ---
|
||||
// test error returned in database processing
|
||||
// ---
|
||||
|
@ -1930,7 +1899,6 @@ mod tests {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: aggregate::AggregateType::Sum as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
// Note we don't set the response on the test database, so we expect an error
|
||||
|
|
|
@ -159,6 +159,7 @@ mod tests {
|
|||
Some(remote_template),
|
||||
Some(Arc::new(RingBufferTraceCollector::new(1))),
|
||||
time_provider,
|
||||
None,
|
||||
)
|
||||
.await,
|
||||
);
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use data_types::non_empty::NonEmptyString;
|
||||
use dml::{DmlDelete, DmlMeta, DmlOperation};
|
||||
use generated_types::google::{FieldViolationExt, NotFound};
|
||||
use generated_types::influxdata::iox::delete::v1::*;
|
||||
use predicate::delete_predicate::parse_delete_predicate;
|
||||
use router::server::RouterServer;
|
||||
use tonic::Response;
|
||||
|
||||
struct DeleteService {
|
||||
server: Arc<RouterServer>,
|
||||
}
|
||||
|
||||
#[tonic::async_trait]
|
||||
impl delete_service_server::DeleteService for DeleteService {
|
||||
async fn delete(
|
||||
&self,
|
||||
request: tonic::Request<DeleteRequest>,
|
||||
) -> Result<tonic::Response<DeleteResponse>, tonic::Status> {
|
||||
let span_ctx = request.extensions().get().cloned();
|
||||
let DeleteRequest {
|
||||
db_name,
|
||||
table_name,
|
||||
start_time,
|
||||
stop_time,
|
||||
predicate,
|
||||
} = request.into_inner();
|
||||
|
||||
let predicate =
|
||||
parse_delete_predicate(&start_time, &stop_time, &predicate).scope("predicate")?;
|
||||
let table_name = NonEmptyString::new(table_name);
|
||||
let meta = DmlMeta::unsequenced(span_ctx);
|
||||
let op = DmlOperation::Delete(DmlDelete::new(predicate, table_name, meta));
|
||||
|
||||
let router = self.server.router(&db_name).ok_or_else(NotFound::default)?;
|
||||
router
|
||||
.write(op)
|
||||
.await
|
||||
.map_err(|e| tonic::Status::internal(e.to_string()))?;
|
||||
|
||||
Ok(Response::new(DeleteResponse {}))
|
||||
}
|
||||
}
|
||||
|
||||
pub fn make_server(
|
||||
server: Arc<RouterServer>,
|
||||
) -> delete_service_server::DeleteServiceServer<impl delete_service_server::DeleteService> {
|
||||
delete_service_server::DeleteServiceServer::new(DeleteService { server })
|
||||
}
|
|
@ -7,6 +7,7 @@ use crate::influxdb_ioxd::{
|
|||
|
||||
use super::RouterServerType;
|
||||
|
||||
mod delete;
|
||||
mod deployment;
|
||||
mod remote;
|
||||
mod router;
|
||||
|
@ -33,6 +34,10 @@ pub async fn server_grpc(
|
|||
builder,
|
||||
router::make_server(Arc::clone(&server_type.server))
|
||||
);
|
||||
add_gated_service!(
|
||||
builder,
|
||||
delete::make_server(Arc::clone(&server_type.server))
|
||||
);
|
||||
add_gated_service!(
|
||||
builder,
|
||||
write_pb::make_server(Arc::clone(&server_type.server))
|
||||
|
|
|
@ -1,11 +1,18 @@
|
|||
use arrow_util::assert_batches_sorted_eq;
|
||||
use data_types::{
|
||||
delete_predicate::{DeleteExpr, DeletePredicate},
|
||||
non_empty::NonEmptyString,
|
||||
timestamp::TimestampRange,
|
||||
};
|
||||
use dml::{test_util::assert_delete_op_eq, DmlDelete};
|
||||
use futures::StreamExt;
|
||||
use influxdb_iox_client::management::generated_types::DatabaseRules;
|
||||
|
||||
use super::scenario::rand_name;
|
||||
use super::scenario::{create_router_to_write_buffer, rand_name};
|
||||
use crate::common::server_fixture::{ServerFixture, ServerType};
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_delete() {
|
||||
async fn test_delete_on_database() {
|
||||
test_helpers::maybe_start_logging();
|
||||
let fixture = ServerFixture::create_shared(ServerType::Database).await;
|
||||
let mut write_client = fixture.write_client();
|
||||
|
@ -149,3 +156,41 @@ async fn test_delete() {
|
|||
];
|
||||
assert_batches_sorted_eq!(&disk_expected, &batches);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_delete_on_router() {
|
||||
let fixture = ServerFixture::create_shared(ServerType::Router).await;
|
||||
|
||||
let db_name = rand_name();
|
||||
let (_tmpdir, mut write_buffer) = create_router_to_write_buffer(&fixture, &db_name).await;
|
||||
|
||||
let table = "cpu";
|
||||
let start = "100";
|
||||
let stop = "120";
|
||||
let pred = "region = west";
|
||||
let _del = fixture
|
||||
.delete_client()
|
||||
.delete(db_name.clone(), table, start, stop, pred)
|
||||
.await
|
||||
.expect("cannot delete");
|
||||
|
||||
let mut stream = write_buffer.streams().into_values().next().unwrap();
|
||||
let delete_actual = stream.stream.next().await.unwrap().unwrap();
|
||||
let delete_expected = DmlDelete::new(
|
||||
DeletePredicate {
|
||||
range: TimestampRange {
|
||||
start: 100,
|
||||
end: 120,
|
||||
},
|
||||
exprs: vec![DeleteExpr {
|
||||
column: String::from("region"),
|
||||
op: data_types::delete_predicate::Op::Eq,
|
||||
scalar: data_types::delete_predicate::Scalar::String(String::from("west")),
|
||||
}],
|
||||
},
|
||||
NonEmptyString::new(table),
|
||||
// We don't care about the metadata here, timestamps and sequence numbers are hard to guess
|
||||
delete_actual.meta().clone(),
|
||||
);
|
||||
assert_delete_op_eq(&delete_actual, &delete_expected);
|
||||
}
|
||||
|
|
|
@ -1,4 +1,3 @@
|
|||
use arrow_util::assert_batches_sorted_eq;
|
||||
use data_types::chunk_metadata::ChunkId;
|
||||
use generated_types::{
|
||||
google::protobuf::{Duration, Empty},
|
||||
|
@ -23,8 +22,6 @@ use crate::{
|
|||
fixture_broken_catalog, wait_for_exact_chunk_states, DatabaseBuilder,
|
||||
},
|
||||
};
|
||||
use chrono::{DateTime, Utc};
|
||||
use std::convert::TryInto;
|
||||
use std::time::Instant;
|
||||
use tonic::Code;
|
||||
use uuid::Uuid;
|
||||
|
@ -1456,74 +1453,6 @@ async fn test_unload_read_buffer() {
|
|||
assert_eq!(chunks[0].storage, storage);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_chunk_access_time() {
|
||||
let fixture = ServerFixture::create_shared(ServerType::Database).await;
|
||||
let mut write_client = fixture.write_client();
|
||||
let mut management_client = fixture.management_client();
|
||||
let mut flight_client = fixture.flight_client();
|
||||
|
||||
let db_name = rand_name();
|
||||
DatabaseBuilder::new(db_name.clone())
|
||||
.build(fixture.grpc_channel())
|
||||
.await;
|
||||
|
||||
write_client
|
||||
.write_lp(&db_name, "cpu foo=1 10", 0)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let to_datetime = |a: Option<&generated_types::google::protobuf::Timestamp>| -> DateTime<Utc> {
|
||||
a.unwrap().clone().try_into().unwrap()
|
||||
};
|
||||
|
||||
let chunks = management_client.list_chunks(&db_name).await.unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let t0 = to_datetime(chunks[0].time_of_last_access.as_ref());
|
||||
|
||||
flight_client
|
||||
.perform_query(&db_name, "select * from cpu;")
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let chunks = management_client.list_chunks(&db_name).await.unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let t1 = to_datetime(chunks[0].time_of_last_access.as_ref());
|
||||
|
||||
flight_client
|
||||
.perform_query(&db_name, "select * from cpu;")
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let chunks = management_client.list_chunks(&db_name).await.unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let t2 = to_datetime(chunks[0].time_of_last_access.as_ref());
|
||||
|
||||
write_client
|
||||
.write_lp(&db_name, "cpu foo=1 20", 0)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let chunks = management_client.list_chunks(&db_name).await.unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let t3 = to_datetime(chunks[0].time_of_last_access.as_ref());
|
||||
|
||||
// This chunk should be pruned out and therefore not accessed by the query
|
||||
flight_client
|
||||
.perform_query(&db_name, "select * from cpu where foo = 2;")
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let chunks = management_client.list_chunks(&db_name).await.unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let t4 = to_datetime(chunks[0].time_of_last_access.as_ref());
|
||||
|
||||
assert!(t0 < t1, "{} {}", t0, t1);
|
||||
assert!(t1 < t2, "{} {}", t1, t2);
|
||||
assert!(t2 < t3, "{} {}", t2, t3);
|
||||
assert_eq!(t3, t4)
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_drop_partition() {
|
||||
use data_types::chunk_metadata::ChunkStorage;
|
||||
|
@ -1624,151 +1553,6 @@ async fn test_drop_partition_error() {
|
|||
assert_contains!(err.to_string(), "Cannot drop unpersisted chunk");
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_delete() {
|
||||
test_helpers::maybe_start_logging();
|
||||
let fixture = ServerFixture::create_shared(ServerType::Database).await;
|
||||
let mut write_client = fixture.write_client();
|
||||
let mut management_client = fixture.management_client();
|
||||
let mut flight_client = fixture.flight_client();
|
||||
|
||||
// DB name and rules
|
||||
let db_name = rand_name();
|
||||
let rules = DatabaseRules {
|
||||
name: db_name.clone(),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
// create that db
|
||||
management_client
|
||||
.create_database(rules.clone())
|
||||
.await
|
||||
.expect("create database failed");
|
||||
|
||||
// Load a few rows of data
|
||||
let lp_lines = vec![
|
||||
"cpu,region=west user=23.2 100",
|
||||
"cpu,region=west user=21.0 150",
|
||||
"disk,region=east bytes=99i 200",
|
||||
];
|
||||
|
||||
let num_lines_written = write_client
|
||||
.write_lp(&db_name, lp_lines.join("\n"), 0)
|
||||
.await
|
||||
.expect("write succeded");
|
||||
|
||||
assert_eq!(num_lines_written, 3);
|
||||
|
||||
// Query cpu
|
||||
let mut query_results = flight_client
|
||||
.perform_query(db_name.clone(), "select * from cpu")
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
let expected = [
|
||||
"+--------+--------------------------------+------+",
|
||||
"| region | time | user |",
|
||||
"+--------+--------------------------------+------+",
|
||||
"| west | 1970-01-01T00:00:00.000000100Z | 23.2 |",
|
||||
"| west | 1970-01-01T00:00:00.000000150Z | 21 |",
|
||||
"+--------+--------------------------------+------+",
|
||||
];
|
||||
assert_batches_sorted_eq!(&expected, &batches);
|
||||
|
||||
// Delete some data
|
||||
let table = "cpu";
|
||||
let start = "100";
|
||||
let stop = "120";
|
||||
let pred = "region = west";
|
||||
let _del = management_client
|
||||
.delete(db_name.clone(), table, start, stop, pred)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
// query to verify data deleted
|
||||
let mut query_results = flight_client
|
||||
.perform_query(db_name.clone(), "select * from cpu")
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
let expected = [
|
||||
"+--------+--------------------------------+------+",
|
||||
"| region | time | user |",
|
||||
"+--------+--------------------------------+------+",
|
||||
"| west | 1970-01-01T00:00:00.000000150Z | 21 |",
|
||||
"+--------+--------------------------------+------+",
|
||||
];
|
||||
assert_batches_sorted_eq!(&expected, &batches);
|
||||
|
||||
// Query cpu again with a selection predicate
|
||||
let mut query_results = flight_client
|
||||
.perform_query(
|
||||
db_name.clone(),
|
||||
r#"select * from cpu where cpu.region='west';"#,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
// result should be as above
|
||||
assert_batches_sorted_eq!(&expected, &batches);
|
||||
|
||||
// Query cpu again with a differentselection predicate
|
||||
let mut query_results = flight_client
|
||||
.perform_query(db_name.clone(), "select * from cpu where user!=21")
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
// result should be nothing
|
||||
let expected = ["++", "++"];
|
||||
assert_batches_sorted_eq!(&expected, &batches);
|
||||
|
||||
// ------------------------------------------
|
||||
// Negative Delete test to get error messages
|
||||
|
||||
// Delete from non-existing table
|
||||
let table = "notable";
|
||||
let start = "100";
|
||||
let stop = "120";
|
||||
let pred = "region = west";
|
||||
let del = management_client
|
||||
.delete(db_name.clone(), table, start, stop, pred)
|
||||
.await
|
||||
.unwrap_err()
|
||||
.to_string();
|
||||
assert!(del.contains("Cannot delete data from table"));
|
||||
|
||||
// Verify both existing tables still have the same data
|
||||
// query to verify data deleted
|
||||
// cpu
|
||||
let mut query_results = flight_client
|
||||
.perform_query(db_name.clone(), "select * from cpu")
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
let cpu_expected = [
|
||||
"+--------+--------------------------------+------+",
|
||||
"| region | time | user |",
|
||||
"+--------+--------------------------------+------+",
|
||||
"| west | 1970-01-01T00:00:00.000000150Z | 21 |",
|
||||
"+--------+--------------------------------+------+",
|
||||
];
|
||||
assert_batches_sorted_eq!(&cpu_expected, &batches);
|
||||
// disk
|
||||
let mut query_results = flight_client
|
||||
.perform_query(db_name.clone(), "select * from disk")
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = query_results.to_batches().await.unwrap();
|
||||
let disk_expected = [
|
||||
"+-------+--------+--------------------------------+",
|
||||
"| bytes | region | time |",
|
||||
"+-------+--------+--------------------------------+",
|
||||
"| 99 | east | 1970-01-01T00:00:00.000000200Z |",
|
||||
"+-------+--------+--------------------------------+",
|
||||
];
|
||||
assert_batches_sorted_eq!(&disk_expected, &batches);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_persist_partition() {
|
||||
use data_types::chunk_metadata::ChunkStorage;
|
||||
|
|
|
@ -63,6 +63,7 @@ async fn read_filter_endpoint(storage_client: &mut StorageClient<Connection>, sc
|
|||
read_source,
|
||||
range,
|
||||
predicate,
|
||||
..Default::default()
|
||||
});
|
||||
let read_response = storage_client
|
||||
.read_filter(read_filter_request)
|
||||
|
@ -77,15 +78,15 @@ async fn read_filter_endpoint(storage_client: &mut StorageClient<Connection>, sc
|
|||
.collect();
|
||||
|
||||
let expected_frames = substitute_nanos(scenario.ns_since_epoch(), &[
|
||||
"SeriesFrame, tags: _field=value,_measurement=cpu_load_short,host=server01, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu_load_short,host=server01,_field=value, type: 0",
|
||||
"FloatPointsFrame, timestamps: [ns1], values: \"27.99\"",
|
||||
"SeriesFrame, tags: _field=value,_measurement=cpu_load_short,host=server01,region=us-east, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu_load_short,host=server01,region=us-east,_field=value, type: 0",
|
||||
"FloatPointsFrame, timestamps: [ns3], values: \"1234567.891011\"",
|
||||
"SeriesFrame, tags: _field=value,_measurement=cpu_load_short,host=server01,region=us-west, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu_load_short,host=server01,region=us-west,_field=value, type: 0",
|
||||
"FloatPointsFrame, timestamps: [ns0, ns4], values: \"0.64,0.000003\"",
|
||||
"SeriesFrame, tags: _field=in,_measurement=swap,host=server01,name=disk0, type: 1",
|
||||
"SeriesFrame, tags: _measurement=swap,host=server01,name=disk0,_field=in, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [ns6], values: \"3\"",
|
||||
"SeriesFrame, tags: _field=out,_measurement=swap,host=server01,name=disk0, type: 1",
|
||||
"SeriesFrame, tags: _measurement=swap,host=server01,name=disk0,_field=out, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [ns6], values: \"4\""
|
||||
]);
|
||||
|
||||
|
@ -316,16 +317,17 @@ pub async fn regex_operator_test() {
|
|||
end: 2001, // include all data
|
||||
}),
|
||||
predicate: Some(make_regex_match_predicate("host", "^b.+")),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let expected_frames = vec![
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"20,21\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"81,82\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"40,41\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"51,52\"",
|
||||
];
|
||||
|
||||
|
@ -391,27 +393,26 @@ async fn test_read_group_none_agg() {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: AggregateType::None as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let expected_group_frames = vec![
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"20,21\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"10,11\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"81,82\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"10,11\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"71,72\"",
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"40,41\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"30,31\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"51,52\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"30,31\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000, 2000], values: \"61,62\"",
|
||||
];
|
||||
|
||||
|
@ -442,19 +443,18 @@ async fn test_read_group_none_agg_with_predicate() {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: AggregateType::None as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let expected_group_frames = vec![
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000], values: \"20\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000], values: \"10\"",
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000], values: \"40\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [1000], values: \"30\"",
|
||||
];
|
||||
|
||||
|
@ -488,27 +488,26 @@ async fn test_read_group_sum_agg() {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: AggregateType::Sum as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let expected_group_frames = vec![
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"41\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"21\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"163\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"21\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"143\"",
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"81\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"61\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"103\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"61\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"123\"",
|
||||
];
|
||||
|
||||
|
@ -541,27 +540,26 @@ async fn test_read_group_count_agg() {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: AggregateType::Count as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let expected_group_frames = vec![
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 1",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=foo, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_user, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=bar, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_system, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=foo, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_user, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 1",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=foo, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_user, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=bar, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_system, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=foo, type: 1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_user, type: 1",
|
||||
"IntegerPointsFrame, timestamps: [2000], values: \"2\"",
|
||||
];
|
||||
|
||||
|
@ -595,27 +593,26 @@ async fn test_read_group_last_agg() {
|
|||
aggregate: Some(Aggregate {
|
||||
r#type: AggregateType::Last as i32,
|
||||
}),
|
||||
hints: 0,
|
||||
};
|
||||
|
||||
let expected_group_frames = vec![
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu1",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"21\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"11\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"82\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu1,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"11\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu1,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"72\"",
|
||||
"GroupFrame, tag_keys: _field,_measurement,cpu,host, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"GroupFrame, tag_keys: _measurement,cpu,host,_field, partition_key_vals: cpu2",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"41\"",
|
||||
"SeriesFrame, tags: _field=usage_system,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"31\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=bar, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=bar,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"52\"",
|
||||
"SeriesFrame, tags: _field=usage_user,_measurement=cpu,cpu=cpu2,host=foo, type: 0",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_system, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"31\"",
|
||||
"SeriesFrame, tags: _measurement=cpu,cpu=cpu2,host=foo,_field=usage_user, type: 0",
|
||||
"FloatPointsFrame, timestamps: [2000], values: \"62\"",
|
||||
];
|
||||
|
||||
|
@ -697,9 +694,9 @@ pub async fn read_window_aggregate_test() {
|
|||
.collect();
|
||||
|
||||
let expected_frames = vec![
|
||||
"SeriesFrame, tags: _field=temp,_measurement=h2o,city=Boston,state=MA, type: 0",
|
||||
"SeriesFrame, tags: _measurement=h2o,city=Boston,state=MA,_field=temp, type: 0",
|
||||
"FloatPointsFrame, timestamps: [400, 600], values: \"143,147\"",
|
||||
"SeriesFrame, tags: _field=temp,_measurement=h2o,city=Cambridge,state=MA, type: 0",
|
||||
"SeriesFrame, tags: _measurement=h2o,city=Cambridge,state=MA,_field=temp, type: 0",
|
||||
"FloatPointsFrame, timestamps: [400, 600], values: \"163,167\"",
|
||||
];
|
||||
|
||||
|
|
|
@ -85,6 +85,7 @@ pub async fn test_tracing_storage_api() {
|
|||
read_source,
|
||||
range,
|
||||
predicate,
|
||||
..Default::default()
|
||||
});
|
||||
let mut storage_client = StorageClient::new(server_fixture.grpc_channel());
|
||||
let read_response = storage_client
|
||||
|
|
|
@ -400,25 +400,6 @@ pub enum DropPartitionError {
|
|||
ServerError(tonic::Status),
|
||||
}
|
||||
|
||||
/// Errors returned by [`Client::delete`]
|
||||
#[derive(Debug, Error)]
|
||||
pub enum DeleteError {
|
||||
/// Database not found
|
||||
#[error("Not found: {}", .0)]
|
||||
NotFound(String),
|
||||
|
||||
/// Response contained no payload
|
||||
#[error("Server returned an empty response")]
|
||||
EmptyResponse,
|
||||
|
||||
/// Server indicated that it is not (yet) available
|
||||
#[error("Server unavailable: {}", .0.message())]
|
||||
Unavailable(tonic::Status),
|
||||
|
||||
/// Client received an unexpected error from the server
|
||||
#[error("Unexpected server error: {}: {}", .0.code(), .0.message())]
|
||||
ServerError(tonic::Status),
|
||||
}
|
||||
/// Errors returned by [`Client::persist_partition`]
|
||||
#[derive(Debug, Error)]
|
||||
pub enum PersistPartitionError {
|
||||
|
@ -1042,39 +1023,6 @@ impl Client {
|
|||
Ok(())
|
||||
}
|
||||
|
||||
/// Delete data from a table on a specified predicate
|
||||
pub async fn delete(
|
||||
&mut self,
|
||||
db_name: impl Into<String> + Send,
|
||||
table_name: impl Into<String> + Send,
|
||||
start_time: impl Into<String> + Send,
|
||||
stop_time: impl Into<String> + Send,
|
||||
predicate: impl Into<String> + Send,
|
||||
) -> Result<(), DeleteError> {
|
||||
let db_name = db_name.into();
|
||||
let table_name = table_name.into();
|
||||
let start_time = start_time.into();
|
||||
let stop_time = stop_time.into();
|
||||
let predicate = predicate.into();
|
||||
|
||||
self.inner
|
||||
.delete(DeleteRequest {
|
||||
db_name,
|
||||
table_name,
|
||||
start_time,
|
||||
stop_time,
|
||||
predicate,
|
||||
})
|
||||
.await
|
||||
.map_err(|status| match status.code() {
|
||||
tonic::Code::NotFound => DeleteError::NotFound(status.message().to_string()),
|
||||
tonic::Code::Unavailable => DeleteError::Unavailable(status),
|
||||
_ => DeleteError::ServerError(status),
|
||||
})?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Persist given partition.
|
||||
///
|
||||
/// Errors if there is nothing to persist at the moment as per the lifecycle rules. If successful it returns the
|
||||
|
|
|
@ -21,7 +21,6 @@ use std::collections::HashMap;
|
|||
|
||||
/// Re-export generated_types
|
||||
pub mod generated_types {
|
||||
pub use generated_types::com::github::influxdata::idpe::storage::read::*;
|
||||
pub use generated_types::influxdata::platform::storage::*;
|
||||
}
|
||||
|
||||
|
@ -97,8 +96,7 @@ impl Client {
|
|||
.encode(&mut d)
|
||||
.expect("encoded read source appropriately");
|
||||
Any {
|
||||
type_url: "type.googleapis.com/com.github.influxdata.idpe.storage.read.ReadSource"
|
||||
.to_string(),
|
||||
type_url: "type.googleapis.com/influxdata.platform.storage.read.ReadSource".to_string(),
|
||||
value: d.freeze(),
|
||||
}
|
||||
}
|
||||
|
|
|
@ -454,7 +454,7 @@ mod test {
|
|||
let reset_after = Some(3);
|
||||
let mut i64fg = I64FieldGenerator::new(
|
||||
"i64fg",
|
||||
&(3..10),
|
||||
&(3..8),
|
||||
true,
|
||||
reset_after,
|
||||
SmallRng::from_entropy(),
|
||||
|
|
|
@ -97,6 +97,7 @@ use tracker::{RwLock, RwLockUpgradableReadGuard, RwLockWriteGuard};
|
|||
///
|
||||
/// Note: The `LifecycleReadGuard` will not block other readers to `RwLock<P>` but
|
||||
/// they will block other upgradeable readers, e.g. other `LifecycleReadGuard`
|
||||
#[derive(Debug)]
|
||||
pub struct LifecycleReadGuard<'a, P, D> {
|
||||
data: D,
|
||||
guard: RwLockUpgradableReadGuard<'a, P>,
|
||||
|
@ -129,15 +130,9 @@ impl<'a, P, D> LifecycleReadGuard<'a, P, D> {
|
|||
}
|
||||
}
|
||||
|
||||
impl<'a, P, D> Debug for LifecycleReadGuard<'a, P, D> {
|
||||
impl<'a, P: Display, D> Display for LifecycleReadGuard<'a, P, D> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "LifecycleReadGuard{{..}}")
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, P, D: Display> Display for LifecycleReadGuard<'a, P, D> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{} locked for read", self.data)
|
||||
self.guard.fmt(f)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -156,6 +151,7 @@ impl<'a, P, D> Deref for LifecycleReadGuard<'a, P, D> {
|
|||
/// a normal read guard or smart pointer, and also mutably through
|
||||
/// `std::ops::DerefMut` akin to a normal write guard
|
||||
///
|
||||
#[derive(Debug)]
|
||||
pub struct LifecycleWriteGuard<'a, P, D> {
|
||||
data: D,
|
||||
guard: RwLockWriteGuard<'a, P>,
|
||||
|
@ -179,14 +175,9 @@ impl<'a, P, D> LifecycleWriteGuard<'a, P, D> {
|
|||
}
|
||||
}
|
||||
|
||||
impl<'a, P, D> Debug for LifecycleWriteGuard<'a, P, D> {
|
||||
impl<'a, P: Display, D> Display for LifecycleWriteGuard<'a, P, D> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "LifecycleWriteGuard{{..}}")
|
||||
}
|
||||
}
|
||||
impl<'a, P, D: Display> Display for LifecycleWriteGuard<'a, P, D> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{} locked for write", self.data)
|
||||
self.guard.fmt(f)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ pub trait LifecycleDb {
|
|||
|
||||
/// A `LockablePartition` is a wrapper around a `LifecyclePartition` that allows
|
||||
/// for planning and executing lifecycle actions on the partition
|
||||
pub trait LockablePartition: Sized + std::fmt::Display {
|
||||
pub trait LockablePartition: Sized {
|
||||
type Partition: LifecyclePartition;
|
||||
type Chunk: LockableChunk;
|
||||
type PersistHandle: PersistHandle + Send + Sync + 'static;
|
||||
|
@ -156,7 +156,7 @@ pub trait LockableChunk: Sized {
|
|||
fn order(&self) -> ChunkOrder;
|
||||
}
|
||||
|
||||
pub trait LifecyclePartition {
|
||||
pub trait LifecyclePartition: std::fmt::Display {
|
||||
fn partition_key(&self) -> &str;
|
||||
|
||||
/// Returns true if all chunks in the partition are persisted.
|
||||
|
|
|
@ -127,10 +127,10 @@ where
|
|||
loop {
|
||||
let buffer_size = self.db.buffer_size();
|
||||
if buffer_size < soft_limit {
|
||||
info!(%db_name, buffer_size, %soft_limit, "memory use under soft limit");
|
||||
trace!(%db_name, buffer_size, %soft_limit, "memory use under soft limit");
|
||||
break;
|
||||
}
|
||||
info!(%db_name, buffer_size, %soft_limit, "memory use over soft limit");
|
||||
trace!(%db_name, buffer_size, %soft_limit, "memory use over soft limit");
|
||||
|
||||
match candidates.next() {
|
||||
Some(candidate) => {
|
||||
|
@ -139,7 +139,7 @@ where
|
|||
Some(chunk) => {
|
||||
let chunk = chunk.read();
|
||||
if chunk.lifecycle_action().is_some() {
|
||||
info!(
|
||||
debug!(
|
||||
%db_name,
|
||||
chunk_id=%candidate.chunk_id.get(),
|
||||
%partition,
|
||||
|
@ -183,7 +183,7 @@ where
|
|||
},
|
||||
}
|
||||
}
|
||||
None => info!(
|
||||
None => debug!(
|
||||
%db_name,
|
||||
chunk_id=%candidate.chunk_id.get(),
|
||||
%partition,
|
||||
|
@ -192,7 +192,7 @@ where
|
|||
}
|
||||
}
|
||||
None => {
|
||||
warn!(%db_name, soft_limit, buffer_size,
|
||||
debug!(%db_name, soft_limit, buffer_size,
|
||||
"soft limited exceeded, but no chunks found that can be evicted. Check lifecycle rules");
|
||||
break;
|
||||
}
|
||||
|
@ -363,9 +363,9 @@ where
|
|||
"considering for persistence");
|
||||
|
||||
if persistable_row_count >= rules.persist_row_threshold.get() {
|
||||
info!(%db_name, %partition, persistable_row_count, "persisting partition as exceeds row threshold");
|
||||
debug!(%db_name, %partition, persistable_row_count, "persisting partition as exceeds row threshold");
|
||||
} else if persistable_age_seconds >= rules.persist_age_threshold_seconds.get() as u64 {
|
||||
info!(%db_name, %partition, persistable_age_seconds, "persisting partition as exceeds age threshold");
|
||||
debug!(%db_name, %partition, persistable_age_seconds, "persisting partition as exceeds age threshold");
|
||||
} else {
|
||||
trace!(%db_name, %partition, persistable_row_count, "partition not eligible for persist");
|
||||
return false;
|
||||
|
@ -459,7 +459,7 @@ where
|
|||
let stall_compaction_persisting = if rules.persist && !self.suppress_persistence {
|
||||
let persisting = self.maybe_persist_chunks(&db_name, partition, &rules, now);
|
||||
if persisting {
|
||||
debug!(%db_name, %partition, reason="persisting", "stalling compaction");
|
||||
debug!(%db_name, partition=%partition.read(), reason="persisting", "stalling compaction");
|
||||
}
|
||||
persisting
|
||||
} else {
|
||||
|
@ -476,7 +476,7 @@ where
|
|||
let max_compactions = self.db.rules().max_active_compactions.get();
|
||||
let slots_full = self.active_compactions >= max_compactions as usize;
|
||||
if slots_full {
|
||||
debug!(%db_name, %partition, ?max_compactions, reason="slots_full", "stalling compaction");
|
||||
debug!(%db_name, partition=%partition.read(), ?max_compactions, reason="slots_full", "stalling compaction");
|
||||
}
|
||||
slots_full
|
||||
};
|
||||
|
@ -659,7 +659,7 @@ where
|
|||
// see if we should stall subsequent pull it is
|
||||
// preventing us from persisting
|
||||
let stall = action.metadata() == &ChunkLifecycleAction::Compacting;
|
||||
info!(?action, chunk=%chunk.addr(), "Chunk to persist has outstanding action");
|
||||
debug!(?action, chunk=%chunk.addr(), "Chunk to persist has outstanding action");
|
||||
|
||||
// NOTE: This early exit also ensures that we are not "jumping" over chunks sorted by `order`.
|
||||
return Err(stall);
|
||||
|
@ -735,6 +735,12 @@ mod tests {
|
|||
}
|
||||
}
|
||||
|
||||
impl std::fmt::Display for TestPartition {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{:?}", self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct TestChunk {
|
||||
addr: ChunkAddr,
|
||||
|
@ -803,12 +809,6 @@ mod tests {
|
|||
partition: Arc<RwLock<TestPartition>>,
|
||||
}
|
||||
|
||||
impl<'a> std::fmt::Display for TestLockablePartition<'a> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{}", self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct TestLockableChunk<'a> {
|
||||
db: &'a TestDb,
|
||||
|
|
|
@ -146,7 +146,7 @@ mod tests {
|
|||
op: Op::Eq,
|
||||
scalar: Scalar::Bool(true),
|
||||
},
|
||||
"foo=true",
|
||||
r#""foo"=true"#,
|
||||
);
|
||||
assert_expr_works(
|
||||
DeleteExpr {
|
||||
|
@ -154,7 +154,7 @@ mod tests {
|
|||
op: Op::Ne,
|
||||
scalar: Scalar::I64(-1),
|
||||
},
|
||||
"bar!=-1",
|
||||
r#""bar"!=-1"#,
|
||||
);
|
||||
assert_expr_works(
|
||||
DeleteExpr {
|
||||
|
@ -162,7 +162,7 @@ mod tests {
|
|||
op: Op::Eq,
|
||||
scalar: Scalar::F64((-1.1).into()),
|
||||
},
|
||||
"baz=-1.1",
|
||||
r#""baz"=-1.1"#,
|
||||
);
|
||||
assert_expr_works(
|
||||
DeleteExpr {
|
||||
|
@ -170,7 +170,7 @@ mod tests {
|
|||
op: Op::Eq,
|
||||
scalar: Scalar::String("foo".to_string()),
|
||||
},
|
||||
"col='foo'",
|
||||
r#""col"='foo'"#,
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@ use std::sync::Arc;
|
|||
|
||||
use datafusion::{
|
||||
self,
|
||||
logical_plan::{normalize_col, Expr, LogicalPlan},
|
||||
logical_plan::{normalize_col, plan::ExtensionPlan, Expr, LogicalPlan},
|
||||
};
|
||||
|
||||
pub use context::{IOxExecutionConfig, IOxExecutionContext};
|
||||
|
@ -140,7 +140,7 @@ impl Drop for Executor {
|
|||
pub fn make_schema_pivot(input: LogicalPlan) -> LogicalPlan {
|
||||
let node = Arc::new(SchemaPivotNode::new(input));
|
||||
|
||||
LogicalPlan::Extension { node }
|
||||
LogicalPlan::Extension(ExtensionPlan { node })
|
||||
}
|
||||
|
||||
/// Make a NonNullChecker node takes an arbitrary input array and
|
||||
|
@ -178,7 +178,7 @@ pub fn make_schema_pivot(input: LogicalPlan) -> LogicalPlan {
|
|||
pub fn make_non_null_checker(table_name: &str, input: LogicalPlan) -> LogicalPlan {
|
||||
let node = Arc::new(NonNullCheckerNode::new(table_name, input));
|
||||
|
||||
LogicalPlan::Extension { node }
|
||||
LogicalPlan::Extension(ExtensionPlan { node })
|
||||
}
|
||||
|
||||
/// Create a StreamSplit node which takes an input stream of record
|
||||
|
@ -214,7 +214,7 @@ pub fn make_stream_split(input: LogicalPlan, split_expr: Expr) -> LogicalPlan {
|
|||
let split_expr = normalize_col(split_expr, &input).expect("normalize is infallable");
|
||||
|
||||
let node = Arc::new(StreamSplitNode::new(input, split_expr));
|
||||
LogicalPlan::Extension { node }
|
||||
LogicalPlan::Extension(ExtensionPlan { node })
|
||||
}
|
||||
|
||||
/// A type that can provide `IOxExecutionContext` for query
|
||||
|
|
|
@ -12,7 +12,10 @@ use arrow::{
|
|||
datatypes::DataType as ArrowDataType,
|
||||
};
|
||||
|
||||
use crate::exec::{field::FieldIndex, seriesset::SeriesSet};
|
||||
use crate::{
|
||||
exec::{field::FieldIndex, seriesset::SeriesSet},
|
||||
frontend::influxrpc::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME},
|
||||
};
|
||||
use snafu::Snafu;
|
||||
|
||||
#[derive(Debug, Snafu)]
|
||||
|
@ -250,19 +253,23 @@ impl SeriesSet {
|
|||
// Special case "measurement" name which is modeled as a tag of
|
||||
// "_measurement" and "field" which is modeled as a tag of "_field"
|
||||
//
|
||||
// Note by placing these tags at the front of the keys, it
|
||||
// means the output will be sorted first by _field and then
|
||||
// _measurement even when there are no groups requested
|
||||
let mut converted_tags = vec![
|
||||
Tag {
|
||||
key: "_field".into(),
|
||||
value: field_name.into(),
|
||||
},
|
||||
Tag {
|
||||
key: "_measurement".into(),
|
||||
value: Arc::clone(&self.table_name),
|
||||
},
|
||||
];
|
||||
// N.B., in order to emit series sets in the same "tag order" as they
|
||||
// would be in a TSM model we need to emit "_measurement" at the front
|
||||
// and "_field" at the end. Whilst this does not appear to be the
|
||||
// correct lexicographical order, in a TSM data-model these tags are
|
||||
// actually stored as `\x00` and `\xff` respectively. Therefore the
|
||||
// expectation is that "_measurement" is emitted first and "_field"
|
||||
// last.
|
||||
//
|
||||
// This also ensures that the output will be sorted first by
|
||||
// "_measurement" and then "_field" even when there are no groups
|
||||
// requested.
|
||||
|
||||
// Prepend key with "_measurement"
|
||||
let mut converted_tags = vec![Tag {
|
||||
key: MEASUREMENT_COLUMN_NAME.into(),
|
||||
value: Arc::clone(&self.table_name),
|
||||
}];
|
||||
|
||||
// convert the rest of the tags
|
||||
converted_tags.extend(self.tags.iter().map(|(k, v)| Tag {
|
||||
|
@ -270,6 +277,11 @@ impl SeriesSet {
|
|||
value: Arc::clone(v),
|
||||
}));
|
||||
|
||||
// Add "_field" to end of key.
|
||||
converted_tags.push(Tag {
|
||||
key: FIELD_COLUMN_NAME.into(),
|
||||
value: field_name.into(),
|
||||
});
|
||||
converted_tags
|
||||
}
|
||||
}
|
||||
|
@ -442,15 +454,15 @@ mod tests {
|
|||
let series_strings = series_set_to_series_strings(series_set);
|
||||
|
||||
let expected = vec![
|
||||
"Series tags={_field=string_field, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=string_field}",
|
||||
" StringPoints timestamps: [2000, 3000], values: [\"bar\", \"baz\"]",
|
||||
"Series tags={_field=int_field, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=int_field}",
|
||||
" IntegerPoints timestamps: [2000, 3000], values: [2, 3]",
|
||||
"Series tags={_field=uint_field, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=uint_field}",
|
||||
" UnsignedPoints timestamps: [2000, 3000], values: [22, 33]",
|
||||
"Series tags={_field=float_field, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=float_field}",
|
||||
" FloatPoints timestamps: [2000, 3000], values: [20.1, 30.1]",
|
||||
"Series tags={_field=boolean_field, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=boolean_field}",
|
||||
" BooleanPoints timestamps: [2000, 3000], values: [false, true]",
|
||||
];
|
||||
|
||||
|
@ -491,9 +503,9 @@ mod tests {
|
|||
let series_strings = series_set_to_series_strings(series_set);
|
||||
|
||||
let expected = vec![
|
||||
"Series tags={_field=string_field2, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=string_field2}",
|
||||
" StringPoints timestamps: [4, 5], values: [\"far\", \"faz\"]",
|
||||
"Series tags={_field=string_field1, _measurement=the_table, tag1=val1}",
|
||||
"Series tags={_measurement=the_table, tag1=val1, _field=string_field1}",
|
||||
" StringPoints timestamps: [2, 3], values: [\"bar\", \"baz\"]",
|
||||
];
|
||||
|
||||
|
@ -543,7 +555,7 @@ mod tests {
|
|||
let series_strings = series_set_to_series_strings(series_set);
|
||||
|
||||
let expected = vec![
|
||||
"Series tags={_field=float_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=float_field}",
|
||||
" FloatPoints timestamps: [1000, 2000, 4000], values: [10.1, 20.1, 40.1]",
|
||||
];
|
||||
|
||||
|
@ -568,7 +580,7 @@ mod tests {
|
|||
|
||||
let batch = RecordBatch::try_from_iter_with_nullable(vec![
|
||||
("state", Arc::new(tag_array) as ArrayRef, true),
|
||||
("srting_field", Arc::new(string_array), true),
|
||||
("string_field", Arc::new(string_array), true),
|
||||
("float_field", Arc::new(float_array), true),
|
||||
("int_field", Arc::new(int_array), true),
|
||||
("uint_field", Arc::new(uint_array), true),
|
||||
|
@ -591,15 +603,15 @@ mod tests {
|
|||
let series_strings = series_set_to_series_strings(series_set);
|
||||
|
||||
let expected = vec![
|
||||
"Series tags={_field=srting_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=string_field}",
|
||||
" StringPoints timestamps: [2000], values: [\"foo\"]",
|
||||
"Series tags={_field=float_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=float_field}",
|
||||
" FloatPoints timestamps: [2000], values: [1.0]",
|
||||
"Series tags={_field=int_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=int_field}",
|
||||
" IntegerPoints timestamps: [2000], values: [-10]",
|
||||
"Series tags={_field=uint_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=uint_field}",
|
||||
" UnsignedPoints timestamps: [2000], values: [100]",
|
||||
"Series tags={_field=bool_field, _measurement=the_table, state=MA}",
|
||||
"Series tags={_measurement=the_table, state=MA, _field=bool_field}",
|
||||
" BooleanPoints timestamps: [2000], values: [true]",
|
||||
];
|
||||
|
||||
|
|
|
@ -3,10 +3,10 @@
|
|||
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| plan_type | plan |
|
||||
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| logical_plan | Sort: #h2o.time ASC NULLS FIRST, #h2o.state ASC NULLS FIRST, #h2o.city ASC NULLS FIRST |
|
||||
| logical_plan | Sort: #h2o.time ASC NULLS LAST, #h2o.state ASC NULLS LAST, #h2o.city ASC NULLS LAST |
|
||||
| | Projection: #h2o.time, #h2o.state, #h2o.city, #h2o.min_temp, #h2o.max_temp, #h2o.area |
|
||||
| | TableScan: h2o projection=Some([0, 1, 2, 3, 4, 5]) |
|
||||
| physical_plan | SortExec: [time@0 ASC,state@1 ASC,city@2 ASC] |
|
||||
| physical_plan | SortExec: [time@0 ASC NULLS LAST,state@1 ASC NULLS LAST,city@2 ASC NULLS LAST] |
|
||||
| | CoalescePartitionsExec |
|
||||
| | ProjectionExec: expr=[time@5 as time, state@4 as state, city@1 as city, min_temp@3 as min_temp, max_temp@2 as max_temp, area@0 as area] |
|
||||
| | UnionExec |
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
| | TableScan: h2o projection=Some([0]) |
|
||||
| physical_plan | ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))] |
|
||||
| | RepartitionExec: partitioning=RoundRobinBatch(4) |
|
||||
| | ProjectionExec: expr=[3 as COUNT(Uint8(1))] |
|
||||
| | ProjectionExec: expr=[3 as COUNT(UInt8(1))] |
|
||||
| | EmptyExec: produce_one_row=true |
|
||||
| | |
|
||||
+---------------+-------------------------------------------------------------+
|
||||
|
|
|
@ -165,16 +165,16 @@
|
|||
| | |
|
||||
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
-- SQL: EXPLAIN SELECT * from restaurant where 5.0 < system and town != 'tewsbury' and system < 7.0 and (count = 632 or town = 'reading') and time > to_timestamp('1970-01-01T00:00:00.000000130+00:00');
|
||||
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| plan_type | plan |
|
||||
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| logical_plan | Projection: #restaurant.count, #restaurant.system, #restaurant.time, #restaurant.town |
|
||||
| | Filter: Float64(5) < #restaurant.system AND #restaurant.town != Utf8("tewsbury") AND #restaurant.system < Float64(7) AND #restaurant.count = Int64(632) OR #restaurant.town = Utf8("reading") AND #restaurant.time > TimestampNanosecond(130) |
|
||||
| | TableScan: restaurant projection=Some([0, 1, 2, 3]), filters=[Float64(5) < #restaurant.system, #restaurant.town != Utf8("tewsbury"), #restaurant.system < Float64(7), #restaurant.count = Int64(632) OR #restaurant.town = Utf8("reading"), #restaurant.time > TimestampNanosecond(130)] |
|
||||
| physical_plan | ProjectionExec: expr=[count@0 as count, system@1 as system, time@2 as time, town@3 as town] |
|
||||
| | CoalesceBatchesExec: target_batch_size=500 |
|
||||
| | FilterExec: 5 < system@1 AND CAST(town@3 AS Utf8) != tewsbury AND system@1 < 7 AND CAST(count@0 AS Int64) = 632 OR CAST(town@3 AS Utf8) = reading AND time@2 > 130 |
|
||||
| | RepartitionExec: partitioning=RoundRobinBatch(4) |
|
||||
| | IOxReadFilterNode: table_name=restaurant, chunks=0 predicate=Predicate exprs: [Float64(5) < #system, #town != Utf8("tewsbury"), #system < Float64(7), #time > TimestampNanosecond(130)] |
|
||||
| | |
|
||||
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| plan_type | plan |
|
||||
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
| logical_plan | Projection: #restaurant.count, #restaurant.system, #restaurant.time, #restaurant.town |
|
||||
| | Filter: Float64(5) < #restaurant.system AND #restaurant.town != Utf8("tewsbury") AND #restaurant.system < Float64(7) AND #restaurant.count = Int64(632) OR #restaurant.town = Utf8("reading") AND #restaurant.time > TimestampNanosecond(130) AS Float64(5) < restaurant.system AND restaurant.town != Utf8("tewsbury") AND restaurant.system < Float64(7) AND restaurant.count = Int64(632) OR restaurant.town = Utf8("reading") AND restaurant.time > totimestamp(Utf8("1970-01-01T00:00:00.000000130+00:00")) |
|
||||
| | TableScan: restaurant projection=Some([0, 1, 2, 3]), filters=[Float64(5) < #restaurant.system AND #restaurant.town != Utf8("tewsbury") AND #restaurant.system < Float64(7) AND #restaurant.count = Int64(632) OR #restaurant.town = Utf8("reading") AND #restaurant.time > TimestampNanosecond(130) AS Float64(5) < restaurant.system AND restaurant.town != Utf8("tewsbury") AND restaurant.system < Float64(7) AND restaurant.count = Int64(632) OR restaurant.town = Utf8("reading") AND restaurant.time > totimestamp(Utf8("1970-01-01T00:00:00.000000130+00:00"))] |
|
||||
| physical_plan | ProjectionExec: expr=[count@0 as count, system@1 as system, time@2 as time, town@3 as town] |
|
||||
| | CoalesceBatchesExec: target_batch_size=500 |
|
||||
| | FilterExec: 5 < system@1 AND CAST(town@3 AS Utf8) != tewsbury AND system@1 < 7 AND CAST(count@0 AS Int64) = 632 OR CAST(town@3 AS Utf8) = reading AND time@2 > 130 |
|
||||
| | RepartitionExec: partitioning=RoundRobinBatch(4) |
|
||||
| | IOxReadFilterNode: table_name=restaurant, chunks=0 predicate=Predicate exprs: [Float64(5) < #system, #town != Utf8("tewsbury"), #system < Float64(7), #time > TimestampNanosecond(130)] |
|
||||
| | |
|
||||
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
| | TableScan: h2o projection=Some([0]) |
|
||||
| physical_plan | ProjectionExec: expr=[COUNT(UInt8(1))@0 as COUNT(UInt8(1))] |
|
||||
| | RepartitionExec: partitioning=RoundRobinBatch(4) |
|
||||
| | ProjectionExec: expr=[3 as COUNT(Uint8(1))] |
|
||||
| | ProjectionExec: expr=[3 as COUNT(UInt8(1))] |
|
||||
| | EmptyExec: produce_one_row=true |
|
||||
| | |
|
||||
+---------------+-------------------------------------------------------------+
|
||||
|
|
|
@ -168,10 +168,10 @@ async fn test_read_filter_no_data_no_pred() {
|
|||
async fn test_read_filter_data_no_pred() {
|
||||
let predicate = EMPTY_PREDICATE;
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [70.4, 72.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [70.4, 72.4]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await;
|
||||
|
@ -181,10 +181,10 @@ async fn test_read_filter_data_no_pred() {
|
|||
async fn test_read_filter_data_no_pred_with_delete() {
|
||||
let predicate = EMPTY_PREDICATE;
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100], values: [70.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -200,8 +200,8 @@ async fn test_read_filter_data_no_pred_with_delete_all() {
|
|||
let predicate = EMPTY_PREDICATE;
|
||||
// nothing from h2o table because all rows were deleted
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -221,7 +221,7 @@ async fn test_read_filter_data_filter() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -278,7 +278,7 @@ async fn test_read_filter_data_filter_with_delete() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -299,7 +299,7 @@ async fn test_read_filter_data_filter_fields() {
|
|||
|
||||
// Only expect other_temp in this location
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=other_temp, _measurement=h2o, city=Boston, state=CA}\n FloatPoints timestamps: [350], values: [72.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=CA, _field=other_temp}\n FloatPoints timestamps: [350], values: [72.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsManyFields {}, predicate, expected_results).await;
|
||||
|
@ -317,7 +317,7 @@ async fn test_read_filter_data_filter_measurement_pred() {
|
|||
|
||||
// Only expect other_temp in this location
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=o2, state=CA}\n FloatPoints timestamps: [300], values: [79.0]",
|
||||
"Series tags={_measurement=o2, state=CA, _field=temp}\n FloatPoints timestamps: [300], values: [79.0]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsManyFields {}, predicate, expected_results).await;
|
||||
|
@ -353,8 +353,8 @@ async fn test_read_filter_data_pred_no_columns() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=user, _measurement=cpu, region=west}\n FloatPoints timestamps: [100, 150], values: [23.2, 21.0]",
|
||||
"Series tags={_field=bytes, _measurement=disk, region=east}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
"Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100, 150], values: [23.2, 21.0]",
|
||||
"Series tags={_measurement=disk, region=east, _field=bytes}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurements {}, predicate, expected_results).await;
|
||||
|
@ -368,8 +368,8 @@ async fn test_read_filter_data_pred_no_columns_with_delete() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=user, _measurement=cpu, region=west}\n FloatPoints timestamps: [100], values: [23.2]",
|
||||
"Series tags={_field=bytes, _measurement=disk, region=east}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
"Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100], values: [23.2]",
|
||||
"Series tags={_measurement=disk, region=east, _field=bytes}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsWithDelete {}, predicate, expected_results).await;
|
||||
|
@ -384,7 +384,7 @@ async fn test_read_filter_data_pred_no_columns_with_delete_all() {
|
|||
|
||||
// Only table disk has no deleted data
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bytes, _measurement=disk, region=east}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
"Series tags={_measurement=disk, region=east, _field=bytes}\n IntegerPoints timestamps: [200], values: [99]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsWithDeleteAll {}, predicate, expected_results).await;
|
||||
|
@ -424,7 +424,7 @@ async fn test_read_filter_data_pred_using_regex_match() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await;
|
||||
|
@ -455,7 +455,7 @@ async fn test_read_filter_data_pred_using_regex_match_with_delete() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
];
|
||||
run_read_filter_test_case(
|
||||
TwoMeasurementsMultiSeriesWithDelete {},
|
||||
|
@ -483,9 +483,9 @@ async fn test_read_filter_data_pred_using_regex_not_match() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [72.4]",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [53.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [72.4]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [250], values: [51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await;
|
||||
|
@ -505,9 +505,9 @@ async fn test_read_filter_data_pred_unsupported_in_scan() {
|
|||
|
||||
// Note these results include data from both o2 and h2o
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await;
|
||||
|
@ -527,9 +527,9 @@ async fn test_read_filter_data_pred_unsupported_in_scan_with_delete() {
|
|||
|
||||
// Note these results include data from both o2 and h2o
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -541,8 +541,8 @@ async fn test_read_filter_data_pred_unsupported_in_scan_with_delete() {
|
|||
|
||||
// With delete all from h2o, no rows from h2p should be returned
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
|
||||
];
|
||||
run_read_filter_test_case(
|
||||
TwoMeasurementsMultiSeriesWithDeleteAll {},
|
||||
|
@ -617,12 +617,12 @@ async fn test_read_filter_data_plan_order() {
|
|||
test_helpers::maybe_start_logging();
|
||||
let predicate = Predicate::default();
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=CA}\n FloatPoints timestamps: [250], values: [70.3]",
|
||||
"Series tags={_field=other, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [5.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [70.5]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA, zz_tag=A}\n FloatPoints timestamps: [1000], values: [70.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=A}\n FloatPoints timestamps: [800], values: [70.1]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=B}\n FloatPoints timestamps: [100], values: [70.2]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=CA, _field=temp}\n FloatPoints timestamps: [250], values: [70.3]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [70.5]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, zz_tag=A, _field=temp}\n FloatPoints timestamps: [1000], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=A, _field=temp}\n FloatPoints timestamps: [800], values: [70.1]",
|
||||
"Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=B, _field=temp}\n FloatPoints timestamps: [100], values: [70.2]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(MeasurementsSortableTags {}, predicate, expected_results).await;
|
||||
|
@ -633,11 +633,11 @@ async fn test_read_filter_data_plan_order_with_delete() {
|
|||
test_helpers::maybe_start_logging();
|
||||
let predicate = Predicate::default();
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=other, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [5.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [70.5]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA, zz_tag=A}\n FloatPoints timestamps: [1000], values: [70.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=A}\n FloatPoints timestamps: [800], values: [70.1]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=B}\n FloatPoints timestamps: [100], values: [70.2]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [70.5]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, zz_tag=A, _field=temp}\n FloatPoints timestamps: [1000], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=A, _field=temp}\n FloatPoints timestamps: [800], values: [70.1]",
|
||||
"Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=B, _field=temp}\n FloatPoints timestamps: [100], values: [70.2]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(
|
||||
|
@ -679,7 +679,7 @@ async fn test_read_filter_filter_on_value_2845() {
|
|||
.build();
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=load4, _measurement=system, host=host.local}\n FloatPoints timestamps: [1527018806000000000, 1527018826000000000], values: [1.77, 1.77]",
|
||||
"Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018806000000000, 1527018826000000000], values: [1.77, 1.77]",
|
||||
];
|
||||
|
||||
run_read_filter_test_case(MeasurementsForDefect2845 {}, predicate, expected_results).await;
|
||||
|
|
|
@ -142,8 +142,8 @@ async fn test_read_group_data_no_tag_columns() {
|
|||
let agg = Aggregate::Count;
|
||||
let group_columns = vec![];
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement partition_key_vals: ",
|
||||
"Series tags={_field=foo, _measurement=m0}\n IntegerPoints timestamps: [2], values: [2]",
|
||||
"Group tag_keys: _measurement, _field partition_key_vals: ",
|
||||
"Series tags={_measurement=m0, _field=foo}\n IntegerPoints timestamps: [2], values: [2]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -158,8 +158,8 @@ async fn test_read_group_data_no_tag_columns() {
|
|||
// min
|
||||
let agg = Aggregate::Min;
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement partition_key_vals: ",
|
||||
"Series tags={_field=foo, _measurement=m0}\n FloatPoints timestamps: [1], values: [1.0]",
|
||||
"Group tag_keys: _measurement, _field partition_key_vals: ",
|
||||
"Series tags={_measurement=m0, _field=foo}\n FloatPoints timestamps: [1], values: [1.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -179,8 +179,8 @@ async fn test_read_group_data_no_tag_columns_count_with_delete() {
|
|||
let agg = Aggregate::Count;
|
||||
let group_columns = vec![];
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement partition_key_vals: ",
|
||||
"Series tags={_field=foo, _measurement=m0}\n IntegerPoints timestamps: [2], values: [1]",
|
||||
"Group tag_keys: _measurement, _field partition_key_vals: ",
|
||||
"Series tags={_measurement=m0, _field=foo}\n IntegerPoints timestamps: [2], values: [1]",
|
||||
];
|
||||
run_read_group_test_case(
|
||||
OneMeasurementNoTagsWithDelete {},
|
||||
|
@ -199,8 +199,8 @@ async fn test_read_group_data_no_tag_columns_min_with_delete() {
|
|||
let agg = Aggregate::Min;
|
||||
let group_columns = vec![];
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement partition_key_vals: ",
|
||||
"Series tags={_field=foo, _measurement=m0}\n FloatPoints timestamps: [2], values: [2.0]",
|
||||
"Group tag_keys: _measurement, _field partition_key_vals: ",
|
||||
"Series tags={_measurement=m0, _field=foo}\n FloatPoints timestamps: [2], values: [2.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -277,8 +277,8 @@ async fn test_read_group_data_pred() {
|
|||
let agg = Aggregate::Sum;
|
||||
let group_columns = vec!["state"];
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: CA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: CA",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -300,10 +300,10 @@ async fn test_read_group_data_field_restriction() {
|
|||
let agg = Aggregate::Sum;
|
||||
let group_columns = vec!["state"];
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: CA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [180.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [142.8]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: CA",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [180.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [142.8]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -359,9 +359,9 @@ async fn test_grouped_series_set_plan_sum() {
|
|||
// The null field (after predicates) are not sent as series
|
||||
// Note order of city key (boston --> cambridge)
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [200], values: [163.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n FloatPoints timestamps: [200], values: [163.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -391,11 +391,11 @@ async fn test_grouped_series_set_plan_count() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=humidity, _measurement=h2o, city=Boston, state=MA}\n IntegerPoints timestamps: [400], values: [0]",
|
||||
"Series tags={_field=humidity, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [200], values: [0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n IntegerPoints timestamps: [400], values: [2]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=humidity}\n IntegerPoints timestamps: [400], values: [0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n IntegerPoints timestamps: [400], values: [2]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=humidity}\n IntegerPoints timestamps: [200], values: [0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -425,9 +425,9 @@ async fn test_grouped_series_set_plan_mean() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [400], values: [70.5]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [200], values: [81.5]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [400], values: [70.5]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n FloatPoints timestamps: [200], values: [81.5]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -476,10 +476,10 @@ async fn test_grouped_series_set_plan_count_measurement_pred() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: CA",
|
||||
"Series tags={_field=temp, _measurement=o2, city=LA, state=CA}\n IntegerPoints timestamps: [350], values: [2]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n IntegerPoints timestamps: [250], values: [2]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: CA",
|
||||
"Series tags={_measurement=o2, city=LA, state=CA, _field=temp}\n IntegerPoints timestamps: [350], values: [2]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n IntegerPoints timestamps: [250], values: [2]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -522,11 +522,11 @@ async fn test_grouped_series_set_plan_first() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=b, _measurement=h2o, city=Cambridge, state=MA}\n BooleanPoints timestamps: [2000], values: [true]",
|
||||
"Series tags={_field=f, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [2000], values: [7.0]",
|
||||
"Series tags={_field=i, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [2000], values: [7]",
|
||||
"Series tags={_field=s, _measurement=h2o, city=Cambridge, state=MA}\n StringPoints timestamps: [2000], values: [\"c\"]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=b}\n BooleanPoints timestamps: [2000], values: [true]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=f}\n FloatPoints timestamps: [2000], values: [7.0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=i}\n IntegerPoints timestamps: [2000], values: [7]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=s}\n StringPoints timestamps: [2000], values: [\"c\"]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -556,10 +556,10 @@ async fn test_grouped_series_set_plan_first_with_nulls() {
|
|||
|
||||
// expect timestamps to be present for all three series
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=moisture, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100000], values: [43.0]",
|
||||
"Series tags={_field=other_temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [70.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [50], values: [70.4]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=moisture}\n FloatPoints timestamps: [100000], values: [43.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other_temp}\n FloatPoints timestamps: [250], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [50], values: [70.4]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -583,11 +583,11 @@ async fn test_grouped_series_set_plan_last() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=b, _measurement=h2o, city=Cambridge, state=MA}\n BooleanPoints timestamps: [3000], values: [false]",
|
||||
"Series tags={_field=f, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [3000], values: [6.0]",
|
||||
"Series tags={_field=i, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [3000], values: [6]",
|
||||
"Series tags={_field=s, _measurement=h2o, city=Cambridge, state=MA}\n StringPoints timestamps: [3000], values: [\"b\"]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=b}\n BooleanPoints timestamps: [3000], values: [false]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=f}\n FloatPoints timestamps: [3000], values: [6.0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=i}\n IntegerPoints timestamps: [3000], values: [6]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=s}\n StringPoints timestamps: [3000], values: [\"b\"]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -617,10 +617,10 @@ async fn test_grouped_series_set_plan_last_with_nulls() {
|
|||
|
||||
// expect timestamps to be present for all three series
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=moisture, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100000], values: [43.0]",
|
||||
"Series tags={_field=other_temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [70.4]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100000], values: [70.4]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=moisture}\n FloatPoints timestamps: [100000], values: [43.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other_temp}\n FloatPoints timestamps: [250], values: [70.4]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100000], values: [70.4]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -665,11 +665,11 @@ async fn test_grouped_series_set_plan_min() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=b, _measurement=h2o, city=Cambridge, state=MA}\n BooleanPoints timestamps: [1000], values: [false]",
|
||||
"Series tags={_field=f, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [3000], values: [6.0]",
|
||||
"Series tags={_field=i, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [3000], values: [6]",
|
||||
"Series tags={_field=s, _measurement=h2o, city=Cambridge, state=MA}\n StringPoints timestamps: [2000], values: [\"a\"]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=b}\n BooleanPoints timestamps: [1000], values: [false]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=f}\n FloatPoints timestamps: [3000], values: [6.0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=i}\n IntegerPoints timestamps: [3000], values: [6]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=s}\n StringPoints timestamps: [2000], values: [\"a\"]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -712,11 +712,11 @@ async fn test_grouped_series_set_plan_max() {
|
|||
let group_columns = vec!["state"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA",
|
||||
"Series tags={_field=b, _measurement=h2o, city=Cambridge, state=MA}\n BooleanPoints timestamps: [3000], values: [true]",
|
||||
"Series tags={_field=f, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [2000], values: [7.0]",
|
||||
"Series tags={_field=i, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [2000], values: [7]",
|
||||
"Series tags={_field=s, _measurement=h2o, city=Cambridge, state=MA}\n StringPoints timestamps: [4000], values: [\"z\"]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=b}\n BooleanPoints timestamps: [3000], values: [true]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=f}\n FloatPoints timestamps: [2000], values: [7.0]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=i}\n IntegerPoints timestamps: [2000], values: [7]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=s}\n StringPoints timestamps: [4000], values: [\"z\"]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -762,13 +762,13 @@ async fn test_grouped_series_set_plan_group_by_state_city() {
|
|||
let group_columns = vec!["state", "city"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: CA, LA",
|
||||
"Series tags={_field=humidity, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [600], values: [21.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [600], values: [181.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA, Boston",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA, Cambridge",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [200], values: [243.0]"
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: CA, LA",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=humidity}\n FloatPoints timestamps: [600], values: [21.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [600], values: [181.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA, Boston",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA, Cambridge",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n FloatPoints timestamps: [200], values: [243.0]"
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -791,13 +791,13 @@ async fn test_grouped_series_set_plan_group_by_city_state() {
|
|||
|
||||
// Test with alternate group key order (note the order of columns is different)
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: Boston, MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: Cambridge, MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [200], values: [243.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: LA, CA",
|
||||
"Series tags={_field=humidity, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [600], values: [21.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [600], values: [181.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: Boston, MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [400], values: [141.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: Cambridge, MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n FloatPoints timestamps: [200], values: [243.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: LA, CA",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=humidity}\n FloatPoints timestamps: [600], values: [21.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [600], values: [181.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -820,13 +820,13 @@ async fn test_grouped_series_set_plan_group_aggregate_none() {
|
|||
|
||||
// Expect order of the columns to begin with city/state
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: Boston, MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [300, 400], values: [70.0, 71.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: Cambridge, MA",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n FloatPoints timestamps: [50, 100, 200], values: [80.0, 81.0, 82.0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: LA, CA",
|
||||
"Series tags={_field=humidity, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [500, 600], values: [10.0, 11.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [500, 600], values: [90.0, 91.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: Boston, MA",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [300, 400], values: [70.0, 71.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: Cambridge, MA",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n FloatPoints timestamps: [50, 100, 200], values: [80.0, 81.0, 82.0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: LA, CA",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=humidity}\n FloatPoints timestamps: [500, 600], values: [10.0, 11.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [500, 600], values: [90.0, 91.0]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -872,16 +872,16 @@ async fn test_grouped_series_set_plan_group_by_field_none() {
|
|||
// Expect the data is grouped so all the distinct values of load1
|
||||
// are before the values for load2
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load1",
|
||||
"Series tags={_field=load1, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_field=load1, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load2",
|
||||
"Series tags={_field=load2, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_field=load2, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load1",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load1}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load1}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load2",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -905,20 +905,20 @@ async fn test_grouped_series_set_plan_group_by_field_and_tag_none() {
|
|||
// Expect the data is grouped so all the distinct values of load1
|
||||
// are before the values for load2, grouped by region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load1, A",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load1, B",
|
||||
"Series tags={_field=load1, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load1, C",
|
||||
"Series tags={_field=load1, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load2, A",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load2, B",
|
||||
"Series tags={_field=load2, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: load2, C",
|
||||
"Series tags={_field=load2, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load1, A",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load1}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load1, B",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load1}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load1, C",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load2, A",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load2, B",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: load2, C",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -942,20 +942,20 @@ async fn test_grouped_series_set_plan_group_by_tag_and_field_none() {
|
|||
let group_columns = vec!["region", "_field"];
|
||||
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: A, load1",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: A, load2",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=A}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: B, load1",
|
||||
"Series tags={_field=load1, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: B, load2",
|
||||
"Series tags={_field=load2, _measurement=system, host=remote, region=B}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: C, load1",
|
||||
"Series tags={_field=load1, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: C, load2",
|
||||
"Series tags={_field=load2, _measurement=aa_system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=C}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: A, load1",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load1}\n FloatPoints timestamps: [100, 200], values: [1.1, 1.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: A, load2",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 2.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: B, load1",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load1}\n FloatPoints timestamps: [100, 200], values: [10.1, 10.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: B, load2",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load2}\n FloatPoints timestamps: [100, 200], values: [2.1, 20.2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: C, load1",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load1}\n FloatPoints timestamps: [100], values: [100.1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: C, load2",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load2}\n FloatPoints timestamps: [100], values: [200.1]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -978,18 +978,18 @@ async fn test_grouped_series_set_plan_group_measurement_tag_count() {
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement and then region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: aa_system, C",
|
||||
"Series tags={_field=load1, _measurement=aa_system, host=local, region=C}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Series tags={_field=load2, _measurement=aa_system, host=local, region=C}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: system, A",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=A}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=A}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: system, B",
|
||||
"Series tags={_field=load1, _measurement=system, host=remote, region=B}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Series tags={_field=load2, _measurement=system, host=remote, region=B}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Group tag_keys: _field, _measurement, host, region partition_key_vals: system, C",
|
||||
"Series tags={_field=load1, _measurement=system, host=local, region=C}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Series tags={_field=load2, _measurement=system, host=local, region=C}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: aa_system, C",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load1}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Series tags={_measurement=aa_system, host=local, region=C, _field=load2}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: system, A",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load1}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Series tags={_measurement=system, host=local, region=A, _field=load2}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: system, B",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load1}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Series tags={_measurement=system, host=remote, region=B, _field=load2}\n IntegerPoints timestamps: [200], values: [2]",
|
||||
"Group tag_keys: _measurement, host, region, _field partition_key_vals: system, C",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load1}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
"Series tags={_measurement=system, host=local, region=C, _field=load2}\n IntegerPoints timestamps: [100], values: [1]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -1012,12 +1012,12 @@ async fn test_grouped_series_set_plan_group_field_start_stop() {
|
|||
// Expect the data is grouped so output is sorted by state, with
|
||||
// blank partition values for _start and _stop (mirroring TSM)
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, state partition_key_vals: , , CA",
|
||||
"Series tags={_field=reading, _measurement=o2, state=CA}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Series tags={_field=temp, _measurement=o2, state=CA}\n IntegerPoints timestamps: [300], values: [1]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: , , MA",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Group tag_keys: _measurement, state, _field partition_key_vals: , , CA",
|
||||
"Series tags={_measurement=o2, state=CA, _field=reading}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Series tags={_measurement=o2, state=CA, _field=temp}\n IntegerPoints timestamps: [300], values: [1]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: , , MA",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
];
|
||||
|
||||
let group_columns = vec!["_start", "_stop", "state"];
|
||||
|
@ -1053,14 +1053,14 @@ async fn test_grouped_series_set_plan_group_field_pred_and_null_fields() {
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement state
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, state partition_key_vals: CA, reading",
|
||||
"Series tags={_field=reading, _measurement=o2, state=CA}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Group tag_keys: _field, _measurement, state partition_key_vals: CA, temp",
|
||||
"Series tags={_field=temp, _measurement=o2, state=CA}\n IntegerPoints timestamps: [300], values: [1]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA, reading",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA, temp",
|
||||
"Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Group tag_keys: _measurement, state, _field partition_key_vals: CA, reading",
|
||||
"Series tags={_measurement=o2, state=CA, _field=reading}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Group tag_keys: _measurement, state, _field partition_key_vals: CA, temp",
|
||||
"Series tags={_measurement=o2, state=CA, _field=temp}\n IntegerPoints timestamps: [300], values: [1]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA, reading",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA, temp",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -1090,10 +1090,10 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_field() {
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement and then region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, state partition_key_vals: CA, reading",
|
||||
"Series tags={_field=reading, _measurement=o2, state=CA}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Group tag_keys: _field, _measurement, city, state partition_key_vals: MA, reading",
|
||||
"Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
"Group tag_keys: _measurement, state, _field partition_key_vals: CA, reading",
|
||||
"Series tags={_measurement=o2, state=CA, _field=reading}\n IntegerPoints timestamps: [300], values: [0]",
|
||||
"Group tag_keys: _measurement, city, state, _field partition_key_vals: MA, reading",
|
||||
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n IntegerPoints timestamps: [50], values: [1]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -1145,8 +1145,8 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_value() {
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement and then region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host partition_key_vals: load4",
|
||||
"Series tags={_field=load4, _measurement=system, host=host.local}\n FloatPoints timestamps: [1527018806000000000], values: [1.77]",
|
||||
"Group tag_keys: _measurement, host, _field partition_key_vals: load4",
|
||||
"Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018806000000000], values: [1.77]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -1177,10 +1177,10 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_multiple_value(
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement and then region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host partition_key_vals: load3",
|
||||
"Series tags={_field=load3, _measurement=system, host=host.local}\n FloatPoints timestamps: [1527018806000000000], values: [1.72]",
|
||||
"Group tag_keys: _field, _measurement, host partition_key_vals: load4",
|
||||
"Series tags={_field=load4, _measurement=system, host=host.local}\n FloatPoints timestamps: [1527018806000000000], values: [1.77]",
|
||||
"Group tag_keys: _measurement, host, _field partition_key_vals: load3",
|
||||
"Series tags={_measurement=system, host=host.local, _field=load3}\n FloatPoints timestamps: [1527018806000000000], values: [1.72]",
|
||||
"Group tag_keys: _measurement, host, _field partition_key_vals: load4",
|
||||
"Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018806000000000], values: [1.77]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
@ -1207,8 +1207,8 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_value_sum() {
|
|||
|
||||
// Expect the data is grouped so output is sorted by measurement and then region
|
||||
let expected_results = vec![
|
||||
"Group tag_keys: _field, _measurement, host partition_key_vals: load4",
|
||||
"Series tags={_field=load4, _measurement=system, host=host.local}\n FloatPoints timestamps: [1527018826000000000], values: [3.54]",
|
||||
"Group tag_keys: _measurement, host, _field partition_key_vals: load4",
|
||||
"Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018826000000000], values: [3.54]",
|
||||
];
|
||||
|
||||
run_read_group_test_case(
|
||||
|
|
|
@ -116,8 +116,8 @@ async fn test_read_window_aggregate_nanoseconds() {
|
|||
|
||||
// note the name of the field is "temp" even though it is the average
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [200, 400, 600], values: [70.0, 71.5, 73.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200, 400, 600], values: [90.0, 91.5, 93.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [200, 400, 600], values: [70.0, 71.5, 73.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 400, 600], values: [90.0, 91.5, 93.0]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -148,8 +148,8 @@ async fn test_read_window_aggregate_nanoseconds_measurement_pred() {
|
|||
let offset = WindowDuration::from_nanoseconds(0);
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [200, 400, 600], values: [70.0, 71.5, 73.0]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [200, 400, 600], values: [90.0, 91.5, 93.0]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [200, 400, 600], values: [70.0, 71.5, 73.0]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 400, 600], values: [90.0, 91.5, 93.0]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -175,9 +175,9 @@ async fn test_read_window_aggregate_nanoseconds_measurement_count() {
|
|||
let offset = WindowDuration::from_nanoseconds(0);
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Cambridge, state=MA}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
"Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
"Series tags={_measurement=h2o, city=Cambridge, state=MA, _field=temp}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n IntegerPoints timestamps: [200, 400, 600], values: [1, 2, 1]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -257,7 +257,7 @@ async fn test_read_window_aggregate_months() {
|
|||
|
||||
// note the name of the field is "temp" even though it is the average
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [1585699200000000000, 1588291200000000000], values: [70.5, 72.5]",
|
||||
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [1585699200000000000, 1588291200000000000], values: [70.5, 72.5]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -376,10 +376,10 @@ async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697() {
|
|||
// MAX, FIRST, LAST) we need to run a plan that brings along the timestamps
|
||||
// for the chosen aggregate in the window.
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000011], values: [5.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000001, 1609459201000000024], values: [1.0, 11.24]",
|
||||
"Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000009, 1609459201000000015, 1609459201000000022], values: [4.0, 6.0, 1.2]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000002], values: [2.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=bar}\n FloatPoints timestamps: [1609459201000000011], values: [5.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=foo}\n FloatPoints timestamps: [1609459201000000001, 1609459201000000024], values: [1.0, 11.24]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=bar}\n FloatPoints timestamps: [1609459201000000009, 1609459201000000015, 1609459201000000022], values: [4.0, 6.0, 1.2]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=foo}\n FloatPoints timestamps: [1609459201000000002], values: [2.0]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -406,10 +406,10 @@ async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697_with_delet
|
|||
|
||||
// one row deleted
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000011], values: [5.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000001, 1609459201000000024], values: [1.0, 11.24]",
|
||||
"Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000009, 1609459201000000015], values: [4.0, 6.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000002], values: [2.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=bar}\n FloatPoints timestamps: [1609459201000000011], values: [5.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=foo}\n FloatPoints timestamps: [1609459201000000001, 1609459201000000024], values: [1.0, 11.24]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=bar}\n FloatPoints timestamps: [1609459201000000009, 1609459201000000015], values: [4.0, 6.0]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=foo}\n FloatPoints timestamps: [1609459201000000002], values: [2.0]",
|
||||
];
|
||||
run_read_window_aggregate_test_case(
|
||||
MeasurementForDefect2697WithDelete {},
|
||||
|
@ -449,10 +449,10 @@ async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697() {
|
|||
// The windowed aggregate is using a non-selector aggregate (SUM, COUNT, MEAD).
|
||||
// For each distinct series the window defines the `time` column
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000020], values: [5.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000020, 1609459201000000030], values: [4.0, 6.0, 1.2]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=bar}\n FloatPoints timestamps: [1609459201000000020], values: [5.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=foo}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=bar}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000020, 1609459201000000030], values: [4.0, 6.0, 1.2]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=foo}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -484,8 +484,8 @@ async fn test_grouped_series_set_plan_group_aggregate_filter_on_field() {
|
|||
// The windowed aggregate is using a non-selector aggregate (SUM, COUNT, MEAD).
|
||||
// For each distinct series the window defines the `time` column
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=foo}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=foo}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
];
|
||||
|
||||
run_read_window_aggregate_test_case(
|
||||
|
@ -515,10 +515,10 @@ async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697_with_delet
|
|||
// The windowed aggregate is using a non-selector aggregate (SUM, COUNT, MEAD).
|
||||
// For each distinct series the window defines the `time` column
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000020], values: [5.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000020], values: [4.0, 6.0]",
|
||||
"Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=bar}\n FloatPoints timestamps: [1609459201000000020], values: [5.0]",
|
||||
"Series tags={_measurement=mm, section=1a, _field=foo}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=bar}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000020], values: [4.0, 6.0]",
|
||||
"Series tags={_measurement=mm, section=2b, _field=foo}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]",
|
||||
];
|
||||
run_read_window_aggregate_test_case(
|
||||
MeasurementForDefect2697WithDelete {},
|
||||
|
@ -579,8 +579,8 @@ async fn test_read_window_aggregate_overflow() {
|
|||
let offset = WindowDuration::from_nanoseconds(0);
|
||||
|
||||
let expected_results = vec![
|
||||
"Series tags={_field=bar, _measurement=mm}\n FloatPoints timestamps: [1609459201000000015], values: [6.0]",
|
||||
"Series tags={_field=foo, _measurement=mm}\n FloatPoints timestamps: [1609459201000000005], values: [3.0]",
|
||||
"Series tags={_measurement=mm, _field=bar}\n FloatPoints timestamps: [1609459201000000015], values: [6.0]",
|
||||
"Series tags={_measurement=mm, _field=foo}\n FloatPoints timestamps: [1609459201000000005], values: [3.0]",
|
||||
];
|
||||
run_read_window_aggregate_test_case(
|
||||
MeasurementForDefect2890 {},
|
||||
|
|
|
@ -46,7 +46,7 @@ impl ConnectionPool {
|
|||
/// Create new connection pool.
|
||||
///
|
||||
/// If `use_mock_grpc` is set only mock gRPC clients are created.
|
||||
pub async fn new(use_mock_grpc: bool, wb_factory: WriteBufferConfigFactory) -> Self {
|
||||
pub async fn new(use_mock_grpc: bool, wb_factory: Arc<WriteBufferConfigFactory>) -> Self {
|
||||
// Note: this function is async even though it does not contain any `.await` calls because `LoadingCache::new`
|
||||
// requires tokio to be running and even if documented people will forget about this.
|
||||
|
||||
|
@ -69,7 +69,6 @@ impl ConnectionPool {
|
|||
})
|
||||
};
|
||||
|
||||
let wb_factory = Arc::new(wb_factory);
|
||||
let write_buffer_producers = LoadingCache::new(move |key: KeyWriteBufferProducer| {
|
||||
let wb_factory = Arc::clone(&wb_factory);
|
||||
async move {
|
||||
|
@ -95,7 +94,10 @@ impl ConnectionPool {
|
|||
let metric_registry = Arc::new(metric::Registry::new());
|
||||
Self::new(
|
||||
true,
|
||||
WriteBufferConfigFactory::new(time_provider, metric_registry),
|
||||
Arc::new(WriteBufferConfigFactory::new(
|
||||
time_provider,
|
||||
metric_registry,
|
||||
)),
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
@ -145,7 +147,10 @@ mod tests {
|
|||
|
||||
let pool1 = ConnectionPool::new(
|
||||
false,
|
||||
WriteBufferConfigFactory::new(Arc::clone(&time_provider), Arc::clone(&metric_registry)),
|
||||
Arc::new(WriteBufferConfigFactory::new(
|
||||
Arc::clone(&time_provider),
|
||||
Arc::clone(&metric_registry),
|
||||
)),
|
||||
)
|
||||
.await;
|
||||
// connection will fail
|
||||
|
@ -153,7 +158,10 @@ mod tests {
|
|||
|
||||
let pool2 = ConnectionPool::new(
|
||||
true,
|
||||
WriteBufferConfigFactory::new(time_provider, metric_registry),
|
||||
Arc::new(WriteBufferConfigFactory::new(
|
||||
time_provider,
|
||||
metric_registry,
|
||||
)),
|
||||
)
|
||||
.await;
|
||||
let client2 = pool2.grpc_client("foo").await.unwrap();
|
||||
|
|
|
@ -26,6 +26,9 @@ pub trait GrpcClient: Sync + Send + std::fmt::Debug + 'static {
|
|||
/// A real, network-driven gRPC client.
|
||||
#[derive(Debug)]
|
||||
pub struct RealClient {
|
||||
/// Delete client for IOx.
|
||||
delete_client: influxdb_iox_client::delete::Client,
|
||||
|
||||
/// Write client for IOx.
|
||||
write_client: influxdb_iox_client::write::Client,
|
||||
}
|
||||
|
@ -34,6 +37,7 @@ impl RealClient {
|
|||
/// Create new client from established connection.
|
||||
pub fn new(connection: influxdb_iox_client::connection::Connection) -> Self {
|
||||
Self {
|
||||
delete_client: influxdb_iox_client::delete::Client::new(connection.clone()),
|
||||
write_client: influxdb_iox_client::write::Client::new(connection),
|
||||
}
|
||||
}
|
||||
|
@ -59,7 +63,24 @@ impl GrpcClient for RealClient {
|
|||
.await
|
||||
.map_err(|e| Box::new(e) as _)
|
||||
}
|
||||
DmlOperation::Delete(_) => unimplemented!(),
|
||||
DmlOperation::Delete(delete) => {
|
||||
// cheap, see https://docs.rs/tonic/0.4.2/tonic/client/index.html#concurrent-usage
|
||||
let mut client = self.delete_client.clone();
|
||||
|
||||
client
|
||||
.delete(
|
||||
db_name.to_owned(),
|
||||
delete
|
||||
.table_name()
|
||||
.map(|s| s.to_owned())
|
||||
.unwrap_or_default(),
|
||||
delete.predicate().range.start.to_string(),
|
||||
delete.predicate().range.end.to_string(),
|
||||
delete.predicate().expr_sql_string(),
|
||||
)
|
||||
.await
|
||||
.map_err(|e| Box::new(e) as _)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -118,6 +118,7 @@ impl Router {
|
|||
}
|
||||
}
|
||||
|
||||
/// Write operation to the specified shard.
|
||||
async fn write_shard(
|
||||
&self,
|
||||
shard_id: ShardId,
|
||||
|
@ -137,14 +138,17 @@ mod tests {
|
|||
use super::*;
|
||||
|
||||
use data_types::{
|
||||
delete_predicate::DeletePredicate,
|
||||
non_empty::NonEmptyString,
|
||||
router::{
|
||||
Matcher, MatcherToShard, ShardConfig, WriteSink as WriteSinkConfig,
|
||||
WriteSinkSet as WriteSinkSetConfig, WriteSinkVariant as WriteSinkVariantConfig,
|
||||
},
|
||||
sequence::Sequence,
|
||||
server_id::ServerId,
|
||||
timestamp::TimestampRange,
|
||||
};
|
||||
use dml::{DmlMeta, DmlWrite};
|
||||
use dml::{DmlDelete, DmlMeta, DmlWrite};
|
||||
use mutable_batch_lp::lines_to_batches;
|
||||
use regex::Regex;
|
||||
use time::Time;
|
||||
|
@ -344,6 +348,82 @@ mod tests {
|
|||
)]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_delete() {
|
||||
let server_id_1 = ServerId::try_from(1).unwrap();
|
||||
let server_id_2 = ServerId::try_from(2).unwrap();
|
||||
|
||||
let resolver = Arc::new(Resolver::new(Some(RemoteTemplate::new("{id}"))));
|
||||
let connection_pool = Arc::new(ConnectionPool::new_testing().await);
|
||||
|
||||
let client_1 = connection_pool.grpc_client("1").await.unwrap();
|
||||
let client_2 = connection_pool.grpc_client("2").await.unwrap();
|
||||
let client_1 = client_1.as_any().downcast_ref::<MockClient>().unwrap();
|
||||
let client_2 = client_2.as_any().downcast_ref::<MockClient>().unwrap();
|
||||
|
||||
let cfg = RouterConfig {
|
||||
name: String::from("my_router"),
|
||||
write_sharder: ShardConfig {
|
||||
specific_targets: vec![
|
||||
MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: Some(Regex::new("foo_bar").unwrap()),
|
||||
},
|
||||
shard: ShardId::new(10),
|
||||
},
|
||||
MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: Some(Regex::new("foo_.*").unwrap()),
|
||||
},
|
||||
shard: ShardId::new(20),
|
||||
},
|
||||
],
|
||||
hash_ring: None,
|
||||
},
|
||||
write_sinks: BTreeMap::from([
|
||||
(
|
||||
ShardId::new(10),
|
||||
WriteSinkSetConfig {
|
||||
sinks: vec![WriteSinkConfig {
|
||||
sink: WriteSinkVariantConfig::GrpcRemote(server_id_1),
|
||||
ignore_errors: false,
|
||||
}],
|
||||
},
|
||||
),
|
||||
(
|
||||
ShardId::new(20),
|
||||
WriteSinkSetConfig {
|
||||
sinks: vec![WriteSinkConfig {
|
||||
sink: WriteSinkVariantConfig::GrpcRemote(server_id_2),
|
||||
ignore_errors: false,
|
||||
}],
|
||||
},
|
||||
),
|
||||
]),
|
||||
query_sinks: Default::default(),
|
||||
};
|
||||
let router = Router::new(cfg.clone(), resolver, connection_pool);
|
||||
|
||||
// clean write
|
||||
let meta = DmlMeta::sequenced(
|
||||
Sequence::new(1, 2),
|
||||
Time::from_timestamp_nanos(1337),
|
||||
None,
|
||||
10,
|
||||
);
|
||||
let delete = DmlOperation::Delete(DmlDelete::new(
|
||||
DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![],
|
||||
},
|
||||
Some(NonEmptyString::new("foo_foo").unwrap()),
|
||||
meta,
|
||||
));
|
||||
router.write(delete.clone()).await.unwrap();
|
||||
client_1.assert_writes(&[]);
|
||||
client_2.assert_writes(&[(String::from("my_router"), delete)]);
|
||||
}
|
||||
|
||||
fn db_write(lines: &[&str], meta: &DmlMeta) -> DmlOperation {
|
||||
DmlOperation::Write(DmlWrite::new(
|
||||
lines_to_batches(&lines.join("\n"), 0).unwrap(),
|
||||
|
|
|
@ -38,31 +38,40 @@ impl RouterServer {
|
|||
trace_collector: Option<Arc<dyn TraceCollector>>,
|
||||
time_provider: Arc<dyn TimeProvider>,
|
||||
) -> Self {
|
||||
Self::new_inner(remote_template, trace_collector, time_provider, false).await
|
||||
Self::new_inner(remote_template, trace_collector, time_provider, None, false).await
|
||||
}
|
||||
|
||||
pub async fn for_testing(
|
||||
remote_template: Option<RemoteTemplate>,
|
||||
trace_collector: Option<Arc<dyn TraceCollector>>,
|
||||
time_provider: Arc<dyn TimeProvider>,
|
||||
wb_factory: Option<Arc<WriteBufferConfigFactory>>,
|
||||
) -> Self {
|
||||
Self::new_inner(remote_template, trace_collector, time_provider, true).await
|
||||
Self::new_inner(
|
||||
remote_template,
|
||||
trace_collector,
|
||||
time_provider,
|
||||
wb_factory,
|
||||
true,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
async fn new_inner(
|
||||
remote_template: Option<RemoteTemplate>,
|
||||
trace_collector: Option<Arc<dyn TraceCollector>>,
|
||||
time_provider: Arc<dyn TimeProvider>,
|
||||
wb_factory: Option<Arc<WriteBufferConfigFactory>>,
|
||||
use_mock_grpc: bool,
|
||||
) -> Self {
|
||||
let metric_registry = Arc::new(metric::Registry::new());
|
||||
let connection_pool = Arc::new(
|
||||
ConnectionPool::new(
|
||||
use_mock_grpc,
|
||||
WriteBufferConfigFactory::new(time_provider, Arc::clone(&metric_registry)),
|
||||
)
|
||||
.await,
|
||||
);
|
||||
let wb_factory = wb_factory.unwrap_or_else(|| {
|
||||
Arc::new(WriteBufferConfigFactory::new(
|
||||
time_provider,
|
||||
Arc::clone(&metric_registry),
|
||||
))
|
||||
});
|
||||
let connection_pool = Arc::new(ConnectionPool::new(use_mock_grpc, wb_factory).await);
|
||||
|
||||
Self {
|
||||
server_id: RwLock::new(None),
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
use std::collections::BTreeMap;
|
||||
use std::collections::{BTreeMap, HashSet};
|
||||
|
||||
use data_types::router::{ShardConfig, ShardId};
|
||||
use dml::{DmlOperation, DmlWrite};
|
||||
use dml::{DmlDelete, DmlOperation, DmlWrite};
|
||||
use hashbrown::HashMap;
|
||||
use mutable_batch::MutableBatch;
|
||||
|
||||
|
@ -9,40 +9,25 @@ use mutable_batch::MutableBatch;
|
|||
pub fn shard_operation(
|
||||
operation: DmlOperation,
|
||||
config: &ShardConfig,
|
||||
) -> impl Iterator<Item = (ShardId, DmlOperation)> {
|
||||
) -> BTreeMap<ShardId, DmlOperation> {
|
||||
match operation {
|
||||
DmlOperation::Write(write) => shard_write(&write, config)
|
||||
.into_iter()
|
||||
.map(|(shard, write)| (shard, DmlOperation::Write(write))),
|
||||
DmlOperation::Delete(_) => unimplemented!(),
|
||||
.map(|(shard, write)| (shard, DmlOperation::Write(write)))
|
||||
.collect(),
|
||||
DmlOperation::Delete(delete) => shard_delete(&delete, config)
|
||||
.into_iter()
|
||||
.map(|(shard, delete)| (shard, DmlOperation::Delete(delete)))
|
||||
.collect(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Shard given write according to config.
|
||||
pub fn shard_write(write: &DmlWrite, config: &ShardConfig) -> BTreeMap<ShardId, DmlWrite> {
|
||||
fn shard_write(write: &DmlWrite, config: &ShardConfig) -> BTreeMap<ShardId, DmlWrite> {
|
||||
let mut batches: HashMap<ShardId, HashMap<String, MutableBatch>> = HashMap::new();
|
||||
|
||||
for (table, batch) in write.tables() {
|
||||
let mut shard_id = None;
|
||||
|
||||
'match_loop: for matcher2shard in &config.specific_targets {
|
||||
if let Some(regex) = &matcher2shard.matcher.table_name_regex {
|
||||
if regex.is_match(table) {
|
||||
shard_id = Some(matcher2shard.shard);
|
||||
break 'match_loop;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if shard_id.is_none() {
|
||||
if let Some(hash_ring) = &config.hash_ring {
|
||||
if let Some(id) = hash_ring.shards.find(table) {
|
||||
shard_id = Some(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(shard_id) = shard_id {
|
||||
if let Some(shard_id) = shard_table(table, config) {
|
||||
assert!(batches
|
||||
.entry(shard_id)
|
||||
.or_default()
|
||||
|
@ -57,20 +42,74 @@ pub fn shard_write(write: &DmlWrite, config: &ShardConfig) -> BTreeMap<ShardId,
|
|||
.collect()
|
||||
}
|
||||
|
||||
/// Shard given delete according to config.
|
||||
fn shard_delete(delete: &DmlDelete, config: &ShardConfig) -> BTreeMap<ShardId, DmlDelete> {
|
||||
if let Some(table) = delete.table_name() {
|
||||
if let Some(shard_id) = shard_table(table, config) {
|
||||
BTreeMap::from([(shard_id, delete.clone())])
|
||||
} else {
|
||||
BTreeMap::default()
|
||||
}
|
||||
} else {
|
||||
let shards: HashSet<ShardId> = config
|
||||
.specific_targets
|
||||
.iter()
|
||||
.map(|matcher2shard| matcher2shard.shard)
|
||||
.chain(
|
||||
config
|
||||
.hash_ring
|
||||
.iter()
|
||||
.map(|hashring| Vec::<ShardId>::from(hashring.shards.clone()).into_iter())
|
||||
.flatten(),
|
||||
)
|
||||
.collect();
|
||||
|
||||
shards
|
||||
.into_iter()
|
||||
.map(|shard| (shard, delete.clone()))
|
||||
.collect()
|
||||
}
|
||||
}
|
||||
|
||||
/// Shard only based on table name
|
||||
fn shard_table(table: &str, config: &ShardConfig) -> Option<ShardId> {
|
||||
for matcher2shard in &config.specific_targets {
|
||||
if let Some(regex) = &matcher2shard.matcher.table_name_regex {
|
||||
if regex.is_match(table) {
|
||||
return Some(matcher2shard.shard);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(hash_ring) = &config.hash_ring {
|
||||
if let Some(id) = hash_ring.shards.find(table) {
|
||||
return Some(id);
|
||||
}
|
||||
}
|
||||
|
||||
None
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use data_types::{
|
||||
consistent_hasher::ConsistentHasher,
|
||||
delete_predicate::DeletePredicate,
|
||||
non_empty::NonEmptyString,
|
||||
router::{HashRing, Matcher, MatcherToShard},
|
||||
timestamp::TimestampRange,
|
||||
};
|
||||
use dml::{
|
||||
test_util::{assert_deletes_eq, assert_writes_eq},
|
||||
DmlMeta,
|
||||
};
|
||||
use dml::{test_util::assert_writes_eq, DmlMeta};
|
||||
use mutable_batch_lp::lines_to_batches;
|
||||
use regex::Regex;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_sharding() {
|
||||
fn test_write_sharding() {
|
||||
let config = ShardConfig {
|
||||
specific_targets: vec![
|
||||
MatcherToShard {
|
||||
|
@ -151,7 +190,7 @@ mod tests {
|
|||
}
|
||||
|
||||
#[test]
|
||||
fn test_no_mach() {
|
||||
fn test_write_no_match() {
|
||||
let config = ShardConfig::default();
|
||||
|
||||
let meta = DmlMeta::default();
|
||||
|
@ -161,10 +200,108 @@ mod tests {
|
|||
assert!(actual.is_empty());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_delete_sharding() {
|
||||
let config = ShardConfig {
|
||||
specific_targets: vec![
|
||||
MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: None,
|
||||
},
|
||||
shard: ShardId::new(1),
|
||||
},
|
||||
MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: Some(Regex::new("some_foo").unwrap()),
|
||||
},
|
||||
shard: ShardId::new(2),
|
||||
},
|
||||
MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: Some(Regex::new("some_.*").unwrap()),
|
||||
},
|
||||
shard: ShardId::new(3),
|
||||
},
|
||||
],
|
||||
hash_ring: Some(HashRing {
|
||||
shards: ConsistentHasher::new(&[
|
||||
ShardId::new(11),
|
||||
ShardId::new(12),
|
||||
ShardId::new(13),
|
||||
]),
|
||||
}),
|
||||
};
|
||||
|
||||
// Deletes w/o table name go to all shards
|
||||
let meta = DmlMeta::unsequenced(None);
|
||||
let delete = DmlDelete::new(
|
||||
DeletePredicate {
|
||||
range: TimestampRange { start: 1, end: 2 },
|
||||
exprs: vec![],
|
||||
},
|
||||
None,
|
||||
meta,
|
||||
);
|
||||
|
||||
let actual = shard_delete(&delete, &config);
|
||||
let expected = BTreeMap::from([
|
||||
(ShardId::new(1), delete.clone()),
|
||||
(ShardId::new(2), delete.clone()),
|
||||
(ShardId::new(3), delete.clone()),
|
||||
(ShardId::new(11), delete.clone()),
|
||||
(ShardId::new(12), delete.clone()),
|
||||
(ShardId::new(13), delete),
|
||||
]);
|
||||
assert_sharded_deletes_eq(&actual, &expected);
|
||||
|
||||
// Deletes are matched by table name regex
|
||||
let meta = DmlMeta::unsequenced(None);
|
||||
let delete = DmlDelete::new(
|
||||
DeletePredicate {
|
||||
range: TimestampRange { start: 3, end: 4 },
|
||||
exprs: vec![],
|
||||
},
|
||||
Some(NonEmptyString::new("some_foo").unwrap()),
|
||||
meta,
|
||||
);
|
||||
|
||||
let actual = shard_delete(&delete, &config);
|
||||
let expected = BTreeMap::from([(ShardId::new(2), delete)]);
|
||||
assert_sharded_deletes_eq(&actual, &expected);
|
||||
|
||||
// Deletes can be matched by hash-ring
|
||||
let meta = DmlMeta::unsequenced(None);
|
||||
let delete = DmlDelete::new(
|
||||
DeletePredicate {
|
||||
range: TimestampRange { start: 5, end: 6 },
|
||||
exprs: vec![],
|
||||
},
|
||||
Some(NonEmptyString::new("bar").unwrap()),
|
||||
meta,
|
||||
);
|
||||
|
||||
let actual = shard_delete(&delete, &config);
|
||||
let expected = BTreeMap::from([(ShardId::new(13), delete)]);
|
||||
assert_sharded_deletes_eq(&actual, &expected);
|
||||
}
|
||||
|
||||
fn db_write(lines: &[&str], meta: &DmlMeta) -> DmlWrite {
|
||||
DmlWrite::new(
|
||||
lines_to_batches(&lines.join("\n"), 0).unwrap(),
|
||||
meta.clone(),
|
||||
)
|
||||
}
|
||||
|
||||
fn assert_sharded_deletes_eq(
|
||||
actual: &BTreeMap<ShardId, DmlDelete>,
|
||||
expected: &BTreeMap<ShardId, DmlDelete>,
|
||||
) {
|
||||
let actual_shard_ids: Vec<_> = actual.keys().cloned().collect();
|
||||
let expected_shard_ids: Vec<_> = expected.keys().cloned().collect();
|
||||
assert_eq!(actual_shard_ids, expected_shard_ids);
|
||||
|
||||
for (actual_delete, expected_delete) in actual.values().zip(expected.values()) {
|
||||
assert_deletes_eq(actual_delete, expected_delete);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -223,7 +223,10 @@ mod tests {
|
|||
|
||||
let time_provider = Arc::new(SystemProvider::new());
|
||||
let metric_registry = Arc::new(metric::Registry::new());
|
||||
let wb_factory = WriteBufferConfigFactory::new(time_provider, metric_registry);
|
||||
let wb_factory = Arc::new(WriteBufferConfigFactory::new(
|
||||
time_provider,
|
||||
metric_registry,
|
||||
));
|
||||
wb_factory.register_always_fail_mock(String::from("failing_wb"));
|
||||
let connection_pool = Arc::new(ConnectionPool::new(true, wb_factory).await);
|
||||
|
||||
|
|
|
@ -59,6 +59,8 @@ write_buffer = { path = "../write_buffer" }
|
|||
workspace-hack = { path = "../workspace-hack"}
|
||||
|
||||
[dev-dependencies] # In alphabetical order
|
||||
regex = "1.4"
|
||||
router = { path = "../router" }
|
||||
test_helpers = { path = "../test_helpers" }
|
||||
|
||||
[features]
|
||||
|
|
|
@ -647,8 +647,7 @@ impl Db {
|
|||
// Use explicit scope to ensure the async generator doesn't
|
||||
// assume the locks have to possibly live across the `await`
|
||||
let fut = {
|
||||
let partition = self.partition(table_name, partition_key)?;
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(self), partition);
|
||||
let partition = self.lockable_partition(table_name, partition_key)?;
|
||||
|
||||
// Do lock dance to get a write lock on the partition as well
|
||||
// as on all of the chunks
|
||||
|
@ -688,8 +687,7 @@ impl Db {
|
|||
// Use explicit scope to ensure the async generator doesn't
|
||||
// assume the locks have to possibly live across the `await`
|
||||
let fut = {
|
||||
let partition = self.partition(table_name, partition_key)?;
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(self), partition);
|
||||
let partition = self.lockable_partition(table_name, partition_key)?;
|
||||
let partition = partition.read();
|
||||
|
||||
// todo: set these chunks
|
||||
|
@ -3699,6 +3697,56 @@ mod tests {
|
|||
assert_batches_sorted_eq!(&expected, &batches);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn chunk_times() {
|
||||
let t0 = Time::from_timestamp(11, 22);
|
||||
let time = Arc::new(time::MockProvider::new(t0));
|
||||
let db = TestDb::builder()
|
||||
.time_provider(Arc::<time::MockProvider>::clone(&time))
|
||||
.build()
|
||||
.await
|
||||
.db;
|
||||
|
||||
write_lp(db.as_ref(), "cpu foo=1 10").await;
|
||||
|
||||
let chunks = db.chunk_summaries().unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
assert_eq!(chunks[0].time_of_first_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_access.unwrap(), t0);
|
||||
|
||||
let t1 = time.inc(Duration::from_secs(1));
|
||||
|
||||
run_query(Arc::clone(&db), "select * from cpu").await;
|
||||
|
||||
let chunks = db.chunk_summaries().unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
assert_eq!(chunks[0].time_of_first_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_access.unwrap(), t1);
|
||||
|
||||
let t2 = time.inc(Duration::from_secs(1));
|
||||
|
||||
write_lp(db.as_ref(), "cpu foo=1 20").await;
|
||||
|
||||
let chunks = db.chunk_summaries().unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
assert_eq!(chunks[0].time_of_first_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_write, t2);
|
||||
assert_eq!(chunks[0].time_of_last_access.unwrap(), t2);
|
||||
|
||||
time.inc(Duration::from_secs(1));
|
||||
|
||||
// This chunk should be pruned out and therefore not accessed by the query
|
||||
run_query(Arc::clone(&db), "select * from cpu where foo = 2;").await;
|
||||
|
||||
let chunks = db.chunk_summaries().unwrap();
|
||||
assert_eq!(chunks.len(), 1);
|
||||
assert_eq!(chunks[0].time_of_first_write, t0);
|
||||
assert_eq!(chunks[0].time_of_last_write, t2);
|
||||
assert_eq!(chunks[0].time_of_last_access.unwrap(), t2);
|
||||
}
|
||||
|
||||
async fn create_parquet_chunk(db: &Arc<Db>) -> (String, String, ChunkId) {
|
||||
write_lp(db, "cpu bar=1 10").await;
|
||||
let partition_key = "1970-01-01T00";
|
||||
|
|
|
@ -22,7 +22,7 @@ use observability_deps::tracing::{info, trace};
|
|||
use persistence_windows::persistence_windows::FlushHandle;
|
||||
use query::QueryChunkMeta;
|
||||
use schema::{merge::SchemaMerger, Schema, TIME_COLUMN_NAME};
|
||||
use std::{fmt::Display, sync::Arc};
|
||||
use std::sync::Arc;
|
||||
use time::{Time, TimeProvider};
|
||||
use tracker::{RwLock, TaskTracker};
|
||||
|
||||
|
@ -41,8 +41,6 @@ mod unload;
|
|||
mod write;
|
||||
|
||||
/// A newtype wrapper around `Arc<Db>` to workaround trait orphan rules
|
||||
///
|
||||
/// TODO: Pull LifecyclePolicy out of Db to allow strong reference (#2242)
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct ArcDb(Arc<Db>);
|
||||
|
||||
|
@ -123,30 +121,11 @@ impl lifecycle::PersistHandle for CatalogPersistHandle {
|
|||
pub struct LockableCatalogPartition {
|
||||
pub db: Arc<Db>,
|
||||
pub partition: Arc<RwLock<Partition>>,
|
||||
/// Human readable description of what this CatalogPartiton is
|
||||
pub display_string: String,
|
||||
}
|
||||
|
||||
impl LockableCatalogPartition {
|
||||
pub fn new(db: Arc<Db>, partition: Arc<RwLock<Partition>>) -> Self {
|
||||
let display_string = {
|
||||
partition
|
||||
.try_read()
|
||||
.map(|partition| partition.to_string())
|
||||
.unwrap_or_else(|| "UNKNOWN (could not get lock)".into())
|
||||
};
|
||||
|
||||
Self {
|
||||
db,
|
||||
partition,
|
||||
display_string,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Display for LockableCatalogPartition {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
write!(f, "{}", self.display_string)
|
||||
Self { db, partition }
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -191,22 +191,20 @@ mod tests {
|
|||
let partition_keys = db.partition_keys().unwrap();
|
||||
assert_eq!(partition_keys.len(), 1);
|
||||
|
||||
let db_partition = db.partition("cpu", &partition_keys[0]).unwrap();
|
||||
let partition = db.lockable_partition("cpu", &partition_keys[0]).unwrap();
|
||||
let partition_guard = partition.read();
|
||||
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = partition.read();
|
||||
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
let chunks = LockablePartition::chunks(&partition_guard);
|
||||
assert_eq!(chunks.len(), 1);
|
||||
let chunk = chunks[0].read();
|
||||
|
||||
let (_, fut) = compact_chunks(partition.upgrade(), vec![chunk.upgrade()]).unwrap();
|
||||
let (_, fut) = compact_chunks(partition_guard.upgrade(), vec![chunk.upgrade()]).unwrap();
|
||||
// NB: perform the write before spawning the background task that performs the compaction
|
||||
let t_later_write = time.inc(Duration::from_secs(1));
|
||||
write_lp(db.as_ref(), "cpu,tag1=bongo,tag2=a bar=2 40").await;
|
||||
tokio::spawn(fut).await.unwrap().unwrap().unwrap();
|
||||
|
||||
let mut chunk_summaries: Vec<_> = db_partition.read().chunk_summaries().collect();
|
||||
let mut chunk_summaries: Vec<_> = partition.read().chunk_summaries().collect();
|
||||
|
||||
chunk_summaries.sort_unstable();
|
||||
|
||||
|
@ -308,9 +306,7 @@ mod tests {
|
|||
assert_eq!(partition_keys.len(), 1);
|
||||
let partition_key: &str = partition_keys[0].as_ref();
|
||||
|
||||
let db_partition = db.partition("cpu", partition_key).unwrap();
|
||||
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = db.lockable_partition("cpu", partition_key).unwrap();
|
||||
let partition = partition.read();
|
||||
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
|
|
|
@ -392,8 +392,7 @@ mod tests {
|
|||
let partition_key = "1970-01-01T00";
|
||||
write_lp(&db, "cpu,tag1=cupcakes bar=1 10").await;
|
||||
|
||||
let db_partition = db.partition("cpu", partition_key).unwrap();
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = db.lockable_partition("cpu", partition_key).unwrap();
|
||||
let partition = partition.write();
|
||||
|
||||
let (_, registration) = db.jobs.register(Job::CompactObjectStoreChunks {
|
||||
|
@ -418,10 +417,8 @@ mod tests {
|
|||
let partition_key = "1970-01-01T00";
|
||||
write_lp(&db, "cpu,tag1=cupcakes bar=1 10").await;
|
||||
|
||||
let db_partition = db.partition("cpu", partition_key).unwrap();
|
||||
|
||||
// persisted non persisted chunks
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = db.lockable_partition("cpu", partition_key).unwrap();
|
||||
let partition = partition.read();
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
assert_eq!(chunks.len(), 1);
|
||||
|
@ -451,8 +448,6 @@ mod tests {
|
|||
let partition_key = "1970-01-01T00";
|
||||
write_lp(&db, "cpu,tag1=cupcakes bar=1 10").await;
|
||||
|
||||
let db_partition = db.partition("cpu", partition_key).unwrap();
|
||||
|
||||
// persist chunk 1
|
||||
db.persist_partition("cpu", partition_key, true)
|
||||
.await
|
||||
|
@ -480,7 +475,7 @@ mod tests {
|
|||
write_lp(db.as_ref(), "cpu,tag1=chunk4,tag2=a bar=2 40").await;
|
||||
|
||||
// let compact 2 non contiguous chunk 1 and chunk 3
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = db.lockable_partition("cpu", partition_key).unwrap();
|
||||
let partition = partition.read();
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
assert_eq!(chunks.len(), 4);
|
||||
|
|
|
@ -271,36 +271,35 @@ mod tests {
|
|||
|
||||
let partition_keys = db.partition_keys().unwrap();
|
||||
assert_eq!(partition_keys.len(), 1);
|
||||
let db_partition = db.partition("cpu", &partition_keys[0]).unwrap();
|
||||
|
||||
// Close window
|
||||
time.inc(Duration::from_secs(2));
|
||||
|
||||
write_lp(db.as_ref(), "cpu,tag1=lagged bar=1 10").await;
|
||||
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = partition.read();
|
||||
let partition = db.lockable_partition("cpu", &partition_keys[0]).unwrap();
|
||||
let partition_guard = partition.read();
|
||||
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
let chunks = LockablePartition::chunks(&partition_guard);
|
||||
let chunks = chunks.iter().map(|x| x.read());
|
||||
|
||||
let mut partition = partition.upgrade();
|
||||
let mut partition_guard = partition_guard.upgrade();
|
||||
|
||||
let handle = LockablePartition::prepare_persist(&mut partition, false)
|
||||
let handle = LockablePartition::prepare_persist(&mut partition_guard, false)
|
||||
.unwrap()
|
||||
.0;
|
||||
|
||||
assert_eq!(handle.timestamp(), Time::from_timestamp_nanos(10));
|
||||
let chunks: Vec<_> = chunks.map(|x| x.upgrade()).collect();
|
||||
|
||||
persist_chunks(partition, chunks, handle)
|
||||
persist_chunks(partition_guard, chunks, handle)
|
||||
.unwrap()
|
||||
.1
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
assert!(db_partition
|
||||
assert!(partition
|
||||
.read()
|
||||
.persistence_windows()
|
||||
.unwrap()
|
||||
|
@ -519,10 +518,9 @@ mod tests {
|
|||
// start persistence job (but don't poll the future yet)
|
||||
let partition_keys = db.partition_keys().unwrap();
|
||||
assert_eq!(partition_keys.len(), 1);
|
||||
let db_partition = db.partition("cpu", &partition_keys[0]).unwrap();
|
||||
|
||||
// Wait for the persistence window to be closed
|
||||
let partition = LockableCatalogPartition::new(Arc::clone(&db), Arc::clone(&db_partition));
|
||||
let partition = db.lockable_partition("cpu", &partition_keys[0]).unwrap();
|
||||
let partition = partition.read();
|
||||
|
||||
let chunks = LockablePartition::chunks(&partition);
|
||||
|
|
|
@ -1,20 +1,28 @@
|
|||
use std::collections::BTreeMap;
|
||||
use std::num::NonZeroU32;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use arrow_util::assert_batches_eq;
|
||||
use data_types::delete_predicate::{DeleteExpr, DeletePredicate, Op, Scalar};
|
||||
use data_types::router::{
|
||||
Matcher, MatcherToShard, QuerySinks, Router as RouterConfig, ShardConfig, ShardId, WriteSink,
|
||||
WriteSinkSet, WriteSinkVariant,
|
||||
};
|
||||
use data_types::server_id::ServerId;
|
||||
use data_types::timestamp::TimestampRange;
|
||||
use data_types::DatabaseName;
|
||||
use dml::{DmlDelete, DmlOperation, DmlWrite};
|
||||
use generated_types::influxdata::iox::{
|
||||
management::v1::{DatabaseRules, LifecycleRules},
|
||||
management::v1::DatabaseRules,
|
||||
write_buffer::v1::{write_buffer_connection::Direction, WriteBufferConnection},
|
||||
};
|
||||
use mutable_batch_lp::lines_to_batches;
|
||||
use query::exec::ExecutionContextProvider;
|
||||
use query::frontend::sql::SqlQueryPlanner;
|
||||
use regex::Regex;
|
||||
use router::router::Router;
|
||||
use router::server::RouterServer;
|
||||
use server::connection::test_helpers::TestConnectionManager;
|
||||
use server::rules::ProvidedDatabaseRules;
|
||||
use server::test_utils::{make_application, make_initialized_server};
|
||||
|
@ -34,10 +42,7 @@ use write_buffer::mock::MockBufferSharedState;
|
|||
/// It primarily exists to test the routing logic.
|
||||
///
|
||||
struct DistributedTest {
|
||||
db_name: DatabaseName<'static>,
|
||||
router: Arc<Server<TestConnectionManager>>,
|
||||
// TODO: Replace with router (#2980)
|
||||
router_db: Arc<Db>,
|
||||
router: Arc<Router>,
|
||||
|
||||
consumer: Arc<Server<TestConnectionManager>>,
|
||||
consumer_db: Arc<Db>,
|
||||
|
@ -63,26 +68,41 @@ impl DistributedTest {
|
|||
};
|
||||
|
||||
// Create a router
|
||||
let router_server = RouterServer::for_testing(
|
||||
None,
|
||||
None,
|
||||
Arc::clone(application.time_provider()),
|
||||
Some(Arc::clone(application.write_buffer_factory())),
|
||||
)
|
||||
.await;
|
||||
let router_id = ServerId::new(NonZeroU32::new(1).unwrap());
|
||||
let router = make_initialized_server(router_id, Arc::clone(&application)).await;
|
||||
router_server.set_server_id(router_id).unwrap();
|
||||
|
||||
let router_db = router
|
||||
.create_database(
|
||||
ProvidedDatabaseRules::new_rules(DatabaseRules {
|
||||
name: db_name.to_string(),
|
||||
write_buffer_connection: Some(write_buffer_connection.clone()),
|
||||
lifecycle_rules: Some(LifecycleRules {
|
||||
immutable: true,
|
||||
..Default::default()
|
||||
}),
|
||||
..Default::default()
|
||||
})
|
||||
.unwrap(),
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
.initialized_db()
|
||||
.unwrap();
|
||||
router_server.update_router(RouterConfig {
|
||||
name: db_name.to_string(),
|
||||
write_sharder: ShardConfig {
|
||||
specific_targets: vec![MatcherToShard {
|
||||
matcher: Matcher {
|
||||
table_name_regex: Some(Regex::new(".*").unwrap()),
|
||||
},
|
||||
shard: ShardId::new(1),
|
||||
}],
|
||||
hash_ring: None,
|
||||
},
|
||||
write_sinks: BTreeMap::from([(
|
||||
ShardId::new(1),
|
||||
WriteSinkSet {
|
||||
sinks: vec![WriteSink {
|
||||
sink: WriteSinkVariant::WriteBuffer(
|
||||
write_buffer_connection.clone().try_into().unwrap(),
|
||||
),
|
||||
ignore_errors: false,
|
||||
}],
|
||||
},
|
||||
)]),
|
||||
query_sinks: QuerySinks::default(),
|
||||
});
|
||||
let router = router_server.router(db_name).unwrap();
|
||||
|
||||
// Create a consumer
|
||||
let consumer_id = ServerId::new(NonZeroU32::new(2).unwrap());
|
||||
|
@ -105,9 +125,7 @@ impl DistributedTest {
|
|||
.unwrap();
|
||||
|
||||
Self {
|
||||
db_name: db_name.clone(),
|
||||
router,
|
||||
router_db,
|
||||
consumer,
|
||||
consumer_db,
|
||||
}
|
||||
|
@ -136,18 +154,18 @@ impl DistributedTest {
|
|||
/// Write line protocol
|
||||
pub async fn write(&self, lp: &str) {
|
||||
self.router
|
||||
.write(
|
||||
&self.db_name,
|
||||
DmlWrite::new(lines_to_batches(lp, 0).unwrap(), Default::default()),
|
||||
)
|
||||
.write(DmlOperation::Write(DmlWrite::new(
|
||||
lines_to_batches(lp, 0).unwrap(),
|
||||
Default::default(),
|
||||
)))
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub async fn delete(&self, delete: DmlDelete) {
|
||||
// TODO: Write to router not Db (#2980)
|
||||
self.router_db
|
||||
.route_operation(&DmlOperation::Delete(delete))
|
||||
self.router
|
||||
.write(DmlOperation::Delete(delete))
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
@ -164,9 +182,7 @@ impl DistributedTest {
|
|||
|
||||
/// Shuts down the fixture and waits for the servers to exit
|
||||
pub async fn drain(&self) {
|
||||
self.router.shutdown();
|
||||
self.consumer.shutdown();
|
||||
self.router.join().await.unwrap();
|
||||
self.consumer.join().await.unwrap();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -97,6 +97,16 @@ impl SpanContext {
|
|||
}
|
||||
}
|
||||
|
||||
impl PartialEq for SpanContext {
|
||||
fn eq(&self, other: &Self) -> bool {
|
||||
self.trace_id == other.trace_id
|
||||
&& self.parent_span_id == other.parent_span_id
|
||||
&& self.span_id == other.span_id
|
||||
&& self.links == other.links
|
||||
&& self.collector.is_some() == other.collector.is_some()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
@ -114,4 +124,60 @@ mod tests {
|
|||
assert_ne!(ctx1.trace_id, ctx2.trace_id);
|
||||
assert_ne!(ctx1.span_id, ctx2.span_id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_partial_eq() {
|
||||
let collector_1 = Arc::new(RingBufferTraceCollector::new(5)) as _;
|
||||
let collector_2 = Arc::new(RingBufferTraceCollector::new(5)) as _;
|
||||
|
||||
let ctx_ref = SpanContext {
|
||||
trace_id: TraceId::new(1).unwrap(),
|
||||
parent_span_id: Some(SpanId::new(2).unwrap()),
|
||||
span_id: SpanId::new(3).unwrap(),
|
||||
links: vec![
|
||||
(TraceId::new(4).unwrap(), SpanId::new(5).unwrap()),
|
||||
(TraceId::new(6).unwrap(), SpanId::new(7).unwrap()),
|
||||
],
|
||||
collector: Some(collector_1),
|
||||
};
|
||||
|
||||
let ctx = SpanContext { ..ctx_ref.clone() };
|
||||
assert_eq!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
trace_id: TraceId::new(10).unwrap(),
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_ne!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
parent_span_id: Some(SpanId::new(10).unwrap()),
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_ne!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
span_id: SpanId::new(10).unwrap(),
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_ne!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
links: vec![(TraceId::new(4).unwrap(), SpanId::new(5).unwrap())],
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_ne!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
collector: None,
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_ne!(ctx_ref, ctx);
|
||||
|
||||
let ctx = SpanContext {
|
||||
collector: Some(collector_2),
|
||||
..ctx_ref.clone()
|
||||
};
|
||||
assert_eq!(ctx_ref, ctx);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -156,6 +156,7 @@ impl KafkaBufferProducer {
|
|||
// these configs can be overwritten
|
||||
cfg.set("message.timeout.ms", "5000");
|
||||
cfg.set("message.max.bytes", "31457280");
|
||||
cfg.set("message.send.max.retries", "10");
|
||||
cfg.set("queue.buffering.max.kbytes", "31457280");
|
||||
cfg.set("request.required.acks", "all"); // equivalent to acks=-1
|
||||
cfg.set("compression.type", "snappy");
|
||||
|
|
Loading…
Reference in New Issue