influxdb/query/src/exec.rs

530 lines
18 KiB
Rust

//! This module handles the manipulation / execution of storage
//! plans. This is currently implemented using DataFusion, and this
//! interface abstracts away many of the details
pub(crate) mod context;
mod counters;
pub mod field;
pub mod fieldlist;
mod schema_pivot;
pub mod seriesset;
pub mod stringset;
use std::sync::Arc;
use arrow_deps::{
arrow::record_batch::RecordBatch,
datafusion::{self, logical_plan::LogicalPlan},
};
use counters::ExecutionCounters;
use context::IOxExecutionContext;
use schema_pivot::SchemaPivotNode;
use fieldlist::{FieldList, IntoFieldList};
use seriesset::{Error as SeriesSetError, SeriesSetConverter, SeriesSetItem};
use stringset::{IntoStringSet, StringSetRef};
use tokio::sync::mpsc::{self, error::SendError};
use snafu::{ResultExt, Snafu};
use crate::plan::{
fieldlist::FieldListPlan,
seriesset::{SeriesSetPlan, SeriesSetPlans},
stringset::StringSetPlan,
};
#[derive(Debug, Snafu)]
pub enum Error {
#[snafu(display("Plan Execution Error: {}", source))]
Execution {
source: Box<dyn std::error::Error + Send + Sync + 'static>,
},
#[snafu(display("Internal error optimizing plan: {}", source))]
DataFusionOptimization {
source: datafusion::error::DataFusionError,
},
#[snafu(display("Internal error during physical planning: {}", source))]
DataFusionPhysicalPlanning {
source: datafusion::error::DataFusionError,
},
#[snafu(display("Internal error executing plan: {}", source))]
DataFusionExecution {
source: datafusion::error::DataFusionError,
},
#[snafu(display("Internal error executing series set set plan: {}", source))]
SeriesSetExecution {
source: datafusion::error::DataFusionError,
},
#[snafu(display("Internal error executing field set plan: {}", source))]
FieldListExectuon {
source: datafusion::error::DataFusionError,
},
#[snafu(display("Internal error extracting results from Record Batches: {}", message))]
InternalResultsExtraction { message: String },
#[snafu(display("Internal error creating StringSet: {}", source))]
StringSetConversion { source: stringset::Error },
#[snafu(display("Error converting results to SeriesSet: {}", source))]
SeriesSetConversion { source: seriesset::Error },
#[snafu(display("Internal error creating FieldList: {}", source))]
FieldListConversion { source: fieldlist::Error },
#[snafu(display("Sending series set results during conversion: {:?}", source))]
SendingDuringConversion {
source: Box<SendError<Result<SeriesSetItem, SeriesSetError>>>,
},
#[snafu(display("Joining execution task: {}", source))]
JoinError { source: tokio::task::JoinError },
}
pub type Result<T, E = Error> = std::result::Result<T, E>;
/// Handles executing plans, and marshalling the results into rust
/// native structures.
#[derive(Debug, Default)]
pub struct Executor {
counters: Arc<ExecutionCounters>,
}
impl Executor {
pub fn new() -> Self {
Self::default()
}
/// Executes this plan and returns the resulting set of strings
pub async fn to_string_set(&self, plan: StringSetPlan) -> Result<StringSetRef> {
match plan {
StringSetPlan::Known(ss) => Ok(ss),
StringSetPlan::Plan(plans) => self
.run_logical_plans(plans)
.await?
.into_stringset()
.context(StringSetConversion),
}
}
/// Executes the embedded plans, each as separate tasks, sending
/// the resulting `SeriesSet`s one by one to the `tx` channel.
///
/// The SeriesSets are guaranteed to come back ordered by table_name
///
/// Note that the returned future resolves (e.g. "returns") once
/// all plans have been sent to `tx`. This means that the future
/// will not resolve if there is nothing hooked up receiving
/// results from the other end of the channel and the channel
/// can't hold all the resulting series.
pub async fn to_series_set(
&self,
series_set_plans: SeriesSetPlans,
tx: mpsc::Sender<Result<SeriesSetItem, SeriesSetError>>,
) -> Result<()> {
let SeriesSetPlans { mut plans } = series_set_plans;
if plans.is_empty() {
return Ok(());
}
// sort by table name and send the results to separate
// channels
plans.sort_by(|a, b| a.table_name.cmp(&b.table_name));
let mut rx_channels = Vec::new(); // sorted by table names
// Run the plans in parallel
let handles = plans
.into_iter()
.map(|plan| {
// TODO run these on some executor other than the main tokio pool (maybe?)
let ctx = self.new_context();
let (plan_tx, plan_rx) = mpsc::channel(1);
rx_channels.push(plan_rx);
tokio::task::spawn(async move {
let SeriesSetPlan {
table_name,
plan,
tag_columns,
field_columns,
num_prefix_tag_group_columns,
} = plan;
let tag_columns = Arc::new(tag_columns);
let physical_plan = ctx
.prepare_plan(&plan)
.await
.context(DataFusionPhysicalPlanning)?;
let it = ctx
.execute(physical_plan)
.await
.context(SeriesSetExecution)?;
SeriesSetConverter::new(plan_tx)
.convert(
table_name,
tag_columns,
field_columns,
num_prefix_tag_group_columns,
it,
)
.await
.context(SeriesSetConversion)?;
Ok(())
})
})
.collect::<Vec<_>>();
// transfer data from the rx streams in order
for mut rx in rx_channels {
while let Some(r) = rx.recv().await {
tx.send(r)
.await
.map_err(|e| Error::SendingDuringConversion {
source: Box::new(e),
})?
}
}
// now, wait for all the values to resolve so we can report
// any errors
for join_handle in handles {
join_handle.await.context(JoinError)??;
}
Ok(())
}
/// Executes `plan` and return the resulting FieldList
pub async fn to_field_list(&self, plan: FieldListPlan) -> Result<FieldList> {
let FieldListPlan { plans } = plan;
// Run the plans in parallel
let handles = plans
.into_iter()
.map(|plan| {
let counters = Arc::clone(&self.counters);
tokio::task::spawn(async move {
let ctx = IOxExecutionContext::new(counters);
let physical_plan = ctx
.prepare_plan(&plan)
.await
.context(DataFusionPhysicalPlanning)?;
// TODO: avoid this buffering
let fieldlist = ctx
.collect(physical_plan)
.await
.context(FieldListExectuon)?
.into_fieldlist()
.context(FieldListConversion);
Ok(fieldlist)
})
})
.collect::<Vec<_>>();
// collect them all up and combine them
let mut results = Vec::new();
for join_handle in handles {
let fieldlist = join_handle.await.context(JoinError)???;
results.push(fieldlist);
}
results.into_fieldlist().context(FieldListConversion)
}
/// Run the plan and return a record batch reader for reading the results
pub async fn run_logical_plan(&self, plan: LogicalPlan) -> Result<Vec<RecordBatch>> {
self.run_logical_plans(vec![plan]).await
}
/// Create a new execution context, suitable for executing a new query
pub fn new_context(&self) -> IOxExecutionContext {
IOxExecutionContext::new(Arc::clone(&self.counters))
}
/// plans and runs the plans in parallel and collects the results
/// run each plan in parallel and collect the results
async fn run_logical_plans(&self, plans: Vec<LogicalPlan>) -> Result<Vec<RecordBatch>> {
let value_futures = plans
.into_iter()
.map(|plan| {
let ctx = self.new_context();
// TODO run these on some executor other than the main tokio pool
tokio::task::spawn(async move {
let physical_plan = ctx
.prepare_plan(&plan)
.await
.context(DataFusionPhysicalPlanning)?;
// TODO: avoid this buffering
ctx.collect(physical_plan)
.await
.context(DataFusionExecution)
})
})
.collect::<Vec<_>>();
// now, wait for all the values to resolve and collect them together
let mut results = Vec::new();
for join_handle in value_futures {
let mut plan_result = join_handle.await.context(JoinError)??;
results.append(&mut plan_result);
}
Ok(results)
}
}
/// Create a SchemaPivot node which an arbitrary input like
/// ColA | ColB | ColC
/// ------+------+------
/// 1 | NULL | NULL
/// 2 | 2 | NULL
/// 3 | 2 | NULL
///
/// And pivots it to a table with a single string column for any
/// columns that had non null values.
///
/// non_null_column
/// -----------------
/// "ColA"
/// "ColB"
pub fn make_schema_pivot(input: LogicalPlan) -> LogicalPlan {
let node = Arc::new(SchemaPivotNode::new(input));
LogicalPlan::Extension { node }
}
#[cfg(test)]
mod tests {
use arrow_deps::{
arrow::{
array::Int64Array,
array::StringArray,
array::StringBuilder,
datatypes::DataType,
datatypes::{Field, Schema, SchemaRef},
},
datafusion::logical_plan::LogicalPlanBuilder,
};
use stringset::StringSet;
use super::*;
#[tokio::test]
async fn executor_known_string_set_plan_ok() -> Result<()> {
let expected_strings = to_set(&["Foo", "Bar"]);
let plan = StringSetPlan::Known(Arc::clone(&expected_strings));
let executor = Executor::default();
let result_strings = executor.to_string_set(plan).await?;
assert_eq!(result_strings, expected_strings);
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_single_plan_no_batches() -> Result<()> {
// Test with a single plan that produces no batches
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let scan = make_plan(schema, vec![]);
let plan: StringSetPlan = vec![scan].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await?;
assert_eq!(results, StringSetRef::new(StringSet::new()));
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_single_plan_one_batch() -> Result<()> {
// Test with a single plan that produces one record batch
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let data = to_string_array(&["foo", "bar", "baz", "foo"]);
let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])
.expect("created new record batch");
let scan = make_plan(schema, vec![batch]);
let plan: StringSetPlan = vec![scan].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await?;
assert_eq!(results, to_set(&["foo", "bar", "baz"]));
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_single_plan_two_batch() -> Result<()> {
// Test with a single plan that produces multiple record batches
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let data1 = to_string_array(&["foo", "bar"]);
let batch1 = RecordBatch::try_new(Arc::clone(&schema), vec![data1])
.expect("created new record batch");
let data2 = to_string_array(&["baz", "foo"]);
let batch2 = RecordBatch::try_new(Arc::clone(&schema), vec![data2])
.expect("created new record batch");
let scan = make_plan(schema, vec![batch1, batch2]);
let plan: StringSetPlan = vec![scan].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await?;
assert_eq!(results, to_set(&["foo", "bar", "baz"]));
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_multi_plan() -> Result<()> {
// Test with multiple datafusion logical plans
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let data1 = to_string_array(&["foo", "bar"]);
let batch1 = RecordBatch::try_new(Arc::clone(&schema), vec![data1])
.expect("created new record batch");
let scan1 = make_plan(Arc::clone(&schema), vec![batch1]);
let data2 = to_string_array(&["baz", "foo"]);
let batch2 = RecordBatch::try_new(Arc::clone(&schema), vec![data2])
.expect("created new record batch");
let scan2 = make_plan(schema, vec![batch2]);
let plan: StringSetPlan = vec![scan1, scan2].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await?;
assert_eq!(results, to_set(&["foo", "bar", "baz"]));
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_nulls() -> Result<()> {
// Ensure that nulls in the output set are handled reasonably
// (error, rather than silently ignored)
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let mut builder = StringBuilder::new(2);
builder.append_value("foo").unwrap();
builder.append_null().unwrap();
let data = Arc::new(builder.finish());
let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])
.expect("created new record batch");
let scan = make_plan(schema, vec![batch]);
let plan: StringSetPlan = vec![scan].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await;
let actual_error = match results {
Ok(_) => "Unexpected Ok".into(),
Err(e) => format!("{}", e),
};
let expected_error = "unexpected null value";
assert!(
actual_error.contains(expected_error),
"expected error '{}' not found in '{:?}'",
expected_error,
actual_error,
);
Ok(())
}
#[tokio::test]
async fn executor_datafusion_string_set_bad_schema() -> Result<()> {
// Ensure that an incorect schema (an int) gives a reasonable error
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, true)]));
let data = Arc::new(Int64Array::from(vec![1]));
let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])
.expect("created new record batch");
let scan = make_plan(schema, vec![batch]);
let plan: StringSetPlan = vec![scan].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await;
let actual_error = match results {
Ok(_) => "Unexpected Ok".into(),
Err(e) => format!("{}", e),
};
let expected_error = "schema not a single Utf8";
assert!(
actual_error.contains(expected_error),
"expected error '{}' not found in '{:?}'",
expected_error,
actual_error
);
Ok(())
}
#[tokio::test]
async fn make_schema_pivot_is_planned() -> Result<()> {
// Test that all the planning logic is wired up and that we
// can make a plan using a SchemaPivot node
let schema = Arc::new(Schema::new(vec![
Field::new("f1", DataType::Utf8, true),
Field::new("f2", DataType::Utf8, true),
]));
let batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![
to_string_array(&["foo", "bar"]),
to_string_array(&["baz", "bzz"]),
],
)
.expect("created new record batch");
let scan = make_plan(schema, vec![batch]);
let pivot = make_schema_pivot(scan);
let plan = vec![pivot].into();
let executor = Executor::new();
let results = executor.to_string_set(plan).await.expect("Executed plan");
assert_eq!(results, to_set(&["f1", "f2"]));
Ok(())
}
/// return a set for testing
fn to_set(strs: &[&str]) -> StringSetRef {
StringSetRef::new(strs.iter().map(|s| s.to_string()).collect::<StringSet>())
}
fn to_string_array(strs: &[&str]) -> Arc<StringArray> {
let mut builder = StringBuilder::new(strs.len());
for s in strs {
builder.append_value(s).expect("appending string");
}
Arc::new(builder.finish())
}
// creates a DataFusion plan that reads the RecordBatches into memory
fn make_plan(schema: SchemaRef, data: Vec<RecordBatch>) -> LogicalPlan {
let projection = None;
LogicalPlanBuilder::scan_memory(
vec![data], // model one partition,
schema,
projection,
)
.unwrap()
.build()
.unwrap()
}
}