trino
trino copied to clipboard
SpillableHashAggregationBuilder#updateMemory might set negative bytes
Caused by: java.sql.SQLException: Query failed (#20211105_031649_15049_gbiv4): bytes cannot be negative
at io.prestosql.jdbc.PrestoResultSet.resultsException(PrestoResultSet.java:1915)
at io.prestosql.jdbc.PrestoResultSet$ResultsPageIterator.computeNext(PrestoResultSet.java:1904)
at io.prestosql.jdbc.PrestoResultSet$ResultsPageIterator.computeNext(PrestoResultSet.java:1862)
at io.prestosql.jdbc.$internal.guava.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:141)
at io.prestosql.jdbc.$internal.guava.collect.AbstractIterator.hasNext(AbstractIterator.java:136)
at java.util.Spliterators$IteratorSpliterator.tryAdvance(Spliterators.java:1811)
at java.util.stream.StreamSpliterators$WrappingSpliterator.lambda$initPartialTraversalState$0(StreamSpliterators.java:294)
at java.util.stream.StreamSpliterators$AbstractWrappingSpliterator.fillBuffer(StreamSpliterators.java:206)
at java.util.stream.StreamSpliterators$AbstractWrappingSpliterator.doAdvance(StreamSpliterators.java:161)
at java.util.stream.StreamSpliterators$WrappingSpliterator.tryAdvance(StreamSpliterators.java:300)
at java.util.Spliterators$1Adapter.hasNext(Spliterators.java:681)
at io.prestosql.jdbc.PrestoResultSet$AsyncIterator.lambda$new$0(PrestoResultSet.java:1804)
at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
... 3 more
Caused by: java.lang.IllegalArgumentException: bytes cannot be negative
at com.google.common.base.Preconditions.checkArgument(Preconditions.java:142)
at io.prestosql.memory.context.SimpleLocalMemoryContext.setBytes(SimpleLocalMemoryContext.java:59)
at io.prestosql.operator.OperatorContext$InternalLocalMemoryContext.setBytes(OperatorContext.java:671)
at io.prestosql.operator.aggregation.builder.SpillableHashAggregationBuilder.updateMemory(SpillableHashAggregationBuilder.java:130)
at io.prestosql.operator.HashAggregationOperator.addInput(HashAggregationOperator.java:439)
at io.prestosql.operator.Driver.processInternal(Driver.java:384)
at io.prestosql.operator.Driver.lambda$processFor$8(Driver.java:283)
at io.prestosql.operator.Driver.tryWithLock(Driver.java:675)
at io.prestosql.operator.Driver.processFor(Driver.java:276)
at io.prestosql.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1076)
at io.prestosql.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
at io.prestosql.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
at io.prestosql.$gen.Presto_347_70_geee8d34____20211026_120515_2.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:834)
@sopel39 @raunaqmorarka @skrzypo987 did you happen to have fixed this in the mean time?
I don't think so. @lzeiming do you know how to repro this?
I don't think so. @lzeiming do you know how to repro this?
I don't know how to reproduce
This seems still existing on 426, and it's on worker so hard to reproduce, still in trying...
{
"type":"java.lang.IllegalArgumentException",
"message":"bytes cannot be negative",
"suppressed":[
],
"stack":[
"com.google.common.base.Preconditions.checkArgument(Preconditions.java:143)",
"io.trino.memory.context.SimpleLocalMemoryContext.setBytes(SimpleLocalMemoryContext.java:58)",
"io.trino.operator.OperatorContext$InternalLocalMemoryContext.setBytes(OperatorContext.java:709)",
"io.trino.operator.HashAggregationOperator.lambda$addInput$0(HashAggregationOperator.java:412)",
"io.trino.operator.FlatHash.tryRehash(FlatHash.java:321)",
"io.trino.operator.FlatHash.ensureAvailableCapacity(FlatHash.java:299)",
"io.trino.operator.FlatGroupByHash$AddNonDictionaryPageWork.process(FlatGroupByHash.java:310)",
"io.trino.operator.HashAggregationOperator.addInput(HashAggregationOperator.java:444)",
"io.trino.operator.Driver.processInternal(Driver.java:400)",
"io.trino.operator.Driver.lambda$process$8(Driver.java:298)",
"io.trino.operator.Driver.tryWithLock(Driver.java:694)",
"io.trino.operator.Driver.process(Driver.java:290)",
"io.trino.operator.Driver.processForDuration(Driver.java:261)",
"io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:911)",
"io.trino.execution.executor.timesharing.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:188)",
"io.trino.execution.executor.timesharing.TimeSharingTaskExecutor$TaskRunner.run(TimeSharingTaskExecutor.java:569)",
"io.trino.$gen.Trino_trino426_sql_005_dirty____20231117_083941_2.run(Unknown Source)",
"java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)",
"java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)",
"java.base/java.lang.Thread.run(Thread.java:833)"
],
"errorCode":{
"code":65536,
"name":"GENERIC_INTERNAL_ERROR",
"type":"INTERNAL_ERROR"
}
}
@jerryleooo anything specific about this query? Do you have spill enabled?
Hi @sopel39 after some research I think my case has been fixed in https://github.com/trinodb/trino/commit/e40b016e4568f34580e0f8a1777c134267f1d127