You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "backkem (via GitHub)" <gi...@apache.org> on 2024/03/09 14:25:03 UTC

[PR] feat: convert Expr to SQL string [arrow-datafusion]

backkem opened a new pull request, #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517

   Relates to #9495.
   
   This is an initial attempt at porting over the SQL Unparser as described in the tracking ticket.
   
   Open question: Do we need this to be feature complete before landing?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "backkem (via GitHub)" <gi...@apache.org>.
backkem commented on PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#issuecomment-1988468553

   I believe I addressed the points that we didn't decide to ticket out. The CI indicates that some workflow still requires approval from a maintainer


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: convert Expr to SQL string [arrow-datafusion]

Posted by "backkem (via GitHub)" <gi...@apache.org>.
backkem commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1518753999


##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))

Review Comment:
   Does DF represent this nesting explicitly as well? Or does it use the Expr hierarchy to capture the order?
   
   If it's the latter, do we want to inject brackets for every operations or have a heuristic to only add them when mathematically needed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#issuecomment-1989441105

   Onwards and upwards (and thank you for your first contribution @backkem )


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "backkem (via GitHub)" <gi...@apache.org>.
backkem commented on PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#issuecomment-1989010844

   I see the CI fails on some dead code. I'll look into it. There is another CI failure but it seems related to a network timeout.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1518829105


##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`

Review Comment:
   ```suggestion
   /// This function is the opposite of `SqlToRel::sql_to_expr` and can
   /// be used to, among other things, convert `Expr`s to strings. 
   ```



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))

Review Comment:
   DF represents the nesting via `Expr` children
   
   In the SQL parser, there are precident rules to resolve how `a + b + c` into a tree and then DataFusion simply gets the same tree structure
   
   So the difference between `(a + b) + c` ad `a + (b + c)` is a function of what the `left` and `right` of the `BinaryExpr` are
   
   I agree with @devinjdangelo  that we should file a ticket to track this issue and work on in a subsequent PR



##########
datafusion/sql/src/unparser/mod.rs:
##########
@@ -0,0 +1,24 @@
+mod expr;

Review Comment:
   The new files in this PR need the ASF header (there is a RAT CI check that will fail)



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example

Review Comment:
   ```suggestion
   /// # Example
   ```



