feat: special case max timestamp range for table_names and field_columns (#3642)
parent
a9fe3362bd
commit
c18ad4ac97
|
@ -219,6 +219,9 @@ impl InfluxRpcPlanner {
|
|||
{
|
||||
debug!(?rpc_predicate, "planning table_names");
|
||||
|
||||
// Special case predicates that span the entire valid timestamp range
|
||||
let rpc_predicate = rpc_predicate.clear_timestamp_if_max_range();
|
||||
|
||||
let mut builder = StringSetPlanBuilder::new();
|
||||
|
||||
// Mapping between table and chunks that need full plan
|
||||
|
@ -617,6 +620,9 @@ impl InfluxRpcPlanner {
|
|||
{
|
||||
debug!(?rpc_predicate, "planning field_columns");
|
||||
|
||||
// Special case predicates that span the entire valid timestamp range
|
||||
let rpc_predicate = rpc_predicate.clear_timestamp_if_max_range();
|
||||
|
||||
// Algorithm is to run a "select field_cols from table where
|
||||
// <predicate> type plan for each table in the chunks"
|
||||
//
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
use arrow::datatypes::DataType;
|
||||
use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME};
|
||||
use datafusion::logical_plan::{col, lit};
|
||||
use predicate::rpc_predicate::InfluxRpcPredicate;
|
||||
use predicate::PredicateBuilder;
|
||||
|
@ -216,3 +217,64 @@ async fn test_field_name_plan_with_delete() {
|
|||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_field_columns_max_time() {
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(MIN_NANO_TIME, MAX_NANO_TIME)
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
|
||||
let expected_fields = FieldList {
|
||||
fields: vec![Field {
|
||||
name: "value".into(),
|
||||
data_type: DataType::Float64,
|
||||
last_timestamp: MAX_NANO_TIME,
|
||||
}],
|
||||
};
|
||||
|
||||
run_field_columns_test_case(MeasurementWithMaxTime {}, predicate, expected_fields).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_field_columns_max_i64() {
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(i64::MIN, i64::MAX)
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
|
||||
let expected_fields = FieldList {
|
||||
fields: vec![Field {
|
||||
name: "value".into(),
|
||||
data_type: DataType::Float64,
|
||||
last_timestamp: MAX_NANO_TIME,
|
||||
}],
|
||||
};
|
||||
|
||||
run_field_columns_test_case(MeasurementWithMaxTime {}, predicate, expected_fields).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_field_columns_max_time_less_one() {
|
||||
let predicate = PredicateBuilder::default()
|
||||
// one less than max timestamp
|
||||
.timestamp_range(MIN_NANO_TIME, MAX_NANO_TIME - 1)
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
|
||||
let expected_fields = FieldList { fields: vec![] };
|
||||
|
||||
run_field_columns_test_case(MeasurementWithMaxTime {}, predicate, expected_fields).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_field_columns_max_time_greater_one() {
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(MIN_NANO_TIME + 1, MAX_NANO_TIME)
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
|
||||
let expected_fields = FieldList { fields: vec![] };
|
||||
|
||||
run_field_columns_test_case(MeasurementWithMaxTime {}, predicate, expected_fields).await;
|
||||
}
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
//! Tests for the Influx gRPC queries
|
||||
use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME};
|
||||
use datafusion::logical_plan::{col, lit};
|
||||
use predicate::rpc_predicate::InfluxRpcPredicate;
|
||||
use predicate::PredicateBuilder;
|
||||
|
@ -207,6 +208,48 @@ async fn list_table_names_data_pred_250_300_with_delete_all() {
|
|||
run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(250, 300), vec![]).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_table_names_max_time() {
|
||||
run_table_names_test_case(
|
||||
MeasurementWithMaxTime {},
|
||||
tsp(MIN_NANO_TIME, MAX_NANO_TIME),
|
||||
vec!["cpu"],
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_table_names_max_i64() {
|
||||
run_table_names_test_case(
|
||||
MeasurementWithMaxTime {},
|
||||
// outside valid timestamp range
|
||||
tsp(i64::MIN, i64::MAX),
|
||||
vec!["cpu"],
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_table_names_time_less_one() {
|
||||
run_table_names_test_case(
|
||||
MeasurementWithMaxTime {},
|
||||
tsp(MIN_NANO_TIME, MAX_NANO_TIME - 1),
|
||||
vec![],
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn list_table_names_max_time_greater_one() {
|
||||
run_table_names_test_case(
|
||||
MeasurementWithMaxTime {},
|
||||
// one more than max timestamp
|
||||
tsp(MIN_NANO_TIME + 1, MAX_NANO_TIME),
|
||||
vec![],
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
// Note when table names supports general purpose predicates, add a
|
||||
// test here with a `_measurement` predicate
|
||||
// https://github.com/influxdata/influxdb_iox/issues/762
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use data_types::timestamp::{MAX_NANO_TIME, MIN_NANO_TIME};
|
||||
use datafusion::logical_plan::{col, lit};
|
||||
use predicate::rpc_predicate::InfluxRpcPredicate;
|
||||
use predicate::PredicateBuilder;
|
||||
|
@ -186,7 +187,7 @@ async fn list_tag_name_end_to_end_with_delete() {
|
|||
async fn list_tag_name_max_time() {
|
||||
test_helpers::maybe_start_logging();
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(-9223372036854775806, 9223372036854775806)
|
||||
.timestamp_range(MIN_NANO_TIME, MAX_NANO_TIME)
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
let expected_tag_keys = vec!["host"];
|
||||
|
@ -209,7 +210,7 @@ async fn list_tag_name_max_i64() {
|
|||
async fn list_tag_name_max_time_less_one() {
|
||||
test_helpers::maybe_start_logging();
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(-9223372036854775806, 9223372036854775805) // one less than max timestamp
|
||||
.timestamp_range(MIN_NANO_TIME, MAX_NANO_TIME - 1) // one less than max timestamp
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
let expected_tag_keys = vec![];
|
||||
|
@ -220,7 +221,7 @@ async fn list_tag_name_max_time_less_one() {
|
|||
async fn list_tag_name_max_time_greater_one() {
|
||||
test_helpers::maybe_start_logging();
|
||||
let predicate = PredicateBuilder::default()
|
||||
.timestamp_range(-9223372036854775805, 9223372036854775806) // one more than min timestamp
|
||||
.timestamp_range(MIN_NANO_TIME + 1, MAX_NANO_TIME) // one more than min timestamp
|
||||
.build();
|
||||
let predicate = InfluxRpcPredicate::new(None, predicate);
|
||||
let expected_tag_keys = vec![];
|
||||
|
|
Loading…
Reference in New Issue