You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "jonmmease (via GitHub)" <gi...@apache.org> on 2023/05/25 19:20:24 UTC

[GitHub] [arrow-datafusion] jonmmease opened a new issue, #6447: Cannot have column named "COUNT(*)"

jonmmease opened a new issue, #6447:
URL: https://github.com/apache/arrow-datafusion/issues/6447

   ### Describe the bug
   
   After the changes in https://github.com/apache/arrow-datafusion/pull/6010, it's no longer possible to have a column named "COUNT(*)" (even with quotes).  A column with this name is renamed to "COUNT(UInt8(1))".
   
   ### To Reproduce
   
   Before #6010:
   
   ```
   ❯ with tbl as (SELECT 0 as "COUNT(*)") SELECT * from tbl;
   +----------+
   | COUNT(*) |
   +----------+
   | 0        |
   +----------+
   ```
   
   After #6010
   
   ```
   ❯ with tbl as (SELECT 0 as "COUNT(*)") SELECT * from tbl;
   +-----------------+
   | COUNT(UInt8(1)) |
   +-----------------+
   | 0               |
   +-----------------+
   ```
   
   ### Expected behavior
   
   I expect it to be possible to have a column named "COUNT(*)" without it being automatically renamed.
   
   ### Additional context
   
   _No response_


-- 
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.apache.org

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


[GitHub] [arrow-datafusion] jiangzhx commented on issue #6447: Cannot have column named "COUNT(*)"

Posted by "jiangzhx (via GitHub)" <gi...@apache.org>.
jiangzhx commented on issue #6447:
URL: https://github.com/apache/arrow-datafusion/issues/6447#issuecomment-1569426106

   thank you @jonmmease  i will find way to fix this .


-- 
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] jackwener commented on issue #6447: Cannot have column named "COUNT(*)"

Posted by "jackwener (via GitHub)" <gi...@apache.org>.
jackwener commented on issue #6447:
URL: https://github.com/apache/arrow-datafusion/issues/6447#issuecomment-1567788096

   I will resolve it, thank you @jonmmease 


-- 
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 closed issue #6447: Cannot have column named "COUNT(*)"

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb closed issue #6447: Cannot have column named "COUNT(*)"
URL: https://github.com/apache/arrow-datafusion/issues/6447


-- 
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] jiangzhx commented on issue #6447: Cannot have column named "COUNT(*)"

