r4ntix commented on issue #5:
URL: https://github.com/apache/arrow-ballista/issues/5#issuecomment-1281906098

   @andygrove @comphead 
   I tried to analyze the problem and found that `SELECT count(distinct c1) as 
cnt_distinct FROM aggregate_test_100` is also expected in distributed mode. 
   
   Because `single distinct` is optimized by the optimizer as `group by` in 
datafusion:
   ```
   example sql: select count(distinct c_name) from customer_1;
   
   // Logic plan before optimization: 
   Projection: COUNT(DISTINCT customer_1.c_name)                                
                                                                                
           
      Aggregate: groupBy=[[]], aggr=[[COUNT(DISTINCT customer_1.c_name)]]       
                                                                                
            
        TableScan: customer_1 projection=[c_name]
   
   // Logic plan after optimization:
   Projection: COUNT(DISTINCT customer_1.c_name)                                
                                                                                
                     
      Projection: COUNT(alias1) AS COUNT(DISTINCT customer_1.c_name)            
                                                                                
                      
        Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]]                         
                                                                                
                      
          Aggregate: groupBy=[[customer_1.c_name AS alias1]], aggr=[[]]         
                                                                                
                      
            TableScan: customer_1 projection=[c_name]
   ```
   
   The current problem with ballista is that it does not support 
`DistinctCount` in non-single distinct scenarios.
   Example sql `select count(distinct c_name), max(c_name) from customer_1`:
   ```
   [2022-10-18T06:32:14Z ERROR 
ballista_core::execution_plans::distributed_query] Job 3N8dtpp failed: Error 
planning job 3N8dtpp: NotImplemented("Aggregate function not supported: 
DistinctCount { name: \"COUNT(DISTINCT customer_1.c_name)\", data_type: Int64, 
state_data_types: [Utf8], exprs: [Column { name: \"c_name\", index: 0 }] }")
   DataFusionError(ArrowError(ExternalError(Execution("Job 3N8dtpp failed: 
Error planning job 3N8dtpp: NotImplemented(\"Aggregate function not supported: 
DistinctCount { name: \\\"COUNT(DISTINCT customer_1.c_name)\\\", data_type: 
Int64, state_data_types: [Utf8], exprs: [Column { name: \\\"c_name\\\", index: 
0 }] }\")"))))
   ```
   
   I will sumit a PR for 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to