trinodb / trino

Official repository of Trino, the distributed SQL query engine for big data, formerly known as PrestoSQL (https://trino.io)
https://trino.io
Apache License 2.0
9.85k stars 2.85k forks source link

ThreadPerDriverTaskExecutor causes workes to stuck #21512

Open davseitsev opened 2 months ago

davseitsev commented 2 months ago

After Trino upgrade from version 409 to 444 we started wo experience issues with stuck workers. They refuse HTTP requests or the requests start to hang forever and desappear from discovery service.

Logs are full of messages like:

2024-04-09T13:31:03.821Z    WARN    node-state-poller-0 io.trino.metadata.RemoteNodeState   Node state update request to http://10.42.110.232:8181/v1/info/state has not returned in 35.08s
...
2024-04-09T13:33:38.872Z    ERROR   http-worker-529 io.trino.server.InternalAuthenticationManager   Internal authentication failed
io.jsonwebtoken.ExpiredJwtException: JWT expired 12872 milliseconds ago at 2024-04-09T13:33:26.000Z. Current time: 2024-04-09T13:33:38.872Z. Allowed clock skew: 0 milliseconds.
    at io.jsonwebtoken.impl.DefaultJwtParser.parse(DefaultJwtParser.java:682)

Where 10.42.110.232 is an IP of problematic worker.

Number of open file descriptors on the problematic worker dramatically increases: image

Sometimes we reach open sockets limit and got a lot of exceptions in logs like this:

2024-04-09T09:37:55.827Z    WARN    http-worker-124-acceptor-0@4e826fd4-http@44e08a7a{HTTP/1.1, (http/1.1, h2c)}{0.0.0.0:8181}  org.eclipse.jetty.server.AbstractConnector  Accept Failure
java.io.IOException: Too many open files
    at java.base/sun.nio.ch.Net.accept(Native Method)
    at java.base/sun.nio.ch.ServerSocketChannelImpl.implAccept(ServerSocketChannelImpl.java:433)
    at java.base/sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:399)
    at org.eclipse.jetty.server.ServerConnector.accept(ServerConnector.java:397)
    at org.eclipse.jetty.server.AbstractConnector$Acceptor.run(AbstractConnector.java:698)
    at org.eclipse.jetty.util.thread.MonitoredQueuedThreadPool$1.run(MonitoredQueuedThreadPool.java:73)
    at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:971)
    at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.doRunJob(QueuedThreadPool.java:1201)
    at org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1156)
    at java.base/java.lang.Thread.run(Thread.java:1583)

But it happens not always. On Trino 409 we had limit for open file descriptors 128K on workes. When we met this issue we increased the limit to 256K, but it doesn't help. This limit is exceeded very quickly.

Number of threads also start to increase: image

Thread dump shows that single random thread blocks many other threads. Example 1: image

Blocking thread stack trace (Full thread dump trino-worker-D0409-T0940-25683.tdump.txt) :

