refactor: simplify `QueryChunk` data access (#6015)
* refactor: simplify `QueryChunk` data access We have only two types for chunks (now that the RUB is gone): 1. In-memory RecordBatches 2. Parquet files Loads of logic is duplicated in the different `read_filter` implementations. Also `read_filter` hides a solid amount of logic from DataFusion, which will prevent certain (future) optimizations. To enable #5897 and to simplify the interface, let the chunks return the data (batches or metadata for parquet files) directly and let `iox_query` perform the actual heavy-lifting. * docs: improve Co-authored-by: Andrew Lamb <alamb@influxdata.com> * docs: improve Co-authored-by: Andrew Lamb <alamb@influxdata.com> Co-authored-by: Andrew Lamb <alamb@influxdata.com> Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>pull/24376/head
parent
1eb0d64210
commit
45b3984aa3
|
@ -4,40 +4,18 @@ use data_types::{
|
||||||
ChunkId, ChunkOrder, CompactionLevel, DeletePredicate, PartitionId, SequenceNumber,
|
ChunkId, ChunkOrder, CompactionLevel, DeletePredicate, PartitionId, SequenceNumber,
|
||||||
TableSummary, Timestamp, Tombstone,
|
TableSummary, Timestamp, Tombstone,
|
||||||
};
|
};
|
||||||
use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream};
|
use datafusion::error::DataFusionError;
|
||||||
use iox_query::{
|
use iox_query::{
|
||||||
exec::{stringset::StringSet, IOxSessionContext},
|
exec::{stringset::StringSet, IOxSessionContext},
|
||||||
util::create_basic_summary,
|
util::create_basic_summary,
|
||||||
QueryChunk, QueryChunkMeta,
|
QueryChunk, QueryChunkData, QueryChunkMeta,
|
||||||
};
|
};
|
||||||
use observability_deps::tracing::trace;
|
|
||||||
use parquet_file::chunk::ParquetChunk;
|
use parquet_file::chunk::ParquetChunk;
|
||||||
use predicate::{delete_predicate::tombstones_to_delete_predicates, Predicate};
|
use predicate::{delete_predicate::tombstones_to_delete_predicates, Predicate};
|
||||||
use schema::{merge::SchemaMerger, selection::Selection, sort::SortKey, Schema};
|
use schema::{merge::SchemaMerger, selection::Selection, sort::SortKey, Schema};
|
||||||
use snafu::{ResultExt, Snafu};
|
|
||||||
use std::{any::Any, sync::Arc};
|
use std::{any::Any, sync::Arc};
|
||||||
use uuid::Uuid;
|
use uuid::Uuid;
|
||||||
|
|
||||||
#[derive(Debug, Snafu)]
|
|
||||||
#[allow(missing_copy_implementations, missing_docs)]
|
|
||||||
pub enum Error {
|
|
||||||
#[snafu(display("Failed to read parquet: {}", source))]
|
|
||||||
ReadParquet {
|
|
||||||
source: parquet_file::storage::ReadError,
|
|
||||||
},
|
|
||||||
|
|
||||||
#[snafu(display(
|
|
||||||
"Error reading IOx Metadata from Parquet IoxParquetMetadata: {}",
|
|
||||||
source
|
|
||||||
))]
|
|
||||||
ReadParquetMeta {
|
|
||||||
source: parquet_file::storage::ReadError,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
/// A specialized `Error` for Compactor's query errors
|
|
||||||
pub type Result<T, E = Error> = std::result::Result<T, E>;
|
|
||||||
|
|
||||||
/// QueryableParquetChunk that implements QueryChunk and QueryMetaChunk for building query plan
|
/// QueryableParquetChunk that implements QueryChunk and QueryMetaChunk for building query plan
|
||||||
#[derive(Debug, Clone)]
|
#[derive(Debug, Clone)]
|
||||||
pub struct QueryableParquetChunk {
|
pub struct QueryableParquetChunk {
|
||||||
|
@ -213,33 +191,8 @@ impl QueryChunk for QueryableParquetChunk {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Provides access to raw `QueryChunk` data as an
|
fn data(&self) -> QueryChunkData {
|
||||||
/// asynchronous stream of `RecordBatch`es filtered by a *required*
|
QueryChunkData::Parquet(self.data.parquet_exec_input())
|
||||||
/// predicate. Note that not all chunks can evaluate all types of
|
|
||||||
/// predicates and this function will return an error
|
|
||||||
/// if requested to evaluate with a predicate that is not supported
|
|
||||||
///
|
|
||||||
/// This is the analog of the `TableProvider` in DataFusion
|
|
||||||
///
|
|
||||||
/// The reason we can't simply use the `TableProvider` trait
|
|
||||||
/// directly is that the data for a particular Table lives in
|
|
||||||
/// several chunks within a partition, so there needs to be an
|
|
||||||
/// implementation of `TableProvider` that stitches together the
|
|
||||||
/// streams from several different `QueryChunk`s.
|
|
||||||
fn read_filter(
|
|
||||||
&self,
|
|
||||||
mut ctx: IOxSessionContext,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<SendableRecordBatchStream, DataFusionError> {
|
|
||||||
ctx.set_metadata("storage", "compactor");
|
|
||||||
ctx.set_metadata("projection", format!("{}", selection));
|
|
||||||
trace!(?selection, "selection");
|
|
||||||
|
|
||||||
self.data
|
|
||||||
.read_filter(predicate, selection, ctx.inner())
|
|
||||||
.context(ReadParquetSnafu)
|
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns chunk type
|
/// Returns chunk type
|
||||||
|
|
|
@ -6,24 +6,16 @@ use std::{any::Any, sync::Arc};
|
||||||
use arrow::record_batch::RecordBatch;
|
use arrow::record_batch::RecordBatch;
|
||||||
use arrow_util::util::ensure_schema;
|
use arrow_util::util::ensure_schema;
|
||||||
use data_types::{ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary};
|
use data_types::{ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary};
|
||||||
use datafusion::{
|
use datafusion::error::DataFusionError;
|
||||||
error::DataFusionError,
|
|
||||||
physical_plan::{
|
|
||||||
common::SizedRecordBatchStream,
|
|
||||||
metrics::{ExecutionPlanMetricsSet, MemTrackingMetrics},
|
|
||||||
SendableRecordBatchStream,
|
|
||||||
},
|
|
||||||
};
|
|
||||||
use iox_query::{
|
use iox_query::{
|
||||||
exec::{stringset::StringSet, IOxSessionContext},
|
exec::{stringset::StringSet, IOxSessionContext},
|
||||||
util::{compute_timenanosecond_min_max, create_basic_summary},
|
util::{compute_timenanosecond_min_max, create_basic_summary},
|
||||||
QueryChunk, QueryChunkMeta,
|
QueryChunk, QueryChunkData, QueryChunkMeta,
|
||||||
};
|
};
|
||||||
use observability_deps::tracing::trace;
|
|
||||||
use once_cell::sync::OnceCell;
|
use once_cell::sync::OnceCell;
|
||||||
use predicate::Predicate;
|
use predicate::Predicate;
|
||||||
use schema::{merge::merge_record_batch_schemas, selection::Selection, sort::SortKey, Schema};
|
use schema::{merge::merge_record_batch_schemas, selection::Selection, sort::SortKey, Schema};
|
||||||
use snafu::{ResultExt, Snafu};
|
use snafu::Snafu;
|
||||||
|
|
||||||
use crate::data::table::TableName;
|
use crate::data::table::TableName;
|
||||||
|
|
||||||
|
@ -230,42 +222,15 @@ impl QueryChunk for QueryAdaptor {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Provides access to raw `QueryChunk` data as an
|
fn data(&self) -> QueryChunkData {
|
||||||
/// asynchronous stream of `RecordBatch`es
|
let schema = self.schema().as_arrow();
|
||||||
fn read_filter(
|
|
||||||
&self,
|
|
||||||
mut ctx: IOxSessionContext,
|
|
||||||
_predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<SendableRecordBatchStream, DataFusionError> {
|
|
||||||
ctx.set_metadata("storage", "ingester");
|
|
||||||
ctx.set_metadata("projection", format!("{}", selection));
|
|
||||||
trace!(?selection, "selection");
|
|
||||||
|
|
||||||
let schema = self
|
QueryChunkData::RecordBatches(
|
||||||
.schema()
|
self.data
|
||||||
.select(selection)
|
.iter()
|
||||||
.context(SchemaSnafu)
|
.map(|b| ensure_schema(&schema, b).expect("schema handling broken"))
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
.collect(),
|
||||||
|
)
|
||||||
// Apply the projection over all the data in self, ensuring each batch
|
|
||||||
// has the specified schema.
|
|
||||||
let batches = self
|
|
||||||
.project_selection(selection)
|
|
||||||
.into_iter()
|
|
||||||
.map(|batch| {
|
|
||||||
ensure_schema(&schema.as_arrow(), &batch)
|
|
||||||
.context(ConcatBatchesSnafu {})
|
|
||||||
.map(Arc::new)
|
|
||||||
})
|
|
||||||
.collect::<Result<Vec<_>, _>>()
|
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))?;
|
|
||||||
|
|
||||||
// Return stream of data
|
|
||||||
let dummy_metrics = ExecutionPlanMetricsSet::new();
|
|
||||||
let mem_metrics = MemTrackingMetrics::new(&dummy_metrics, 0);
|
|
||||||
let stream = SizedRecordBatchStream::new(schema.as_arrow(), batches, mem_metrics);
|
|
||||||
Ok(Box::pin(stream))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns chunk type
|
/// Returns chunk type
|
||||||
|
|
|
@ -10,12 +10,14 @@
|
||||||
clippy::dbg_macro
|
clippy::dbg_macro
|
||||||
)]
|
)]
|
||||||
|
|
||||||
|
use arrow::record_batch::RecordBatch;
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use data_types::{ChunkId, ChunkOrder, DeletePredicate, InfluxDbType, PartitionId, TableSummary};
|
use data_types::{ChunkId, ChunkOrder, DeletePredicate, InfluxDbType, PartitionId, TableSummary};
|
||||||
use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream};
|
use datafusion::{error::DataFusionError, prelude::SessionContext};
|
||||||
use exec::{stringset::StringSet, IOxSessionContext};
|
use exec::{stringset::StringSet, IOxSessionContext};
|
||||||
use hashbrown::HashMap;
|
use hashbrown::HashMap;
|
||||||
use observability_deps::tracing::{debug, trace};
|
use observability_deps::tracing::{debug, trace};
|
||||||
|
use parquet_file::storage::ParquetExecInput;
|
||||||
use predicate::{rpc_predicate::QueryDatabaseMeta, Predicate, PredicateMatch};
|
use predicate::{rpc_predicate::QueryDatabaseMeta, Predicate, PredicateMatch};
|
||||||
use schema::{
|
use schema::{
|
||||||
selection::Selection,
|
selection::Selection,
|
||||||
|
@ -173,6 +175,37 @@ pub trait QueryDatabase: QueryDatabaseMeta + Debug + Send + Sync {
|
||||||
fn as_meta(&self) -> &dyn QueryDatabaseMeta;
|
fn as_meta(&self) -> &dyn QueryDatabaseMeta;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Raw data of a [`QueryChunk`].
|
||||||
|
#[derive(Debug)]
|
||||||
|
pub enum QueryChunkData {
|
||||||
|
/// In-memory record batches.
|
||||||
|
///
|
||||||
|
/// **IMPORTANT: All batches MUST have the schema that the [chunk reports](QueryChunkMeta::schema).**
|
||||||
|
RecordBatches(Vec<RecordBatch>),
|
||||||
|
|
||||||
|
/// Parquet file.
|
||||||
|
///
|
||||||
|
/// See [`ParquetExecInput`] for details.
|
||||||
|
Parquet(ParquetExecInput),
|
||||||
|
}
|
||||||
|
|
||||||
|
impl QueryChunkData {
|
||||||
|
/// Read data into [`RecordBatch`]es. This is mostly meant for testing!
|
||||||
|
pub async fn read_to_batches(
|
||||||
|
self,
|
||||||
|
schema: Arc<Schema>,
|
||||||
|
session_ctx: &SessionContext,
|
||||||
|
) -> Vec<RecordBatch> {
|
||||||
|
match self {
|
||||||
|
Self::RecordBatches(batches) => batches,
|
||||||
|
Self::Parquet(exec_input) => exec_input
|
||||||
|
.read_to_batches(schema.as_arrow(), Selection::All, session_ctx)
|
||||||
|
.await
|
||||||
|
.unwrap(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Collection of data that shares the same partition key
|
/// Collection of data that shares the same partition key
|
||||||
pub trait QueryChunk: QueryChunkMeta + Debug + Send + Sync + 'static {
|
pub trait QueryChunk: QueryChunkMeta + Debug + Send + Sync + 'static {
|
||||||
/// returns the Id of this chunk. Ids are unique within a
|
/// returns the Id of this chunk. Ids are unique within a
|
||||||
|
@ -222,25 +255,10 @@ pub trait QueryChunk: QueryChunkMeta + Debug + Send + Sync + 'static {
|
||||||
predicate: &Predicate,
|
predicate: &Predicate,
|
||||||
) -> Result<Option<StringSet>, DataFusionError>;
|
) -> Result<Option<StringSet>, DataFusionError>;
|
||||||
|
|
||||||
/// Provides access to raw `QueryChunk` data as an
|
/// Provides access to raw [`QueryChunk`] data.
|
||||||
/// asynchronous stream of `RecordBatch`es filtered by a *required*
|
|
||||||
/// predicate. Note that not all chunks can evaluate all types of
|
|
||||||
/// predicates and this function will return an error
|
|
||||||
/// if requested to evaluate with a predicate that is not supported
|
|
||||||
///
|
///
|
||||||
/// This is the analog of the `TableProvider` in DataFusion
|
/// The engine assume that minimal work shall be performed to gather the `QueryChunkData`.
|
||||||
///
|
fn data(&self) -> QueryChunkData;
|
||||||
/// The reason we can't simply use the `TableProvider` trait
|
|
||||||
/// directly is that the data for a particular Table lives in
|
|
||||||
/// several chunks within a partition, so there needs to be an
|
|
||||||
/// implementation of `TableProvider` that stitches together the
|
|
||||||
/// streams from several different `QueryChunk`s.
|
|
||||||
fn read_filter(
|
|
||||||
&self,
|
|
||||||
ctx: IOxSessionContext,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<SendableRecordBatchStream, DataFusionError>;
|
|
||||||
|
|
||||||
/// Returns chunk type. Useful in tests and debug logs.
|
/// Returns chunk type. Useful in tests and debug logs.
|
||||||
fn chunk_type(&self) -> &str;
|
fn chunk_type(&self) -> &str;
|
||||||
|
|
|
@ -1,22 +1,28 @@
|
||||||
//! Implementation of a DataFusion PhysicalPlan node across partition chunks
|
//! Implementation of a DataFusion PhysicalPlan node across partition chunks
|
||||||
|
|
||||||
use super::adapter::SchemaAdapterStream;
|
use super::adapter::SchemaAdapterStream;
|
||||||
use crate::{exec::IOxSessionContext, QueryChunk};
|
use crate::{exec::IOxSessionContext, QueryChunk, QueryChunkData};
|
||||||
use arrow::datatypes::SchemaRef;
|
use arrow::datatypes::SchemaRef;
|
||||||
use data_types::TableSummary;
|
use data_types::TableSummary;
|
||||||
use datafusion::{
|
use datafusion::{
|
||||||
|
datasource::listing::PartitionedFile,
|
||||||
error::DataFusionError,
|
error::DataFusionError,
|
||||||
execution::context::TaskContext,
|
execution::context::TaskContext,
|
||||||
physical_plan::{
|
physical_plan::{
|
||||||
|
execute_stream,
|
||||||
expressions::PhysicalSortExpr,
|
expressions::PhysicalSortExpr,
|
||||||
|
file_format::{FileScanConfig, ParquetExec},
|
||||||
|
memory::MemoryStream,
|
||||||
metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet},
|
metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet},
|
||||||
|
stream::RecordBatchStreamAdapter,
|
||||||
DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics,
|
DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics,
|
||||||
},
|
},
|
||||||
};
|
};
|
||||||
|
use futures::TryStreamExt;
|
||||||
use observability_deps::tracing::trace;
|
use observability_deps::tracing::trace;
|
||||||
use predicate::Predicate;
|
use predicate::Predicate;
|
||||||
use schema::{selection::Selection, Schema};
|
use schema::Schema;
|
||||||
use std::{fmt, sync::Arc};
|
use std::{collections::HashSet, fmt, sync::Arc};
|
||||||
|
|
||||||
/// Implements the DataFusion physical plan interface
|
/// Implements the DataFusion physical plan interface
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
|
@ -104,16 +110,13 @@ impl ExecutionPlan for IOxReadFilterNode {
|
||||||
fn execute(
|
fn execute(
|
||||||
&self,
|
&self,
|
||||||
partition: usize,
|
partition: usize,
|
||||||
_context: Arc<TaskContext>,
|
context: Arc<TaskContext>,
|
||||||
) -> datafusion::error::Result<SendableRecordBatchStream> {
|
) -> datafusion::error::Result<SendableRecordBatchStream> {
|
||||||
trace!(partition, "Start IOxReadFilterNode::execute");
|
trace!(partition, "Start IOxReadFilterNode::execute");
|
||||||
|
|
||||||
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
|
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
|
||||||
let timer = baseline_metrics.elapsed_compute().timer();
|
|
||||||
|
|
||||||
let schema = self.schema();
|
let schema = self.schema();
|
||||||
let fields = schema.fields();
|
|
||||||
let selection_cols = fields.iter().map(|f| f.name() as &str).collect::<Vec<_>>();
|
|
||||||
|
|
||||||
let chunk = Arc::clone(&self.chunks[partition]);
|
let chunk = Arc::clone(&self.chunks[partition]);
|
||||||
|
|
||||||
|
@ -125,32 +128,79 @@ impl ExecutionPlan for IOxReadFilterNode {
|
||||||
// restrict the requested selection to the actual columns
|
// restrict the requested selection to the actual columns
|
||||||
// available, and use SchemaAdapterStream to pad the rest of
|
// available, and use SchemaAdapterStream to pad the rest of
|
||||||
// the columns with NULLs if necessary
|
// the columns with NULLs if necessary
|
||||||
let selection_cols = restrict_selection(selection_cols, &chunk_table_schema);
|
let final_output_column_names: HashSet<_> =
|
||||||
let selection = Selection::Some(&selection_cols);
|
schema.fields().iter().map(|f| f.name()).collect();
|
||||||
|
let projection: Vec<_> = chunk_table_schema
|
||||||
let stream = chunk
|
.iter()
|
||||||
.read_filter(
|
.enumerate()
|
||||||
self.ctx.child_ctx("chunk read_filter"),
|
.filter(|(_idx, (_t, field))| final_output_column_names.contains(field.name()))
|
||||||
&self.predicate,
|
.map(|(idx, _)| idx)
|
||||||
selection,
|
.collect();
|
||||||
|
let projection = (!((projection.len() == chunk_table_schema.len())
|
||||||
|
&& (projection.iter().enumerate().all(|(a, b)| a == *b))))
|
||||||
|
.then_some(projection);
|
||||||
|
let incomplete_output_schema = projection
|
||||||
|
.as_ref()
|
||||||
|
.map(|projection| {
|
||||||
|
Arc::new(
|
||||||
|
chunk_table_schema
|
||||||
|
.as_arrow()
|
||||||
|
.project(projection)
|
||||||
|
.expect("projection broken"),
|
||||||
)
|
)
|
||||||
.map_err(|e| {
|
})
|
||||||
DataFusionError::Execution(format!(
|
.unwrap_or_else(|| chunk_table_schema.as_arrow());
|
||||||
"Error creating scan for table {} chunk {}: {}",
|
|
||||||
self.table_name,
|
|
||||||
chunk.id(),
|
|
||||||
e
|
|
||||||
))
|
|
||||||
})?;
|
|
||||||
|
|
||||||
// all CPU time is now done, pass in baseline metrics to adapter
|
|
||||||
timer.done();
|
|
||||||
|
|
||||||
|
let stream = match chunk.data() {
|
||||||
|
QueryChunkData::RecordBatches(batches) => {
|
||||||
|
let stream = Box::pin(MemoryStream::try_new(
|
||||||
|
batches,
|
||||||
|
incomplete_output_schema,
|
||||||
|
projection,
|
||||||
|
)?);
|
||||||
let adapter = SchemaAdapterStream::try_new(stream, schema, baseline_metrics)
|
let adapter = SchemaAdapterStream::try_new(stream, schema, baseline_metrics)
|
||||||
.map_err(|e| DataFusionError::Internal(e.to_string()))?;
|
.map_err(|e| DataFusionError::Internal(e.to_string()))?;
|
||||||
|
Box::pin(adapter) as SendableRecordBatchStream
|
||||||
|
}
|
||||||
|
QueryChunkData::Parquet(exec_input) => {
|
||||||
|
let base_config = FileScanConfig {
|
||||||
|
object_store_url: exec_input.object_store_url,
|
||||||
|
file_schema: Arc::clone(&schema),
|
||||||
|
file_groups: vec![vec![PartitionedFile {
|
||||||
|
object_meta: exec_input.object_meta,
|
||||||
|
partition_values: vec![],
|
||||||
|
range: None,
|
||||||
|
extensions: None,
|
||||||
|
}]],
|
||||||
|
statistics: Statistics::default(),
|
||||||
|
projection: None,
|
||||||
|
limit: None,
|
||||||
|
table_partition_cols: vec![],
|
||||||
|
config_options: context.session_config().config_options(),
|
||||||
|
};
|
||||||
|
let delete_predicates: Vec<_> = chunk
|
||||||
|
.delete_predicates()
|
||||||
|
.iter()
|
||||||
|
.map(|pred| Arc::new(pred.as_ref().clone().into()))
|
||||||
|
.collect();
|
||||||
|
let predicate = self
|
||||||
|
.predicate
|
||||||
|
.clone()
|
||||||
|
.with_delete_predicates(&delete_predicates);
|
||||||
|
let exec = ParquetExec::new(base_config, predicate.filter_expr(), None);
|
||||||
|
let stream = RecordBatchStreamAdapter::new(
|
||||||
|
schema,
|
||||||
|
futures::stream::once(execute_stream(Arc::new(exec), context)).try_flatten(),
|
||||||
|
);
|
||||||
|
|
||||||
|
// Note: No SchemaAdapterStream required here because `ParquetExec` already creates NULL columns for us.
|
||||||
|
|
||||||
|
Box::pin(stream)
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
trace!(partition, "End IOxReadFilterNode::execute");
|
trace!(partition, "End IOxReadFilterNode::execute");
|
||||||
Ok(Box::pin(adapter))
|
Ok(stream)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||||
|
@ -190,17 +240,3 @@ impl ExecutionPlan for IOxReadFilterNode {
|
||||||
.unwrap_or_default()
|
.unwrap_or_default()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Removes any columns that are not present in schema, returning a possibly
|
|
||||||
/// restricted set of columns
|
|
||||||
fn restrict_selection<'a>(
|
|
||||||
selection_cols: Vec<&'a str>,
|
|
||||||
chunk_table_schema: &'a Schema,
|
|
||||||
) -> Vec<&'a str> {
|
|
||||||
let arrow_schema = chunk_table_schema.as_arrow();
|
|
||||||
|
|
||||||
selection_cols
|
|
||||||
.into_iter()
|
|
||||||
.filter(|col| arrow_schema.fields().iter().any(|f| f.name() == col))
|
|
||||||
.collect()
|
|
||||||
}
|
|
||||||
|
|
|
@ -8,15 +8,14 @@ use crate::{
|
||||||
stringset::{StringSet, StringSetRef},
|
stringset::{StringSet, StringSetRef},
|
||||||
ExecutionContextProvider, Executor, ExecutorType, IOxSessionContext,
|
ExecutionContextProvider, Executor, ExecutorType, IOxSessionContext,
|
||||||
},
|
},
|
||||||
Predicate, PredicateMatch, QueryChunk, QueryChunkMeta, QueryCompletedToken, QueryDatabase,
|
Predicate, PredicateMatch, QueryChunk, QueryChunkData, QueryChunkMeta, QueryCompletedToken,
|
||||||
QueryText,
|
QueryDatabase, QueryText,
|
||||||
};
|
};
|
||||||
use arrow::{
|
use arrow::{
|
||||||
array::{
|
array::{
|
||||||
ArrayRef, DictionaryArray, Int64Array, StringArray, TimestampNanosecondArray, UInt64Array,
|
ArrayRef, DictionaryArray, Int64Array, StringArray, TimestampNanosecondArray, UInt64Array,
|
||||||
},
|
},
|
||||||
datatypes::{DataType, Int32Type, TimeUnit},
|
datatypes::{DataType, Int32Type, TimeUnit},
|
||||||
error::ArrowError,
|
|
||||||
record_batch::RecordBatch,
|
record_batch::RecordBatch,
|
||||||
};
|
};
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
|
@ -24,9 +23,7 @@ use data_types::{
|
||||||
ChunkId, ChunkOrder, ColumnSummary, DeletePredicate, InfluxDbType, PartitionId, StatValues,
|
ChunkId, ChunkOrder, ColumnSummary, DeletePredicate, InfluxDbType, PartitionId, StatValues,
|
||||||
Statistics, TableSummary,
|
Statistics, TableSummary,
|
||||||
};
|
};
|
||||||
use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream};
|
use datafusion::error::DataFusionError;
|
||||||
use datafusion_util::stream_from_batches;
|
|
||||||
use futures::StreamExt;
|
|
||||||
use hashbrown::HashSet;
|
use hashbrown::HashSet;
|
||||||
use observability_deps::tracing::debug;
|
use observability_deps::tracing::debug;
|
||||||
use parking_lot::Mutex;
|
use parking_lot::Mutex;
|
||||||
|
@ -949,34 +946,8 @@ impl QueryChunk for TestChunk {
|
||||||
self.may_contain_pk_duplicates
|
self.may_contain_pk_duplicates
|
||||||
}
|
}
|
||||||
|
|
||||||
fn read_filter(
|
fn data(&self) -> QueryChunkData {
|
||||||
&self,
|
QueryChunkData::RecordBatches(self.table_data.iter().map(|b| b.as_ref().clone()).collect())
|
||||||
_ctx: IOxSessionContext,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<SendableRecordBatchStream, DataFusionError> {
|
|
||||||
self.check_error()?;
|
|
||||||
|
|
||||||
// save the predicate
|
|
||||||
self.predicates.lock().push(predicate.clone());
|
|
||||||
|
|
||||||
let batches = match self
|
|
||||||
.schema
|
|
||||||
.df_projection(selection)
|
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))?
|
|
||||||
{
|
|
||||||
None => self.table_data.clone(),
|
|
||||||
Some(projection) => self
|
|
||||||
.table_data
|
|
||||||
.iter()
|
|
||||||
.map(|batch| {
|
|
||||||
let batch = batch.project(&projection)?;
|
|
||||||
Ok(Arc::new(batch))
|
|
||||||
})
|
|
||||||
.collect::<std::result::Result<Vec<_>, ArrowError>>()?,
|
|
||||||
};
|
|
||||||
|
|
||||||
Ok(stream_from_batches(self.schema().as_arrow(), batches))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
fn chunk_type(&self) -> &str {
|
fn chunk_type(&self) -> &str {
|
||||||
|
@ -1071,17 +1042,10 @@ impl QueryChunkMeta for TestChunk {
|
||||||
|
|
||||||
/// Return the raw data from the list of chunks
|
/// Return the raw data from the list of chunks
|
||||||
pub async fn raw_data(chunks: &[Arc<dyn QueryChunk>]) -> Vec<RecordBatch> {
|
pub async fn raw_data(chunks: &[Arc<dyn QueryChunk>]) -> Vec<RecordBatch> {
|
||||||
|
let ctx = IOxSessionContext::with_testing();
|
||||||
let mut batches = vec![];
|
let mut batches = vec![];
|
||||||
for c in chunks {
|
for c in chunks {
|
||||||
let pred = Predicate::default();
|
batches.append(&mut c.data().read_to_batches(c.schema(), ctx.inner()).await);
|
||||||
let selection = Selection::All;
|
|
||||||
let mut stream = c
|
|
||||||
.read_filter(IOxSessionContext::with_testing(), &pred, selection)
|
|
||||||
.expect("Error in read_filter");
|
|
||||||
while let Some(b) = stream.next().await {
|
|
||||||
let b = b.expect("Error in stream");
|
|
||||||
batches.push(b)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
batches
|
batches
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,7 +30,6 @@ use parquet_file::{
|
||||||
metadata::IoxMetadata,
|
metadata::IoxMetadata,
|
||||||
storage::{ParquetStorage, StorageId},
|
storage::{ParquetStorage, StorageId},
|
||||||
};
|
};
|
||||||
use predicate::Predicate;
|
|
||||||
use schema::{
|
use schema::{
|
||||||
selection::Selection,
|
selection::Selection,
|
||||||
sort::{adjust_sort_key_columns, compute_sort_key, SortKey},
|
sort::{adjust_sort_key_columns, compute_sort_key, SortKey},
|
||||||
|
@ -389,14 +388,13 @@ impl TestTable {
|
||||||
Arc::new(schema),
|
Arc::new(schema),
|
||||||
self.catalog.parquet_store.clone(),
|
self.catalog.parquet_store.clone(),
|
||||||
);
|
);
|
||||||
let rx = chunk
|
chunk
|
||||||
.read_filter(
|
.parquet_exec_input()
|
||||||
&Predicate::default(),
|
.read_to_batches(
|
||||||
|
chunk.schema().as_arrow(),
|
||||||
Selection::All,
|
Selection::All,
|
||||||
&chunk.store().test_df_context(),
|
&chunk.store().test_df_context(),
|
||||||
)
|
)
|
||||||
.unwrap();
|
|
||||||
datafusion::physical_plan::common::collect(rx)
|
|
||||||
.await
|
.await
|
||||||
.unwrap()
|
.unwrap()
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,10 +1,11 @@
|
||||||
//! A metadata summary of a Parquet file in object storage, with the ability to
|
//! A metadata summary of a Parquet file in object storage, with the ability to
|
||||||
//! download & execute a scan.
|
//! download & execute a scan.
|
||||||
|
|
||||||
use crate::{storage::ParquetStorage, ParquetFilePath};
|
use crate::{
|
||||||
|
storage::{ParquetExecInput, ParquetStorage},
|
||||||
|
ParquetFilePath,
|
||||||
|
};
|
||||||
use data_types::{ParquetFile, TimestampMinMax};
|
use data_types::{ParquetFile, TimestampMinMax};
|
||||||
use datafusion::{physical_plan::SendableRecordBatchStream, prelude::SessionContext};
|
|
||||||
use predicate::Predicate;
|
|
||||||
use schema::{selection::Selection, Schema};
|
use schema::{selection::Selection, Schema};
|
||||||
use std::{collections::BTreeSet, mem, sync::Arc};
|
use std::{collections::BTreeSet, mem, sync::Arc};
|
||||||
use uuid::Uuid;
|
use uuid::Uuid;
|
||||||
|
@ -78,21 +79,14 @@ impl ParquetChunk {
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Return stream of data read from parquet file
|
/// Return stream of data read from parquet file
|
||||||
pub fn read_filter(
|
/// Inputs for [`ParquetExec`].
|
||||||
&self,
|
///
|
||||||
predicate: &Predicate,
|
/// See [`ParquetExecInput`] for more information.
|
||||||
selection: Selection<'_>,
|
///
|
||||||
session_ctx: &SessionContext,
|
/// [`ParquetExec`]: datafusion::physical_plan::file_format::ParquetExec
|
||||||
) -> Result<SendableRecordBatchStream, crate::storage::ReadError> {
|
pub fn parquet_exec_input(&self) -> ParquetExecInput {
|
||||||
let path: ParquetFilePath = self.parquet_file.as_ref().into();
|
let path: ParquetFilePath = self.parquet_file.as_ref().into();
|
||||||
self.store.read_filter(
|
self.store.parquet_exec_input(&path, self.file_size_bytes())
|
||||||
predicate,
|
|
||||||
selection,
|
|
||||||
Arc::clone(&self.schema.as_arrow()),
|
|
||||||
&path,
|
|
||||||
self.file_size_bytes(),
|
|
||||||
session_ctx,
|
|
||||||
)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// The total number of rows in all row groups in this chunk.
|
/// The total number of rows in all row groups in this chunk.
|
||||||
|
|
|
@ -6,27 +6,26 @@ use crate::{
|
||||||
serialize::{self, CodecError},
|
serialize::{self, CodecError},
|
||||||
ParquetFilePath,
|
ParquetFilePath,
|
||||||
};
|
};
|
||||||
use arrow::datatypes::{Field, SchemaRef};
|
use arrow::{
|
||||||
|
datatypes::{Field, SchemaRef},
|
||||||
|
record_batch::RecordBatch,
|
||||||
|
};
|
||||||
use bytes::Bytes;
|
use bytes::Bytes;
|
||||||
use datafusion::{
|
use datafusion::{
|
||||||
datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl},
|
datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl},
|
||||||
|
error::DataFusionError,
|
||||||
execution::context::TaskContext,
|
execution::context::TaskContext,
|
||||||
physical_plan::{
|
physical_plan::{
|
||||||
execute_stream,
|
|
||||||
file_format::{FileScanConfig, ParquetExec},
|
file_format::{FileScanConfig, ParquetExec},
|
||||||
stream::RecordBatchStreamAdapter,
|
ExecutionPlan, SendableRecordBatchStream, Statistics,
|
||||||
SendableRecordBatchStream, Statistics,
|
|
||||||
},
|
},
|
||||||
prelude::SessionContext,
|
prelude::SessionContext,
|
||||||
};
|
};
|
||||||
use datafusion_util::config::iox_session_config;
|
use datafusion_util::config::iox_session_config;
|
||||||
use futures::TryStreamExt;
|
|
||||||
use object_store::{DynObjectStore, ObjectMeta};
|
use object_store::{DynObjectStore, ObjectMeta};
|
||||||
use observability_deps::tracing::*;
|
use observability_deps::tracing::*;
|
||||||
use predicate::Predicate;
|
|
||||||
use schema::selection::{select_schema, Selection};
|
use schema::selection::{select_schema, Selection};
|
||||||
use std::{
|
use std::{
|
||||||
num::TryFromIntError,
|
|
||||||
sync::Arc,
|
sync::Arc,
|
||||||
time::{Duration, Instant},
|
time::{Duration, Instant},
|
||||||
};
|
};
|
||||||
|
@ -53,38 +52,6 @@ pub enum UploadError {
|
||||||
Upload(#[from] object_store::Error),
|
Upload(#[from] object_store::Error),
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Errors during Parquet file download & scan.
|
|
||||||
#[derive(Debug, Error)]
|
|
||||||
#[allow(clippy::large_enum_variant)]
|
|
||||||
pub enum ReadError {
|
|
||||||
/// Error writing the bytes fetched from object store to the temporary
|
|
||||||
/// parquet file on disk.
|
|
||||||
#[error("i/o error writing downloaded parquet: {0}")]
|
|
||||||
IO(#[from] std::io::Error),
|
|
||||||
|
|
||||||
/// An error fetching Parquet file bytes from object store.
|
|
||||||
#[error("failed to read data from object store: {0}")]
|
|
||||||
ObjectStore(#[from] object_store::Error),
|
|
||||||
|
|
||||||
/// An error reading the downloaded Parquet file.
|
|
||||||
#[error("invalid parquet file: {0}")]
|
|
||||||
Parquet(#[from] parquet::errors::ParquetError),
|
|
||||||
|
|
||||||
/// Schema mismatch
|
|
||||||
#[error("Schema mismatch (expected VS actual parquet file) for file '{path}': {source}")]
|
|
||||||
SchemaMismatch {
|
|
||||||
/// Path of the affected parquet file.
|
|
||||||
path: object_store::path::Path,
|
|
||||||
|
|
||||||
/// Source error
|
|
||||||
source: ProjectionError,
|
|
||||||
},
|
|
||||||
|
|
||||||
/// Malformed integer data for row count
|
|
||||||
#[error("Malformed row count integer")]
|
|
||||||
MalformedRowCount(#[from] TryFromIntError),
|
|
||||||
}
|
|
||||||
|
|
||||||
/// ID for an object store hooked up into DataFusion.
|
/// ID for an object store hooked up into DataFusion.
|
||||||
#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)]
|
#[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)]
|
||||||
pub struct StorageId(&'static str);
|
pub struct StorageId(&'static str);
|
||||||
|
@ -107,6 +74,68 @@ impl std::fmt::Display for StorageId {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Inputs required to build a [`ParquetExec`] for one or multiple files.
|
||||||
|
///
|
||||||
|
/// The files shall be grouped by [`object_store_url`](Self::object_store_url). For each each object store, you shall
|
||||||
|
/// create one [`ParquetExec`] and put each file into its own "file group".
|
||||||
|
///
|
||||||
|
/// [`ParquetExec`]: datafusion::physical_plan::file_format::ParquetExec
|
||||||
|
#[derive(Debug)]
|
||||||
|
pub struct ParquetExecInput {
|
||||||
|
/// Store where the file is located.
|
||||||
|
pub object_store_url: ObjectStoreUrl,
|
||||||
|
|
||||||
|
/// Object metadata.
|
||||||
|
pub object_meta: ObjectMeta,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl ParquetExecInput {
|
||||||
|
/// Read parquet file into [`RecordBatch`]es.
|
||||||
|
///
|
||||||
|
/// This should only be used for testing purposes.
|
||||||
|
pub async fn read_to_batches(
|
||||||
|
&self,
|
||||||
|
schema: SchemaRef,
|
||||||
|
selection: Selection<'_>,
|
||||||
|
session_ctx: &SessionContext,
|
||||||
|
) -> Result<Vec<RecordBatch>, DataFusionError> {
|
||||||
|
// Compute final (output) schema after selection
|
||||||
|
let schema = Arc::new(
|
||||||
|
select_schema(selection, &schema)
|
||||||
|
.as_ref()
|
||||||
|
.clone()
|
||||||
|
.with_metadata(Default::default()),
|
||||||
|
);
|
||||||
|
|
||||||
|
let base_config = FileScanConfig {
|
||||||
|
object_store_url: self.object_store_url.clone(),
|
||||||
|
file_schema: schema,
|
||||||
|
file_groups: vec![vec![PartitionedFile {
|
||||||
|
object_meta: self.object_meta.clone(),
|
||||||
|
partition_values: vec![],
|
||||||
|
range: None,
|
||||||
|
extensions: None,
|
||||||
|
}]],
|
||||||
|
statistics: Statistics::default(),
|
||||||
|
projection: None,
|
||||||
|
limit: None,
|
||||||
|
table_partition_cols: vec![],
|
||||||
|
// TODO avoid this `copied_config` when config_options are directly available on context
|
||||||
|
config_options: session_ctx.copied_config().config_options(),
|
||||||
|
};
|
||||||
|
let exec = ParquetExec::new(base_config, None, None);
|
||||||
|
let exec_schema = exec.schema();
|
||||||
|
datafusion::physical_plan::collect(Arc::new(exec), session_ctx.task_ctx())
|
||||||
|
.await
|
||||||
|
.map(|batches| {
|
||||||
|
for batch in &batches {
|
||||||
|
assert_eq!(batch.schema(), exec_schema);
|
||||||
|
}
|
||||||
|
batches
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// The [`ParquetStorage`] type encapsulates [`RecordBatch`] persistence to an
|
/// The [`ParquetStorage`] type encapsulates [`RecordBatch`] persistence to an
|
||||||
/// underlying [`ObjectStore`].
|
/// underlying [`ObjectStore`].
|
||||||
///
|
///
|
||||||
|
@ -220,72 +249,22 @@ impl ParquetStorage {
|
||||||
Ok((parquet_meta, file_size))
|
Ok((parquet_meta, file_size))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Pull the Parquet-encoded [`RecordBatch`] at the file path derived from
|
/// Inputs for [`ParquetExec`].
|
||||||
/// the provided [`ParquetFilePath`].
|
|
||||||
///
|
///
|
||||||
/// The `selection` projection is pushed down to the Parquet deserializer.
|
/// See [`ParquetExecInput`] for more information.
|
||||||
///
|
///
|
||||||
/// This impl fetches the associated Parquet file bytes from object storage,
|
/// [`ParquetExec`]: datafusion::physical_plan::file_format::ParquetExec
|
||||||
/// temporarily persisting them to a local temp file to feed to the arrow
|
pub fn parquet_exec_input(&self, path: &ParquetFilePath, file_size: usize) -> ParquetExecInput {
|
||||||
/// reader.
|
ParquetExecInput {
|
||||||
///
|
object_store_url: ObjectStoreUrl::parse(format!("iox://{}/", self.id))
|
||||||
/// No caching is performed by `read_filter()`, and each call to
|
.expect("valid object store URL"),
|
||||||
/// `read_filter()` will re-download the parquet file unless the underlying
|
object_meta: ObjectMeta {
|
||||||
/// object store impl caches the fetched bytes.
|
location: path.object_store_path(),
|
||||||
///
|
|
||||||
/// [`RecordBatch`]: arrow::record_batch::RecordBatch
|
|
||||||
pub fn read_filter(
|
|
||||||
&self,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
schema: SchemaRef,
|
|
||||||
path: &ParquetFilePath,
|
|
||||||
file_size: usize,
|
|
||||||
session_ctx: &SessionContext,
|
|
||||||
) -> Result<SendableRecordBatchStream, ReadError> {
|
|
||||||
let path = path.object_store_path();
|
|
||||||
trace!(path=?path, "fetching parquet data for filtered read");
|
|
||||||
|
|
||||||
// Compute final (output) schema after selection
|
|
||||||
let schema = Arc::new(
|
|
||||||
select_schema(selection, &schema)
|
|
||||||
.as_ref()
|
|
||||||
.clone()
|
|
||||||
.with_metadata(Default::default()),
|
|
||||||
);
|
|
||||||
|
|
||||||
// create ParquetExec node
|
|
||||||
let object_meta = ObjectMeta {
|
|
||||||
location: path,
|
|
||||||
// we don't care about the "last modified" field
|
// we don't care about the "last modified" field
|
||||||
last_modified: Default::default(),
|
last_modified: Default::default(),
|
||||||
size: file_size,
|
size: file_size,
|
||||||
};
|
},
|
||||||
let expr = predicate.filter_expr();
|
}
|
||||||
let base_config = FileScanConfig {
|
|
||||||
object_store_url: ObjectStoreUrl::parse(format!("iox://{}/", self.id))
|
|
||||||
.expect("valid object store URL"),
|
|
||||||
file_schema: Arc::clone(&schema),
|
|
||||||
file_groups: vec![vec![PartitionedFile {
|
|
||||||
object_meta,
|
|
||||||
partition_values: vec![],
|
|
||||||
range: None,
|
|
||||||
extensions: None,
|
|
||||||
}]],
|
|
||||||
statistics: Statistics::default(),
|
|
||||||
projection: None,
|
|
||||||
limit: None,
|
|
||||||
table_partition_cols: vec![],
|
|
||||||
// TODO avoid this `copied_config` when config_options are directly available on context
|
|
||||||
config_options: session_ctx.copied_config().config_options(),
|
|
||||||
};
|
|
||||||
let exec = ParquetExec::new(base_config, expr, None);
|
|
||||||
|
|
||||||
Ok(Box::pin(RecordBatchStreamAdapter::new(
|
|
||||||
Arc::clone(&schema),
|
|
||||||
futures::stream::once(execute_stream(Arc::new(exec), session_ctx.task_ctx()))
|
|
||||||
.try_flatten(),
|
|
||||||
)))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -598,24 +577,13 @@ mod tests {
|
||||||
file_size: usize,
|
file_size: usize,
|
||||||
) -> Result<RecordBatch, DataFusionError> {
|
) -> Result<RecordBatch, DataFusionError> {
|
||||||
let path: ParquetFilePath = meta.into();
|
let path: ParquetFilePath = meta.into();
|
||||||
let rx = store
|
store
|
||||||
.read_filter(
|
.parquet_exec_input(&path, file_size)
|
||||||
&Predicate::default(),
|
.read_to_batches(expected_schema, selection, &store.test_df_context())
|
||||||
selection,
|
|
||||||
expected_schema,
|
|
||||||
&path,
|
|
||||||
file_size,
|
|
||||||
&store.test_df_context(),
|
|
||||||
)
|
|
||||||
.expect("should read record batches from object store");
|
|
||||||
let schema = rx.schema();
|
|
||||||
datafusion::physical_plan::common::collect(rx)
|
|
||||||
.await
|
.await
|
||||||
.map(|mut batches| {
|
.map(|mut batches| {
|
||||||
assert_eq!(batches.len(), 1);
|
assert_eq!(batches.len(), 1);
|
||||||
let batch = batches.remove(0);
|
batches.remove(0)
|
||||||
assert_eq!(batch.schema(), schema);
|
|
||||||
batch
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -340,14 +340,13 @@ pub mod tests {
|
||||||
use arrow::{datatypes::DataType, record_batch::RecordBatch};
|
use arrow::{datatypes::DataType, record_batch::RecordBatch};
|
||||||
use arrow_util::assert_batches_eq;
|
use arrow_util::assert_batches_eq;
|
||||||
use data_types::{ColumnType, NamespaceSchema};
|
use data_types::{ColumnType, NamespaceSchema};
|
||||||
use futures::StreamExt;
|
|
||||||
use iox_query::{
|
use iox_query::{
|
||||||
exec::{ExecutorType, IOxSessionContext},
|
exec::{ExecutorType, IOxSessionContext},
|
||||||
QueryChunk, QueryChunkMeta,
|
QueryChunk, QueryChunkMeta,
|
||||||
};
|
};
|
||||||
use iox_tests::util::{TestCatalog, TestNamespace, TestParquetFileBuilder};
|
use iox_tests::util::{TestCatalog, TestNamespace, TestParquetFileBuilder};
|
||||||
use metric::{Attributes, Observation, RawReporter};
|
use metric::{Attributes, Observation, RawReporter};
|
||||||
use schema::{builder::SchemaBuilder, selection::Selection, sort::SortKeyBuilder};
|
use schema::{builder::SchemaBuilder, sort::SortKeyBuilder};
|
||||||
use test_helpers::maybe_start_logging;
|
use test_helpers::maybe_start_logging;
|
||||||
use tokio::runtime::Handle;
|
use tokio::runtime::Handle;
|
||||||
|
|
||||||
|
@ -397,13 +396,9 @@ pub mod tests {
|
||||||
ctx: IOxSessionContext,
|
ctx: IOxSessionContext,
|
||||||
) -> Vec<RecordBatch> {
|
) -> Vec<RecordBatch> {
|
||||||
chunk
|
chunk
|
||||||
.read_filter(ctx, &Default::default(), Selection::All)
|
.data()
|
||||||
.unwrap()
|
.read_to_batches(chunk.schema(), ctx.inner())
|
||||||
.collect::<Vec<_>>()
|
|
||||||
.await
|
.await
|
||||||
.into_iter()
|
|
||||||
.map(Result::unwrap)
|
|
||||||
.collect()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
struct TestData {
|
struct TestData {
|
||||||
|
|
|
@ -1,35 +1,13 @@
|
||||||
use crate::chunk::QuerierChunk;
|
use crate::chunk::QuerierChunk;
|
||||||
use data_types::{ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary};
|
use data_types::{ChunkId, ChunkOrder, DeletePredicate, PartitionId, TableSummary};
|
||||||
use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream};
|
use datafusion::error::DataFusionError;
|
||||||
use iox_query::{
|
use iox_query::{
|
||||||
exec::{stringset::StringSet, IOxSessionContext},
|
exec::{stringset::StringSet, IOxSessionContext},
|
||||||
QueryChunk, QueryChunkMeta,
|
QueryChunk, QueryChunkData, QueryChunkMeta,
|
||||||
};
|
};
|
||||||
use observability_deps::tracing::debug;
|
|
||||||
use predicate::Predicate;
|
use predicate::Predicate;
|
||||||
use schema::{selection::Selection, sort::SortKey, Schema};
|
use schema::{selection::Selection, sort::SortKey, Schema};
|
||||||
use snafu::{ResultExt, Snafu};
|
|
||||||
use std::{any::Any, sync::Arc};
|
use std::{any::Any, sync::Arc};
|
||||||
use trace::span::SpanRecorder;
|
|
||||||
|
|
||||||
#[derive(Debug, Snafu)]
|
|
||||||
pub enum Error {
|
|
||||||
#[snafu(display("Parquet File Error in chunk {}: {}", chunk_id, source))]
|
|
||||||
ParquetFileChunk {
|
|
||||||
source: Box<parquet_file::storage::ReadError>,
|
|
||||||
chunk_id: ChunkId,
|
|
||||||
},
|
|
||||||
|
|
||||||
#[snafu(display(
|
|
||||||
"Could not find column name '{}' in read buffer column_values results for chunk {}",
|
|
||||||
column_name,
|
|
||||||
chunk_id,
|
|
||||||
))]
|
|
||||||
ColumnNameNotFound {
|
|
||||||
column_name: String,
|
|
||||||
chunk_id: ChunkId,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
impl QueryChunkMeta for QuerierChunk {
|
impl QueryChunkMeta for QuerierChunk {
|
||||||
fn summary(&self) -> Arc<TableSummary> {
|
fn summary(&self) -> Arc<TableSummary> {
|
||||||
|
@ -103,42 +81,8 @@ impl QueryChunk for QuerierChunk {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn read_filter(
|
fn data(&self) -> QueryChunkData {
|
||||||
&self,
|
QueryChunkData::Parquet(self.parquet_chunk.parquet_exec_input())
|
||||||
mut ctx: IOxSessionContext,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<SendableRecordBatchStream, DataFusionError> {
|
|
||||||
let span_recorder = SpanRecorder::new(
|
|
||||||
ctx.span()
|
|
||||||
.map(|span| span.child("QuerierChunk::read_filter")),
|
|
||||||
);
|
|
||||||
let delete_predicates: Vec<_> = self
|
|
||||||
.delete_predicates()
|
|
||||||
.iter()
|
|
||||||
.map(|pred| Arc::new(pred.as_ref().clone().into()))
|
|
||||||
.collect();
|
|
||||||
ctx.set_metadata("delete_predicates", delete_predicates.len() as i64);
|
|
||||||
|
|
||||||
// merge the negated delete predicates into the select predicate
|
|
||||||
let pred_with_deleted_exprs = predicate.clone().with_delete_predicates(&delete_predicates);
|
|
||||||
debug!(?pred_with_deleted_exprs, "Merged negated predicate");
|
|
||||||
|
|
||||||
ctx.set_metadata("predicate", format!("{}", &pred_with_deleted_exprs));
|
|
||||||
ctx.set_metadata("projection", format!("{}", selection));
|
|
||||||
ctx.set_metadata("storage", "parquet");
|
|
||||||
|
|
||||||
let chunk_id = self.id();
|
|
||||||
debug!(?predicate, "parquet read_filter");
|
|
||||||
|
|
||||||
// TODO(marco): propagate span all the way down to the object store cache access
|
|
||||||
let _span_recorder = span_recorder;
|
|
||||||
|
|
||||||
self.parquet_chunk
|
|
||||||
.read_filter(&pred_with_deleted_exprs, selection, ctx.inner())
|
|
||||||
.map_err(Box::new)
|
|
||||||
.context(ParquetFileChunkSnafu { chunk_id })
|
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
fn chunk_type(&self) -> &str {
|
fn chunk_type(&self) -> &str {
|
||||||
|
|
|
@ -12,7 +12,6 @@ use data_types::{
|
||||||
TableSummary, TimestampMinMax,
|
TableSummary, TimestampMinMax,
|
||||||
};
|
};
|
||||||
use datafusion::error::DataFusionError;
|
use datafusion::error::DataFusionError;
|
||||||
use datafusion_util::MemoryStream;
|
|
||||||
use futures::{stream::FuturesUnordered, TryStreamExt};
|
use futures::{stream::FuturesUnordered, TryStreamExt};
|
||||||
use generated_types::{
|
use generated_types::{
|
||||||
influxdata::iox::ingester::v1::GetWriteInfoResponse,
|
influxdata::iox::ingester::v1::GetWriteInfoResponse,
|
||||||
|
@ -25,7 +24,7 @@ use influxdb_iox_client::flight::{
|
||||||
use iox_query::{
|
use iox_query::{
|
||||||
exec::{stringset::StringSet, IOxSessionContext},
|
exec::{stringset::StringSet, IOxSessionContext},
|
||||||
util::{compute_timenanosecond_min_max, create_basic_summary},
|
util::{compute_timenanosecond_min_max, create_basic_summary},
|
||||||
QueryChunk, QueryChunkMeta,
|
QueryChunk, QueryChunkData, QueryChunkMeta,
|
||||||
};
|
};
|
||||||
use iox_time::{Time, TimeProvider};
|
use iox_time::{Time, TimeProvider};
|
||||||
use metric::{DurationHistogram, Metric};
|
use metric::{DurationHistogram, Metric};
|
||||||
|
@ -1111,30 +1110,8 @@ impl QueryChunk for IngesterChunk {
|
||||||
Ok(None)
|
Ok(None)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn read_filter(
|
fn data(&self) -> QueryChunkData {
|
||||||
&self,
|
QueryChunkData::RecordBatches(self.batches.clone())
|
||||||
_ctx: IOxSessionContext,
|
|
||||||
predicate: &Predicate,
|
|
||||||
selection: Selection<'_>,
|
|
||||||
) -> Result<datafusion::physical_plan::SendableRecordBatchStream, DataFusionError> {
|
|
||||||
trace!(?predicate, ?selection, input_batches=?self.batches, "Reading data");
|
|
||||||
|
|
||||||
// Apply selection to in-memory batch
|
|
||||||
let batches = match self
|
|
||||||
.schema
|
|
||||||
.df_projection(selection)
|
|
||||||
.map_err(|e| DataFusionError::External(Box::new(e)))?
|
|
||||||
{
|
|
||||||
None => self.batches.clone(),
|
|
||||||
Some(projection) => self
|
|
||||||
.batches
|
|
||||||
.iter()
|
|
||||||
.map(|batch| batch.project(&projection))
|
|
||||||
.collect::<std::result::Result<Vec<_>, ArrowError>>()?,
|
|
||||||
};
|
|
||||||
trace!(?predicate, ?selection, output_batches=?batches, input_batches=?self.batches, "Reading data");
|
|
||||||
|
|
||||||
Ok(Box::pin(MemoryStream::new(batches)))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
fn chunk_type(&self) -> &str {
|
fn chunk_type(&self) -> &str {
|
||||||
|
|
|
@ -1,10 +1,8 @@
|
||||||
use super::IngesterConnection;
|
use super::IngesterConnection;
|
||||||
use arrow::record_batch::RecordBatch;
|
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use data_types::ShardIndex;
|
use data_types::ShardIndex;
|
||||||
use futures::StreamExt;
|
|
||||||
use generated_types::influxdata::iox::ingester::v1::GetWriteInfoResponse;
|
use generated_types::influxdata::iox::ingester::v1::GetWriteInfoResponse;
|
||||||
use iox_query::{exec::IOxSessionContext, util::create_basic_summary, QueryChunk};
|
use iox_query::util::create_basic_summary;
|
||||||
use parking_lot::Mutex;
|
use parking_lot::Mutex;
|
||||||
use schema::selection::Selection;
|
use schema::selection::Selection;
|
||||||
use schema::Schema as IOxSchema;
|
use schema::Schema as IOxSchema;
|
||||||
|
@ -38,7 +36,7 @@ impl IngesterConnection for MockIngesterConnection {
|
||||||
_namespace_name: Arc<str>,
|
_namespace_name: Arc<str>,
|
||||||
_table_name: Arc<str>,
|
_table_name: Arc<str>,
|
||||||
columns: Vec<String>,
|
columns: Vec<String>,
|
||||||
predicate: &predicate::Predicate,
|
_predicate: &predicate::Predicate,
|
||||||
_expected_schema: Arc<schema::Schema>,
|
_expected_schema: Arc<schema::Schema>,
|
||||||
_span: Option<Span>,
|
_span: Option<Span>,
|
||||||
) -> super::Result<Vec<super::IngesterPartition>> {
|
) -> super::Result<Vec<super::IngesterPartition>> {
|
||||||
|
@ -77,14 +75,14 @@ impl IngesterConnection for MockIngesterConnection {
|
||||||
.chunks
|
.chunks
|
||||||
.into_iter()
|
.into_iter()
|
||||||
.map(|ic| async move {
|
.map(|ic| async move {
|
||||||
let mut batches: Vec<RecordBatch> = vec![];
|
let batches: Vec<_> = ic
|
||||||
let mut stream = ic
|
.batches
|
||||||
.read_filter(IOxSessionContext::with_testing(), predicate, selection)
|
.iter()
|
||||||
.expect("Error in read_filter");
|
.map(|batch| match ic.schema.df_projection(selection).unwrap() {
|
||||||
while let Some(b) = stream.next().await {
|
Some(projection) => batch.project(&projection).unwrap(),
|
||||||
let b = b.expect("Error in stream");
|
None => batch.clone(),
|
||||||
batches.push(b)
|
})
|
||||||
}
|
.collect();
|
||||||
|
|
||||||
assert!(!batches.is_empty(), "Error: empty batches");
|
assert!(!batches.is_empty(), "Error: empty batches");
|
||||||
let new_schema = IOxSchema::try_from(batches[0].schema()).unwrap();
|
let new_schema = IOxSchema::try_from(batches[0].schema()).unwrap();
|
||||||
|
|
|
@ -520,7 +520,7 @@ mod tests {
|
||||||
use iox_query::exec::IOxSessionContext;
|
use iox_query::exec::IOxSessionContext;
|
||||||
use iox_tests::util::{TestCatalog, TestParquetFileBuilder, TestTable};
|
use iox_tests::util::{TestCatalog, TestParquetFileBuilder, TestTable};
|
||||||
use predicate::Predicate;
|
use predicate::Predicate;
|
||||||
use schema::{builder::SchemaBuilder, selection::Selection, InfluxFieldType};
|
use schema::{builder::SchemaBuilder, InfluxFieldType};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use test_helpers::maybe_start_logging;
|
use test_helpers::maybe_start_logging;
|
||||||
use trace::{span::SpanStatus, RingBufferTraceCollector};
|
use trace::{span::SpanStatus, RingBufferTraceCollector};
|
||||||
|
@ -711,8 +711,8 @@ mod tests {
|
||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
assert_eq!(chunks.len(), 1);
|
assert_eq!(chunks.len(), 1);
|
||||||
|
|
||||||
let chunk = &chunks[0];
|
let chunk = &chunks[0];
|
||||||
|
assert_eq!(chunk.chunk_type(), "IngesterPartition");
|
||||||
|
|
||||||
// verify chunk schema
|
// verify chunk schema
|
||||||
let schema = chunk.schema();
|
let schema = chunk.schema();
|
||||||
|
@ -739,17 +739,9 @@ mod tests {
|
||||||
|
|
||||||
// verify chunk data
|
// verify chunk data
|
||||||
let batches = chunk
|
let batches = chunk
|
||||||
.read_filter(
|
.data()
|
||||||
IOxSessionContext::with_testing(),
|
.read_to_batches(chunk.schema(), IOxSessionContext::with_testing().inner())
|
||||||
&Default::default(),
|
.await;
|
||||||
Selection::All,
|
|
||||||
)
|
|
||||||
.unwrap()
|
|
||||||
.collect::<Vec<_>>()
|
|
||||||
.await
|
|
||||||
.into_iter()
|
|
||||||
.map(Result::unwrap)
|
|
||||||
.collect::<Vec<_>>();
|
|
||||||
let expected = vec![
|
let expected = vec![
|
||||||
"+-----+------+------+--------------------------------+",
|
"+-----+------+------+--------------------------------+",
|
||||||
"| foo | tag1 | tag2 | time |",
|
"| foo | tag1 | tag2 | time |",
|
||||||
|
|
Loading…
Reference in New Issue