vogievetsky opened a new issue #8958: Calcite is too aggressive in simplifying 
a having filter on multi-value dimensions
URL: https://github.com/apache/incubator-druid/issues/8958
 
 
   ### Affected Version
   
   Druid 0.16.0 (and prob earlier)
   
   ### Description
   
   With this dataset (as `fs`):
   
   ```json
   
{"time":"2019-08-14T00:00:00.000Z","srcGroups":["x","y","z"],"dstGroups":["a","b","c","d"]}
   
{"time":"2019-08-14T00:00:00.000Z","srcGroups":["x","y","z"],"dstGroups":["a","c","d"]}
   
{"time":"2019-08-14T00:00:00.000Z","srcGroups":["x","y","z"],"dstGroups":["a","g"]}
   ```
   
   Doing this query:
   
   ```sql
   SELECT
     "dstGroups",
     COUNT(*) AS "Count"
   FROM "fs"
   GROUP BY 1
   HAVING "dstGroups" = 'a'
   ORDER BY "Count" DESC
   ```
   
   Yields:
   
   
![image](https://user-images.githubusercontent.com/177816/69725823-f0a96a80-10d3-11ea-97e3-f8dd1ab9e0e2.png)
   
   Which is not what is expected.
   
   The plan is:
   
   ```json
   {
     "queryType": "topN",
     "dataSource": {
       "type": "table",
       "name": "fs"
     },
     "virtualColumns": [],
     "dimension": {
       "type": "default",
       "dimension": "dstGroups",
       "outputName": "d0",
       "outputType": "STRING"
     },
     "metric": {
       "type": "numeric",
       "metric": "a0"
     },
     "threshold": 100,
     "intervals": {
       "type": "intervals",
       "intervals": [
         "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
       ]
     },
     "filter": {
       "type": "selector",
       "dimension": "dstGroups",
       "value": "a",
       "extractionFn": null
     },
     "granularity": {
       "type": "all"
     },
     "aggregations": [
       {
         "type": "count",
         "name": "a0"
       }
     ],
     "postAggregations": [],
     "descending": false
   }
   ```
   
   This is because Calcite converted the HAVING on a grouping key into a WHERE, 
which given Druid multi-value semantics is not correct to do.
   
   A workaround is to prevent Calcite from simplifying the HAVING filter by 
complicating the HAVING into something that can not be simplified as easily.
   
   ```sql
   SELECT
     "dstGroups",
     COUNT(*) AS "Count"
   FROM "fs"
   GROUP BY 1
   HAVING "dstGroups" = 'a' OR "Count" < 0
   ORDER BY "Count" DESC
   ```
   
   Which yields:
   
   
![image](https://user-images.githubusercontent.com/177816/69726160-c60be180-10d4-11ea-93d7-373e26da88c0.png)
   
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to