You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
SELECT multimap_agg(x, y) FROM (VALUES (nan(), 2), (2, 3)) t(x,y);
will fail with the following stack trace.
java.lang.ArrayIndexOutOfBoundsException: Index -1 out of bounds for length 1024
at com.facebook.presto.common.array.ObjectBigArray.set(ObjectBigArray.java:108)
at com.facebook.presto.operator.aggregation.multimapagg.MultimapAggregationFunction.lambda$output$0(MultimapAggregationFunction.java:155)
at com.facebook.presto.operator.aggregation.multimapagg.SingleMultimapAggregationState.forEach(SingleMultimapAggregationState.java:54)
at com.facebook.presto.operator.aggregation.multimapagg.MultimapAggregationFunction.output(MultimapAggregationFunction.java:150)
at com.facebook.presto.$gen.Map_doubleDoubleIntegerMultimapAggAccumulator_20240506_160754_335279.evaluateFinal(Unknown Source)
at com.facebook.presto.operator.Aggregator.evaluate(Aggregator.java:71)
at com.facebook.presto.operator.AggregationOperator.getOutput(AggregationOperator.java:174)
at com.facebook.presto.operator.Driver.processInternal(Driver.java:441)
at com.facebook.presto.operator.Driver.lambda$processFor$10(Driver.java:324)
at com.facebook.presto.operator.Driver.tryWithLock(Driver.java:750)
at com.facebook.presto.operator.Driver.processFor(Driver.java:317)
at com.facebook.presto.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1079)
at com.facebook.presto.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:165)
at com.facebook.presto.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:621)
at com.facebook.presto.$gen.Presto_0_288_edge1_2_SNAPSHOT_$_git_commit_id_abbrev___0_288_edge1_2____20240505_014942_1.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
Queries like the below
will fail with the following stack trace.
expected behavior is to aggregate nan() keys the same as other keys as per https://github.com/prestodb/rfcs/blob/main/RFC-0001-nan-definition.md
This will be fixed by #22386
The text was updated successfully, but these errors were encountered: