Trait datafusion_catalog::Session

source ·
pub trait Session: Send + Sync {
    // Required methods
    fn session_id(&self) -> &str;
    fn config(&self) -> &SessionConfig;
    fn create_physical_plan<'life0, 'life1, 'async_trait>(
        &'life0 self,
        logical_plan: &'life1 LogicalPlan,
    ) -> Pin<Box<dyn Future<Output = Result<Arc<dyn ExecutionPlan>>> + Send + 'async_trait>>
       where Self: 'async_trait,
             'life0: 'async_trait,
             'life1: 'async_trait;
    fn create_physical_expr(
        &self,
        expr: Expr,
        df_schema: &DFSchema,
    ) -> Result<Arc<dyn PhysicalExpr>>;
    fn scalar_functions(&self) -> &HashMap<String, Arc<ScalarUDF>>;
    fn aggregate_functions(&self) -> &HashMap<String, Arc<AggregateUDF>>;
    fn window_functions(&self) -> &HashMap<String, Arc<WindowUDF>>;
    fn runtime_env(&self) -> &Arc<RuntimeEnv>;
    fn execution_props(&self) -> &ExecutionProps;
    fn as_any(&self) -> &dyn Any;

    // Provided method
    fn config_options(&self) -> &ConfigOptions { ... }
}
Expand description

Interface for accessing SessionState from the catalog.

This trait provides access to the information needed to plan and execute queries, such as configuration, functions, and runtime environment. See the documentation on SessionState for more information.

Historically, the SessionState struct was passed directly to catalog traits such as TableProvider, which required a direct dependency on the DataFusion core. The interface required is now defined by this trait. See #10782 for more details.

§Migration from SessionState

Using trait methods is preferred, as the implementation may change in future versions. However, you can downcast a Session to a SessionState as shown in the example below. If you find yourself needing to do this, please open an issue on the DataFusion repository so we can extend the trait to provide the required information.

// Given a `Session` reference, get the concrete `SessionState` reference
// Note: this may stop working in future versions,
fn session_state_from_session(session: &dyn Session) -> Result<&SessionState> {
   session.as_any()
    .downcast_ref::<SessionState>()
    .ok_or_else(|| exec_datafusion_err!("Failed to downcast Session to SessionState"))
}

Required Methods§

source

fn session_id(&self) -> &str

Return the session ID

source

fn config(&self) -> &SessionConfig

Return the SessionConfig

source

fn create_physical_plan<'life0, 'life1, 'async_trait>( &'life0 self, logical_plan: &'life1 LogicalPlan, ) -> Pin<Box<dyn Future<Output = Result<Arc<dyn ExecutionPlan>>> + Send + 'async_trait>>
where Self: 'async_trait, 'life0: 'async_trait, 'life1: 'async_trait,

Creates a physical ExecutionPlan plan from a LogicalPlan.

Note: this will optimize the provided plan first.

This function will error for LogicalPlans such as catalog DDL like CREATE TABLE, which do not have corresponding physical plans and must be handled by another layer, typically the SessionContext.

source

fn create_physical_expr( &self, expr: Expr, df_schema: &DFSchema, ) -> Result<Arc<dyn PhysicalExpr>>

Create a PhysicalExpr from an Expr after applying type coercion, and function rewrites.

Note: The expression is not simplified or otherwise optimized: `a = 1

  • 2will not be simplified toa = 3` as this is a more involved process. See the expr_api example for how to simplify expressions.
source

fn scalar_functions(&self) -> &HashMap<String, Arc<ScalarUDF>>

Return reference to scalar_functions

source

fn aggregate_functions(&self) -> &HashMap<String, Arc<AggregateUDF>>

Return reference to aggregate_functions

source

fn window_functions(&self) -> &HashMap<String, Arc<WindowUDF>>

Return reference to window functions

source

fn runtime_env(&self) -> &Arc<RuntimeEnv>

Return the runtime env

source

fn execution_props(&self) -> &ExecutionProps

Return the execution properties

source

fn as_any(&self) -> &dyn Any

Provided Methods§

Trait Implementations§

source§

impl From<&dyn Session> for TaskContext

Create a new task context instance from Session

source§

fn from(state: &dyn Session) -> Self

Converts to this type from the input type.

Implementors§