Skip to content

Commit c8b8c74

Browse files
alambwestonpace
andauthored
Add SessionContext/SessionState::create_physical_expr() to create PhysicalExpressions from Exprs (#10330)
* Improve coerce API so it does not need DFSchema * Add `SessionContext::create_physical_expr()` and `SessionState::create_physical_expr()` * Apply suggestions from code review Co-authored-by: Weston Pace <[email protected]> * Add note on simplification --------- Co-authored-by: Weston Pace <[email protected]>
1 parent f0e96c6 commit c8b8c74

File tree

6 files changed

+337
-28
lines changed

6 files changed

+337
-28
lines changed

datafusion-examples/examples/expr_api.rs

Lines changed: 11 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -25,9 +25,7 @@ use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit};
2525
use datafusion::common::DFSchema;
2626
use datafusion::error::Result;
2727
use datafusion::optimizer::simplify_expressions::ExprSimplifier;
28-
use datafusion::physical_expr::{
29-
analyze, create_physical_expr, AnalysisContext, ExprBoundaries, PhysicalExpr,
30-
};
28+
use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries};
3129
use datafusion::prelude::*;
3230
use datafusion_common::{ScalarValue, ToDFSchema};
3331
use datafusion_expr::execution_props::ExecutionProps;
@@ -92,7 +90,8 @@ fn evaluate_demo() -> Result<()> {
9290
let expr = col("a").lt(lit(5)).or(col("a").eq(lit(8)));
9391

9492
// First, you make a "physical expression" from the logical `Expr`
95-
let physical_expr = physical_expr(&batch.schema(), expr)?;
93+
let df_schema = DFSchema::try_from(batch.schema())?;
94+
let physical_expr = SessionContext::new().create_physical_expr(expr, &df_schema)?;
9695

9796
// Now, you can evaluate the expression against the RecordBatch
9897
let result = physical_expr.evaluate(&batch)?;
@@ -213,7 +212,7 @@ fn range_analysis_demo() -> Result<()> {
213212
// `date < '2020-10-01' AND date > '2020-09-01'`
214213

215214
// As always, we need to tell DataFusion the type of column "date"
216-
let schema = Schema::new(vec![make_field("date", DataType::Date32)]);
215+
let schema = Arc::new(Schema::new(vec![make_field("date", DataType::Date32)]));
217216

218217
// You can provide DataFusion any known boundaries on the values of `date`
219218
// (for example, maybe you know you only have data up to `2020-09-15`), but
@@ -222,9 +221,13 @@ fn range_analysis_demo() -> Result<()> {
222221
let boundaries = ExprBoundaries::try_new_unbounded(&schema)?;
223222

224223
// Now, we invoke the analysis code to perform the range analysis
225-
let physical_expr = physical_expr(&schema, expr)?;
226-
let analysis_result =
227-
analyze(&physical_expr, AnalysisContext::new(boundaries), &schema)?;
224+
let df_schema = DFSchema::try_from(schema)?;
225+
let physical_expr = SessionContext::new().create_physical_expr(expr, &df_schema)?;
226+
let analysis_result = analyze(
227+
&physical_expr,
228+
AnalysisContext::new(boundaries),
229+
df_schema.as_ref(),
230+
)?;
228231

229232
// The results of the analysis is an range, encoded as an `Interval`, for
230233
// each column in the schema, that must be true in order for the predicate
@@ -248,21 +251,6 @@ fn make_ts_field(name: &str) -> Field {
248251
make_field(name, DataType::Timestamp(TimeUnit::Nanosecond, tz))
249252
}
250253

251-
/// Build a physical expression from a logical one, after applying simplification and type coercion
252-
pub fn physical_expr(schema: &Schema, expr: Expr) -> Result<Arc<dyn PhysicalExpr>> {
253-
let df_schema = schema.clone().to_dfschema_ref()?;
254-
255-
// Simplify
256-
let props = ExecutionProps::new();
257-
let simplifier =
258-
ExprSimplifier::new(SimplifyContext::new(&props).with_schema(df_schema.clone()));
259-
260-
// apply type coercion here to ensure types match
261-
let expr = simplifier.coerce(expr, &df_schema)?;
262-
263-
create_physical_expr(&expr, df_schema.as_ref(), &props)
264-
}
265-
266254
/// This function shows how to use `Expr::get_type` to retrieve the DataType
267255
/// of an expression
268256
fn expression_type_demo() -> Result<()> {

datafusion/common/src/dfschema.rs

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -125,6 +125,20 @@ impl DFSchema {
125125
}
126126
}
127127

128+
/// Return a reference to the inner Arrow [`Schema`]
129+
///
130+
/// Note this does not have the qualifier information
131+
pub fn as_arrow(&self) -> &Schema {
132+
self.inner.as_ref()
133+
}
134+
135+
/// Return a reference to the inner Arrow [`SchemaRef`]
136+
///
137+
/// Note this does not have the qualifier information
138+
pub fn inner(&self) -> &SchemaRef {
139+
&self.inner
140+
}
141+
128142
/// Create a `DFSchema` from an Arrow schema where all the fields have a given qualifier
129143
pub fn new_with_metadata(
130144
qualified_fields: Vec<(Option<TableReference>, Arc<Field>)>,
@@ -806,6 +820,21 @@ impl From<&DFSchema> for Schema {
806820
}
807821
}
808822

823+
/// Allow DFSchema to be converted into an Arrow `&Schema`
824+
impl AsRef<Schema> for DFSchema {
825+
fn as_ref(&self) -> &Schema {
826+
self.as_arrow()
827+
}
828+
}
829+
830+
/// Allow DFSchema to be converted into an Arrow `&SchemaRef` (to clone, for
831+
/// example)
832+
impl AsRef<SchemaRef> for DFSchema {
833+
fn as_ref(&self) -> &SchemaRef {
834+
self.inner()
835+
}
836+
}
837+
809838
/// Create a `DFSchema` from an Arrow schema
810839
impl TryFrom<Schema> for DFSchema {
811840
type Error = DataFusionError;

datafusion/core/src/execution/context/mod.rs

Lines changed: 108 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -71,13 +71,13 @@ use datafusion_common::{
7171
config::{ConfigExtension, TableOptions},
7272
exec_err, not_impl_err, plan_datafusion_err, plan_err,
7373
tree_node::{TreeNodeRecursion, TreeNodeVisitor},
74-
SchemaReference, TableReference,
74+
DFSchema, SchemaReference, TableReference,
7575
};
7676
use datafusion_execution::registry::SerializerRegistry;
7777
use datafusion_expr::{
7878
logical_plan::{DdlStatement, Statement},
7979
var_provider::is_system_variables,
80-
Expr, StringifiedPlan, UserDefinedLogicalNode, WindowUDF,
80+
Expr, ExprSchemable, StringifiedPlan, UserDefinedLogicalNode, WindowUDF,
8181
};
8282
use datafusion_sql::{
8383
parser::{CopyToSource, CopyToStatement, DFParser},
@@ -87,15 +87,20 @@ use datafusion_sql::{
8787

8888
use async_trait::async_trait;
8989
use chrono::{DateTime, Utc};
90+
use datafusion_common::tree_node::TreeNode;
9091
use parking_lot::RwLock;
9192
use sqlparser::dialect::dialect_from_str;
9293
use url::Url;
9394
use uuid::Uuid;
9495

96+
use crate::physical_expr::PhysicalExpr;
9597
pub use datafusion_execution::config::SessionConfig;
9698
pub use datafusion_execution::TaskContext;
9799
pub use datafusion_expr::execution_props::ExecutionProps;
98100
use datafusion_expr::expr_rewriter::FunctionRewrite;
101+
use datafusion_expr::simplify::SimplifyInfo;
102+
use datafusion_optimizer::simplify_expressions::ExprSimplifier;
103+
use datafusion_physical_expr::create_physical_expr;
99104

100105
mod avro;
101106
mod csv;
@@ -523,6 +528,41 @@ impl SessionContext {
523528
}
524529
}
525530

531+
/// Create a [`PhysicalExpr`] from an [`Expr`] after applying type
532+
/// coercion and function rewrites.
533+
///
534+
/// Note: The expression is not [simplified] or otherwise optimized: `a = 1
535+
/// + 2` will not be simplified to `a = 3` as this is a more involved process.
536+
/// See the [expr_api] example for how to simplify expressions.
537+
///
538+
/// # Example
539+
/// ```
540+
/// # use std::sync::Arc;
541+
/// # use arrow::datatypes::{DataType, Field, Schema};
542+
/// # use datafusion::prelude::*;
543+
/// # use datafusion_common::DFSchema;
544+
/// // a = 1 (i64)
545+
/// let expr = col("a").eq(lit(1i64));
546+
/// // provide type information that `a` is an Int32
547+
/// let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]);
548+
/// let df_schema = DFSchema::try_from(schema).unwrap();
549+
/// // Create a PhysicalExpr. Note DataFusion automatically coerces (casts) `1i64` to `1i32`
550+
/// let physical_expr = SessionContext::new()
551+
/// .create_physical_expr(expr, &df_schema).unwrap();
552+
/// ```
553+
/// # See Also
554+
/// * [`SessionState::create_physical_expr`] for a lower level API
555+
///
556+
/// [simplified]: datafusion_optimizer::simplify_expressions
557+
/// [expr_api]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/expr_api.rs
558+
pub fn create_physical_expr(
559+
&self,
560+
expr: Expr,
561+
df_schema: &DFSchema,
562+
) -> Result<Arc<dyn PhysicalExpr>> {
563+
self.state.read().create_physical_expr(expr, df_schema)
564+
}
565+
526566
// return an empty dataframe
527567
fn return_empty_dataframe(&self) -> Result<DataFrame> {
528568
let plan = LogicalPlanBuilder::empty(false).build()?;
@@ -1946,13 +1986,14 @@ impl SessionState {
19461986
}
19471987
}
19481988

1949-
/// Creates a physical plan from a logical plan.
1989+
/// Creates a physical [`ExecutionPlan`] plan from a [`LogicalPlan`].
19501990
///
19511991
/// Note: this first calls [`Self::optimize`] on the provided
19521992
/// plan.
19531993
///
1954-
/// This function will error for [`LogicalPlan`]s such as catalog
1955-
/// DDL `CREATE TABLE` must be handled by another layer.
1994+
/// This function will error for [`LogicalPlan`]s such as catalog DDL like
1995+
/// `CREATE TABLE`, which do not have corresponding physical plans and must
1996+
/// be handled by another layer, typically [`SessionContext`].
19561997
pub async fn create_physical_plan(
19571998
&self,
19581999
logical_plan: &LogicalPlan,
@@ -1963,6 +2004,39 @@ impl SessionState {
19632004
.await
19642005
}
19652006

2007+
/// Create a [`PhysicalExpr`] from an [`Expr`] after applying type
2008+
/// coercion, and function rewrites.
2009+
///
2010+
/// Note: The expression is not [simplified] or otherwise optimized: `a = 1
2011+
/// + 2` will not be simplified to `a = 3` as this is a more involved process.
2012+
/// See the [expr_api] example for how to simplify expressions.
2013+
///
2014+
/// # See Also:
2015+
/// * [`SessionContext::create_physical_expr`] for a higher-level API
2016+
/// * [`create_physical_expr`] for a lower-level API
2017+
///
2018+
/// [simplified]: datafusion_optimizer::simplify_expressions
2019+
/// [expr_api]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/expr_api.rs
2020+
pub fn create_physical_expr(
2021+
&self,
2022+
expr: Expr,
2023+
df_schema: &DFSchema,
2024+
) -> Result<Arc<dyn PhysicalExpr>> {
2025+
let simplifier =
2026+
ExprSimplifier::new(SessionSimplifyProvider::new(self, df_schema));
2027+
// apply type coercion here to ensure types match
2028+
let mut expr = simplifier.coerce(expr, df_schema)?;
2029+
2030+
// rewrite Exprs to functions if necessary
2031+
let config_options = self.config_options();
2032+
for rewrite in self.analyzer.function_rewrites() {
2033+
expr = expr
2034+
.transform_up(|expr| rewrite.rewrite(expr, df_schema, config_options))?
2035+
.data;
2036+
}
2037+
create_physical_expr(&expr, df_schema, self.execution_props())
2038+
}
2039+
19662040
/// Return the session ID
19672041
pub fn session_id(&self) -> &str {
19682042
&self.session_id
@@ -2040,6 +2114,35 @@ impl SessionState {
20402114
}
20412115
}
20422116

2117+
struct SessionSimplifyProvider<'a> {
2118+
state: &'a SessionState,
2119+
df_schema: &'a DFSchema,
2120+
}
2121+
2122+
impl<'a> SessionSimplifyProvider<'a> {
2123+
fn new(state: &'a SessionState, df_schema: &'a DFSchema) -> Self {
2124+
Self { state, df_schema }
2125+
}
2126+
}
2127+
2128+
impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> {
2129+
fn is_boolean_type(&self, expr: &Expr) -> Result<bool> {
2130+
Ok(expr.get_type(self.df_schema)? == DataType::Boolean)
2131+
}
2132+
2133+
fn nullable(&self, expr: &Expr) -> Result<bool> {
2134+
expr.nullable(self.df_schema)
2135+
}
2136+
2137+
fn execution_props(&self) -> &ExecutionProps {
2138+
self.state.execution_props()
2139+
}
2140+
2141+
fn get_data_type(&self, expr: &Expr) -> Result<DataType> {
2142+
expr.get_type(self.df_schema)
2143+
}
2144+
}
2145+
20432146
struct SessionContextProvider<'a> {
20442147
state: &'a SessionState,
20452148
tables: HashMap<String, Arc<dyn TableSource>>,

datafusion/core/tests/core_integration.rs

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,9 @@ mod dataframe;
2424
/// Run all tests that are found in the `macro_hygiene` directory
2525
mod macro_hygiene;
2626

27+
/// Run all tests that are found in the `expr_api` directory
28+
mod expr_api;
29+
2730
#[cfg(test)]
2831
#[ctor::ctor]
2932
fn init() {

0 commit comments

Comments
 (0)