You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/08/06 15:14:20 UTC

[GitHub] [incubator-pinot] lgo opened a new issue #5820: Query execution error when grouping by multi-value dimension and non-dictionary encoded dimension

lgo opened a new issue #5820:
URL: https://github.com/apache/incubator-pinot/issues/5820


   I've been playing around with queries and ran into an execution problem (this isn't really blocking anything, just doing toying with table setups). Specifically, it seems to happen when a query is grouping by a multi-value string dimension (`fields`) as well as a non-dictionary encoded string dimension (`id`). The setup below will reproduce it. The error doesn't happen with `"noDictionaryColumns": ["id"]`.
   
   ### Query
   
   ```
   SELECT fields, id, SUM(amount)
   FROM testdata
   GROUP BY fields, id
   ```
   
   ### Exception
   
   ```
   QueryExecutionError:
   java.lang.NullPointerException
       at org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator.getNextBlock(GroupByOrderByCombineOperator.java:215)
       at org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator.getNextBlock(GroupByOrderByCombineOperator.java:62)
       at org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49)
       at org.apache.pinot.core.operator.InstanceResponseOperator.getNextBlock(InstanceResponseOperator.java:37)
       at org.apache.pinot.core.operator.InstanceResponseOperator.getNextBlock(InstanceResponseOperator.java:26)
       at org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49)
       at org.apache.pinot.core.plan.GlobalPlanImplV0.execute(GlobalPlanImplV0.java:48)
       at org.apache.pinot.core.query.executor.ServerQueryExecutorV1Impl.processQuery(ServerQueryExecutorV1Impl.java:221)
       at org.apache.pinot.core.query.scheduler.QueryScheduler.processQueryAndSerialize(QueryScheduler.java:155)
       at org.apache.pinot.core.query.scheduler.QueryScheduler.lambda$createQueryFutureTask$0(QueryScheduler.java:139)
       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
       at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
       at shaded.com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:111)
       at shaded.com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:58)
   ```
   
   ### Table
   
   ```
   {
     "tableName": "testdata",
     "tableType": "OFFLINE",
     "routing": {
       "segmentPrunerType": "partition"
     },
     "segmentsConfig": {
       "timeColumnName": "created_at",
       "timeType": "SECONDS",
       "replication": "1",
       "schemaName": "testdata",
       "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy",
       "segmentPushFrequency": "HOURLY",
       "segmentPushType": "APPEND"
     },
     "tableIndexConfig": {
       "loadMode": "MMAP",
       "createInvertedIndexDuringSegmentGeneration": true,
       "invertedIndexColumns": ["fields"],
       "noDictionaryColumns": ["id"]
     },
     "tenants": {},
     "metadata": {}
   }
   ```
   
   ### Schema
   
   ```
   {
     "schemaName": "testdata",
     "dimensionFieldSpecs": [
       {
         "name": "fields",
         "dataType": "STRING",
         "singleValueField": false
       },
       {
         "name": "id",
         "dataType": "STRING"
       }
     ],
     "metricFieldSpecs": [
       {
         "name": "amount",
         "dataType": "DOUBLE"
       }
     ],
     "dateTimeFieldSpecs": [
       {
         "name": "created_at",
         "dataType": "LONG",
         "format": "1:SECONDS:EPOCH",
         "granularity": "15:MINUTES"
       }
     ]
   }
   ```
   
   ### Test data (JSON)
   
   ```
   {"fields": ["foo"], "id": "id_123", "amount": -1.12, "created_at": 1569293987}
   {"fields": ["foo", "bar"], "id": "id_987", "amount": 1.12, "created_at": 1569293987}
   {"fields": ["foo"], "id": "id_123", "amount": 1, "created_at": 1569293930}
   ```
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] fx19880617 commented on issue #5820: Query execution error when grouping by multi-value dimension and non-dictionary encoded dimension

Posted by GitBox <gi...@apache.org>.
fx19880617 commented on issue #5820:
URL: https://github.com/apache/incubator-pinot/issues/5820#issuecomment-672493742


   Currently we are not supporting No-Dictionary multi-value column in groupby right now.
   
   The relative code path is in `org.apache.pinot.core.query.aggregation.groupby.NoDictionaryMultiColumnGroupKeyGenerator`:
   ```
   @Override
     public void generateKeysForBlock(TransformBlock transformBlock, int[][] groupKeys) {
       // TODO: Support generating keys for multi-valued columns.
       throw new UnsupportedOperationException("Operation not supported");
     }
   ```
   
   I'm looking into the support for this.
   
   cc: @kishoreg @Jackie-Jiang 
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] fx19880617 commented on issue #5820: Query execution error when grouping by multi-value dimension and non-dictionary encoded dimension

Posted by GitBox <gi...@apache.org>.
fx19880617 commented on issue #5820:
URL: https://github.com/apache/incubator-pinot/issues/5820#issuecomment-673164217


   Pinot doesn't support non-dictionary index on multi-value column, so the dictionary is still there even if it's configured.
   
   The real issue is that the code goes to the above branch when querying:  GROUP BY a multi-value column and a single-value non-dictionary value column.
   
   Fixed in #5851   
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] kishoreg closed issue #5820: Query execution error when grouping by multi-value dimension and non-dictionary encoded dimension

Posted by GitBox <gi...@apache.org>.
kishoreg closed issue #5820:
URL: https://github.com/apache/incubator-pinot/issues/5820


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org