Posted by "jiangzhx (via GitHub)" <gi...@apache.org>.
jiangzhx commented on issue #6447:
URL: https://github.com/apache/arrow-datafusion/issues/6447#issuecomment-1569762717

   I did some testing and found that 
   the `count_wildcard_rule` did not replace the `alias` from `count(*)` to `COUNT(UInt8(1))`. 
   It looks like `push_down_projection` performed this replacement. 
   I will investigate why this happened.
   
   ```
   +------------------------------------------------------------+---------------------------------------------+
   | plan_type                                                  | plan                                        |
   +------------------------------------------------------------+---------------------------------------------+
   | initial_logical_plan                                       | Projection: tbl.COUNT(*)                    |
   |                                                            |   SubqueryAlias: tbl                        |
   |                                                            |     Projection: Int64(0) AS COUNT(*)        |
   |                                                            |       EmptyRelation                         |
   | logical_plan after inline_table_scan                       | SAME TEXT AS ABOVE                          |
   | logical_plan after type_coercion                           | SAME TEXT AS ABOVE                          |
   | logical_plan after count_wildcard_rule                     | Projection: tbl.COUNT(UInt8(1))             |
   |                                                            |   SubqueryAlias: tbl                        |
   |                                                            |     Projection: Int64(0) AS COUNT(*)        |
   |                                                            |       EmptyRelation                         |
   | analyzed_logical_plan                                      | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after unwrap_cast_in_comparison               | SAME TEXT AS ABOVE                          |
   | logical_plan after replace_distinct_aggregate              | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_join                          | SAME TEXT AS ABOVE                          |
   | logical_plan after decorrelate_predicate_subquery          | SAME TEXT AS ABOVE                          |
   | logical_plan after scalar_subquery_to_join                 | SAME TEXT AS ABOVE                          |
   | logical_plan after extract_equijoin_predicate              | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after merge_projection                        | SAME TEXT AS ABOVE                          |
   | logical_plan after rewrite_disjunctive_predicate           | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_duplicated_expr               | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_filter                        | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_cross_join                    | SAME TEXT AS ABOVE                          |
   | logical_plan after common_sub_expression_eliminate         | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan after propagate_empty_relation                | SAME TEXT AS ABOVE                          |
   | logical_plan after filter_null_join_keys                   | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_outer_join                    | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_filter                        | SAME TEXT AS ABOVE                          |
   | logical_plan after single_distinct_aggregation_to_group_by | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after unwrap_cast_in_comparison               | SAME TEXT AS ABOVE                          |
   | logical_plan after common_sub_expression_eliminate         | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_projection                    | Projection: tbl.COUNT(UInt8(1))             |
   |                                                            |   SubqueryAlias: tbl                        |
   |                                                            |     Projection: Int64(0) AS COUNT(UInt8(1)) |
   |                                                            |       EmptyRelation                         |
   | logical_plan after eliminate_projection                    | SubqueryAlias: tbl                          |
   |                                                            |   Projection: Int64(0) AS COUNT(UInt8(1))   |
   |                                                            |     EmptyRelation                           |
   | logical_plan after push_down_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after unwrap_cast_in_comparison               | SAME TEXT AS ABOVE                          |
   | logical_plan after replace_distinct_aggregate              | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_join                          | SAME TEXT AS ABOVE                          |
   | logical_plan after decorrelate_predicate_subquery          | SAME TEXT AS ABOVE                          |
   | logical_plan after scalar_subquery_to_join                 | SAME TEXT AS ABOVE                          |
   | logical_plan after extract_equijoin_predicate              | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after merge_projection                        | SAME TEXT AS ABOVE                          |
   | logical_plan after rewrite_disjunctive_predicate           | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_duplicated_expr               | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_filter                        | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_cross_join                    | SAME TEXT AS ABOVE                          |
   | logical_plan after common_sub_expression_eliminate         | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan after propagate_empty_relation                | SAME TEXT AS ABOVE                          |
   | logical_plan after filter_null_join_keys                   | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_outer_join                    | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_filter                        | SAME TEXT AS ABOVE                          |
   | logical_plan after single_distinct_aggregation_to_group_by | SAME TEXT AS ABOVE                          |
   | logical_plan after simplify_expressions                    | SAME TEXT AS ABOVE                          |
   | logical_plan after unwrap_cast_in_comparison               | SAME TEXT AS ABOVE                          |
   | logical_plan after common_sub_expression_eliminate         | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_projection                    | SAME TEXT AS ABOVE                          |
   | logical_plan after eliminate_projection                    | SAME TEXT AS ABOVE                          |
   | logical_plan after push_down_limit                         | SAME TEXT AS ABOVE                          |
   | logical_plan                                               | SubqueryAlias: tbl                          |
   |                                                            |   Projection: Int64(0) AS COUNT(UInt8(1))   |
   |                                                            |     EmptyRelation                           |
   | initial_physical_plan                                      | ProjectionExec: expr=[0 as COUNT(UInt8(1))] |
   |                                                            |   EmptyExec: produce_one_row=true           |
   |                                                            |                                             |
   | physical_plan after aggregate_statistics                   | SAME TEXT AS ABOVE                          |
   | physical_plan after join_selection                         | SAME TEXT AS ABOVE                          |
   | physical_plan after PipelineFixer                          | SAME TEXT AS ABOVE                          |
   | physical_plan after repartition                            | SAME TEXT AS ABOVE                          |
   | physical_plan after global_sort_selection                  | SAME TEXT AS ABOVE                          |
   | physical_plan after EnforceDistribution                    | SAME TEXT AS ABOVE                          |
   | physical_plan after EnforceSorting                         | SAME TEXT AS ABOVE                          |
   | physical_plan after CombinePartialFinalAggregate           | SAME TEXT AS ABOVE                          |
   | physical_plan after coalesce_batches                       | SAME TEXT AS ABOVE                          |
   | physical_plan after PipelineChecker                        | SAME TEXT AS ABOVE                          |
   | physical_plan                                              | ProjectionExec: expr=[0 as COUNT(UInt8(1))] |
   |                                                            |   EmptyExec: produce_one_row=true           |
   |                                                            |                                             |
   +------------------------------------------------------------+---------------------------------------------+
   
   ```


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