##########
datafusion/sql/src/unparser/dialect.rs:
##########
@@ -0,0 +1,52 @@
+pub trait Dialect {

Review Comment:
   Shall we make a change upstream? I would be happy to review such a PR (or maybe just make it?) We could leave a comment in the PR to remove this trait when the new sqlparser cate was released



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))
+            }
+            Expr::Case(Case {
+                expr,
+                when_then_expr: _,
+                else_expr: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Cast(Cast { expr, data_type: _ }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Literal(value) => Ok(ast::Expr::Value(self.scalar_to_sql(value)?)),
+            Expr::Alias(Alias { expr, name: _, .. }) => self.expr_to_sql(expr),
+            Expr::WindowFunction(WindowFunction {
+                fun: _,
+                args: _,
+                partition_by: _,
+                order_by: _,
+                window_frame: _,
+                null_treatment: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Like(Like {
+                negated: _,
+                expr,
+                pattern: _,
+                escape_char: _,
+                case_insensitive: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            _ => not_impl_err!("Unsupported expression: {expr:?}"),
+        }
+    }
+
+    fn col_to_sql(&self, col: &Column) -> Result<ast::Expr> {
+        if let Some(table_ref) = &col.relation {
+            let mut id = table_ref.to_vec();
+            id.push(col.name.to_string());
+            return Ok(ast::Expr::CompoundIdentifier(
+                id.iter().map(|i| self.new_ident(i.to_string())).collect(),
+            ));
+        }
+        Ok(ast::Expr::Identifier(self.new_ident(col.name.to_string())))
+    }
+
+    fn new_ident(&self, str: String) -> ast::Ident {
+        ast::Ident {
+            value: str,
+            quote_style: self.dialect.identifier_quote_style(),
+        }
+    }
+
+    fn binary_op_to_sql(
+        &self,
+        lhs: ast::Expr,
+        rhs: ast::Expr,
+        op: ast::BinaryOperator,
+    ) -> ast::Expr {
+        ast::Expr::BinaryOp {
+            left: Box::new(lhs),
+            op,
+            right: Box::new(rhs),
+        }
+    }
+
+    fn op_to_sql(&self, op: &Operator) -> Result<ast::BinaryOperator> {
+        match op {
+            Operator::Eq => Ok(ast::BinaryOperator::Eq),
+            Operator::NotEq => Ok(ast::BinaryOperator::NotEq),
+            Operator::Lt => Ok(ast::BinaryOperator::Lt),
+            Operator::LtEq => Ok(ast::BinaryOperator::LtEq),
+            Operator::Gt => Ok(ast::BinaryOperator::Gt),
+            Operator::GtEq => Ok(ast::BinaryOperator::GtEq),
+            Operator::Plus => Ok(ast::BinaryOperator::Plus),
+            Operator::Minus => Ok(ast::BinaryOperator::Minus),
+            Operator::Multiply => Ok(ast::BinaryOperator::Multiply),
+            Operator::Divide => Ok(ast::BinaryOperator::Divide),
+            Operator::Modulo => Ok(ast::BinaryOperator::Modulo),
+            Operator::And => Ok(ast::BinaryOperator::And),
+            Operator::Or => Ok(ast::BinaryOperator::Or),
+            Operator::IsDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::IsNotDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::RegexMatch => Ok(ast::BinaryOperator::PGRegexMatch),
+            Operator::RegexIMatch => Ok(ast::BinaryOperator::PGRegexIMatch),
+            Operator::RegexNotMatch => Ok(ast::BinaryOperator::PGRegexNotMatch),
+            Operator::RegexNotIMatch => Ok(ast::BinaryOperator::PGRegexNotIMatch),
+            Operator::ILikeMatch => Ok(ast::BinaryOperator::PGILikeMatch),
+            Operator::NotLikeMatch => Ok(ast::BinaryOperator::PGNotLikeMatch),
+            Operator::LikeMatch => Ok(ast::BinaryOperator::PGLikeMatch),
+            Operator::NotILikeMatch => Ok(ast::BinaryOperator::PGNotILikeMatch),
+            Operator::BitwiseAnd => Ok(ast::BinaryOperator::BitwiseAnd),
+            Operator::BitwiseOr => Ok(ast::BinaryOperator::BitwiseOr),
+            Operator::BitwiseXor => Ok(ast::BinaryOperator::BitwiseXor),
+            Operator::BitwiseShiftRight => Ok(ast::BinaryOperator::PGBitwiseShiftRight),
+            Operator::BitwiseShiftLeft => Ok(ast::BinaryOperator::PGBitwiseShiftLeft),
+            Operator::StringConcat => Ok(ast::BinaryOperator::StringConcat),
+            Operator::AtArrow => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::ArrowAt => not_impl_err!("unsupported operation: {op:?}"),
+        }
+    }
+
+    fn scalar_to_sql(&self, v: &ScalarValue) -> Result<ast::Value> {
+        match v {
+            ScalarValue::Null => Ok(ast::Value::Null),
+            ScalarValue::Boolean(Some(b)) => Ok(ast::Value::Boolean(b.to_owned())),
+            ScalarValue::Boolean(None) => Ok(ast::Value::Null),
+            ScalarValue::Float32(Some(f)) => Ok(ast::Value::Number(f.to_string(), false)),
+            ScalarValue::Float32(None) => Ok(ast::Value::Null),
+            ScalarValue::Float64(Some(f)) => Ok(ast::Value::Number(f.to_string(), false)),
+            ScalarValue::Float64(None) => Ok(ast::Value::Null),
+            ScalarValue::Decimal128(Some(_), ..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Decimal128(None, ..) => Ok(ast::Value::Null),
+            ScalarValue::Decimal256(Some(_), ..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Decimal256(None, ..) => Ok(ast::Value::Null),
+            ScalarValue::Int8(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int8(None) => Ok(ast::Value::Null),
+            ScalarValue::Int16(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int16(None) => Ok(ast::Value::Null),
+            ScalarValue::Int32(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int32(None) => Ok(ast::Value::Null),
+            ScalarValue::Int64(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int64(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt8(Some(ui)) => Ok(ast::Value::Number(ui.to_string(), false)),
+            ScalarValue::UInt8(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt16(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt16(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt32(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt32(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt64(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt64(None) => Ok(ast::Value::Null),
+            ScalarValue::Utf8(Some(str)) => {
+                Ok(ast::Value::SingleQuotedString(str.to_string()))
+            }
+            ScalarValue::Utf8(None) => Ok(ast::Value::Null),
+            ScalarValue::LargeUtf8(Some(str)) => {
+                Ok(ast::Value::SingleQuotedString(str.to_string()))
+            }
+            ScalarValue::LargeUtf8(None) => Ok(ast::Value::Null),
+            ScalarValue::Binary(Some(_)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Binary(None) => Ok(ast::Value::Null),
+            ScalarValue::FixedSizeBinary(..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::LargeBinary(Some(_)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::LargeBinary(None) => Ok(ast::Value::Null),
+            ScalarValue::FixedSizeList(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::List(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::LargeList(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date32(Some(_d)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date32(None) => Ok(ast::Value::Null),
+            ScalarValue::Date64(Some(_d)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date64(None) => Ok(ast::Value::Null),
+            ScalarValue::Time32Second(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time32Second(None) => Ok(ast::Value::Null),
+            ScalarValue::Time32Millisecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time32Millisecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Time64Microsecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time64Microsecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Time64Nanosecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time64Nanosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::TimestampSecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampSecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampMillisecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampMillisecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampMicrosecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampMicrosecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampNanosecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampNanosecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::IntervalYearMonth(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalYearMonth(None) => Ok(ast::Value::Null),
+            ScalarValue::IntervalDayTime(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalDayTime(None) => Ok(ast::Value::Null),
+            ScalarValue::IntervalMonthDayNano(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalMonthDayNano(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationSecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationSecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationMillisecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationMillisecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationMicrosecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationMicrosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationNanosecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationNanosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Struct(_) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Dictionary(..) => not_impl_err!("Unsupported scalar: {v:?}"),
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use datafusion_common::TableReference;
+    use datafusion_expr::{col, lit};
+
+    use crate::unparser::dialect::CustomDialect;
+
+    use super::*;
+
+    #[test]
+    fn expr_to_sql_ok() -> Result<()> {
+        let tests: Vec<(Expr, &str)> = vec![
+            (col("a").gt(lit(4)), r#"a > 4"#),

Review Comment:
   I agree the round trip test would be good (and will make it quite mechanical to add coverage for the missing expressions)
   
   Something like
   
   ```rust
   round_trip("a"); // parse "a" to Expr and then turn back to string
   round_trip("a + b");
   ...
   ```
   
   I think we could do it as a follow on PR as well
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: convert Expr to SQL string [arrow-datafusion]

Posted by "devinjdangelo (via GitHub)" <gi...@apache.org>.
devinjdangelo commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1518822678


##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))

Review Comment:
   Based on playing around with round trips (AST->Logical Plan->AST) I believe that Logical Plans encode operation ordering in the tree itself while AST represents the literal ordering in the SQL string. The easiest way to guarantee mathematical equivalence would be to always add SQLExpr::Nested. A heuristic could work but would be quite a bit more complex I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1520393036


##########
datafusion/sql/src/unparser/dialect.rs:
##########
@@ -0,0 +1,71 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+/// Dialect is used to capture dialect specific syntax.
+/// Note: this trait will be moved to the SQLParser package.

Review Comment:
   ```suggestion
   /// Note: this trait will eventually be replaced by the Dialect in the SQLparser package
   /// 
   /// See <https://github.com/sqlparser-rs/sqlparser-rs/pull/1170>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#issuecomment-1989390923

   I added the follow on tasks to https://github.com/apache/arrow-datafusion/issues/9495. Thanks @backkem 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: convert Expr to SQL string [arrow-datafusion]

Posted by "devinjdangelo (via GitHub)" <gi...@apache.org>.
devinjdangelo commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1518623707


##########
datafusion/sql/src/unparser/dialect.rs:
##########
@@ -0,0 +1,52 @@
+pub trait Dialect {

Review Comment:
   It is too bad we can't simply use the Dialect trait already in sqlparser... but it does appear that trait can't tell you the quote style directly.



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))

Review Comment:
   I have found that we lose order of operations information here, but that wrapping the output in SQLExpr::Nested fixes it.
   
   E.g.
   ```sql
   x+1 / 2
   ```
   is not the same as
   ```sql
   (x+1)/2
   ```
   
   I can open a follow up PR with some tests to show this.



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))
+            }
+            Expr::Case(Case {
+                expr,
+                when_then_expr: _,
+                else_expr: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Cast(Cast { expr, data_type: _ }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Literal(value) => Ok(ast::Expr::Value(self.scalar_to_sql(value)?)),
+            Expr::Alias(Alias { expr, name: _, .. }) => self.expr_to_sql(expr),
+            Expr::WindowFunction(WindowFunction {
+                fun: _,
+                args: _,
+                partition_by: _,
+                order_by: _,
+                window_frame: _,
+                null_treatment: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Like(Like {
+                negated: _,
+                expr,
+                pattern: _,
+                escape_char: _,
+                case_insensitive: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            _ => not_impl_err!("Unsupported expression: {expr:?}"),
+        }
+    }
+
+    fn col_to_sql(&self, col: &Column) -> Result<ast::Expr> {
+        if let Some(table_ref) = &col.relation {
+            let mut id = table_ref.to_vec();
+            id.push(col.name.to_string());
+            return Ok(ast::Expr::CompoundIdentifier(
+                id.iter().map(|i| self.new_ident(i.to_string())).collect(),
+            ));
+        }
+        Ok(ast::Expr::Identifier(self.new_ident(col.name.to_string())))
+    }
+
+    fn new_ident(&self, str: String) -> ast::Ident {
+        ast::Ident {
+            value: str,
+            quote_style: self.dialect.identifier_quote_style(),
+        }
+    }
+
+    fn binary_op_to_sql(
+        &self,
+        lhs: ast::Expr,
+        rhs: ast::Expr,
+        op: ast::BinaryOperator,
+    ) -> ast::Expr {
+        ast::Expr::BinaryOp {
+            left: Box::new(lhs),
+            op,
+            right: Box::new(rhs),
+        }
+    }
+
+    fn op_to_sql(&self, op: &Operator) -> Result<ast::BinaryOperator> {
+        match op {
+            Operator::Eq => Ok(ast::BinaryOperator::Eq),
+            Operator::NotEq => Ok(ast::BinaryOperator::NotEq),
+            Operator::Lt => Ok(ast::BinaryOperator::Lt),
+            Operator::LtEq => Ok(ast::BinaryOperator::LtEq),
+            Operator::Gt => Ok(ast::BinaryOperator::Gt),
+            Operator::GtEq => Ok(ast::BinaryOperator::GtEq),
+            Operator::Plus => Ok(ast::BinaryOperator::Plus),
+            Operator::Minus => Ok(ast::BinaryOperator::Minus),
+            Operator::Multiply => Ok(ast::BinaryOperator::Multiply),
+            Operator::Divide => Ok(ast::BinaryOperator::Divide),
+            Operator::Modulo => Ok(ast::BinaryOperator::Modulo),
+            Operator::And => Ok(ast::BinaryOperator::And),
+            Operator::Or => Ok(ast::BinaryOperator::Or),
+            Operator::IsDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::IsNotDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::RegexMatch => Ok(ast::BinaryOperator::PGRegexMatch),
+            Operator::RegexIMatch => Ok(ast::BinaryOperator::PGRegexIMatch),
+            Operator::RegexNotMatch => Ok(ast::BinaryOperator::PGRegexNotMatch),
+            Operator::RegexNotIMatch => Ok(ast::BinaryOperator::PGRegexNotIMatch),
+            Operator::ILikeMatch => Ok(ast::BinaryOperator::PGILikeMatch),
+            Operator::NotLikeMatch => Ok(ast::BinaryOperator::PGNotLikeMatch),
+            Operator::LikeMatch => Ok(ast::BinaryOperator::PGLikeMatch),
+            Operator::NotILikeMatch => Ok(ast::BinaryOperator::PGNotILikeMatch),
+            Operator::BitwiseAnd => Ok(ast::BinaryOperator::BitwiseAnd),
+            Operator::BitwiseOr => Ok(ast::BinaryOperator::BitwiseOr),
+            Operator::BitwiseXor => Ok(ast::BinaryOperator::BitwiseXor),
+            Operator::BitwiseShiftRight => Ok(ast::BinaryOperator::PGBitwiseShiftRight),
+            Operator::BitwiseShiftLeft => Ok(ast::BinaryOperator::PGBitwiseShiftLeft),
+            Operator::StringConcat => Ok(ast::BinaryOperator::StringConcat),
+            Operator::AtArrow => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::ArrowAt => not_impl_err!("unsupported operation: {op:?}"),
+        }
+    }
+
+    fn scalar_to_sql(&self, v: &ScalarValue) -> Result<ast::Value> {

Review Comment:
   I believe that some DataFusion Scalars will require mapping to a SQLExpr. For example for date scalars, I think the only way is to convert to something like `CAST(quoted_string_val to DATE)`. 
   
   This is a private function though, so does not need to be resolved prior to merging imo.



##########
datafusion/sql/src/unparser/expr.rs:
##########
@@ -0,0 +1,337 @@
+use datafusion_common::{not_impl_err, Column, Result, ScalarValue};
+use datafusion_expr::{
+    expr::{Alias, InList, ScalarFunction, WindowFunction},
+    Between, BinaryExpr, Case, Cast, Expr, Like, Operator,
+};
+use sqlparser::ast;
+
+use super::Unparser;
+
+/// Convert a DataFusion [`Expr`] to `sqlparser::ast::Expr`
+///
+/// This function is the opposite of `SqlToRel::sql_to_expr`
+///
+/// Example
+/// ```
+/// use datafusion_expr::{col, lit};
+/// use datafusion_sql::unparser::expr_to_sql;
+/// let expr = col("a").gt(lit(4));
+/// let sql = expr_to_sql(&expr).unwrap();
+///
+/// assert_eq!(format!("{}", sql), "a > 4")
+/// ```
+pub fn expr_to_sql(expr: &Expr) -> Result<ast::Expr> {
+    let unparser = Unparser::default();
+    unparser.expr_to_sql(expr)
+}
+
+impl Unparser<'_> {
+    pub fn expr_to_sql(&self, expr: &Expr) -> Result<ast::Expr> {
+        match expr {
+            Expr::InList(InList {
+                expr,
+                list: _,
+                negated: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::ScalarFunction(ScalarFunction { .. }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Between(Between {
+                expr,
+                negated: _,
+                low: _,
+                high: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Column(col) => self.col_to_sql(col),
+            Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
+                let l = self.expr_to_sql(left.as_ref())?;
+                let r = self.expr_to_sql(right.as_ref())?;
+                let op = self.op_to_sql(op)?;
+
+                Ok(self.binary_op_to_sql(l, r, op))
+            }
+            Expr::Case(Case {
+                expr,
+                when_then_expr: _,
+                else_expr: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Cast(Cast { expr, data_type: _ }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Literal(value) => Ok(ast::Expr::Value(self.scalar_to_sql(value)?)),
+            Expr::Alias(Alias { expr, name: _, .. }) => self.expr_to_sql(expr),
+            Expr::WindowFunction(WindowFunction {
+                fun: _,
+                args: _,
+                partition_by: _,
+                order_by: _,
+                window_frame: _,
+                null_treatment: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            Expr::Like(Like {
+                negated: _,
+                expr,
+                pattern: _,
+                escape_char: _,
+                case_insensitive: _,
+            }) => {
+                not_impl_err!("Unsupported expression: {expr:?}")
+            }
+            _ => not_impl_err!("Unsupported expression: {expr:?}"),
+        }
+    }
+
+    fn col_to_sql(&self, col: &Column) -> Result<ast::Expr> {
+        if let Some(table_ref) = &col.relation {
+            let mut id = table_ref.to_vec();
+            id.push(col.name.to_string());
+            return Ok(ast::Expr::CompoundIdentifier(
+                id.iter().map(|i| self.new_ident(i.to_string())).collect(),
+            ));
+        }
+        Ok(ast::Expr::Identifier(self.new_ident(col.name.to_string())))
+    }
+
+    fn new_ident(&self, str: String) -> ast::Ident {
+        ast::Ident {
+            value: str,
+            quote_style: self.dialect.identifier_quote_style(),
+        }
+    }
+
+    fn binary_op_to_sql(
+        &self,
+        lhs: ast::Expr,
+        rhs: ast::Expr,
+        op: ast::BinaryOperator,
+    ) -> ast::Expr {
+        ast::Expr::BinaryOp {
+            left: Box::new(lhs),
+            op,
+            right: Box::new(rhs),
+        }
+    }
+
+    fn op_to_sql(&self, op: &Operator) -> Result<ast::BinaryOperator> {
+        match op {
+            Operator::Eq => Ok(ast::BinaryOperator::Eq),
+            Operator::NotEq => Ok(ast::BinaryOperator::NotEq),
+            Operator::Lt => Ok(ast::BinaryOperator::Lt),
+            Operator::LtEq => Ok(ast::BinaryOperator::LtEq),
+            Operator::Gt => Ok(ast::BinaryOperator::Gt),
+            Operator::GtEq => Ok(ast::BinaryOperator::GtEq),
+            Operator::Plus => Ok(ast::BinaryOperator::Plus),
+            Operator::Minus => Ok(ast::BinaryOperator::Minus),
+            Operator::Multiply => Ok(ast::BinaryOperator::Multiply),
+            Operator::Divide => Ok(ast::BinaryOperator::Divide),
+            Operator::Modulo => Ok(ast::BinaryOperator::Modulo),
+            Operator::And => Ok(ast::BinaryOperator::And),
+            Operator::Or => Ok(ast::BinaryOperator::Or),
+            Operator::IsDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::IsNotDistinctFrom => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::RegexMatch => Ok(ast::BinaryOperator::PGRegexMatch),
+            Operator::RegexIMatch => Ok(ast::BinaryOperator::PGRegexIMatch),
+            Operator::RegexNotMatch => Ok(ast::BinaryOperator::PGRegexNotMatch),
+            Operator::RegexNotIMatch => Ok(ast::BinaryOperator::PGRegexNotIMatch),
+            Operator::ILikeMatch => Ok(ast::BinaryOperator::PGILikeMatch),
+            Operator::NotLikeMatch => Ok(ast::BinaryOperator::PGNotLikeMatch),
+            Operator::LikeMatch => Ok(ast::BinaryOperator::PGLikeMatch),
+            Operator::NotILikeMatch => Ok(ast::BinaryOperator::PGNotILikeMatch),
+            Operator::BitwiseAnd => Ok(ast::BinaryOperator::BitwiseAnd),
+            Operator::BitwiseOr => Ok(ast::BinaryOperator::BitwiseOr),
+            Operator::BitwiseXor => Ok(ast::BinaryOperator::BitwiseXor),
+            Operator::BitwiseShiftRight => Ok(ast::BinaryOperator::PGBitwiseShiftRight),
+            Operator::BitwiseShiftLeft => Ok(ast::BinaryOperator::PGBitwiseShiftLeft),
+            Operator::StringConcat => Ok(ast::BinaryOperator::StringConcat),
+            Operator::AtArrow => not_impl_err!("unsupported operation: {op:?}"),
+            Operator::ArrowAt => not_impl_err!("unsupported operation: {op:?}"),
+        }
+    }
+
+    fn scalar_to_sql(&self, v: &ScalarValue) -> Result<ast::Value> {
+        match v {
+            ScalarValue::Null => Ok(ast::Value::Null),
+            ScalarValue::Boolean(Some(b)) => Ok(ast::Value::Boolean(b.to_owned())),
+            ScalarValue::Boolean(None) => Ok(ast::Value::Null),
+            ScalarValue::Float32(Some(f)) => Ok(ast::Value::Number(f.to_string(), false)),
+            ScalarValue::Float32(None) => Ok(ast::Value::Null),
+            ScalarValue::Float64(Some(f)) => Ok(ast::Value::Number(f.to_string(), false)),
+            ScalarValue::Float64(None) => Ok(ast::Value::Null),
+            ScalarValue::Decimal128(Some(_), ..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Decimal128(None, ..) => Ok(ast::Value::Null),
+            ScalarValue::Decimal256(Some(_), ..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Decimal256(None, ..) => Ok(ast::Value::Null),
+            ScalarValue::Int8(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int8(None) => Ok(ast::Value::Null),
+            ScalarValue::Int16(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int16(None) => Ok(ast::Value::Null),
+            ScalarValue::Int32(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int32(None) => Ok(ast::Value::Null),
+            ScalarValue::Int64(Some(i)) => Ok(ast::Value::Number(i.to_string(), false)),
+            ScalarValue::Int64(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt8(Some(ui)) => Ok(ast::Value::Number(ui.to_string(), false)),
+            ScalarValue::UInt8(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt16(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt16(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt32(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt32(None) => Ok(ast::Value::Null),
+            ScalarValue::UInt64(Some(ui)) => {
+                Ok(ast::Value::Number(ui.to_string(), false))
+            }
+            ScalarValue::UInt64(None) => Ok(ast::Value::Null),
+            ScalarValue::Utf8(Some(str)) => {
+                Ok(ast::Value::SingleQuotedString(str.to_string()))
+            }
+            ScalarValue::Utf8(None) => Ok(ast::Value::Null),
+            ScalarValue::LargeUtf8(Some(str)) => {
+                Ok(ast::Value::SingleQuotedString(str.to_string()))
+            }
+            ScalarValue::LargeUtf8(None) => Ok(ast::Value::Null),
+            ScalarValue::Binary(Some(_)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Binary(None) => Ok(ast::Value::Null),
+            ScalarValue::FixedSizeBinary(..) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::LargeBinary(Some(_)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::LargeBinary(None) => Ok(ast::Value::Null),
+            ScalarValue::FixedSizeList(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::List(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::LargeList(_a) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date32(Some(_d)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date32(None) => Ok(ast::Value::Null),
+            ScalarValue::Date64(Some(_d)) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Date64(None) => Ok(ast::Value::Null),
+            ScalarValue::Time32Second(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time32Second(None) => Ok(ast::Value::Null),
+            ScalarValue::Time32Millisecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time32Millisecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Time64Microsecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time64Microsecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Time64Nanosecond(Some(_t)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::Time64Nanosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::TimestampSecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampSecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampMillisecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampMillisecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampMicrosecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampMicrosecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::TimestampNanosecond(Some(_ts), _) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::TimestampNanosecond(None, _) => Ok(ast::Value::Null),
+            ScalarValue::IntervalYearMonth(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalYearMonth(None) => Ok(ast::Value::Null),
+            ScalarValue::IntervalDayTime(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalDayTime(None) => Ok(ast::Value::Null),
+            ScalarValue::IntervalMonthDayNano(Some(_i)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::IntervalMonthDayNano(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationSecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationSecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationMillisecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationMillisecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationMicrosecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationMicrosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::DurationNanosecond(Some(_d)) => {
+                not_impl_err!("Unsupported scalar: {v:?}")
+            }
+            ScalarValue::DurationNanosecond(None) => Ok(ast::Value::Null),
+            ScalarValue::Struct(_) => not_impl_err!("Unsupported scalar: {v:?}"),
+            ScalarValue::Dictionary(..) => not_impl_err!("Unsupported scalar: {v:?}"),
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use datafusion_common::TableReference;
+    use datafusion_expr::{col, lit};
+
+    use crate::unparser::dialect::CustomDialect;
+
+    use super::*;
+
+    #[test]
+    fn expr_to_sql_ok() -> Result<()> {
+        let tests: Vec<(Expr, &str)> = vec![
+            (col("a").gt(lit(4)), r#"a > 4"#),

Review Comment:
   I think these tests look good to start. It might also be a good idea to have some tests that do a full round trip using SqltoRel, e.g. String -> AST -> LogicalPlan -> AST -> String and making sure the Strings and AST match.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] feat: Introduce convert Expr to SQL string API and basic feature [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #9517:
URL: https://github.com/apache/arrow-datafusion/pull/9517#discussion_r1518828958


##########
datafusion/sql/src/unparser/dialect.rs:
##########
@@ -0,0 +1,52 @@
+pub trait Dialect {

Review Comment:
   Shall we make a change upstream to sqlparser? I would be happy to review such a PR (or maybe just make it?) We could leave a comment in the PR to remove this trait when the new sqlparser cate was released



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org