trino
trino copied to clipboard
Flaky test TestHiveFaultTolerantExecutionConnectorTest
Error: io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionConnectorTest.testBucketedExecution Time elapsed: 1.068 s <<< FAILURE!
java.lang.AssertionError: Execution of 'actual' query failed: SELECT count(*) a FROM orders t1 JOIN orders t2 on t1.custkey=t2.custkey
at org.testng.Assert.fail(Assert.java:83)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:150)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:106)
at io.trino.testing.AbstractTestQueryFramework.assertQuery(AbstractTestQueryFramework.java:247)
at io.trino.plugin.hive.BaseHiveConnectorTest.testBucketedExecution(BaseHiveConnectorTest.java:3858)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:568)
at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.lang.RuntimeException: Error fetching next at http://127.0.0.1:37173/v1/statement/executing/20220831_182840_02610_d34nf/ydccd7eab35794dbcb479d945e958f3cfea6d7b2f/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Wed, 31 Aug 2022 18:28:41 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
at com.google.common.util.concurrent.AbstractFuture.cancellationExceptionWithCause(AbstractFuture.java:1543)
at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:586)
at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:547)
at com.google.common.util.concurrent.FluentFuture$TrustedFuture.get(FluentFuture.java:91)
at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:240)
at com.google.common.util.concurrent.Futures.getDone(Futures.java:1181)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1130)
at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
]
at io.trino.client.StatementClientV1.requestFailedException(StatementClientV1.java:453)
at io.trino.client.StatementClientV1.advance(StatementClientV1.java:392)
at io.trino.testing.AbstractTestingTrinoClient.execute(AbstractTestingTrinoClient.java:99)
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:480)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:147)
... 16 more
Suppressed: java.lang.Exception: SQL: SELECT count(*) a FROM orders t1 JOIN orders t2 on t1.custkey=t2.custkey
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:483)
... 17 more
@linzebing @arhimondr @losipiuk is it a new problem, or some other known issue?
same for other tests
https://github.com/trinodb/trino/actions/runs/3051089042/jobs/4919438612
Error: io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionAggregations.test15WayGroupBy Time elapsed: 0.72 s <<< FAILURE!
java.lang.AssertionError: Execution of 'actual' query failed: SELECT orderkey + 1, orderkey + 2, orderkey + 3, orderkey + 4, orderkey + 5, orderkey + 6, orderkey + 7, orderkey + 8, orderkey + 9, orderkey + 10, count(*) FROM orders GROUP BY orderkey + 1, orderkey + 2, orderkey + 3, orderkey + 4, orderkey + 5, orderkey + 6, orderkey + 7, orderkey + 8, orderkey + 9, orderkey + 10
at org.testng.Assert.fail(Assert.java:83)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:150)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:106)
at io.trino.testing.AbstractTestQueryFramework.assertQuery(AbstractTestQueryFramework.java:247)
at io.trino.testing.AbstractTestQueryFramework.assertQuery(AbstractTestQueryFramework.java:242)
at io.trino.testing.AbstractTestAggregations.test15WayGroupBy(AbstractTestAggregations.java:712)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:568)
at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.lang.RuntimeException: Error fetching next at http://127.0.0.1:38379/v1/statement/executing/20220914_085824_00032_qhvwj/y57f12eaa8a5d8f3a511b567362e900209d073f9a/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Wed, 14 Sep 2022 08:58:25 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
at com.google.common.util.concurrent.AbstractFuture.cancellationExceptionWithCause(AbstractFuture.java:1543)
at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:586)
at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:547)
at com.google.common.util.concurrent.FluentFuture$TrustedFuture.get(FluentFuture.java:91)
at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:240)
at com.google.common.util.concurrent.Futures.getDone(Futures.java:1181)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1130)
at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
]
at io.trino.client.StatementClientV1.requestFailedException(StatementClientV1.java:453)
at io.trino.client.StatementClientV1.advance(StatementClientV1.java:392)
at io.trino.testing.AbstractTestingTrinoClient.execute(AbstractTestingTrinoClient.java:99)
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:480)
at io.trino.testing.QueryAssertions.assertQuery(QueryAssertions.java:147)
... 17 more
Suppressed: java.lang.Exception: SQL: SELECT orderkey + 1, orderkey + 2, orderkey + 3, orderkey + 4, orderkey + 5, orderkey + 6, orderkey + 7, orderkey + 8, orderkey + 9, orderkey + 10, count(*) FROM orders GROUP BY orderkey + 1, orderkey + 2, orderkey + 3, orderkey + 4, orderkey + 5, orderkey + 6, orderkey + 7, orderkey + 8, orderkey + 9, orderkey + 10
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:483)
... 18 more
It feels that the future returned from FileSystemExchangeSource.isBlocked() gets canceled.
And when we are trying to compute the ignored here we get an exception. I did not yet figure out why the future can get canceled.
It would happen if the exchange is closed while data is being read from it but it should not happen in happy path IIUC.
@linzebing
When looking at the code I thought for a while that this call to close() could be the problem.
As close() cancels the future which was returned from isBlocked().
Turned out not be the problem as futures returned from ExchangeStorageReader.isBlocked() are wrapped in whenAnyComplete here. And whenAnyComplete does not propagate cancelation upwards.
Still, it look not correct to cancel the future in happy-path flow in S3FileSystemExchangeStorage. Should we just use SettableFuture as inProgressReadFuture in S3FileSystemExchangeStorage and just complete it succesfully in fillBuffer, where we now call cancel()?
@losipiuk I will take a look in a day or two
@losipiuk : it can't be it. We have nonCancellationPropagating wrapping it in FileSystemExchangeSource.java
@losipiuk : it can't be it. We have
nonCancellationPropagatingwrapping it inFileSystemExchangeSource.java
Yeah @linzebing. I know it does not result in a problem tracked with that issue. I noted that here: Turned out not be the problem.... Yet you are right the more obvious reason for that is the use of nonCancellationPropagating.
The comment was more a digression about the fact fillBuffer uses close() which unconditionally calls inProgressReadFuture.calls(). Probably it is not harmful but reduces readability somewhat. In happy path here we should just assert that inProgressReadFuture.isDone() and replace it immediateVoidFuture() for sake of GC.
It can happen during initialization as well:
Error: io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionAggregations.init Time elapsed: 107.604 s <<< FAILURE!
java.lang.RuntimeException:
Error fetching next at http://127.0.0.1:44911/v1/statement/executing/20220919_154842_00025_zxutm/yda6236fa82ee49ec4a22c269dab2d38b8df7afb6/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Mon, 19 Sep 2022 15:48:43 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
https://github.com/trinodb/trino/actions/runs/3083723967/jobs/4985044156
TestHiveFaultTolerantExecutionAggregations.init init is affected too
https://github.com/trinodb/trino/actions/runs/3097801353/jobs/5015430530
Error: io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionAggregations.init Time elapsed: 148.081 s <<< FAILURE!
java.lang.RuntimeException:
Error fetching next at http://127.0.0.1:40181/v1/statement/executing/20220921_141119_00012_dif2k/y8f736a03626b1e519bf12edc5becc43b5179925c/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Wed, 21 Sep 2022 14:11:20 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
at com.google.common.util.concurrent.AbstractFuture.cancellationExceptionWithCause(AbstractFuture.java:1543)
at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:586)
at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:547)
at com.google.common.util.concurrent.FluentFuture$TrustedFuture.get(FluentFuture.java:91)
at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:240)
at com.google.common.util.concurrent.Futures.getDone(Futures.java:1181)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1130)
at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
]
at io.trino.client.StatementClientV1.requestFailedException(StatementClientV1.java:453)
at io.trino.client.StatementClientV1.advance(StatementClientV1.java:392)
at io.trino.testing.AbstractTestingTrinoClient.execute(AbstractTestingTrinoClient.java:99)
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:480)
at io.trino.testing.QueryAssertions.copyTable(QueryAssertions.java:339)
at io.trino.testing.QueryAssertions.copyTable(QueryAssertions.java:331)
at io.trino.testing.QueryAssertions.copyTpchTables(QueryAssertions.java:323)
at io.trino.plugin.hive.HiveQueryRunner$Builder.populateData(HiveQueryRunner.java:273)
at io.trino.plugin.hive.HiveQueryRunner$Builder.build(HiveQueryRunner.java:258)
at io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionAggregations.createQueryRunner(TestHiveFaultTolerantExecutionAggregations.java:48)
at io.trino.testing.AbstractTestFaultTolerantExecutionAggregations.createQueryRunner(AbstractTestFaultTolerantExecutionAggregations.java:25)
at io.trino.testing.AbstractTestQueryFramework.init(AbstractTestQueryFramework.java:95)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:568)
at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
at org.testng.internal.Invoker.invokeConfigurationMethod(Invoker.java:515)
at org.testng.internal.Invoker.invokeConfigurations(Invoker.java:217)
at org.testng.internal.Invoker.invokeConfigurations(Invoker.java:144)
at org.testng.internal.TestMethodWorker.invokeBeforeClassMethods(TestMethodWorker.java:169)
at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:108)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Suppressed: java.lang.Exception: SQL: CREATE TABLE IF NOT EXISTS partsupp AS SELECT * FROM tpch.tiny.partsupp
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:483)
... 21 more
TestHiveFaultTolerantExecutionConnectorTest.init seems affected too https://github.com/trinodb/trino/actions/runs/3128897703/jobs/5078465823
Error: io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionConnectorTest.init Time elapsed: 126.58 s <<< FAILURE!
java.lang.RuntimeException:
Error fetching next at http://127.0.0.1:45013/v1/statement/executing/20220926_174346_00014_b3d39/y88a152df384dc4700f30a74aa76affd161e39b4c/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Mon, 26 Sep 2022 17:43:48 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
at com.google.common.util.concurrent.AbstractFuture.cancellationExceptionWithCause(AbstractFuture.java:1543)
at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:586)
at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:547)
at com.google.common.util.concurrent.FluentFuture$TrustedFuture.get(FluentFuture.java:91)
at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:240)
at com.google.common.util.concurrent.Futures.getDone(Futures.java:1181)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1130)
at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
]
at io.trino.client.StatementClientV1.requestFailedException(StatementClientV1.java:453)
at io.trino.client.StatementClientV1.advance(StatementClientV1.java:392)
at io.trino.testing.AbstractTestingTrinoClient.execute(AbstractTestingTrinoClient.java:99)
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:480)
at io.trino.plugin.hive.HiveQueryRunner.copyTableBucketed(HiveQueryRunner.java:375)
at io.trino.plugin.hive.HiveQueryRunner.copyTpchTablesBucketed(HiveQueryRunner.java:339)
at io.trino.plugin.hive.HiveQueryRunner$Builder.populateData(HiveQueryRunner.java:279)
at io.trino.plugin.hive.HiveQueryRunner$Builder.build(HiveQueryRunner.java:258)
at io.trino.plugin.hive.BaseHiveConnectorTest.createHiveQueryRunner(BaseHiveConnectorTest.java:213)
at io.trino.faulttolerant.hive.TestHiveFaultTolerantExecutionConnectorTest.createQueryRunner(TestHiveFaultTolerantExecutionConnectorTest.java:44)
at io.trino.testing.AbstractTestQueryFramework.init(AbstractTestQueryFramework.java:95)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:568)
at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
at org.testng.internal.Invoker.invokeConfigurationMethod(Invoker.java:515)
at org.testng.internal.Invoker.invokeConfigurations(Invoker.java:217)
at org.testng.internal.Invoker.invokeConfigurations(Invoker.java:144)
at org.testng.internal.TestMethodWorker.invokeBeforeClassMethods(TestMethodWorker.java:169)
at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:108)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Suppressed: java.lang.Exception: SQL: CREATE TABLE orders WITH (bucketed_by=array['custkey'], bucket_count=11) AS SELECT * FROM tpch.tiny.orders
at io.trino.testing.DistributedQueryRunner.execute(DistributedQueryRunner.java:483)
... 20 more
cc @arhimondr @linzebing
https://github.com/trinodb/trino/actions/runs/3128540046/jobs/5077313683
2022-09-26T11:42:37.384-0500 ERROR pool-3-thread-1 io.trino.testng.services.ProgressLoggingListener [TEST FAILURE] io.trino.faulttolerant.hive.TestHiveTaskFailureRecoveryTest.testJoinDynamicFilteringDisabled; (took: 1 minutes and 1 seconds)
java.lang.RuntimeException: Error fetching next at http://127.0.0.1:42205/v1/statement/executing/20220926_164235_00454_wqgmn/y3ac6fdac0dd77ff815993e9a783188ec7f46354f/1 returned an invalid response: JsonResponse{statusCode=500, statusMessage=Internal Server Error, headers={content-length=[1016], content-type=[text/plain], date=[Mon, 26 Sep 2022 16:42:37 GMT]}, hasValue=false} [Error: java.util.concurrent.CancellationException: Task was cancelled.
at com.google.common.util.concurrent.AbstractFuture.cancellationExceptionWithCause(AbstractFuture.java:1543)
at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:586)
at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:547)
at com.google.common.util.concurrent.FluentFuture$TrustedFuture.get(FluentFuture.java:91)
at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:240)
at com.google.common.util.concurrent.Futures.getDone(Futures.java:1181)
at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1130)
at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
]
at io.trino.client.StatementClientV1.requestFailedException(StatementClientV1.java:453)
at io.trino.client.StatementClientV1.advance(StatementClientV1.java:392)
at io.trino.testing.AbstractTestingTrinoClient.execute(AbstractTestingTrinoClient.java:99)
at io.trino.testing.DistributedQueryRunner.executeWithQueryId(DistributedQueryRunner.java:495)
at io.trino.faulttolerant.BaseFailureRecoveryTest$FailureRecoveryAssert.execute(BaseFailureRecoveryTest.java:578)
at io.trino.faulttolerant.BaseFailureRecoveryTest$FailureRecoveryAssert.executeActual(BaseFailureRecoveryTest.java:564)
at io.trino.faulttolerant.BaseFailureRecoveryTest$FailureRecoveryAssert.executeActual(BaseFailureRecoveryTest.java:544)
at io.trino.faulttolerant.BaseFailureRecoveryTest$FailureRecoveryAssert.finishesSuccessfully(BaseFailureRecoveryTest.java:635)
at io.trino.faulttolerant.BaseFailureRecoveryTest$FailureRecoveryAssert.finishesSuccessfully(BaseFailureRecoveryTest.java:626)
at io.trino.faulttolerant.BaseFailureRecoveryTest.testSelect(BaseFailureRecoveryTest.java:228)
at io.trino.faulttolerant.BaseFailureRecoveryTest.testSelect(BaseFailureRecoveryTest.java:198)
at io.trino.faulttolerant.BaseFailureRecoveryTest.testJoinDynamicFilteringDisabled(BaseFailureRecoveryTest.java:165)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:568)
at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
cc @arhimondr @linzebing @losipiuk
Fix: https://github.com/trinodb/trino/pull/14355