You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/10/13 02:53:20 UTC

[GitHub] [druid] jjaen0823 opened a new issue, #13220: MomentSketchAggregatorFactory doesn't implement makeAggregateCombiner

jjaen0823 opened a new issue, #13220:
URL: https://github.com/apache/druid/issues/13220

   In 0.23.0, using a `momentSketch` aggregator at ingestion will result in the following exception during indexing merging:
   
   ```
   2022-10-12T07:35:37,617 WARN [[index_parallel_newyork_momentsketch_test_agfjijan_2022-10-12T07:35:26.205Z]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.AppenderatorImpl - Failed to push merged index for segment[newyork_momentsketch_test_2016-01-01T00:00:00.000Z_2016-02-01T00:00:00.000Z_2022-10-12T07:35:36.144Z].
   org.apache.druid.java.util.common.UOE: [org.apache.druid.query.aggregation.momentsketch.aggregator.MomentSketchAggregatorFactory] does not implement makeAggregateCombiner()
   	at org.apache.druid.query.aggregation.AggregatorFactory.makeAggregateCombiner(AggregatorFactory.java:128) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.query.aggregation.AggregatorFactory.makeNullableAggregateCombiner(AggregatorFactory.java:142) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.RowCombiningTimeAndDimsIterator.lambda$new$1(RowCombiningTimeAndDimsIterator.java:120) ~[druid-processing-0.23.0.jar:0.23.0]
   	at java.util.Arrays.setAll(Arrays.java:4698) ~[?:1.8.0_275]
   	at org.apache.druid.segment.RowCombiningTimeAndDimsIterator.<init>(RowCombiningTimeAndDimsIterator.java:120) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.lambda$merge$3(IndexMergerV9.java:1298) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.makeMergedTimeAndDimsIterator(IndexMergerV9.java:1360) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.makeIndexFiles(IndexMergerV9.java:293) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.merge(IndexMergerV9.java:1303) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.multiphaseMerge(IndexMergerV9.java:1120) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.IndexMergerV9.mergeQueryableIndex(IndexMergerV9.java:1062) ~[druid-processing-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.realtime.appenderator.AppenderatorImpl.mergeAndPush(AppenderatorImpl.java:923) ~[druid-server-0.23.0.jar:0.23.0]
   	at org.apache.druid.segment.realtime.appenderator.AppenderatorImpl.lambda$push$1(AppenderatorImpl.java:784) ~[druid-server-0.23.0.jar:0.23.0]
   	at com.google.common.util.concurrent.Futures$1.apply(Futures.java:713) [guava-16.0.1.jar:?]
   	at com.google.common.util.concurrent.Futures$ChainingListenableFuture.run(Futures.java:861) [guava-16.0.1.jar:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_275]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_275]
   	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_275]
   ```
   
   It looks like the moment sketch aggregator doesn't have `makeAggregateCombiner()`.
   
   The ingestion spec:
   ```
   ...
             {
               "type": "momentSketch",
               "k": 3,
               "name": "ms_passenger_count_true",
               "fieldName": "passenger_count",
               "compress": 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: commits-unsubscribe@druid.apache.org.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 commented on issue #13220: MomentSketchAggregatorFactory doesn't implement makeAggregateCombiner

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on issue #13220:
URL: https://github.com/apache/druid/issues/13220#issuecomment-1306887287

   @jjaen0823 - it would seem that `momentSketch` is not with rollup-enabled ingestion. Happy to help you if you want to raise a PR to fix this gap. 


-- 
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: commits-unsubscribe@druid.apache.org

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


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


[GitHub] [druid] rohangarg commented on issue #13220: MomentSketchAggregatorFactory doesn't implement makeAggregateCombiner

Posted by GitBox <gi...@apache.org>.
rohangarg commented on issue #13220:
URL: https://github.com/apache/druid/issues/13220#issuecomment-1306920930

   @jjaen0823 : Incase you're not looking specifically for moment-sketch, you could also try DataSketches Quantiles Sketch aggregator (https://druid.apache.org/docs/latest/development/extensions-core/datasketches-quantiles.html) for quantiles over arbitrary data distributions with bounded errors (it also provides a bunch of other distribution summary functions).


-- 
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: commits-unsubscribe@druid.apache.org

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


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