mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-10 23:32:55 +00:00
refactor: make table scan return physical plan (#326)
* refactor: return PhysicalPlan in Table trait's scan method, to support partitioned execution in Frontend's distribute read * refactor: pub use necessary DataFusion types * refactor: replace old "PhysicalPlan" and its adapters Co-authored-by: luofucong <luofucong@greptime.com> Co-authored-by: Yingwen <realevenyag@gmail.com>
This commit is contained in:
3
Cargo.lock
generated
3
Cargo.lock
generated
@@ -937,6 +937,7 @@ name = "common-query"
|
||||
version = "0.1.0"
|
||||
dependencies = [
|
||||
"arrow2",
|
||||
"async-trait",
|
||||
"common-base",
|
||||
"common-error",
|
||||
"common-recordbatch",
|
||||
@@ -955,6 +956,7 @@ name = "common-recordbatch"
|
||||
version = "0.1.0"
|
||||
dependencies = [
|
||||
"common-error",
|
||||
"datafusion",
|
||||
"datafusion-common",
|
||||
"datatypes",
|
||||
"futures",
|
||||
@@ -5135,6 +5137,7 @@ dependencies = [
|
||||
"common-recordbatch",
|
||||
"common-telemetry",
|
||||
"common-time",
|
||||
"datafusion",
|
||||
"datafusion-common",
|
||||
"datatypes",
|
||||
"futures",
|
||||
|
||||
@@ -103,6 +103,18 @@ pub enum Error {
|
||||
|
||||
#[snafu(display("Illegal catalog manager state: {}", msg))]
|
||||
IllegalManagerState { backtrace: Backtrace, msg: String },
|
||||
|
||||
#[snafu(display("Failed to scan system catalog table, source: {}", source))]
|
||||
SystemCatalogTableScan {
|
||||
#[snafu(backtrace)]
|
||||
source: table::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to execute system catalog table scan, source: {}", source))]
|
||||
SystemCatalogTableScanExec {
|
||||
#[snafu(backtrace)]
|
||||
source: common_query::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -131,7 +143,10 @@ impl ErrorExt for Error {
|
||||
| Error::CreateSystemCatalog { source, .. }
|
||||
| Error::InsertTableRecord { source, .. }
|
||||
| Error::OpenTable { source, .. }
|
||||
| Error::CreateTable { source, .. } => source.status_code(),
|
||||
| Error::CreateTable { source, .. }
|
||||
| Error::SystemCatalogTableScan { source } => source.status_code(),
|
||||
|
||||
Error::SystemCatalogTableScanExec { source } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -3,6 +3,8 @@ use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use common_telemetry::debug;
|
||||
use common_time::timestamp::Timestamp;
|
||||
@@ -22,7 +24,7 @@ use crate::consts::{
|
||||
SYSTEM_CATALOG_TABLE_NAME,
|
||||
};
|
||||
use crate::error::{
|
||||
CreateSystemCatalogSnafu, EmptyValueSnafu, Error, InvalidEntryTypeSnafu, InvalidKeySnafu,
|
||||
self, CreateSystemCatalogSnafu, EmptyValueSnafu, Error, InvalidEntryTypeSnafu, InvalidKeySnafu,
|
||||
OpenSystemCatalogSnafu, Result, ValueDeserializeSnafu,
|
||||
};
|
||||
|
||||
@@ -51,7 +53,7 @@ impl Table for SystemCatalogTable {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> table::Result<SendableRecordBatchStream> {
|
||||
) -> table::Result<PhysicalPlanRef> {
|
||||
panic!("System catalog table does not support scan!")
|
||||
}
|
||||
|
||||
@@ -111,7 +113,15 @@ impl SystemCatalogTable {
|
||||
/// Create a stream of all entries inside system catalog table
|
||||
pub async fn records(&self) -> Result<SendableRecordBatchStream> {
|
||||
let full_projection = None;
|
||||
let stream = self.table.scan(&full_projection, &[], None).await.unwrap();
|
||||
let scan = self
|
||||
.table
|
||||
.scan(&full_projection, &[], None)
|
||||
.await
|
||||
.context(error::SystemCatalogTableScanSnafu)?;
|
||||
let stream = scan
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.context(error::SystemCatalogTableScanExecSnafu)?;
|
||||
Ok(stream)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,8 +7,9 @@ use std::task::{Context, Poll};
|
||||
|
||||
use async_stream::stream;
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_recordbatch::error::Result as RecordBatchResult;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream};
|
||||
use datatypes::prelude::{ConcreteDataType, VectorBuilder};
|
||||
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
|
||||
use datatypes::value::Value;
|
||||
@@ -17,6 +18,7 @@ use futures::Stream;
|
||||
use snafu::ResultExt;
|
||||
use table::engine::TableEngineRef;
|
||||
use table::metadata::{TableId, TableInfoRef};
|
||||
use table::table::scan::SimpleTableScan;
|
||||
use table::{Table, TableRef};
|
||||
|
||||
use crate::consts::{INFORMATION_SCHEMA_NAME, SYSTEM_CATALOG_TABLE_NAME};
|
||||
@@ -62,7 +64,7 @@ impl Table for Tables {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> table::error::Result<SendableRecordBatchStream> {
|
||||
) -> table::error::Result<PhysicalPlanRef> {
|
||||
let catalogs = self.catalogs.clone();
|
||||
let schema_ref = self.schema.clone();
|
||||
let engine_name = self.engine_name.clone();
|
||||
@@ -89,10 +91,11 @@ impl Table for Tables {
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(TablesRecordBatchStream {
|
||||
let stream = Box::pin(TablesRecordBatchStream {
|
||||
schema: self.schema.clone(),
|
||||
stream: Box::pin(stream),
|
||||
}))
|
||||
});
|
||||
Ok(Arc::new(SimpleTableScan::new(stream)))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -277,6 +280,7 @@ fn build_schema_for_tables() -> Schema {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use datatypes::arrow::array::Utf8Array;
|
||||
use datatypes::arrow::datatypes::DataType;
|
||||
use futures_util::StreamExt;
|
||||
@@ -298,7 +302,11 @@ mod tests {
|
||||
catalog_list.register_catalog("test_catalog".to_string(), catalog_provider);
|
||||
let tables = Tables::new(catalog_list, "test_engine".to_string());
|
||||
|
||||
let mut tables_stream = tables.scan(&None, &[], None).await.unwrap();
|
||||
let tables_stream = tables.scan(&None, &[], None).await.unwrap();
|
||||
let mut tables_stream = tables_stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
if let Some(t) = tables_stream.next().await {
|
||||
let batch = t.unwrap().df_recordbatch;
|
||||
assert_eq!(1, batch.num_rows());
|
||||
|
||||
@@ -4,6 +4,7 @@ version = "0.1.0"
|
||||
edition = "2021"
|
||||
|
||||
[dependencies]
|
||||
async-trait = "0.1"
|
||||
common-error = { path = "../error" }
|
||||
common-recordbatch = { path = "../recordbatch" }
|
||||
common-time = { path = "../time" }
|
||||
|
||||
@@ -62,6 +62,36 @@ pub enum InnerError {
|
||||
|
||||
#[snafu(display("unexpected: not constant column"))]
|
||||
InvalidInputCol { backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Not expected to run ExecutionPlan more than once"))]
|
||||
ExecuteRepeatedly { backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("General DataFusion error, source: {}", source))]
|
||||
GeneralDataFusion {
|
||||
source: DataFusionError,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to execute DataFusion ExecutionPlan, source: {}", source))]
|
||||
DataFusionExecutionPlan {
|
||||
source: DataFusionError,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to convert DataFusion's recordbatch stream, source: {}",
|
||||
source
|
||||
))]
|
||||
ConvertDfRecordBatchStream {
|
||||
#[snafu(backtrace)]
|
||||
source: common_recordbatch::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to convert arrow schema, source: {}", source))]
|
||||
ConvertArrowSchema {
|
||||
#[snafu(backtrace)]
|
||||
source: DataTypeError,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -76,9 +106,17 @@ impl ErrorExt for InnerError {
|
||||
| InnerError::InvalidInputState { .. }
|
||||
| InnerError::InvalidInputCol { .. }
|
||||
| InnerError::BadAccumulatorImpl { .. } => StatusCode::EngineExecuteQuery,
|
||||
InnerError::InvalidInputs { source, .. } => source.status_code(),
|
||||
InnerError::IntoVector { source, .. } => source.status_code(),
|
||||
InnerError::FromScalarValue { source } => source.status_code(),
|
||||
|
||||
InnerError::InvalidInputs { source, .. }
|
||||
| InnerError::IntoVector { source, .. }
|
||||
| InnerError::FromScalarValue { source }
|
||||
| InnerError::ConvertArrowSchema { source } => source.status_code(),
|
||||
|
||||
InnerError::ExecuteRepeatedly { .. }
|
||||
| InnerError::GeneralDataFusion { .. }
|
||||
| InnerError::DataFusionExecutionPlan { .. } => StatusCode::Unexpected,
|
||||
|
||||
InnerError::ConvertDfRecordBatchStream { source, .. } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -105,6 +143,7 @@ impl From<Error> for DataFusionError {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use arrow::error::ArrowError;
|
||||
use snafu::GenerateImplicitData;
|
||||
|
||||
use super::*;
|
||||
@@ -127,6 +166,48 @@ mod tests {
|
||||
.unwrap()
|
||||
.into();
|
||||
assert_error(&err, StatusCode::EngineExecuteQuery);
|
||||
|
||||
let err: Error = throw_df_error()
|
||||
.context(GeneralDataFusionSnafu)
|
||||
.err()
|
||||
.unwrap()
|
||||
.into();
|
||||
assert_error(&err, StatusCode::Unexpected);
|
||||
|
||||
let err: Error = throw_df_error()
|
||||
.context(DataFusionExecutionPlanSnafu)
|
||||
.err()
|
||||
.unwrap()
|
||||
.into();
|
||||
assert_error(&err, StatusCode::Unexpected);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_execute_repeatedly_error() {
|
||||
let error: Error = None::<i32>
|
||||
.context(ExecuteRepeatedlySnafu)
|
||||
.err()
|
||||
.unwrap()
|
||||
.into();
|
||||
assert_eq!(error.inner.status_code(), StatusCode::Unexpected);
|
||||
assert!(error.backtrace_opt().is_some());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_convert_df_recordbatch_stream_error() {
|
||||
let result: std::result::Result<i32, common_recordbatch::error::Error> =
|
||||
Err(common_recordbatch::error::InnerError::PollStream {
|
||||
source: ArrowError::Overflow,
|
||||
backtrace: Backtrace::generate(),
|
||||
}
|
||||
.into());
|
||||
let error: Error = result
|
||||
.context(ConvertDfRecordBatchStreamSnafu)
|
||||
.err()
|
||||
.unwrap()
|
||||
.into();
|
||||
assert_eq!(error.inner.status_code(), StatusCode::Internal);
|
||||
assert!(error.backtrace_opt().is_some());
|
||||
}
|
||||
|
||||
fn raise_datatype_error() -> std::result::Result<(), DataTypeError> {
|
||||
|
||||
@@ -4,6 +4,7 @@ pub mod columnar_value;
|
||||
pub mod error;
|
||||
mod function;
|
||||
pub mod logical_plan;
|
||||
pub mod physical_plan;
|
||||
pub mod prelude;
|
||||
mod signature;
|
||||
|
||||
@@ -13,3 +14,5 @@ pub enum Output {
|
||||
RecordBatches(RecordBatches),
|
||||
Stream(SendableRecordBatchStream),
|
||||
}
|
||||
|
||||
pub use datafusion::physical_plan::ExecutionPlan as DfPhysicalPlan;
|
||||
|
||||
@@ -2,7 +2,7 @@ use datafusion::logical_plan::Expr as DfExpr;
|
||||
|
||||
/// Central struct of query API.
|
||||
/// Represent logical expressions such as `A + 1`, or `CAST(c1 AS int)`.
|
||||
#[derive(Clone, PartialEq, Hash)]
|
||||
#[derive(Clone, PartialEq, Hash, Debug)]
|
||||
pub struct Expr {
|
||||
df_expr: DfExpr,
|
||||
}
|
||||
|
||||
325
src/common/query/src/physical_plan.rs
Normal file
325
src/common/query/src/physical_plan.rs
Normal file
@@ -0,0 +1,325 @@
|
||||
use std::any::Any;
|
||||
use std::fmt::Debug;
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_recordbatch::adapter::{DfRecordBatchStreamAdapter, RecordBatchStreamAdapter};
|
||||
use common_recordbatch::DfSendableRecordBatchStream;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use datafusion::arrow::datatypes::SchemaRef as DfSchemaRef;
|
||||
use datafusion::error::Result as DfResult;
|
||||
pub use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
use datafusion::physical_plan::expressions::PhysicalSortExpr;
|
||||
pub use datafusion::physical_plan::Partitioning;
|
||||
use datafusion::physical_plan::Statistics;
|
||||
use datatypes::schema::SchemaRef;
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::DfPhysicalPlan;
|
||||
|
||||
pub type PhysicalPlanRef = Arc<dyn PhysicalPlan>;
|
||||
|
||||
/// `PhysicalPlan` represent nodes in the Physical Plan.
|
||||
///
|
||||
/// Each `PhysicalPlan` is Partition-aware and is responsible for
|
||||
/// creating the actual `async` [`SendableRecordBatchStream`]s
|
||||
/// of [`RecordBatch`] that incrementally compute the operator's
|
||||
/// output from its input partition.
|
||||
#[async_trait]
|
||||
pub trait PhysicalPlan: Debug + Send + Sync {
|
||||
/// Returns the physical plan as [`Any`](std::any::Any) so that it can be
|
||||
/// downcast to a specific implementation.
|
||||
fn as_any(&self) -> &dyn Any;
|
||||
|
||||
/// Get the schema for this physical plan
|
||||
fn schema(&self) -> SchemaRef;
|
||||
|
||||
/// Specifies the output partitioning scheme of this plan
|
||||
fn output_partitioning(&self) -> Partitioning;
|
||||
|
||||
/// Get a list of child physical plans that provide the input for this plan. The returned list
|
||||
/// will be empty for leaf nodes, will contain a single value for unary nodes, or two
|
||||
/// values for binary nodes (such as joins).
|
||||
fn children(&self) -> Vec<PhysicalPlanRef>;
|
||||
|
||||
/// Returns a new plan where all children were replaced by new plans.
|
||||
/// The size of `children` must be equal to the size of `PhysicalPlan::children()`.
|
||||
fn with_new_children(&self, children: Vec<PhysicalPlanRef>) -> Result<PhysicalPlanRef>;
|
||||
|
||||
/// Creates an RecordBatch stream.
|
||||
async fn execute(
|
||||
&self,
|
||||
partition: usize,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
) -> Result<SendableRecordBatchStream>;
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct PhysicalPlanAdapter {
|
||||
schema: SchemaRef,
|
||||
df_plan: Arc<dyn DfPhysicalPlan>,
|
||||
}
|
||||
|
||||
impl PhysicalPlanAdapter {
|
||||
pub fn new(schema: SchemaRef, df_plan: Arc<dyn DfPhysicalPlan>) -> Self {
|
||||
Self { schema, df_plan }
|
||||
}
|
||||
|
||||
pub fn df_plan(&self) -> Arc<dyn DfPhysicalPlan> {
|
||||
self.df_plan.clone()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl PhysicalPlan for PhysicalPlanAdapter {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
self.df_plan.output_partitioning()
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<PhysicalPlanRef> {
|
||||
self.df_plan
|
||||
.children()
|
||||
.into_iter()
|
||||
.map(|x| Arc::new(PhysicalPlanAdapter::new(self.schema(), x)) as _)
|
||||
.collect()
|
||||
}
|
||||
|
||||
fn with_new_children(&self, children: Vec<PhysicalPlanRef>) -> Result<PhysicalPlanRef> {
|
||||
let children = children
|
||||
.into_iter()
|
||||
.map(|x| Arc::new(DfPhysicalPlanAdapter(x)) as _)
|
||||
.collect();
|
||||
let plan = self
|
||||
.df_plan
|
||||
.with_new_children(children)
|
||||
.context(error::GeneralDataFusionSnafu)?;
|
||||
Ok(Arc::new(PhysicalPlanAdapter::new(self.schema(), plan)))
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
partition: usize,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
let stream = self
|
||||
.df_plan
|
||||
.execute(partition, runtime)
|
||||
.await
|
||||
.context(error::DataFusionExecutionPlanSnafu)?;
|
||||
let stream = RecordBatchStreamAdapter::try_new(stream)
|
||||
.context(error::ConvertDfRecordBatchStreamSnafu)?;
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct DfPhysicalPlanAdapter(pub PhysicalPlanRef);
|
||||
|
||||
#[async_trait]
|
||||
impl DfPhysicalPlan for DfPhysicalPlanAdapter {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
self.0.schema().arrow_schema().clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
self.0.output_partitioning()
|
||||
}
|
||||
|
||||
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
|
||||
None
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<Arc<dyn DfPhysicalPlan>> {
|
||||
self.0
|
||||
.children()
|
||||
.into_iter()
|
||||
.map(|x| Arc::new(DfPhysicalPlanAdapter(x)) as _)
|
||||
.collect()
|
||||
}
|
||||
|
||||
fn with_new_children(
|
||||
&self,
|
||||
children: Vec<Arc<dyn DfPhysicalPlan>>,
|
||||
) -> DfResult<Arc<dyn DfPhysicalPlan>> {
|
||||
let df_schema = self.schema();
|
||||
let schema: SchemaRef = Arc::new(
|
||||
df_schema
|
||||
.try_into()
|
||||
.context(error::ConvertArrowSchemaSnafu)
|
||||
.map_err(error::Error::from)?,
|
||||
);
|
||||
let children = children
|
||||
.into_iter()
|
||||
.map(|x| Arc::new(PhysicalPlanAdapter::new(schema.clone(), x)) as _)
|
||||
.collect();
|
||||
let plan = self.0.with_new_children(children)?;
|
||||
Ok(Arc::new(DfPhysicalPlanAdapter(plan)))
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
partition: usize,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
) -> DfResult<DfSendableRecordBatchStream> {
|
||||
let stream = self.0.execute(partition, runtime).await?;
|
||||
Ok(Box::pin(DfRecordBatchStreamAdapter::new(stream)))
|
||||
}
|
||||
|
||||
fn statistics(&self) -> Statistics {
|
||||
// TODO(LFC): impl statistics
|
||||
Statistics::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use arrow::datatypes::{DataType, Field, Schema as ArrowSchema};
|
||||
use common_recordbatch::{RecordBatch, RecordBatches};
|
||||
use datafusion::arrow_print;
|
||||
use datafusion::datasource::TableProvider as DfTableProvider;
|
||||
use datafusion::logical_plan::LogicalPlanBuilder;
|
||||
use datafusion::physical_plan::collect;
|
||||
use datafusion::physical_plan::empty::EmptyExec;
|
||||
use datafusion::prelude::ExecutionContext;
|
||||
use datafusion_common::field_util::SchemaExt;
|
||||
use datafusion_expr::Expr;
|
||||
use datatypes::schema::Schema;
|
||||
use datatypes::vectors::Int32Vector;
|
||||
|
||||
use super::*;
|
||||
|
||||
struct MyDfTableProvider;
|
||||
|
||||
#[async_trait]
|
||||
impl DfTableProvider for MyDfTableProvider {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
Arc::new(ArrowSchema::new(vec![Field::new(
|
||||
"a",
|
||||
DataType::Int32,
|
||||
false,
|
||||
)]))
|
||||
}
|
||||
|
||||
async fn scan(
|
||||
&self,
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> DfResult<Arc<dyn DfPhysicalPlan>> {
|
||||
let schema = Schema::try_from(self.schema()).unwrap();
|
||||
let my_plan = Arc::new(MyExecutionPlan {
|
||||
schema: Arc::new(schema),
|
||||
});
|
||||
let df_plan = DfPhysicalPlanAdapter(my_plan);
|
||||
Ok(Arc::new(df_plan))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct MyExecutionPlan {
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl PhysicalPlan for MyExecutionPlan {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
Partitioning::UnknownPartitioning(1)
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<PhysicalPlanRef> {
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn with_new_children(&self, _children: Vec<PhysicalPlanRef>) -> Result<PhysicalPlanRef> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
_partition: usize,
|
||||
_runtime: Arc<RuntimeEnv>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
let schema = self.schema();
|
||||
let recordbatches = RecordBatches::try_new(
|
||||
schema.clone(),
|
||||
vec![
|
||||
RecordBatch::new(
|
||||
schema.clone(),
|
||||
vec![Arc::new(Int32Vector::from_slice(vec![1])) as _],
|
||||
)
|
||||
.unwrap(),
|
||||
RecordBatch::new(
|
||||
schema,
|
||||
vec![Arc::new(Int32Vector::from_slice(vec![2, 3])) as _],
|
||||
)
|
||||
.unwrap(),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
Ok(recordbatches.as_stream())
|
||||
}
|
||||
}
|
||||
|
||||
// Test our physical plan can be executed by DataFusion, through adapters.
|
||||
#[tokio::test]
|
||||
async fn test_execute_physical_plan() {
|
||||
let ctx = ExecutionContext::new();
|
||||
let logical_plan = LogicalPlanBuilder::scan("test", Arc::new(MyDfTableProvider), None)
|
||||
.unwrap()
|
||||
.build()
|
||||
.unwrap();
|
||||
let physical_plan = ctx.create_physical_plan(&logical_plan).await.unwrap();
|
||||
let df_recordbatches = collect(physical_plan, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let pretty_print = arrow_print::write(&df_recordbatches);
|
||||
let pretty_print = pretty_print.lines().collect::<Vec<&str>>();
|
||||
assert_eq!(
|
||||
pretty_print,
|
||||
vec!["+---+", "| a |", "+---+", "| 1 |", "| 2 |", "| 3 |", "+---+",]
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_physical_plan_adapter() {
|
||||
let df_schema = Arc::new(ArrowSchema::new(vec![Field::new(
|
||||
"name",
|
||||
DataType::Utf8,
|
||||
true,
|
||||
)]));
|
||||
|
||||
let plan = PhysicalPlanAdapter::new(
|
||||
Arc::new(Schema::try_from(df_schema.clone()).unwrap()),
|
||||
Arc::new(EmptyExec::new(true, df_schema.clone())),
|
||||
);
|
||||
assert!(plan.df_plan.as_any().downcast_ref::<EmptyExec>().is_some());
|
||||
|
||||
let df_plan = DfPhysicalPlanAdapter(Arc::new(plan));
|
||||
assert_eq!(df_schema, df_plan.schema());
|
||||
}
|
||||
}
|
||||
@@ -5,6 +5,7 @@ edition = "2021"
|
||||
|
||||
[dependencies]
|
||||
common-error = { path = "../error" }
|
||||
datafusion = { git = "https://github.com/apache/arrow-datafusion.git", branch = "arrow2", features = ["simd"] }
|
||||
datafusion-common = { git = "https://github.com/apache/arrow-datafusion.git", branch = "arrow2" }
|
||||
datatypes = { path = "../../datatypes" }
|
||||
futures = "0.3"
|
||||
|
||||
92
src/common/recordbatch/src/adapter.rs
Normal file
92
src/common/recordbatch/src/adapter.rs
Normal file
@@ -0,0 +1,92 @@
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
use datafusion::arrow::datatypes::SchemaRef as DfSchemaRef;
|
||||
use datafusion::physical_plan::RecordBatchStream as DfRecordBatchStream;
|
||||
use datafusion_common::record_batch::RecordBatch as DfRecordBatch;
|
||||
use datatypes::arrow::error::ArrowError;
|
||||
use datatypes::arrow::error::Result as ArrowResult;
|
||||
use datatypes::schema::{Schema, SchemaRef};
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::DfSendableRecordBatchStream;
|
||||
use crate::{RecordBatch, RecordBatchStream, SendableRecordBatchStream, Stream};
|
||||
|
||||
/// Greptime SendableRecordBatchStream -> DataFusion RecordBatchStream
|
||||
pub struct DfRecordBatchStreamAdapter {
|
||||
stream: SendableRecordBatchStream,
|
||||
}
|
||||
|
||||
impl DfRecordBatchStreamAdapter {
|
||||
pub fn new(stream: SendableRecordBatchStream) -> Self {
|
||||
Self { stream }
|
||||
}
|
||||
}
|
||||
|
||||
impl DfRecordBatchStream for DfRecordBatchStreamAdapter {
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
self.stream.schema().arrow_schema().clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for DfRecordBatchStreamAdapter {
|
||||
type Item = ArrowResult<DfRecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.stream).poll_next(cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Some(recordbatch)) => match recordbatch {
|
||||
Ok(recordbatch) => Poll::Ready(Some(Ok(recordbatch.df_recordbatch))),
|
||||
Err(e) => Poll::Ready(Some(Err(ArrowError::External("".to_owned(), Box::new(e))))),
|
||||
},
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn size_hint(&self) -> (usize, Option<usize>) {
|
||||
self.stream.size_hint()
|
||||
}
|
||||
}
|
||||
|
||||
/// DataFusion SendableRecordBatchStream -> Greptime RecordBatchStream
|
||||
pub struct RecordBatchStreamAdapter {
|
||||
schema: SchemaRef,
|
||||
stream: DfSendableRecordBatchStream,
|
||||
}
|
||||
|
||||
impl RecordBatchStreamAdapter {
|
||||
pub fn try_new(stream: DfSendableRecordBatchStream) -> Result<Self> {
|
||||
let schema =
|
||||
Arc::new(Schema::try_from(stream.schema()).context(error::SchemaConversionSnafu)?);
|
||||
Ok(Self { schema, stream })
|
||||
}
|
||||
}
|
||||
|
||||
impl RecordBatchStream for RecordBatchStreamAdapter {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for RecordBatchStreamAdapter {
|
||||
type Item = Result<RecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.stream).poll_next(cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Some(df_recordbatch)) => Poll::Ready(Some(Ok(RecordBatch {
|
||||
schema: self.schema(),
|
||||
df_recordbatch: df_recordbatch.context(error::PollStreamSnafu)?,
|
||||
}))),
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn size_hint(&self) -> (usize, Option<usize>) {
|
||||
self.stream.size_hint()
|
||||
}
|
||||
}
|
||||
@@ -33,16 +33,32 @@ pub enum InnerError {
|
||||
reason: String,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to convert Arrow schema, source: {}", source))]
|
||||
SchemaConversion {
|
||||
source: datatypes::error::Error,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to poll stream, source: {}", source))]
|
||||
PollStream {
|
||||
source: datatypes::arrow::error::ArrowError,
|
||||
backtrace: Backtrace,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for InnerError {
|
||||
fn status_code(&self) -> StatusCode {
|
||||
match self {
|
||||
InnerError::NewDfRecordBatch { .. } => StatusCode::InvalidArguments,
|
||||
InnerError::DataTypes { .. } | InnerError::CreateRecordBatches { .. } => {
|
||||
StatusCode::Internal
|
||||
}
|
||||
|
||||
InnerError::DataTypes { .. }
|
||||
| InnerError::CreateRecordBatches { .. }
|
||||
| InnerError::PollStream { .. } => StatusCode::Internal,
|
||||
|
||||
InnerError::External { source } => source.status_code(),
|
||||
|
||||
InnerError::SchemaConversion { source, .. } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,9 +1,11 @@
|
||||
pub mod adapter;
|
||||
pub mod error;
|
||||
mod recordbatch;
|
||||
pub mod util;
|
||||
|
||||
use std::pin::Pin;
|
||||
|
||||
pub use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
|
||||
use datatypes::schema::SchemaRef;
|
||||
use error::Result;
|
||||
use futures::task::{Context, Poll};
|
||||
@@ -74,6 +76,41 @@ impl RecordBatches {
|
||||
pub fn take(self) -> Vec<RecordBatch> {
|
||||
self.batches
|
||||
}
|
||||
|
||||
pub fn as_stream(&self) -> SendableRecordBatchStream {
|
||||
Box::pin(SimpleRecordBatchStream {
|
||||
inner: RecordBatches {
|
||||
schema: self.schema(),
|
||||
batches: self.batches.clone(),
|
||||
},
|
||||
index: 0,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
pub struct SimpleRecordBatchStream {
|
||||
inner: RecordBatches,
|
||||
index: usize,
|
||||
}
|
||||
|
||||
impl RecordBatchStream for SimpleRecordBatchStream {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.inner.schema()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for SimpleRecordBatchStream {
|
||||
type Item = Result<RecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
Poll::Ready(if self.index < self.inner.batches.len() {
|
||||
let batch = self.inner.batches[self.index].clone();
|
||||
self.index += 1;
|
||||
Some(Ok(batch))
|
||||
} else {
|
||||
None
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@@ -116,4 +153,27 @@ mod tests {
|
||||
assert_eq!(schema1, batches.schema());
|
||||
assert_eq!(vec![batch1], batches.take());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_simple_recordbatch_stream() {
|
||||
let column_a = ColumnSchema::new("a", ConcreteDataType::int32_datatype(), false);
|
||||
let column_b = ColumnSchema::new("b", ConcreteDataType::string_datatype(), false);
|
||||
let schema = Arc::new(Schema::new(vec![column_a, column_b]));
|
||||
|
||||
let va1: VectorRef = Arc::new(Int32Vector::from_slice(&[1, 2]));
|
||||
let vb1: VectorRef = Arc::new(StringVector::from(vec!["a", "b"]));
|
||||
let batch1 = RecordBatch::new(schema.clone(), vec![va1, vb1]).unwrap();
|
||||
|
||||
let va2: VectorRef = Arc::new(Int32Vector::from_slice(&[3, 4, 5]));
|
||||
let vb2: VectorRef = Arc::new(StringVector::from(vec!["c", "d", "e"]));
|
||||
let batch2 = RecordBatch::new(schema.clone(), vec![va2, vb2]).unwrap();
|
||||
|
||||
let recordbatches =
|
||||
RecordBatches::try_new(schema.clone(), vec![batch1.clone(), batch2.clone()]).unwrap();
|
||||
let stream = recordbatches.as_stream();
|
||||
let collected = util::collect(stream).await.unwrap();
|
||||
assert_eq!(collected.len(), 2);
|
||||
assert_eq!(collected[0], batch1);
|
||||
assert_eq!(collected[1], batch2);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -364,8 +364,8 @@ mod tests {
|
||||
insert_expr, Column, ColumnDataType,
|
||||
};
|
||||
use common_base::BitVec;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_query::prelude::Expr;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use common_time::timestamp::Timestamp;
|
||||
use datatypes::{
|
||||
data_type::ConcreteDataType,
|
||||
@@ -552,7 +552,7 @@ mod tests {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> TableResult<SendableRecordBatchStream> {
|
||||
) -> TableResult<PhysicalPlanRef> {
|
||||
unimplemented!();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2,17 +2,16 @@ use std::sync::Arc;
|
||||
|
||||
use common_grpc::AsExcutionPlan;
|
||||
use common_grpc::DefaultAsPlanImpl;
|
||||
use common_query::physical_plan::PhysicalPlanAdapter;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_query::Output;
|
||||
use datatypes::schema::Schema;
|
||||
use query::PhysicalPlanAdapter;
|
||||
use query::{plan::PhysicalPlan, QueryEngineRef};
|
||||
use query::QueryEngineRef;
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::error::{ConvertSchemaSnafu, ExecutePhysicalPlanSnafu, IntoPhysicalPlanSnafu};
|
||||
|
||||
pub type PhysicalPlanRef = Arc<dyn PhysicalPlan>;
|
||||
|
||||
pub struct PhysicalPlanner {
|
||||
query_engine: QueryEngineRef,
|
||||
}
|
||||
|
||||
@@ -92,7 +92,7 @@ mod tests {
|
||||
|
||||
use catalog::SchemaProvider;
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_time::timestamp::Timestamp;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, SchemaBuilder, SchemaRef};
|
||||
@@ -146,7 +146,7 @@ mod tests {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> TableResult<SendableRecordBatchStream> {
|
||||
) -> TableResult<PhysicalPlanRef> {
|
||||
unimplemented!();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2,7 +2,6 @@
|
||||
name = "meta-client"
|
||||
version = "0.1.0"
|
||||
edition = "2021"
|
||||
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
|
||||
|
||||
[dependencies]
|
||||
api = { path = "../api" }
|
||||
|
||||
@@ -2,7 +2,6 @@
|
||||
name = "meta-srv"
|
||||
version = "0.1.0"
|
||||
edition = "2021"
|
||||
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
|
||||
|
||||
[dependencies]
|
||||
api = { path = "../api" }
|
||||
|
||||
@@ -2,7 +2,6 @@
|
||||
|
||||
mod catalog_adapter;
|
||||
mod error;
|
||||
pub mod plan_adapter;
|
||||
mod planner;
|
||||
|
||||
use std::sync::Arc;
|
||||
@@ -11,9 +10,14 @@ use catalog::CatalogListRef;
|
||||
use common_function::scalars::aggregate::AggregateFunctionMetaRef;
|
||||
use common_function::scalars::udf::create_udf;
|
||||
use common_function::scalars::FunctionRef;
|
||||
use common_query::physical_plan::PhysicalPlanAdapter;
|
||||
use common_query::physical_plan::{DfPhysicalPlanAdapter, PhysicalPlan};
|
||||
use common_query::{prelude::ScalarUdf, Output};
|
||||
use common_recordbatch::adapter::RecordBatchStreamAdapter;
|
||||
use common_recordbatch::{EmptyRecordBatchStream, SendableRecordBatchStream};
|
||||
use common_telemetry::timer;
|
||||
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
|
||||
use datafusion::physical_plan::ExecutionPlan;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use sql::statements::statement::Statement;
|
||||
use sql::{dialect::GenericDialect, parser::ParserContext};
|
||||
@@ -22,14 +26,13 @@ pub use crate::datafusion::catalog_adapter::DfCatalogListAdapter;
|
||||
use crate::metric;
|
||||
use crate::query_engine::{QueryContext, QueryEngineState};
|
||||
use crate::{
|
||||
datafusion::plan_adapter::PhysicalPlanAdapter,
|
||||
datafusion::planner::{DfContextProviderAdapter, DfPlanner},
|
||||
error::Result,
|
||||
executor::QueryExecutor,
|
||||
logical_optimizer::LogicalOptimizer,
|
||||
physical_optimizer::PhysicalOptimizer,
|
||||
physical_planner::PhysicalPlanner,
|
||||
plan::{LogicalPlan, PhysicalPlan},
|
||||
plan::LogicalPlan,
|
||||
planner::Planner,
|
||||
QueryEngine,
|
||||
};
|
||||
@@ -179,8 +182,7 @@ impl PhysicalOptimizer for DatafusionQueryEngine {
|
||||
.as_any()
|
||||
.downcast_ref::<PhysicalPlanAdapter>()
|
||||
.context(error::PhysicalPlanDowncastSnafu)?
|
||||
.df_plan()
|
||||
.clone();
|
||||
.df_plan();
|
||||
|
||||
for optimizer in optimizers {
|
||||
new_plan = optimizer
|
||||
@@ -203,9 +205,24 @@ impl QueryExecutor for DatafusionQueryEngine {
|
||||
let _timer = timer!(metric::METRIC_EXEC_PLAN_ELAPSED);
|
||||
match plan.output_partitioning().partition_count() {
|
||||
0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))),
|
||||
1 => Ok(plan.execute(&ctx.state().runtime(), 0).await?),
|
||||
1 => Ok(plan
|
||||
.execute(0, ctx.state().runtime())
|
||||
.await
|
||||
.context(error::ExecutePhysicalPlanSnafu)?),
|
||||
_ => {
|
||||
unimplemented!();
|
||||
// merge into a single partition
|
||||
let plan =
|
||||
CoalescePartitionsExec::new(Arc::new(DfPhysicalPlanAdapter(plan.clone())));
|
||||
// CoalescePartitionsExec must produce a single partition
|
||||
assert_eq!(1, plan.output_partitioning().partition_count());
|
||||
let df_stream = plan.execute(0, ctx.state().runtime()).await.context(
|
||||
error::DatafusionSnafu {
|
||||
msg: "Failed to execute DataFusion merge exec",
|
||||
},
|
||||
)?;
|
||||
let stream = RecordBatchStreamAdapter::try_new(df_stream)
|
||||
.context(error::ConvertDfRecordBatchStreamSnafu)?;
|
||||
Ok(Box::pin(stream))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,7 +12,6 @@ use datafusion::catalog::{
|
||||
};
|
||||
use datafusion::datasource::TableProvider as DfTableProvider;
|
||||
use datafusion::error::Result as DataFusionResult;
|
||||
use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
use snafu::ResultExt;
|
||||
use table::{
|
||||
table::adapter::{DfTableProviderAdapter, TableAdapter},
|
||||
@@ -22,16 +21,12 @@ use table::{
|
||||
use crate::datafusion::error;
|
||||
|
||||
pub struct DfCatalogListAdapter {
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
catalog_list: CatalogListRef,
|
||||
}
|
||||
|
||||
impl DfCatalogListAdapter {
|
||||
pub fn new(runtime: Arc<RuntimeEnv>, catalog_list: CatalogListRef) -> DfCatalogListAdapter {
|
||||
DfCatalogListAdapter {
|
||||
runtime,
|
||||
catalog_list,
|
||||
}
|
||||
pub fn new(catalog_list: CatalogListRef) -> DfCatalogListAdapter {
|
||||
DfCatalogListAdapter { catalog_list }
|
||||
}
|
||||
}
|
||||
|
||||
@@ -47,16 +42,10 @@ impl DfCatalogList for DfCatalogListAdapter {
|
||||
) -> Option<Arc<dyn DfCatalogProvider>> {
|
||||
let catalog_adapter = Arc::new(CatalogProviderAdapter {
|
||||
df_catalog_provider: catalog,
|
||||
runtime: self.runtime.clone(),
|
||||
});
|
||||
self.catalog_list
|
||||
.register_catalog(name, catalog_adapter)
|
||||
.map(|catalog_provider| {
|
||||
Arc::new(DfCatalogProviderAdapter {
|
||||
catalog_provider,
|
||||
runtime: self.runtime.clone(),
|
||||
}) as _
|
||||
})
|
||||
.map(|catalog_provider| Arc::new(DfCatalogProviderAdapter { catalog_provider }) as _)
|
||||
}
|
||||
|
||||
fn catalog_names(&self) -> Vec<String> {
|
||||
@@ -64,19 +53,15 @@ impl DfCatalogList for DfCatalogListAdapter {
|
||||
}
|
||||
|
||||
fn catalog(&self, name: &str) -> Option<Arc<dyn DfCatalogProvider>> {
|
||||
self.catalog_list.catalog(name).map(|catalog_provider| {
|
||||
Arc::new(DfCatalogProviderAdapter {
|
||||
catalog_provider,
|
||||
runtime: self.runtime.clone(),
|
||||
}) as _
|
||||
})
|
||||
self.catalog_list
|
||||
.catalog(name)
|
||||
.map(|catalog_provider| Arc::new(DfCatalogProviderAdapter { catalog_provider }) as _)
|
||||
}
|
||||
}
|
||||
|
||||
/// Datafusion's CatalogProvider -> greptime CatalogProvider
|
||||
struct CatalogProviderAdapter {
|
||||
df_catalog_provider: Arc<dyn DfCatalogProvider>,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl CatalogProvider for CatalogProviderAdapter {
|
||||
@@ -99,19 +84,13 @@ impl CatalogProvider for CatalogProviderAdapter {
|
||||
fn schema(&self, name: &str) -> Option<Arc<dyn SchemaProvider>> {
|
||||
self.df_catalog_provider
|
||||
.schema(name)
|
||||
.map(|df_schema_provider| {
|
||||
Arc::new(SchemaProviderAdapter {
|
||||
df_schema_provider,
|
||||
runtime: self.runtime.clone(),
|
||||
}) as _
|
||||
})
|
||||
.map(|df_schema_provider| Arc::new(SchemaProviderAdapter { df_schema_provider }) as _)
|
||||
}
|
||||
}
|
||||
|
||||
///Greptime CatalogProvider -> datafusion's CatalogProvider
|
||||
struct DfCatalogProviderAdapter {
|
||||
catalog_provider: CatalogProviderRef,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl DfCatalogProvider for DfCatalogProviderAdapter {
|
||||
@@ -124,19 +103,15 @@ impl DfCatalogProvider for DfCatalogProviderAdapter {
|
||||
}
|
||||
|
||||
fn schema(&self, name: &str) -> Option<Arc<dyn DfSchemaProvider>> {
|
||||
self.catalog_provider.schema(name).map(|schema_provider| {
|
||||
Arc::new(DfSchemaProviderAdapter {
|
||||
schema_provider,
|
||||
runtime: self.runtime.clone(),
|
||||
}) as _
|
||||
})
|
||||
self.catalog_provider
|
||||
.schema(name)
|
||||
.map(|schema_provider| Arc::new(DfSchemaProviderAdapter { schema_provider }) as _)
|
||||
}
|
||||
}
|
||||
|
||||
/// Greptime SchemaProvider -> datafusion SchemaProvider
|
||||
struct DfSchemaProviderAdapter {
|
||||
schema_provider: Arc<dyn SchemaProvider>,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl DfSchemaProvider for DfSchemaProviderAdapter {
|
||||
@@ -159,7 +134,7 @@ impl DfSchemaProvider for DfSchemaProviderAdapter {
|
||||
name: String,
|
||||
table: Arc<dyn DfTableProvider>,
|
||||
) -> DataFusionResult<Option<Arc<dyn DfTableProvider>>> {
|
||||
let table = Arc::new(TableAdapter::new(table, self.runtime.clone())?);
|
||||
let table = Arc::new(TableAdapter::new(table)?);
|
||||
match self.schema_provider.register_table(name, table)? {
|
||||
Some(p) => Ok(Some(Arc::new(DfTableProviderAdapter::new(p)))),
|
||||
None => Ok(None),
|
||||
@@ -181,7 +156,6 @@ impl DfSchemaProvider for DfSchemaProviderAdapter {
|
||||
/// Datafusion SchemaProviderAdapter -> greptime SchemaProviderAdapter
|
||||
struct SchemaProviderAdapter {
|
||||
df_schema_provider: Arc<dyn DfSchemaProvider>,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl SchemaProvider for SchemaProviderAdapter {
|
||||
@@ -203,8 +177,8 @@ impl SchemaProvider for SchemaProviderAdapter {
|
||||
Some(adapter) => adapter.table(),
|
||||
None => {
|
||||
// TODO(yingwen): Avoid panic here.
|
||||
let adapter = TableAdapter::new(table_provider, self.runtime.clone())
|
||||
.expect("convert datafusion table");
|
||||
let adapter =
|
||||
TableAdapter::new(table_provider).expect("convert datafusion table");
|
||||
Arc::new(adapter) as _
|
||||
}
|
||||
}
|
||||
@@ -233,8 +207,7 @@ impl SchemaProvider for SchemaProviderAdapter {
|
||||
msg: "Fail to deregister table from datafusion",
|
||||
})?
|
||||
.map(|table| {
|
||||
let adapter = TableAdapter::new(table, self.runtime.clone())
|
||||
.context(error::TableSchemaMismatchSnafu)?;
|
||||
let adapter = TableAdapter::new(table).context(error::TableSchemaMismatchSnafu)?;
|
||||
Ok(Arc::new(adapter) as _)
|
||||
})
|
||||
.transpose()
|
||||
@@ -259,7 +232,6 @@ mod tests {
|
||||
df_catalog_provider: Arc::new(
|
||||
datafusion::catalog::catalog::MemoryCatalogProvider::new(),
|
||||
),
|
||||
runtime: Arc::new(RuntimeEnv::default()),
|
||||
};
|
||||
|
||||
adapter.register_schema(
|
||||
@@ -271,7 +243,6 @@ mod tests {
|
||||
#[test]
|
||||
pub fn test_register_table() {
|
||||
let adapter = DfSchemaProviderAdapter {
|
||||
runtime: Arc::new(RuntimeEnv::default()),
|
||||
schema_provider: Arc::new(MemorySchemaProvider::new()),
|
||||
};
|
||||
|
||||
@@ -288,9 +259,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
pub fn test_register_catalog() {
|
||||
let rt = Arc::new(RuntimeEnv::default());
|
||||
let catalog_list = DfCatalogListAdapter {
|
||||
runtime: rt.clone(),
|
||||
catalog_list: new_memory_catalog_list().unwrap(),
|
||||
};
|
||||
assert!(catalog_list
|
||||
@@ -298,7 +267,6 @@ mod tests {
|
||||
"test_catalog".to_string(),
|
||||
Arc::new(DfCatalogProviderAdapter {
|
||||
catalog_provider: Arc::new(MemoryCatalogProvider::new()),
|
||||
runtime: rt,
|
||||
}),
|
||||
)
|
||||
.is_none());
|
||||
|
||||
@@ -44,6 +44,21 @@ pub enum InnerError {
|
||||
#[snafu(backtrace)]
|
||||
source: table::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to convert DataFusion's recordbatch stream, source: {}",
|
||||
source
|
||||
))]
|
||||
ConvertDfRecordBatchStream {
|
||||
#[snafu(backtrace)]
|
||||
source: common_recordbatch::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to execute physical plan, source: {}", source))]
|
||||
ExecutePhysicalPlan {
|
||||
#[snafu(backtrace)]
|
||||
source: common_query::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
impl ErrorExt for InnerError {
|
||||
@@ -59,6 +74,8 @@ impl ErrorExt for InnerError {
|
||||
}
|
||||
ParseSql { source, .. } => source.status_code(),
|
||||
PlanSql { .. } => StatusCode::PlanQuery,
|
||||
ConvertDfRecordBatchStream { source } => source.status_code(),
|
||||
ExecutePhysicalPlan { source } => source.status_code(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,215 +0,0 @@
|
||||
use std::any::Any;
|
||||
use std::fmt::Debug;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use datafusion::arrow::datatypes::SchemaRef as DfSchemaRef;
|
||||
use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
use datafusion::{
|
||||
error::Result as DfResult,
|
||||
physical_plan::{
|
||||
expressions::PhysicalSortExpr, ExecutionPlan, Partitioning as DfPartitioning,
|
||||
SendableRecordBatchStream as DfSendableRecordBatchStream, Statistics,
|
||||
},
|
||||
};
|
||||
use datatypes::schema::SchemaRef;
|
||||
use snafu::ResultExt;
|
||||
use table::table::adapter::{DfRecordBatchStreamAdapter, RecordBatchStreamAdapter};
|
||||
|
||||
use crate::datafusion::error;
|
||||
use crate::error::Result;
|
||||
use crate::executor::Runtime;
|
||||
use crate::plan::{Partitioning, PhysicalPlan};
|
||||
|
||||
/// Datafusion ExecutionPlan -> greptime PhysicalPlan
|
||||
#[derive(Debug)]
|
||||
pub struct PhysicalPlanAdapter {
|
||||
plan: Arc<dyn ExecutionPlan>,
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
impl PhysicalPlanAdapter {
|
||||
pub fn new(schema: SchemaRef, plan: Arc<dyn ExecutionPlan>) -> Self {
|
||||
Self { schema, plan }
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn df_plan(&self) -> &Arc<dyn ExecutionPlan> {
|
||||
&self.plan
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl PhysicalPlan for PhysicalPlanAdapter {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
//FIXME(dennis)
|
||||
Partitioning::UnknownPartitioning(1)
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<Arc<dyn PhysicalPlan>> {
|
||||
let mut plans: Vec<Arc<dyn PhysicalPlan>> = vec![];
|
||||
for p in self.plan.children() {
|
||||
let plan = PhysicalPlanAdapter::new(self.schema.clone(), p);
|
||||
plans.push(Arc::new(plan));
|
||||
}
|
||||
plans
|
||||
}
|
||||
|
||||
fn with_new_children(
|
||||
&self,
|
||||
children: Vec<Arc<dyn PhysicalPlan>>,
|
||||
) -> Result<Arc<dyn PhysicalPlan>> {
|
||||
let mut df_children: Vec<Arc<dyn ExecutionPlan>> = Vec::with_capacity(children.len());
|
||||
|
||||
for plan in children {
|
||||
let p = Arc::new(ExecutionPlanAdapter {
|
||||
plan,
|
||||
schema: self.schema.clone(),
|
||||
});
|
||||
df_children.push(p);
|
||||
}
|
||||
|
||||
let plan = self
|
||||
.plan
|
||||
.with_new_children(df_children)
|
||||
.context(error::DatafusionSnafu {
|
||||
msg: "Fail to add children to plan",
|
||||
})?;
|
||||
Ok(Arc::new(PhysicalPlanAdapter::new(
|
||||
self.schema.clone(),
|
||||
plan,
|
||||
)))
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
runtime: &Runtime,
|
||||
partition: usize,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
let df_stream =
|
||||
self.plan
|
||||
.execute(partition, runtime.into())
|
||||
.await
|
||||
.context(error::DatafusionSnafu {
|
||||
msg: "Fail to execute physical plan",
|
||||
})?;
|
||||
|
||||
Ok(Box::pin(
|
||||
RecordBatchStreamAdapter::try_new(df_stream)
|
||||
.context(error::TableSchemaMismatchSnafu)?,
|
||||
))
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
/// Greptime PhysicalPlan -> datafusion ExecutionPlan.
|
||||
#[derive(Debug)]
|
||||
struct ExecutionPlanAdapter {
|
||||
plan: Arc<dyn PhysicalPlan>,
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl ExecutionPlan for ExecutionPlanAdapter {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
self.schema.arrow_schema().clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> DfPartitioning {
|
||||
// FIXME(dennis)
|
||||
DfPartitioning::UnknownPartitioning(1)
|
||||
}
|
||||
|
||||
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
|
||||
// FIXME(dennis)
|
||||
None
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
|
||||
// TODO(dennis)
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn with_new_children(
|
||||
&self,
|
||||
children: Vec<Arc<dyn ExecutionPlan>>,
|
||||
) -> DfResult<Arc<dyn ExecutionPlan>> {
|
||||
let mut gt_children: Vec<Arc<dyn PhysicalPlan>> = Vec::with_capacity(children.len());
|
||||
|
||||
for plan in children {
|
||||
let p = Arc::new(PhysicalPlanAdapter::new(self.schema.clone(), plan));
|
||||
gt_children.push(p);
|
||||
}
|
||||
|
||||
match self.plan.with_new_children(gt_children) {
|
||||
Ok(plan) => Ok(Arc::new(ExecutionPlanAdapter {
|
||||
schema: self.schema.clone(),
|
||||
plan,
|
||||
})),
|
||||
Err(e) => Err(e.into()),
|
||||
}
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
partition: usize,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
) -> DfResult<DfSendableRecordBatchStream> {
|
||||
match self.plan.execute(&runtime.into(), partition).await {
|
||||
Ok(stream) => Ok(Box::pin(DfRecordBatchStreamAdapter::new(stream))),
|
||||
Err(e) => Err(e.into()),
|
||||
}
|
||||
}
|
||||
|
||||
fn statistics(&self) -> Statistics {
|
||||
//TODO(dennis)
|
||||
Statistics::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use arrow::datatypes::Field;
|
||||
use datafusion::physical_plan::empty::EmptyExec;
|
||||
use datafusion_common::field_util::SchemaExt;
|
||||
use datatypes::schema::Schema;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_physical_plan_adapter() {
|
||||
let arrow_schema = arrow::datatypes::Schema::new(vec![Field::new(
|
||||
"name",
|
||||
arrow::datatypes::DataType::Utf8,
|
||||
true,
|
||||
)]);
|
||||
|
||||
let schema = Arc::new(Schema::try_from(arrow_schema.clone()).unwrap());
|
||||
let physical_plan = PhysicalPlanAdapter::new(
|
||||
schema.clone(),
|
||||
Arc::new(EmptyExec::new(true, Arc::new(arrow_schema))),
|
||||
);
|
||||
|
||||
assert!(physical_plan
|
||||
.plan
|
||||
.as_any()
|
||||
.downcast_ref::<EmptyExec>()
|
||||
.is_some());
|
||||
let execution_plan_adapter = ExecutionPlanAdapter {
|
||||
plan: Arc::new(physical_plan),
|
||||
schema: schema.clone(),
|
||||
};
|
||||
assert_eq!(schema, execution_plan_adapter.schema);
|
||||
}
|
||||
}
|
||||
@@ -1,9 +1,9 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::physical_plan::PhysicalPlan;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
|
||||
use crate::{error::Result, plan::PhysicalPlan, query_engine::QueryContext};
|
||||
use crate::{error::Result, query_engine::QueryContext};
|
||||
|
||||
/// Executor to run [ExecutionPlan].
|
||||
#[async_trait::async_trait]
|
||||
@@ -14,27 +14,3 @@ pub trait QueryExecutor {
|
||||
plan: &Arc<dyn PhysicalPlan>,
|
||||
) -> Result<SendableRecordBatchStream>;
|
||||
}
|
||||
|
||||
/// Execution runtime environment
|
||||
#[derive(Clone, Default)]
|
||||
pub struct Runtime {
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl From<Arc<RuntimeEnv>> for Runtime {
|
||||
fn from(runtime: Arc<RuntimeEnv>) -> Self {
|
||||
Runtime { runtime }
|
||||
}
|
||||
}
|
||||
|
||||
impl From<Runtime> for Arc<RuntimeEnv> {
|
||||
fn from(r: Runtime) -> Arc<RuntimeEnv> {
|
||||
r.runtime
|
||||
}
|
||||
}
|
||||
|
||||
impl From<&Runtime> for Arc<RuntimeEnv> {
|
||||
fn from(r: &Runtime) -> Arc<RuntimeEnv> {
|
||||
r.runtime.clone()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,5 +12,4 @@ pub mod plan;
|
||||
pub mod planner;
|
||||
pub mod query_engine;
|
||||
|
||||
pub use crate::datafusion::plan_adapter::PhysicalPlanAdapter;
|
||||
pub use crate::query_engine::{QueryContext, QueryEngine, QueryEngineFactory, QueryEngineRef};
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::{error::Result, plan::PhysicalPlan, query_engine::QueryContext};
|
||||
use common_query::physical_plan::PhysicalPlan;
|
||||
|
||||
use crate::{error::Result, query_engine::QueryContext};
|
||||
|
||||
pub trait PhysicalOptimizer {
|
||||
fn optimize_physical_plan(
|
||||
|
||||
@@ -1,7 +1,9 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::physical_plan::PhysicalPlan;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::plan::{LogicalPlan, PhysicalPlan};
|
||||
use crate::plan::LogicalPlan;
|
||||
use crate::query_engine::QueryContext;
|
||||
|
||||
/// Physical query planner that converts a `LogicalPlan` to an
|
||||
|
||||
@@ -1,13 +1,6 @@
|
||||
use std::any::Any;
|
||||
use std::fmt::Debug;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use datafusion::logical_plan::LogicalPlan as DfLogicalPlan;
|
||||
use datatypes::schema::SchemaRef;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::executor::Runtime;
|
||||
|
||||
/// A LogicalPlan represents the different types of relational
|
||||
/// operators (such as Projection, Filter, etc) and can be created by
|
||||
@@ -21,50 +14,3 @@ use crate::executor::Runtime;
|
||||
pub enum LogicalPlan {
|
||||
DfPlan(DfLogicalPlan),
|
||||
}
|
||||
|
||||
/// Partitioning schemes supported by operators.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum Partitioning {
|
||||
/// Unknown partitioning scheme with a known number of partitions
|
||||
UnknownPartitioning(usize),
|
||||
}
|
||||
|
||||
impl Partitioning {
|
||||
/// Returns the number of partitions in this partitioning scheme
|
||||
pub fn partition_count(&self) -> usize {
|
||||
use Partitioning::*;
|
||||
match self {
|
||||
UnknownPartitioning(n) => *n,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
pub trait PhysicalPlan: Send + Sync + Any + Debug {
|
||||
/// Get the schema for this execution plan
|
||||
fn schema(&self) -> SchemaRef;
|
||||
|
||||
/// Specifies the output partitioning scheme of this plan
|
||||
fn output_partitioning(&self) -> Partitioning;
|
||||
|
||||
/// Get a list of child execution plans that provide the input for this plan. The returned list
|
||||
/// will be empty for leaf nodes, will contain a single value for unary nodes, or two
|
||||
/// values for binary nodes (such as joins).
|
||||
fn children(&self) -> Vec<Arc<dyn PhysicalPlan>>;
|
||||
|
||||
/// Returns a new plan where all children were replaced by new plans.
|
||||
/// The size of `children` must be equal to the size of `ExecutionPlan::children()`.
|
||||
fn with_new_children(
|
||||
&self,
|
||||
children: Vec<Arc<dyn PhysicalPlan>>,
|
||||
) -> Result<Arc<dyn PhysicalPlan>>;
|
||||
|
||||
/// creates an iterator
|
||||
async fn execute(
|
||||
&self,
|
||||
_runtime: &Runtime,
|
||||
partition: usize,
|
||||
) -> Result<SendableRecordBatchStream>;
|
||||
|
||||
fn as_any(&self) -> &dyn Any;
|
||||
}
|
||||
|
||||
@@ -6,13 +6,14 @@ use std::sync::Arc;
|
||||
use catalog::CatalogList;
|
||||
use common_function::scalars::aggregate::AggregateFunctionMetaRef;
|
||||
use common_function::scalars::{FunctionRef, FUNCTION_REGISTRY};
|
||||
use common_query::physical_plan::PhysicalPlan;
|
||||
use common_query::prelude::ScalarUdf;
|
||||
use common_query::Output;
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
use crate::datafusion::DatafusionQueryEngine;
|
||||
use crate::error::Result;
|
||||
use crate::plan::{LogicalPlan, PhysicalPlan};
|
||||
use crate::plan::LogicalPlan;
|
||||
pub use crate::query_engine::context::QueryContext;
|
||||
pub use crate::query_engine::state::QueryEngineState;
|
||||
|
||||
|
||||
@@ -4,6 +4,7 @@ use std::sync::{Arc, RwLock};
|
||||
|
||||
use catalog::CatalogListRef;
|
||||
use common_function::scalars::aggregate::AggregateFunctionMetaRef;
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use common_query::prelude::ScalarUdf;
|
||||
use datafusion::optimizer::common_subexpr_eliminate::CommonSubexprEliminate;
|
||||
use datafusion::optimizer::eliminate_limit::EliminateLimit;
|
||||
@@ -15,7 +16,6 @@ use datafusion::optimizer::to_approx_perc::ToApproxPerc;
|
||||
use datafusion::prelude::{ExecutionConfig, ExecutionContext};
|
||||
|
||||
use crate::datafusion::DfCatalogListAdapter;
|
||||
use crate::executor::Runtime;
|
||||
use crate::optimizer::TypeConversionRule;
|
||||
|
||||
/// Query engine global state
|
||||
@@ -58,10 +58,8 @@ impl QueryEngineState {
|
||||
|
||||
let df_context = ExecutionContext::with_config(config);
|
||||
|
||||
df_context.state.lock().catalog_list = Arc::new(DfCatalogListAdapter::new(
|
||||
df_context.runtime_env(),
|
||||
catalog_list.clone(),
|
||||
));
|
||||
df_context.state.lock().catalog_list =
|
||||
Arc::new(DfCatalogListAdapter::new(catalog_list.clone()));
|
||||
|
||||
Self {
|
||||
df_context,
|
||||
@@ -108,7 +106,7 @@ impl QueryEngineState {
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn runtime(&self) -> Runtime {
|
||||
self.df_context.runtime_env().into()
|
||||
pub(crate) fn runtime(&self) -> Arc<RuntimeEnv> {
|
||||
self.df_context.runtime_env()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,6 +17,7 @@ common-query = { path = "../common/query" }
|
||||
common-recordbatch = { path = "../common/recordbatch" }
|
||||
common-telemetry = { path = "../common/telemetry" }
|
||||
common-time = { path = "../common/time" }
|
||||
datafusion = { git = "https://github.com/apache/arrow-datafusion.git", branch = "arrow2", features = ["simd"] }
|
||||
datafusion-common = { git = "https://github.com/apache/arrow-datafusion.git", branch = "arrow2" }
|
||||
datatypes = { path = "../datatypes" }
|
||||
futures = "0.3"
|
||||
|
||||
@@ -426,6 +426,7 @@ impl<S: StorageEngine> MitoEngineInner<S> {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use common_recordbatch::util;
|
||||
use datafusion_common::field_util::FieldExt;
|
||||
use datafusion_common::field_util::SchemaExt;
|
||||
@@ -520,6 +521,10 @@ mod tests {
|
||||
assert_eq!(2, table.insert(insert_req).await.unwrap());
|
||||
|
||||
let stream = table.scan(&None, &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(1, batches.len());
|
||||
|
||||
@@ -555,6 +560,10 @@ mod tests {
|
||||
assert_eq!(2, table.insert(insert_req).await.unwrap());
|
||||
|
||||
let stream = table.scan(&None, &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(1, batches.len());
|
||||
|
||||
@@ -612,6 +621,10 @@ mod tests {
|
||||
assert_eq!(2, table.insert(insert_req).await.unwrap());
|
||||
|
||||
let stream = table.scan(&None, &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(1, batches.len());
|
||||
assert_eq!(batches[0].df_recordbatch.num_columns(), 4);
|
||||
@@ -633,6 +646,10 @@ mod tests {
|
||||
|
||||
// Scan with projections: cpu and memory
|
||||
let stream = table.scan(&Some(vec![1, 2]), &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(1, batches.len());
|
||||
assert_eq!(batches[0].df_recordbatch.num_columns(), 2);
|
||||
@@ -650,6 +667,10 @@ mod tests {
|
||||
|
||||
// Scan with projections: only ts
|
||||
let stream = table.scan(&Some(vec![3]), &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(1, batches.len());
|
||||
assert_eq!(batches[0].df_recordbatch.num_columns(), 1);
|
||||
@@ -692,6 +713,10 @@ mod tests {
|
||||
assert_eq!(test_batch_size, table.insert(insert_req).await.unwrap());
|
||||
|
||||
let stream = table.scan(&None, &[], None).await.unwrap();
|
||||
let stream = stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let batches = util::collect(stream).await.unwrap();
|
||||
let mut total = 0;
|
||||
for batch in batches {
|
||||
|
||||
@@ -8,8 +8,9 @@ use std::sync::Arc;
|
||||
use arc_swap::ArcSwap;
|
||||
use async_trait::async_trait;
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_recordbatch::error::{Error as RecordBatchError, Result as RecordBatchResult};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream};
|
||||
use common_telemetry::logging;
|
||||
use datatypes::schema::{ColumnSchema, SchemaBuilder};
|
||||
use datatypes::vectors::VectorRef;
|
||||
@@ -25,6 +26,7 @@ use store_api::storage::{
|
||||
use table::error::{Error as TableError, MissingColumnSnafu, Result as TableResult};
|
||||
use table::metadata::{FilterPushDownType, TableInfoRef, TableMetaBuilder};
|
||||
use table::requests::{AddColumnRequest, AlterKind, AlterTableRequest, InsertRequest};
|
||||
use table::table::scan::SimpleTableScan;
|
||||
use table::{
|
||||
metadata::{TableInfo, TableType},
|
||||
table::Table,
|
||||
@@ -154,7 +156,7 @@ impl<R: Region> Table for MitoTable<R> {
|
||||
projection: &Option<Vec<usize>>,
|
||||
filters: &[Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> TableResult<SendableRecordBatchStream> {
|
||||
) -> TableResult<PhysicalPlanRef> {
|
||||
let read_ctx = ReadContext::default();
|
||||
let snapshot = self.region.snapshot(&read_ctx).map_err(TableError::new)?;
|
||||
|
||||
@@ -180,7 +182,8 @@ impl<R: Region> Table for MitoTable<R> {
|
||||
}
|
||||
});
|
||||
|
||||
Ok(Box::pin(ChunkStream { schema, stream }))
|
||||
let stream = Box::pin(ChunkStream { schema, stream });
|
||||
Ok(Arc::new(SimpleTableScan::new(stream)))
|
||||
}
|
||||
|
||||
// Alter table changes the schemas of the table. The altering happens as cloning a new schema,
|
||||
|
||||
@@ -28,9 +28,6 @@ pub enum InnerError {
|
||||
#[snafu(display("Missing column when insert, column: {}", name))]
|
||||
MissingColumn { name: String, backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Not expected to run ExecutionPlan more than once"))]
|
||||
ExecuteRepeatedly { backtrace: Backtrace },
|
||||
|
||||
#[snafu(display("Poll stream failed, source: {}", source))]
|
||||
PollStream {
|
||||
source: ArrowError,
|
||||
@@ -58,7 +55,6 @@ impl ErrorExt for InnerError {
|
||||
| InnerError::SchemaConversion { .. }
|
||||
| InnerError::TableProjection { .. } => StatusCode::EngineExecuteQuery,
|
||||
InnerError::MissingColumn { .. } => StatusCode::InvalidArguments,
|
||||
InnerError::ExecuteRepeatedly { .. } => StatusCode::Unexpected,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -97,10 +93,6 @@ mod tests {
|
||||
Err(DataFusionError::NotImplemented("table test".to_string())).context(DatafusionSnafu)?
|
||||
}
|
||||
|
||||
fn throw_repeatedly() -> Result<()> {
|
||||
ExecuteRepeatedlySnafu {}.fail()?
|
||||
}
|
||||
|
||||
fn throw_missing_column_inner() -> std::result::Result<(), InnerError> {
|
||||
MissingColumnSnafu { name: "test" }.fail()
|
||||
}
|
||||
@@ -119,10 +111,6 @@ mod tests {
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert_eq!(StatusCode::EngineExecuteQuery, err.status_code());
|
||||
|
||||
let err = throw_repeatedly().err().unwrap();
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert_eq!(StatusCode::Unexpected, err.status_code());
|
||||
|
||||
let err = throw_missing_column().err().unwrap();
|
||||
assert!(err.backtrace_opt().is_some());
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
|
||||
@@ -1,11 +1,13 @@
|
||||
pub mod adapter;
|
||||
pub mod numbers;
|
||||
pub mod scan;
|
||||
|
||||
use std::any::Any;
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use datatypes::schema::SchemaRef;
|
||||
|
||||
use crate::error::Result;
|
||||
@@ -13,7 +15,7 @@ use crate::metadata::{FilterPushDownType, TableInfoRef, TableType};
|
||||
use crate::requests::{AlterTableRequest, InsertRequest};
|
||||
|
||||
/// Table abstraction.
|
||||
#[async_trait::async_trait]
|
||||
#[async_trait]
|
||||
pub trait Table: Send + Sync {
|
||||
/// Returns the table as [`Any`](std::any::Any) so that it can be
|
||||
/// downcast to a specific implementation.
|
||||
@@ -45,7 +47,7 @@ pub trait Table: Send + Sync {
|
||||
// If set, it contains the amount of rows needed by the `LogicalPlan`,
|
||||
// The datasource should return *at least* this number of rows if available.
|
||||
limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream>;
|
||||
) -> Result<PhysicalPlanRef>;
|
||||
|
||||
/// Tests whether the table provider can make use of a filter expression
|
||||
/// to optimise data retrieval.
|
||||
|
||||
@@ -1,14 +1,9 @@
|
||||
use core::fmt::Formatter;
|
||||
use core::pin::Pin;
|
||||
use core::task::{Context, Poll};
|
||||
use std::any::Any;
|
||||
use std::fmt::Debug;
|
||||
use std::mem;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::logical_plan::Expr;
|
||||
use common_recordbatch::error::Result as RecordBatchResult;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use common_query::physical_plan::{DfPhysicalPlanAdapter, PhysicalPlanAdapter, PhysicalPlanRef};
|
||||
use common_query::DfPhysicalPlan;
|
||||
use common_telemetry::debug;
|
||||
use datafusion::arrow::datatypes::SchemaRef as DfSchemaRef;
|
||||
/// Datafusion table adpaters
|
||||
@@ -17,92 +12,14 @@ use datafusion::datasource::{
|
||||
TableType as DfTableType,
|
||||
};
|
||||
use datafusion::error::Result as DfResult;
|
||||
use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
use datafusion::logical_plan::Expr as DfExpr;
|
||||
use datafusion::physical_plan::{
|
||||
expressions::PhysicalSortExpr, ExecutionPlan, Partitioning,
|
||||
RecordBatchStream as DfRecordBatchStream,
|
||||
SendableRecordBatchStream as DfSendableRecordBatchStream, Statistics,
|
||||
};
|
||||
use datafusion_common::record_batch::RecordBatch as DfRecordBatch;
|
||||
use datatypes::arrow::error::{ArrowError, Result as ArrowResult};
|
||||
use datatypes::schema::SchemaRef;
|
||||
use datatypes::schema::{Schema, SchemaRef as TableSchemaRef};
|
||||
use futures::Stream;
|
||||
use datatypes::schema::{SchemaRef as TableSchemaRef, SchemaRef};
|
||||
use snafu::prelude::*;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::metadata::TableInfoRef;
|
||||
use crate::table::{FilterPushDownType, Table, TableRef, TableType};
|
||||
|
||||
/// Greptime SendableRecordBatchStream -> datafusion ExecutionPlan.
|
||||
struct ExecutionPlanAdapter {
|
||||
stream: Mutex<Option<SendableRecordBatchStream>>,
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
impl Debug for ExecutionPlanAdapter {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> core::fmt::Result {
|
||||
f.debug_struct("ExecutionPlanAdapter")
|
||||
.field("schema", &self.schema)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl ExecutionPlan for ExecutionPlanAdapter {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
self.schema.arrow_schema().clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
// FIXME(dennis)
|
||||
Partitioning::UnknownPartitioning(1)
|
||||
}
|
||||
|
||||
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
|
||||
// FIXME(dennis)
|
||||
None
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
|
||||
// TODO(dennis)
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn with_new_children(
|
||||
&self,
|
||||
_children: Vec<Arc<dyn ExecutionPlan>>,
|
||||
) -> DfResult<Arc<dyn ExecutionPlan>> {
|
||||
// TODO(dennis)
|
||||
todo!();
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
_partition: usize,
|
||||
_runtime: Arc<RuntimeEnv>,
|
||||
) -> DfResult<DfSendableRecordBatchStream> {
|
||||
let mut stream = self.stream.lock().unwrap();
|
||||
|
||||
if stream.is_some() {
|
||||
let stream = mem::replace(&mut *stream, None);
|
||||
Ok(Box::pin(DfRecordBatchStreamAdapter::new(stream.unwrap())))
|
||||
} else {
|
||||
error::ExecuteRepeatedlySnafu.fail()?
|
||||
}
|
||||
}
|
||||
|
||||
fn statistics(&self) -> Statistics {
|
||||
//TODO(dennis)
|
||||
Statistics::default()
|
||||
}
|
||||
}
|
||||
|
||||
/// Greptime Table -> datafusion TableProvider
|
||||
pub struct DfTableProviderAdapter {
|
||||
table: TableRef,
|
||||
@@ -141,14 +58,10 @@ impl TableProvider for DfTableProviderAdapter {
|
||||
projection: &Option<Vec<usize>>,
|
||||
filters: &[DfExpr],
|
||||
limit: Option<usize>,
|
||||
) -> DfResult<Arc<dyn ExecutionPlan>> {
|
||||
) -> DfResult<Arc<dyn DfPhysicalPlan>> {
|
||||
let filters: Vec<Expr> = filters.iter().map(Clone::clone).map(Into::into).collect();
|
||||
|
||||
let stream = self.table.scan(projection, &filters, limit).await?;
|
||||
Ok(Arc::new(ExecutionPlanAdapter {
|
||||
schema: stream.schema(),
|
||||
stream: Mutex::new(Some(stream)),
|
||||
}))
|
||||
let inner = self.table.scan(projection, &filters, limit).await?;
|
||||
Ok(Arc::new(DfPhysicalPlanAdapter(inner)))
|
||||
}
|
||||
|
||||
fn supports_filter_pushdown(&self, filter: &DfExpr) -> DfResult<DfTableProviderFilterPushDown> {
|
||||
@@ -167,15 +80,18 @@ impl TableProvider for DfTableProviderAdapter {
|
||||
pub struct TableAdapter {
|
||||
schema: TableSchemaRef,
|
||||
table_provider: Arc<dyn TableProvider>,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl TableAdapter {
|
||||
pub fn new(table_provider: Arc<dyn TableProvider>, runtime: Arc<RuntimeEnv>) -> Result<Self> {
|
||||
pub fn new(table_provider: Arc<dyn TableProvider>) -> Result<Self> {
|
||||
Ok(Self {
|
||||
schema: Arc::new(table_provider.schema().try_into().unwrap()),
|
||||
schema: Arc::new(
|
||||
table_provider
|
||||
.schema()
|
||||
.try_into()
|
||||
.context(error::SchemaConversionSnafu)?,
|
||||
),
|
||||
table_provider,
|
||||
runtime,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -207,7 +123,7 @@ impl Table for TableAdapter {
|
||||
projection: &Option<Vec<usize>>,
|
||||
filters: &[Expr],
|
||||
limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
) -> Result<PhysicalPlanRef> {
|
||||
let filters: Vec<DfExpr> = filters.iter().map(|e| e.df_expr().clone()).collect();
|
||||
debug!("TableScan filter size: {}", filters.len());
|
||||
let execution_plan = self
|
||||
@@ -215,14 +131,13 @@ impl Table for TableAdapter {
|
||||
.scan(projection, &filters, limit)
|
||||
.await
|
||||
.context(error::DatafusionSnafu)?;
|
||||
|
||||
// FIXME(dennis) Partitioning
|
||||
let df_stream = execution_plan
|
||||
.execute(0, self.runtime.clone())
|
||||
.await
|
||||
.context(error::DatafusionSnafu)?;
|
||||
|
||||
Ok(Box::pin(RecordBatchStreamAdapter::try_new(df_stream)?))
|
||||
let schema: SchemaRef = Arc::new(
|
||||
execution_plan
|
||||
.schema()
|
||||
.try_into()
|
||||
.context(error::SchemaConversionSnafu)?,
|
||||
);
|
||||
Ok(Arc::new(PhysicalPlanAdapter::new(schema, execution_plan)))
|
||||
}
|
||||
|
||||
fn supports_filter_pushdown(&self, filter: &Expr) -> Result<FilterPushDownType> {
|
||||
@@ -238,83 +153,6 @@ impl Table for TableAdapter {
|
||||
}
|
||||
}
|
||||
|
||||
/// Greptime SendableRecordBatchStream -> datafusion RecordBatchStream
|
||||
pub struct DfRecordBatchStreamAdapter {
|
||||
stream: SendableRecordBatchStream,
|
||||
}
|
||||
|
||||
impl DfRecordBatchStreamAdapter {
|
||||
pub fn new(stream: SendableRecordBatchStream) -> Self {
|
||||
Self { stream }
|
||||
}
|
||||
}
|
||||
|
||||
impl DfRecordBatchStream for DfRecordBatchStreamAdapter {
|
||||
fn schema(&self) -> DfSchemaRef {
|
||||
self.stream.schema().arrow_schema().clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for DfRecordBatchStreamAdapter {
|
||||
type Item = ArrowResult<DfRecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.stream).poll_next(cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Some(recordbatch)) => match recordbatch {
|
||||
Ok(recordbatch) => Poll::Ready(Some(Ok(recordbatch.df_recordbatch))),
|
||||
Err(e) => Poll::Ready(Some(Err(ArrowError::External("".to_owned(), Box::new(e))))),
|
||||
},
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn size_hint(&self) -> (usize, Option<usize>) {
|
||||
self.stream.size_hint()
|
||||
}
|
||||
}
|
||||
|
||||
/// Datafusion SendableRecordBatchStream to greptime RecordBatchStream
|
||||
pub struct RecordBatchStreamAdapter {
|
||||
schema: SchemaRef,
|
||||
stream: DfSendableRecordBatchStream,
|
||||
}
|
||||
|
||||
impl RecordBatchStreamAdapter {
|
||||
pub fn try_new(stream: DfSendableRecordBatchStream) -> Result<Self> {
|
||||
let schema =
|
||||
Arc::new(Schema::try_from(stream.schema()).context(error::SchemaConversionSnafu)?);
|
||||
Ok(Self { schema, stream })
|
||||
}
|
||||
}
|
||||
|
||||
impl RecordBatchStream for RecordBatchStreamAdapter {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for RecordBatchStreamAdapter {
|
||||
type Item = RecordBatchResult<RecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.stream).poll_next(cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Some(df_recordbatch)) => Poll::Ready(Some(Ok(RecordBatch {
|
||||
schema: self.schema(),
|
||||
df_recordbatch: df_recordbatch.context(error::PollStreamSnafu)?,
|
||||
}))),
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn size_hint(&self) -> (usize, Option<usize>) {
|
||||
self.stream.size_hint()
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use datafusion::arrow;
|
||||
@@ -328,14 +166,14 @@ mod tests {
|
||||
#[should_panic]
|
||||
fn test_table_adaptor_info() {
|
||||
let df_table = Arc::new(EmptyTable::new(Arc::new(arrow::datatypes::Schema::empty())));
|
||||
let table_adapter = TableAdapter::new(df_table, Arc::new(RuntimeEnv::default())).unwrap();
|
||||
let table_adapter = TableAdapter::new(df_table).unwrap();
|
||||
let _ = table_adapter.table_info();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_table_adaptor_type() {
|
||||
let df_table = Arc::new(EmptyTable::new(Arc::new(arrow::datatypes::Schema::empty())));
|
||||
let table_adapter = TableAdapter::new(df_table, Arc::new(RuntimeEnv::default())).unwrap();
|
||||
let table_adapter = TableAdapter::new(df_table).unwrap();
|
||||
assert_eq!(Base, table_adapter.table_type());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2,8 +2,9 @@ use std::any::Any;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_recordbatch::error::Result as RecordBatchResult;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream};
|
||||
use datafusion_common::record_batch::RecordBatch as DfRecordBatch;
|
||||
use datatypes::arrow::array::UInt32Array;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
@@ -13,6 +14,7 @@ use futures::Stream;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::metadata::TableInfoRef;
|
||||
use crate::table::scan::SimpleTableScan;
|
||||
use crate::table::{Expr, Table};
|
||||
|
||||
/// numbers table for test
|
||||
@@ -53,12 +55,13 @@ impl Table for NumbersTable {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
Ok(Box::pin(NumbersStream {
|
||||
) -> Result<PhysicalPlanRef> {
|
||||
let stream = Box::pin(NumbersStream {
|
||||
limit: limit.unwrap_or(100) as u32,
|
||||
schema: self.schema.clone(),
|
||||
already_run: false,
|
||||
}))
|
||||
});
|
||||
Ok(Arc::new(SimpleTableScan::new(stream)))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
123
src/table/src/table/scan.rs
Normal file
123
src/table/src/table/scan.rs
Normal file
@@ -0,0 +1,123 @@
|
||||
use std::any::Any;
|
||||
use std::fmt::{Debug, Formatter};
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_query::error as query_error;
|
||||
use common_query::error::Result as QueryResult;
|
||||
use common_query::physical_plan::Partitioning;
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use common_query::physical_plan::{PhysicalPlan, PhysicalPlanRef};
|
||||
use common_recordbatch::SendableRecordBatchStream;
|
||||
use datatypes::schema::SchemaRef;
|
||||
use snafu::OptionExt;
|
||||
|
||||
pub struct SimpleTableScan {
|
||||
stream: Mutex<Option<SendableRecordBatchStream>>,
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
impl Debug for SimpleTableScan {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("SimpleTableScan")
|
||||
.field("stream", &"<SendableRecordBatchStream>")
|
||||
.field("schema", &self.schema)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl SimpleTableScan {
|
||||
pub fn new(stream: SendableRecordBatchStream) -> Self {
|
||||
let schema = stream.schema();
|
||||
Self {
|
||||
stream: Mutex::new(Some(stream)),
|
||||
schema,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl PhysicalPlan for SimpleTableScan {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn output_partitioning(&self) -> Partitioning {
|
||||
Partitioning::UnknownPartitioning(1)
|
||||
}
|
||||
|
||||
fn children(&self) -> Vec<PhysicalPlanRef> {
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn with_new_children(&self, _children: Vec<PhysicalPlanRef>) -> QueryResult<PhysicalPlanRef> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
async fn execute(
|
||||
&self,
|
||||
_partition: usize,
|
||||
_runtime: Arc<RuntimeEnv>,
|
||||
) -> QueryResult<SendableRecordBatchStream> {
|
||||
let mut stream = self.stream.lock().unwrap();
|
||||
Ok(stream.take().context(query_error::ExecuteRepeatedlySnafu)?)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use common_recordbatch::util;
|
||||
use common_recordbatch::{RecordBatch, RecordBatches};
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, Schema};
|
||||
use datatypes::vectors::Int32Vector;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_simple_table_scan() {
|
||||
let schema = Arc::new(Schema::new(vec![ColumnSchema::new(
|
||||
"a",
|
||||
ConcreteDataType::int32_datatype(),
|
||||
false,
|
||||
)]));
|
||||
|
||||
let batch1 = RecordBatch::new(
|
||||
schema.clone(),
|
||||
vec![Arc::new(Int32Vector::from_slice(&[1, 2])) as _],
|
||||
)
|
||||
.unwrap();
|
||||
let batch2 = RecordBatch::new(
|
||||
schema.clone(),
|
||||
vec![Arc::new(Int32Vector::from_slice(&[3, 4, 5])) as _],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let recordbatches =
|
||||
RecordBatches::try_new(schema.clone(), vec![batch1.clone(), batch2.clone()]).unwrap();
|
||||
let stream = recordbatches.as_stream();
|
||||
|
||||
let scan = SimpleTableScan::new(stream);
|
||||
|
||||
assert_eq!(scan.schema(), schema);
|
||||
|
||||
let runtime = Arc::new(RuntimeEnv::default());
|
||||
let stream = scan.execute(0, runtime.clone()).await.unwrap();
|
||||
let recordbatches = util::collect(stream).await.unwrap();
|
||||
assert_eq!(recordbatches[0], batch1);
|
||||
assert_eq!(recordbatches[1], batch2);
|
||||
|
||||
let result = scan.execute(0, runtime).await;
|
||||
assert!(result.is_err());
|
||||
match result {
|
||||
Err(e) => assert!(e
|
||||
.to_string()
|
||||
.contains("Not expected to run ExecutionPlan more than once")),
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,11 +1,13 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_recordbatch::{EmptyRecordBatchStream, SendableRecordBatchStream};
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_recordbatch::EmptyRecordBatchStream;
|
||||
|
||||
use crate::metadata::TableInfoBuilder;
|
||||
use crate::metadata::TableInfoRef;
|
||||
use crate::requests::InsertRequest;
|
||||
use crate::table::scan::SimpleTableScan;
|
||||
use crate::Result;
|
||||
use crate::{
|
||||
metadata::{TableMetaBuilder, TableType},
|
||||
@@ -64,7 +66,8 @@ impl Table for EmptyTable {
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[common_query::prelude::Expr],
|
||||
_limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
Ok(Box::pin(EmptyRecordBatchStream::new(self.schema())))
|
||||
) -> Result<PhysicalPlanRef> {
|
||||
let scan = SimpleTableScan::new(Box::pin(EmptyRecordBatchStream::new(self.schema())));
|
||||
Ok(Arc::new(scan))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3,9 +3,10 @@ use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_query::physical_plan::PhysicalPlanRef;
|
||||
use common_query::prelude::Expr;
|
||||
use common_recordbatch::error::Result as RecordBatchResult;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream};
|
||||
use datatypes::prelude::*;
|
||||
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
|
||||
use datatypes::vectors::UInt32Vector;
|
||||
@@ -15,6 +16,7 @@ use snafu::prelude::*;
|
||||
|
||||
use crate::error::{Result, SchemaConversionSnafu, TableProjectionSnafu};
|
||||
use crate::metadata::TableInfoRef;
|
||||
use crate::table::scan::SimpleTableScan;
|
||||
use crate::Table;
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
@@ -71,7 +73,7 @@ impl Table for MemTable {
|
||||
projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
) -> Result<PhysicalPlanRef> {
|
||||
let df_recordbatch = if let Some(indices) = projection {
|
||||
self.recordbatch
|
||||
.df_recordbatch
|
||||
@@ -95,10 +97,10 @@ impl Table for MemTable {
|
||||
),
|
||||
df_recordbatch,
|
||||
};
|
||||
Ok(Box::pin(MemtableStream {
|
||||
Ok(Arc::new(SimpleTableScan::new(Box::pin(MemtableStream {
|
||||
schema: recordbatch.schema.clone(),
|
||||
recordbatch: Some(recordbatch),
|
||||
}))
|
||||
}))))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -126,6 +128,7 @@ impl Stream for MemtableStream {
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use common_query::physical_plan::RuntimeEnv;
|
||||
use common_recordbatch::util;
|
||||
use datatypes::prelude::*;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
@@ -138,6 +141,10 @@ mod test {
|
||||
let table = build_testing_table();
|
||||
|
||||
let scan_stream = table.scan(&Some(vec![1]), &[], None).await.unwrap();
|
||||
let scan_stream = scan_stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let recordbatch = util::collect(scan_stream).await.unwrap();
|
||||
assert_eq!(1, recordbatch.len());
|
||||
let columns = recordbatch[0].df_recordbatch.columns();
|
||||
@@ -157,6 +164,10 @@ mod test {
|
||||
let table = build_testing_table();
|
||||
|
||||
let scan_stream = table.scan(&None, &[], Some(2)).await.unwrap();
|
||||
let scan_stream = scan_stream
|
||||
.execute(0, Arc::new(RuntimeEnv::default()))
|
||||
.await
|
||||
.unwrap();
|
||||
let recordbatch = util::collect(scan_stream).await.unwrap();
|
||||
assert_eq!(1, recordbatch.len());
|
||||
let columns = recordbatch[0].df_recordbatch.columns();
|
||||
|
||||
Reference in New Issue
Block a user