You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "parkma99 (via GitHub)" <gi...@apache.org> on 2023/06/17 09:41:34 UTC

[GitHub] [arrow-datafusion] parkma99 opened a new pull request, #6708: Minor: `derive(Debug)` for `Expr`

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

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #6677 
   
   # Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   # Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->


-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1040,7 +1040,7 @@ mod test {
         let empty = empty_with_type(DataType::Int64);
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
         let expected =
-            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
              \n  EmptyRelation";

Review Comment:
   I think this test case should not change, but current I failed to past this test case.
   When I change the code [0ef94](https://github.com/apache/arrow-datafusion/blob/0ef9478e6675308529d5420e94d70b0f3a32dc98/datafusion/expr/src/expr.rs#L1110) to use `Display` I got failed by `Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)])` missing `Alias`.
   
   I  current could not find a good way to solve it.



-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1040,7 +1040,7 @@ mod test {
         let empty = empty_with_type(DataType::Int64);
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
         let expected =
-            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
              \n  EmptyRelation";

Review Comment:
   ` AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\` is missing



-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on pull request #6708: Minor: `derive(Debug)` for `Expr`

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

   Hello @alamb , I would like to pass all tests in my local machine, but got some error.
   My computer is `Mac mini m2` with `16+256`.
   When I run command `cargo test` in Terminal, it got error with message : 
   ```failures:
   
   ---- fuzz_cases::order_spill_fuzz::test_sort_1k_mem stdout ----
   thread 'fuzz_cases::order_spill_fuzz::test_sort_1k_mem' panicked at 'called `Result::unwrap()` on an `Err` value: Execution("Failed to create partition file at \"/var/folders/c7/nfvz06pj3t5dxq4ycycq8czm0000gn/T/.tmpmetJoJ/.tmpAliZCF\": Os { code: 24, kind: Uncategorized, message: \"Too many open files\" }")', datafusion/core/tests/fuzz_cases/order_spill_fuzz.rs:85:63
   note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace
   
   
   failures:
       fuzz_cases::order_spill_fuzz::test_sort_1k_mem
   
   test result: FAILED. 14 passed; 1 failed; 0 ignored; 0 measured; 0 filtered out; finished in 14.88s
   
   error: test failed, to rerun pass `-p datafusion --test fuzz`
   ```
   When I run command `cargo test` in `VScode Terminal`, it got error with message :
   ```
   thread '<unknown>' has overflowed its stack
   fatal runtime error: stack overflow
   error: test failed, to rerun pass `-p datafusion-proto --lib`
   
   Caused by:
     process didn't exit successfully: `/Users/m/code/arrow-datafusion/target/debug/deps/datafusion_proto-34d869f2f58ec4e8` (signal: 6, SIGABRT: process abort signal)
   ```.
   


-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1040,7 +1040,7 @@ mod test {
         let empty = empty_with_type(DataType::Int64);
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
         let expected =
-            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
              \n  EmptyRelation";

Review Comment:
   I think this test case should not change, but current I failed to past this test case.
   When I change the code [0ef94](https://github.com/apache/arrow-datafusion/blob/0ef9478e6675308529d5420e94d70b0f3a32dc98/datafusion/expr/src/expr.rs#L1110) to use `Display` I got failed by `Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)])` missing `Alias`



-- 
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


[GitHub] [arrow-datafusion] alamb merged pull request #6708: `derive(Debug)` for `Expr`

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


-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6708: `derive(Debug)` for `Expr`

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


##########
datafusion/expr/src/expr.rs:
##########
@@ -869,53 +869,56 @@ impl Expr {
     }
 }
 
-/// Format expressions for display as part of a logical plan. In many cases, this will produce
-/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Display for Expr {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        write!(f, "{self:?}")
-    }
+#[macro_export]

Review Comment:
   I tried https://github.com/apache/arrow-datafusion/pull/6739 -- it isn't great -- any thoughts appreciated



-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/expr/src/expr.rs:
##########
@@ -869,53 +869,56 @@ impl Expr {
     }
 }
 
-/// Format expressions for display as part of a logical plan. In many cases, this will produce
-/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Display for Expr {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        write!(f, "{self:?}")
-    }
+#[macro_export]

Review Comment:
   I can give it a try maybe after this PR is merged



##########
datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part:
##########
@@ -79,21 +79,21 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST]
 ----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(UInt8(1))@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal]
 ------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)]
 --------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4
 ------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)]
 --------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal]
-----------------NestedLoopJoinExec: join_type=Inner, filter=BinaryExpr { left: CastExpr { expr: Column { name: "c_acctbal", index: 0 }, cast_type: Decimal128(19, 6), cast_options: CastOptions { safe: false, format_options: FormatOptions { safe: true, null: "", date_format: None, datetime_format: None, timestamp_format: None, timestamp_tz_format: None, time_format: None } } }, op: Gt, right: Column { name: "AVG(customer.c_acctbal)", index: 1 } }
+----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1

Review Comment:
   this is really much nicer 👍 



-- 
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


[GitHub] [arrow-datafusion] crepererum commented on pull request #6708: Minor: `derive(Debug)` for `Expr`

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

   So I had a quick look at the outputs: I think the changes to the sqllogictests (e.g. `on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]` => `on=[(t1_id@0, t2_id@0)]`) are desired. The new output is way easier to read and matches the formatting that we use for other plan formatting. Please update the tests accordingly.


-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1040,7 +1040,7 @@ mod test {
         let empty = empty_with_type(DataType::Int64);
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
         let expected =
-            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
              \n  EmptyRelation";

Review Comment:
   ` AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\` is missing,I debug find it's a alias.



-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/expr/src/expr.rs:
##########
@@ -869,53 +869,56 @@ impl Expr {
     }
 }
 
-/// Format expressions for display as part of a logical plan. In many cases, this will produce
-/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Display for Expr {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        write!(f, "{self:?}")
-    }
+#[macro_export]

Review Comment:
   I think making a function and moving to `datafusin_util` is a good option. 



-- 
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


[GitHub] [arrow-datafusion] parkma99 commented on pull request #6708: Minor: `derive(Debug)` for `Expr`

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

   > So I had a quick look at the outputs: I think the changes to the sqllogictests (e.g. `on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]` => `on=[(t1_id@0, t2_id@0)]`) are desired. The new output is way easier to read and matches the formatting that we use for other plan formatting. Please update the tests accordingly.
   
   Ok
   
   > [SQL] explain select * from hashjoin_datatype_table_t1 t1 right join hashjoin_datatype_table_t2 t2 on t1.c3 = t2.c3
   [Diff] (-expected|+actual)
       logical_plan
       Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3
       --SubqueryAlias: t1
       ----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4]
       --SubqueryAlias: t2
       ----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4]
       physical_plan
       ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4]
   -   --SortMergeJoin: join_type=Right, on=[(Column { name: "CAST(t1.c3 AS Decimal128(10, 2))", index: 4 }, Column { name: "c3", index: 2 })]
   -   ----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]
   +   --SortMergeJoin: join_type=Right, on=[(Cast(Cast { expr: Column(Column { relation: Some(Bare { table: "t1" }), name: "c3" }), data_type: Decimal128(10, 2) })@4, c3@2)]
   +   ----SortExec: expr=[Cast(Cast { expr: Column(Column { relation: Some(Bare { table: "t1" }), name: "c3" }), data_type: Decimal128(10, 2) })@4 ASC]
       ------CoalesceBatchesExec: target_batch_size=4096
   -   --------RepartitionExec: partitioning=Hash([Column { name: "CAST(t1.c3 AS Decimal128(10, 2))", index: 4 }], 2), input_partitions=2
   -   ----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))]
   +   --------RepartitionExec: partitioning=Hash([Cast(Cast { expr: Column(Column { relation: Some(Bare { table: "t1" }), name: "c3" }), data_type: Decimal128(10, 2) })@4], 2), input_partitions=2
   +   ----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as Cast(Cast { expr: Column(Column { relation: Some(Bare { table: "t1" }), name: "c3" }), data_type: Decimal128(10, 2) })]
       ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
       --------------MemoryExec: partitions=1, partition_sizes=[1]
       ----SortExec: expr=[c3@2 ASC]
       ------CoalesceBatchesExec: target_batch_size=4096
       --------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2
       ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
       ------------MemoryExec: partitions=1, partition_sizes=[1]
   at tests/sqllogictests/test_files/joins.slt:2672
   
   Case 2 I find the reason. I will update right now.


-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/core/src/physical_optimizer/dist_enforcement.rs:
##########
@@ -1252,32 +1253,32 @@ mod tests {
                         &join_type,
                     );
                     let top_join_plan =
-                        format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]");
+                        format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]");
 
                     let expected = match join_type {
                         // Should include 3 RepartitionExecs
                         JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![
                             top_join_plan.as_str(),
                             join_plan.as_str(),
-                            "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+                            "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",

Review Comment:
   this certainly looks nicer 👍 



-- 
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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/expr/src/expr.rs:
##########
@@ -869,53 +869,56 @@ impl Expr {
     }
 }
 
-/// Format expressions for display as part of a logical plan. In many cases, this will produce
-/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Display for Expr {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        write!(f, "{self:?}")
-    }
+#[macro_export]

Review Comment:
   Given how common this functionality is I wonder if we could move this macro (or maybe make a function as I don't see any benefit in making it a macro)  in `datafusion_util` and use it more around the codebase.
   
   Just a thought
   



##########
datafusion/expr/src/expr.rs:
##########
@@ -79,7 +79,7 @@ use std::sync::Arc;
 ///   assert_eq!(binary_expr.op, Operator::Eq);
 /// }
 /// ```
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]

Review Comment:
   ❤️ 



-- 
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


[GitHub] [arrow-datafusion] crepererum commented on a diff in pull request #6708: Minor: `derive(Debug)` for `Expr`

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


##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1040,7 +1040,7 @@ mod test {
         let empty = empty_with_type(DataType::Int64);
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
         let expected =
-            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+            "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
              \n  EmptyRelation";

Review Comment:
   Which alias?



-- 
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


[GitHub] [arrow-datafusion] alamb commented on pull request #6708: Minor: `derive(Debug)` for `Expr`

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

   > When I run command cargo test in Terminal, it got error with message :
   > `message: \"Too many open files\"`
   
   
   That looks like you either have to increase your ulimit to increase the number of open files, or perhaps you can [reduce the parallelism of the tests](https://doc.rust-lang.org/book/ch11-02-running-tests.html#running-tests-in-parallel-or-consecutively): 
   
   ```shell
   $ cargo test -- --test-threads=1
   ```


-- 
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