Task-20240409_093501_07390_nd69d.3.16.0-5123
Stack Trace is:
java.lang.Thread.State: RUNNABLE
at java.lang.Thread.start0(java.base@21.0.2/Native Method)
at java.lang.Thread.start(java.base@21.0.2/Thread.java:1553)
- locked <0x00007f6d32039e00> (a java.lang.Thread)
at java.lang.System$2.start(java.base@21.0.2/System.java:2577)
at jdk.internal.vm.SharedThreadContainer.start(java.base@21.0.2/SharedThreadContainer.java:152)
at java.util.concurrent.ThreadPoolExecutor.addWorker(java.base@21.0.2/ThreadPoolExecutor.java:953)
at java.util.concurrent.ThreadPoolExecutor.execute(java.base@21.0.2/ThreadPoolExecutor.java:1375)
at com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.execute(MoreExecutors.java:640)
at java.util.concurrent.AbstractExecutorService.submit(java.base@21.0.2/AbstractExecutorService.java:134)
at com.google.common.util.concurrent.AbstractListeningExecutorService.submit(AbstractListeningExecutorService.java:73)
at io.trino.execution.executor.scheduler.FairScheduler.submit(FairScheduler.java:161)
- locked <0x00007f57374acb88> (a io.trino.execution.executor.scheduler.FairScheduler)
at io.trino.execution.executor.dedicated.TaskEntry.runSplit(TaskEntry.java:136)
- locked <0x00007f6d32000000> (a io.trino.execution.executor.dedicated.TaskEntry)
at io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor.enqueueSplits(ThreadPerDriverTaskExecutor.java:157)
- locked <0x00007f57373c35b0> (a io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor)
at io.trino.execution.SqlTaskExecution.enqueueDriverSplitRunner(SqlTaskExecution.java:400)
- locked <0x00007f6285ff4580> (a io.trino.execution.SqlTaskExecution)
at io.trino.execution.SqlTaskExecution.scheduleDriversForTaskLifeCycle(SqlTaskExecution.java:389)
at io.trino.execution.SqlTaskExecution.start(SqlTaskExecution.java:205)
- locked <0x00007f6285ff4580> (a io.trino.execution.SqlTaskExecution)
at io.trino.execution.SqlTask.tryCreateSqlTaskExecution(SqlTask.java:561)
- locked <0x00007f5eb809b388> (a java.lang.Object)
at io.trino.execution.SqlTask.updateTask(SqlTask.java:503)
at io.trino.execution.SqlTaskManager.doUpdateTask(SqlTaskManager.java:555)
at io.trino.execution.SqlTaskManager.lambda$updateTask$9(SqlTaskManager.java:489)
at io.trino.execution.SqlTaskManager$$Lambda/0x00007f566ab68000.call(Unknown Source)
at io.trino.$gen.Trino_444____20240409_090553_2.call(Unknown Source)
at io.trino.execution.SqlTaskManager.updateTask(SqlTaskManager.java:489)
at io.trino.server.TaskResource.createOrUpdateTask(TaskResource.java:153)
...

Example 2: image

Blocking thread stack trace (Full thread dump trino-worker-D0409-T0949-25683.tdump.txt):

SplitRunner-24734
Stack Trace is:
java.lang.Thread.State: RUNNABLE
at java.lang.Thread.start0(java.base@21.0.2/Native Method)
at java.lang.Thread.start(java.base@21.0.2/Thread.java:1553)
- locked <0x00007f6db5a217a8> (a java.lang.Thread)
at java.lang.System$2.start(java.base@21.0.2/System.java:2577)
at jdk.internal.vm.SharedThreadContainer.start(java.base@21.0.2/SharedThreadContainer.java:152)
at java.util.concurrent.ThreadPoolExecutor.addWorker(java.base@21.0.2/ThreadPoolExecutor.java:953)
at java.util.concurrent.ThreadPoolExecutor.execute(java.base@21.0.2/ThreadPoolExecutor.java:1375)
at com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.execute(MoreExecutors.java:640)
at java.util.concurrent.AbstractExecutorService.submit(java.base@21.0.2/AbstractExecutorService.java:134)
at com.google.common.util.concurrent.AbstractListeningExecutorService.submit(AbstractListeningExecutorService.java:73)
at io.trino.execution.executor.scheduler.FairScheduler.submit(FairScheduler.java:161)
- locked <0x00007f57374acb88> (a io.trino.execution.executor.scheduler.FairScheduler)
at io.trino.execution.executor.dedicated.TaskEntry.runSplit(TaskEntry.java:136)
- locked <0x00007f5cbd1881d8> (a io.trino.execution.executor.dedicated.TaskEntry)
at io.trino.execution.executor.dedicated.TaskEntry.dequeueAndRunLeafSplit(TaskEntry.java:116)
- locked <0x00007f5cbd1881d8> (a io.trino.execution.executor.dedicated.TaskEntry)
at io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor.scheduleLeafSplit(ThreadPerDriverTaskExecutor.java:170)
at io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor.scheduleMoreLeafSplits(ThreadPerDriverTaskExecutor.java:190)
- locked <0x00007f57373c35b0> (a io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor)
at io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor.leafSplitDone(ThreadPerDriverTaskExecutor.java:181)
- locked <0x00007f57373c35b0> (a io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor)
at io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor$$Lambda/0x00007f566abdfd50.run(Unknown Source)
at io.trino.execution.executor.dedicated.TaskEntry.lambda$dequeueAndRunLeafSplit$0(TaskEntry.java:119)
at io.trino.execution.executor.dedicated.TaskEntry$$Lambda/0x00007f566abe2ad0.run(Unknown Source)
at com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:31)
...

