You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2023/04/07 10:38:14 UTC

[arrow-datafusion] branch main updated: Fix: allow arbitrary exprs in VALUES clause (#5813)

This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git


The following commit(s) were added to refs/heads/main by this push:
     new f00ef9d548 Fix: allow arbitrary exprs in VALUES clause (#5813)
f00ef9d548 is described below

commit f00ef9d5484039dc7013af5b8fa04f7b29ffaba7
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Fri Apr 7 12:38:09 2023 +0200

    Fix: allow arbitrary exprs in VALUES clause (#5813)
    
    * Fix: allow arbitrary exprs in VALUES clause
    
    * update test
---
 .../core/tests/sqllogictests/test_files/ddl.slt    | 16 ++++++++
 .../tests/sqllogictests/test_files/interval.slt    | 15 +++++--
 datafusion/sql/src/set_expr.rs                     |  4 +-
 datafusion/sql/src/values.rs                       | 46 +++-------------------
 4 files changed, 34 insertions(+), 47 deletions(-)

diff --git a/datafusion/core/tests/sqllogictests/test_files/ddl.slt b/datafusion/core/tests/sqllogictests/test_files/ddl.slt
index 4c942c3b78..d21d85267a 100644
--- a/datafusion/core/tests/sqllogictests/test_files/ddl.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/ddl.slt
@@ -565,3 +565,19 @@ query I
 select * from foo;
 ----
 NULL
+
+statement ok
+drop table foo;
+
+
+## should allow any type of exprs as values
+statement ok
+create table t (i interval, x int) as values (interval '5 days 3 nanoseconds', CASE when true then 1 else 0 end);
+
+query ?I
+select * from t;
+----
+0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs 1
+
+statement ok
+drop table t;
diff --git a/datafusion/core/tests/sqllogictests/test_files/interval.slt b/datafusion/core/tests/sqllogictests/test_files/interval.slt
index be5b432e8e..4a1cd4b220 100644
--- a/datafusion/core/tests/sqllogictests/test_files/interval.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/interval.slt
@@ -149,11 +149,20 @@ select
 Interval(MonthDayNano) Interval(MonthDayNano)
 
 
-# Should work tables with interval values
-# https://github.com/apache/arrow-datafusion/issues/5802
-statement error DataFusion error: This feature is not implemented: Unsupported value Interval \{ value: Value\(SingleQuotedString\("5 days 3 nanoseconds"\)\), leading_field: None, leading_precision: None, last_field: None, fractional_seconds_precision: None \} in a values list expression
+statement ok
 create table t (i interval) as values (interval '5 days 3 nanoseconds');
 
+query ?T rowsort
+select
+  i,
+  arrow_typeof(i)
+from t;
+----
+0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs Interval(MonthDayNano)
+
+
+statement ok
+drop table t;
 
 # Create tables with interval values
 statement ok
diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs
index 6fe2ed8e51..48a4fdddc2 100644
--- a/datafusion/sql/src/set_expr.rs
+++ b/datafusion/sql/src/set_expr.rs
@@ -28,9 +28,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
     ) -> Result<LogicalPlan> {
         match set_expr {
             SetExpr::Select(s) => self.select_to_plan(*s, planner_context),
-            SetExpr::Values(v) => {
-                self.sql_values_to_plan(v, planner_context.prepare_param_data_types())
-            }
+            SetExpr::Values(v) => self.sql_values_to_plan(v, planner_context),
             SetExpr::SetOperation {
                 op,
                 left,
diff --git a/datafusion/sql/src/values.rs b/datafusion/sql/src/values.rs
index 05823c4a71..9efb75bd60 100644
--- a/datafusion/sql/src/values.rs
+++ b/datafusion/sql/src/values.rs
@@ -16,16 +16,15 @@
 // under the License.
 
 use crate::planner::{ContextProvider, PlannerContext, SqlToRel};
-use arrow_schema::DataType;
-use datafusion_common::{DFSchema, DataFusionError, Result};
-use datafusion_expr::{lit, Cast, Expr, LogicalPlan, LogicalPlanBuilder};
-use sqlparser::ast::{Expr as SQLExpr, Values as SQLValues};
+use datafusion_common::{DFSchema, Result};
+use datafusion_expr::{LogicalPlan, LogicalPlanBuilder};
+use sqlparser::ast::Values as SQLValues;
 
 impl<'a, S: ContextProvider> SqlToRel<'a, S> {
     pub(super) fn sql_values_to_plan(
         &self,
         values: SQLValues,
-        param_data_types: &[DataType],
+        planner_context: &mut PlannerContext,
     ) -> Result<LogicalPlan> {
         let SQLValues {
             explicit_row: _,
@@ -38,42 +37,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
             .into_iter()
             .map(|row| {
                 row.into_iter()
-                    .map(|v| match v {
-                        SQLExpr::Value(value) => {
-                            self.parse_value(value, param_data_types)
-                        }
-                        SQLExpr::UnaryOp { op, expr } => self.parse_sql_unary_op(
-                            op,
-                            *expr,
-                            &schema,
-                            &mut PlannerContext::new(),
-                        ),
-                        SQLExpr::BinaryOp { left, op, right } => self
-                            .parse_sql_binary_op(
-                                *left,
-                                op,
-                                *right,
-                                &schema,
-                                &mut PlannerContext::new(),
-                            ),
-                        SQLExpr::TypedString { data_type, value } => {
-                            Ok(Expr::Cast(Cast::new(
-                                Box::new(lit(value)),
-                                self.convert_data_type(&data_type)?,
-                            )))
-                        }
-                        SQLExpr::Cast { expr, data_type } => Ok(Expr::Cast(Cast::new(
-                            Box::new(self.sql_expr_to_logical_expr(
-                                *expr,
-                                &schema,
-                                &mut PlannerContext::new(),
-                            )?),
-                            self.convert_data_type(&data_type)?,
-                        ))),
-                        other => Err(DataFusionError::NotImplemented(format!(
-                            "Unsupported value {other:?} in a values list expression"
-                        ))),
-                    })
+                    .map(|v| self.sql_to_expr(v, &schema, planner_context))
                     .collect::<Result<Vec<_>>>()
             })
             .collect::<Result<Vec<_>>>()?;