mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-08 06:12:55 +00:00
feat: impl QueryEngine#execute, adds numbers table and query engine test (#13)
* feat: impl QueryEngine#execute, adds numbers table and query engine test * fix: clippy warning * fix: reuse runtime in context in table adapter * fix: by CR comments
This commit is contained in:
@@ -11,7 +11,7 @@ use datatypes::schema::{Schema, SchemaRef};
|
||||
use crate::error::Result;
|
||||
|
||||
pub mod adapter;
|
||||
pub mod memory;
|
||||
pub mod numbers;
|
||||
|
||||
pub type TableId = u64;
|
||||
pub type TableVersion = u64;
|
||||
|
||||
@@ -17,7 +17,7 @@ use datafusion::datasource::{
|
||||
TableType as DfTableType,
|
||||
};
|
||||
use datafusion::error::{DataFusionError, Result as DfResult};
|
||||
use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv};
|
||||
use datafusion::execution::runtime_env::RuntimeEnv;
|
||||
use datafusion::logical_plan::Expr as DfExpr;
|
||||
use datafusion::physical_plan::{
|
||||
expressions::PhysicalSortExpr, ExecutionPlan, Partitioning,
|
||||
@@ -168,11 +168,15 @@ impl TableProvider for DfTableProviderAdapter {
|
||||
/// Datafusion TableProvider -> greptime Table
|
||||
pub struct TableAdapter {
|
||||
table_provider: Arc<dyn TableProvider>,
|
||||
runtime: Arc<RuntimeEnv>,
|
||||
}
|
||||
|
||||
impl TableAdapter {
|
||||
pub fn new(table_provider: Arc<dyn TableProvider>) -> Self {
|
||||
Self { table_provider }
|
||||
pub fn new(table_provider: Arc<dyn TableProvider>, runtime: Arc<RuntimeEnv>) -> Self {
|
||||
Self {
|
||||
table_provider,
|
||||
runtime,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -208,10 +212,9 @@ impl Table for TableAdapter {
|
||||
.await
|
||||
.context(error::DatafusionSnafu)?;
|
||||
|
||||
// FIXME(dennis) Partitioning and runtime
|
||||
let runtime = RuntimeEnv::new(RuntimeConfig::default()).context(error::DatafusionSnafu)?;
|
||||
// FIXME(dennis) Partitioning
|
||||
let df_stream = execution_plan
|
||||
.execute(0, Arc::new(runtime))
|
||||
.execute(0, self.runtime.clone())
|
||||
.await
|
||||
.context(error::DatafusionSnafu)?;
|
||||
|
||||
|
||||
@@ -1 +0,0 @@
|
||||
|
||||
93
src/table/src/table/numbers.rs
Normal file
93
src/table/src/table/numbers.rs
Normal file
@@ -0,0 +1,93 @@
|
||||
use std::any::Any;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
|
||||
use arrow::array::UInt32Array;
|
||||
use arrow::datatypes::{DataType, Field, Schema as ArrowSchema};
|
||||
use common_recordbatch::error::Result as RecordBatchResult;
|
||||
use common_recordbatch::{RecordBatch, RecordBatchStream, SendableRecordBatchStream};
|
||||
use datafusion::field_util::SchemaExt;
|
||||
use datafusion_common::record_batch::RecordBatch as DfRecordBatch;
|
||||
use datatypes::schema::{Schema, SchemaRef};
|
||||
use futures::task::{Context, Poll};
|
||||
use futures::Stream;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::table::{Expr, Table};
|
||||
|
||||
/// numbers table for test
|
||||
pub struct NumbersTable {
|
||||
schema: SchemaRef,
|
||||
}
|
||||
|
||||
impl Default for NumbersTable {
|
||||
fn default() -> Self {
|
||||
let arrow_schema = Arc::new(ArrowSchema::new(vec![Field::new(
|
||||
"number",
|
||||
DataType::UInt32,
|
||||
false,
|
||||
)]));
|
||||
Self {
|
||||
schema: Arc::new(Schema::new(arrow_schema)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl Table for NumbersTable {
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
self
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
async fn scan(
|
||||
&self,
|
||||
_projection: &Option<Vec<usize>>,
|
||||
_filters: &[Expr],
|
||||
limit: Option<usize>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
Ok(Box::pin(NumbersStream {
|
||||
limit: limit.unwrap_or(100) as u32,
|
||||
schema: self.schema.clone(),
|
||||
already_run: false,
|
||||
}))
|
||||
}
|
||||
}
|
||||
|
||||
// Limited numbers stream
|
||||
struct NumbersStream {
|
||||
limit: u32,
|
||||
schema: SchemaRef,
|
||||
already_run: bool,
|
||||
}
|
||||
|
||||
impl RecordBatchStream for NumbersStream {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for NumbersStream {
|
||||
type Item = RecordBatchResult<RecordBatch>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
if self.already_run {
|
||||
return Poll::Ready(None);
|
||||
}
|
||||
self.already_run = true;
|
||||
let numbers: Vec<u32> = (0..self.limit).collect();
|
||||
let batch = DfRecordBatch::try_new(
|
||||
self.schema.arrow_schema().clone(),
|
||||
vec![Arc::new(UInt32Array::from_slice(&numbers))],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
Poll::Ready(Some(Ok(RecordBatch {
|
||||
schema: self.schema.clone(),
|
||||
df_recordbatch: batch,
|
||||
})))
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user