We have changed value of experimental.thread-per-driver-scheduler-enabled to false and now the issue is not reproduced, our cluster looks stable https://github.com/trinodb/trino/blob/b7a161a422850c35b992f3ba26ac1a2f8bc9eb54/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java#L110-L115 I will update if anything changes.

Let me know if I can provide more information.

raunaqmorarka commented 2 months ago

cc: @martint @findepi

findepi commented 2 months ago

@davseitsev appreciate detailed analysis, that's very helpful!

nesat commented 2 months ago

Thanks a lot for sharing your findings and a potential solution for the problem. We had a similar issue on our Trino cluster started after we upgraded Trino from version 426 to 442. Workers started to have exchange issues more frequently with the logs:

2024-04-25T10:24:55.409Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to https://xxx.ip:8081/v1/info/state has not returned in 10.02s
2024-04-25T10:24:55.410Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to https://xxx.ip:8081/v1/info/state has not returned in 10.02s
2024-04-25T10:24:58.184Z        INFO    Thread-47       io.airlift.bootstrap.LifeCycleManager   JVM is shutting down, cleaning up
2024-04-25T10:24:58.185Z        INFO    Thread-42       io.airlift.bootstrap.LifeCycleManager   JVM is shutting down, cleaning up

I see the experimental.thread-per-driver-scheduler-enabled was enabled since version 438(https://github.com/trinodb/trino/pull/20451) so we decided to disable it and it helped with stabilizing the cluster. We still see some exchange issues but the number of impacted queries used to be 14x more before disabling the feature.

Pluies commented 1 month ago

Chiming in for more feedback - we're seeing exactly the same symptoms here on an update from 436 to 444. Nodes are going down, queries failing left and right, and a very very unstable cluster altogether.

If it helps at all, we're also seeing a ~30% query speed slowdown on smaller benchmarks, before nodes go unresponsive. Running the benchmark for longer (not very long in the grand scheme of things, maybe 5 to 10 minutes at ~half our current prod concurrency) causes unresponsive nodes.

I can also confirm setting experimental.thread-per-driver-scheduler-enabled=false fixes it completely: performance is restored to Trino 436 levels and nodes are stable.

martint commented 1 month ago

Are you able to narrow it down to specific query shapes? It would help immensely to debug if we can get some examples that cause the cluster to fall over, or that perform more slowly.

Pluies commented 1 month ago

@martint let me dig into our benchmark and see if I can narrow it down to specific shapes 👍

Pluies commented 1 month ago

@martint well it's been a bit of a wild goose chase. I have a list of 100 queries that do cause the issue, but I've been unable to pinpoint the issue to any single query in particular. Happy to share this list in private if it helps.

All these queries:

@davseitsev are there any communalities with your workloads? 🤔

I can't imagine this issue affects every Trino workloads or there would have been much more noise about it after the 436 release.

martint commented 1 month ago

Do the queries have joins, union, many stages? Does it happen if you load the cluster with any specific type of query, or only when you mix them up?

Pluies commented 1 month ago

Do the queries have joins, union, many stages?

Yes on all three counts 😅

Does it happen if you load the cluster with any specific type of query, or only when you mix them up?

It appears to be the mixing, or at least some level of concurrency, as I cannot reproduce this by executing queries one-by-one sequentially.

Our clusters can sustain ~40 concurrent queries on average, and the problem appears with ~10 concurrent queries, so it's not like we're pushing the envelope in terms of hardware either.