mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-17 02:32:56 +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:
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user