use std::any::Any;
use std::sync::Arc;
use arrow::datatypes::SchemaRef;
use async_trait::async_trait;
use crate::Session;
use crate::TableProvider;
use datafusion_common::{plan_err, Result};
use datafusion_expr::{Expr, TableType};
use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec};
use datafusion_physical_plan::ExecutionPlan;
use log::debug;
#[derive(Debug)]
pub struct StreamingTable {
schema: SchemaRef,
partitions: Vec<Arc<dyn PartitionStream>>,
infinite: bool,
}
impl StreamingTable {
pub fn try_new(
schema: SchemaRef,
partitions: Vec<Arc<dyn PartitionStream>>,
) -> Result<Self> {
for x in partitions.iter() {
let partition_schema = x.schema();
if !schema.contains(partition_schema) {
debug!(
"target schema does not contain partition schema. \
Target_schema: {schema:?}. Partition Schema: {partition_schema:?}"
);
return plan_err!("Mismatch between schema and batches");
}
}
Ok(Self {
schema,
partitions,
infinite: false,
})
}
pub fn with_infinite_table(mut self, infinite: bool) -> Self {
self.infinite = infinite;
self
}
}
#[async_trait]
impl TableProvider for StreamingTable {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
fn table_type(&self) -> TableType {
TableType::View
}
async fn scan(
&self,
_state: &dyn Session,
projection: Option<&Vec<usize>>,
_filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(StreamingTableExec::try_new(
Arc::clone(&self.schema),
self.partitions.clone(),
projection,
None,
self.infinite,
limit,
)?))
}
}