Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 11 additions & 2 deletions datafusion-examples/examples/dataframe/cache_factory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use std::sync::{Arc, RwLock};

use arrow::array::RecordBatch;
use async_trait::async_trait;
use datafusion::catalog::Session;
use datafusion::catalog::memory::MemorySourceConfig;
use datafusion::common::DFSchemaRef;
use datafusion::error::Result;
Expand All @@ -37,7 +38,7 @@ use datafusion::physical_planner::{
DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner,
};
use datafusion::prelude::*;
use datafusion_common::HashMap;
use datafusion_common::{HashMap, exec_datafusion_err};
use datafusion_examples::utils::{datasets::ExampleDataset, write_csv_to_parquet};

/// This example demonstrates how to leverage [CacheFactory] to implement custom caching strategies for dataframes in DataFusion.
Expand Down Expand Up @@ -198,8 +199,16 @@ impl QueryPlanner for CacheNodeQueryPlanner {
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
session: &dyn Session,
) -> Result<Arc<dyn ExecutionPlan>> {
let session_state =
session
.as_any()
.downcast_ref::<SessionState>()
.ok_or_else(|| {
exec_datafusion_err!("Failed to downcast Session to SessionState")
})?;

let physical_planner =
DefaultPhysicalPlanner::with_extension_planners(vec![Arc::new(
CacheNodePlanner {
Expand Down
18 changes: 14 additions & 4 deletions datafusion-examples/examples/relation_planner/table_sample.rs
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,9 @@ use futures::{
use rand::{Rng, SeedableRng, rngs::StdRng};
use tonic::async_trait;

use datafusion::optimizer::simplify_expressions::simplify_literal::parse_literal;
use datafusion::{
catalog::Session, optimizer::simplify_expressions::simplify_literal::parse_literal,
};
use datafusion::{
execution::{
RecordBatchStream, SendableRecordBatchStream, SessionState, SessionStateBuilder,
Expand All @@ -116,8 +118,8 @@ use datafusion::{
prelude::*,
};
use datafusion_common::{
DFSchemaRef, DataFusionError, Result, Statistics, internal_err, not_impl_err,
plan_datafusion_err, plan_err,
DFSchemaRef, DataFusionError, Result, Statistics, exec_datafusion_err, internal_err,
not_impl_err, plan_datafusion_err, plan_err,
};
use datafusion_expr::{
UserDefinedLogicalNode, UserDefinedLogicalNodeCore,
Expand Down Expand Up @@ -564,8 +566,16 @@ impl QueryPlanner for TableSampleQueryPlanner {
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
session: &dyn Session,
) -> Result<Arc<dyn ExecutionPlan>> {
let session_state =
session
.as_any()
.downcast_ref::<SessionState>()
.ok_or_else(|| {
exec_datafusion_err!("Failed to downcast Session to SessionState")
})?;

let planner = DefaultPhysicalPlanner::with_extension_planners(vec![Arc::new(
TableSampleExtensionPlanner,
)]);
Expand Down
21 changes: 9 additions & 12 deletions datafusion/core/src/execution/context/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ use crate::{

// backwards compatibility
pub use crate::execution::session_state::SessionState;
pub use datafusion_session::QueryPlanner;

use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
Expand Down Expand Up @@ -1970,17 +1971,6 @@ impl From<SessionContext> for SessionStateBuilder {
}
}

/// A planner used to add extensions to DataFusion logical and physical plans.
#[async_trait]
pub trait QueryPlanner: Debug {
/// Given a [`LogicalPlan`], create an [`ExecutionPlan`] suitable for execution
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
) -> Result<Arc<dyn ExecutionPlan>>;
}

/// Interface for handling `CREATE FUNCTION` statements and interacting with
/// [SessionState] to create and register functions ([`ScalarUDF`],
/// [`AggregateUDF`], [`WindowUDF`], and [`TableFunctionImpl`]) dynamically.
Expand Down Expand Up @@ -2162,6 +2152,7 @@ mod tests {
use crate::test_util::{plan_and_collect, populate_csv_partitions};
use arrow::datatypes::{DataType, TimeUnit};
use datafusion_common::DataFusionError;
use datafusion_session::Session;
use std::error::Error;
use std::path::PathBuf;

Expand Down Expand Up @@ -2643,8 +2634,14 @@ mod tests {
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
session: &dyn Session,
) -> Result<Arc<dyn ExecutionPlan>> {
let session_state = session
.as_any()
.downcast_ref::<SessionState>()
.ok_or_else(|| {
exec_datafusion_err!("Failed to downcast Session to SessionState")
})?;
let physical_planner = MyPhysicalPlanner {};
physical_planner
.create_physical_plan(logical_plan, session_state)
Expand Down
15 changes: 11 additions & 4 deletions datafusion/core/src/execution/session_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ use crate::datasource::file_format::FileFormatFactory;
#[cfg(feature = "sql")]
use crate::datasource::provider_as_source;
use crate::execution::SessionStateDefaults;
use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner};
use crate::execution::context::{EmptySerializerRegistry, FunctionFactory};
use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner};
use arrow_schema::{DataType, FieldRef};
use datafusion_catalog::MemoryCatalogProviderList;
Expand All @@ -44,7 +44,7 @@ use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan};
use datafusion_common::tree_node::TreeNode;
use datafusion_common::{
DFSchema, DataFusionError, ResolvedTableReference, TableReference, config_err,
exec_err, plan_datafusion_err,
exec_datafusion_err, exec_err, plan_datafusion_err,
};
use datafusion_execution::TaskContext;
use datafusion_execution::config::SessionConfig;
Expand All @@ -68,7 +68,7 @@ use datafusion_physical_expr_common::physical_expr::PhysicalExpr;
use datafusion_physical_optimizer::PhysicalOptimizerRule;
use datafusion_physical_optimizer::optimizer::PhysicalOptimizer;
use datafusion_physical_plan::ExecutionPlan;
use datafusion_session::Session;
use datafusion_session::{QueryPlanner, Session};
#[cfg(feature = "sql")]
use datafusion_sql::{
parser::{DFParserBuilder, Statement},
Expand Down Expand Up @@ -2115,8 +2115,15 @@ impl QueryPlanner for DefaultQueryPlanner {
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
session: &dyn Session,
) -> datafusion_common::Result<Arc<dyn ExecutionPlan>> {
let session_state =
session
.as_any()
.downcast_ref::<SessionState>()
.ok_or_else(|| {
exec_datafusion_err!("Failed to downcast Session to SessionState")
})?;
let planner = DefaultPhysicalPlanner::default();
planner
.create_physical_plan(logical_plan, session_state)
Expand Down
15 changes: 13 additions & 2 deletions datafusion/core/tests/user_defined/user_defined_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -91,14 +91,17 @@ use datafusion::{
};
use datafusion_common::config::ConfigOptions;
use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode};
use datafusion_common::{ScalarValue, assert_eq_or_internal_err, assert_or_internal_err};
use datafusion_common::{
ScalarValue, assert_eq_or_internal_err, assert_or_internal_err, exec_datafusion_err,
};
use datafusion_expr::{FetchType, InvariantLevel, Projection, SortExpr};
use datafusion_optimizer::AnalyzerRule;
use datafusion_optimizer::optimizer::ApplyOrder;
use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType};

use async_trait::async_trait;
use datafusion_common::cast::as_string_view_array;
use datafusion_session::Session;
use futures::{Stream, StreamExt};

/// Execute the specified sql and return the resulting record batches
Expand Down Expand Up @@ -466,8 +469,16 @@ impl QueryPlanner for TopKQueryPlanner {
async fn create_physical_plan(
&self,
logical_plan: &LogicalPlan,
session_state: &SessionState,
session: &dyn Session,
) -> Result<Arc<dyn ExecutionPlan>> {
let session_state =
session
.as_any()
.downcast_ref::<SessionState>()
.ok_or_else(|| {
exec_datafusion_err!("Failed to downcast Session to SessionState")
})?;

// Teach the default physical planner how to plan TopK nodes.
let physical_planner =
DefaultPhysicalPlanner::with_extension_planners(vec![Arc::new(
Expand Down
2 changes: 1 addition & 1 deletion datafusion/execution/src/task.rs
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ impl FunctionRegistry for TaskContext {
}

/// Produce the [`TaskContext`].
pub trait TaskContextProvider {
pub trait TaskContextProvider: Sync + Send {
fn task_ctx(&self) -> Arc<TaskContext>;
}

Expand Down
9 changes: 7 additions & 2 deletions datafusion/ffi/src/session/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,9 @@ use datafusion_expr::{
};
use datafusion_physical_expr::PhysicalExpr;
use datafusion_physical_plan::ExecutionPlan;
use datafusion_proto::bytes::{logical_plan_from_bytes, logical_plan_to_bytes};
use datafusion_proto::bytes::{
logical_plan_from_bytes, logical_plan_to_bytes_with_extension_codec,
};
use datafusion_proto::logical_plan::LogicalExtensionCodec;
use datafusion_proto::logical_plan::from_proto::parse_expr;
use datafusion_proto::logical_plan::to_proto::serialize_expr;
Expand All @@ -60,6 +62,7 @@ use crate::util::FFIResult;
use crate::{df_result, rresult, rresult_return};

pub mod config;
pub mod planner;

/// A stable struct for sharing [`Session`] across FFI boundaries.
///
Expand Down Expand Up @@ -467,8 +470,10 @@ impl Session for ForeignSession {
&self,
logical_plan: &LogicalPlan,
) -> datafusion_common::Result<Arc<dyn ExecutionPlan>> {
let codec: Arc<dyn LogicalExtensionCodec> = (&self.session.logical_codec).into();
unsafe {
let logical_plan = logical_plan_to_bytes(logical_plan)?;
let logical_plan =
logical_plan_to_bytes_with_extension_codec(logical_plan, codec.as_ref())?;
let physical_plan = df_result!(
(self.session.create_physical_plan)(
&self.session,
Expand Down
Loading