use crate::arrow::record_batch::RecordBatch;
use crate::error::Result;
use crate::logical_plan::{
DFSchema, Expr, FunctionRegistry, JoinType, LogicalPlan, Partitioning,
};
use std::sync::Arc;
use crate::physical_plan::SendableRecordBatchStream;
use async_trait::async_trait;
#[async_trait]
pub trait DataFrame: Send + Sync {
fn select_columns(&self, columns: &[&str]) -> Result<Arc<dyn DataFrame>>;
fn select(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>;
fn filter(&self, expr: Expr) -> Result<Arc<dyn DataFrame>>;
fn aggregate(
&self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>,
) -> Result<Arc<dyn DataFrame>>;
fn limit(&self, n: usize) -> Result<Arc<dyn DataFrame>>;
fn union(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
fn distinct(&self) -> Result<Arc<dyn DataFrame>>;
fn sort(&self, expr: Vec<Expr>) -> Result<Arc<dyn DataFrame>>;
fn join(
&self,
right: Arc<dyn DataFrame>,
join_type: JoinType,
left_cols: &[&str],
right_cols: &[&str],
) -> Result<Arc<dyn DataFrame>>;
fn repartition(
&self,
partitioning_scheme: Partitioning,
) -> Result<Arc<dyn DataFrame>>;
async fn collect(&self) -> Result<Vec<RecordBatch>>;
async fn show(&self) -> Result<()>;
async fn show_limit(&self, n: usize) -> Result<()>;
async fn execute_stream(&self) -> Result<SendableRecordBatchStream>;
async fn collect_partitioned(&self) -> Result<Vec<Vec<RecordBatch>>>;
async fn execute_stream_partitioned(&self) -> Result<Vec<SendableRecordBatchStream>>;
fn schema(&self) -> &DFSchema;
fn to_logical_plan(&self) -> LogicalPlan;
fn explain(&self, verbose: bool, analyze: bool) -> Result<Arc<dyn DataFrame>>;
fn registry(&self) -> Arc<dyn FunctionRegistry>;
fn intersect(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
fn except(&self, dataframe: Arc<dyn DataFrame>) -> Result<Arc<dyn DataFrame>>;
}