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
10.51k stars 3.03k forks source link

Issue of Excessive CPU Usage by Worker Threads #22642

Open youngjinkim0706 opened 4 months ago

youngjinkim0706 commented 4 months ago

When setting the number of max-worker-threads to cpus*2 and executing queries, CPU usage nearly reaches 100%, leading to resource starvation for the thread responsible for communication between worker and coordinator, resulting in a 503 error. This issue persists even when using the hardCpuLimit in the resource-group config. What are the best practices to resolve this issue? Alternatively, can this problem be solved by modifying other configurations?

here is error message

2024-07-04T08:07:13.550Z        WARN    query-management-1      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 10.50s
2024-07-04T08:07:25.528Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to http://172.30.0.4:8080/v1/info/state has not returned in 11.11s
2024-07-04T08:07:53.740Z        WARN    query-management-3      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 59.15s
2024-07-04T08:08:33.059Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to http://172.30.0.4:8080/v1/info/state has not returned in 87.41s
2024-07-04T08:08:35.645Z        WARN    query-management-0      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 100.94s
2024-07-04T08:08:39.222Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to http://172.30.0.4:8080/v1/info/state has not returned in 93.23s
2024-07-04T08:08:39.416Z        WARN    query-management-2      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 104.83s
2024-07-04T08:08:44.261Z        WARN    query-management-3      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 109.67s
2024-07-04T08:08:44.743Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to http://172.30.0.4:8080/v1/info/state has not returned in 99.11s
2024-07-04T08:08:47.148Z        WARN    query-management-1      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 112.56s
2024-07-04T08:08:49.813Z        WARN    query-management-4      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 115.22s
2024-07-04T08:08:49.824Z        WARN    node-state-poller-0     io.trino.metadata.RemoteNodeState       Node state update request to http://172.30.0.4:8080/v1/info/state has not returned in 104.20s
2024-07-04T08:08:50.814Z        WARN    query-management-0      io.trino.memory.RemoteNodeMemory        Memory info update request to http://172.30.0.4:8080/v1/memory has not returned in 116.23s
2024-07-04T08:08:50.847Z        WARN    http-client-node-manager-50     io.trino.metadata.RemoteNodeState       Error fetching node state from http://172.30.0.4:8080/v1/info/state: java.util.concurrent.TimeoutException: Total timeout 10000 ms elapsed
2024-07-04T08:08:50.848Z        WARN    http-client-memoryManager-scheduler-1   io.trino.memory.RemoteNodeMemory        Error fetching memory info from http://172.30.0.4:8080/v1/memory: java.util.concurrent.TimeoutException: Total timeout 10000 ms elapsed
2024-07-04T08:08:50.848Z        WARN    TaskInfoFetcher-20240704_080620_00005_4yit8.6.0.0-506   io.trino.server.remotetask.RequestErrorTracker  Error getting info for task 20240704_080620_00005_4yit8.6.0.0: java.util.concurrent.TimeoutException: Total timeout 10000 ms elapsed: http://172.30.0.4:8080/v1/task/20240704_080620_00005_4yit8.6.0.0
2024-07-04T08:08:50.820Z        WARN    ContinuousTaskStatusFetcher-20240704_080620_00005_4yit8.1.0.0-458       io.trino.server.remotetask.RequestErrorTracker  Error getting task status 20240704_080620_00005_4yit8.1.0.0: http://172.30.0.4:8080/v1/task/20240704_080620_00005_4yit8.1.0.0
io.trino.server.remotetask.SimpleHttpResponseHandler$ServiceUnavailableException: Server returned SERVICE_UNAVAILABLE: http://172.30.0.4:8080/v1/task/20240704_080620_00005_4yit8.1.0.0/status
        at io.trino.server.remotetask.SimpleHttpResponseHandler.onSuccess(SimpleHttpResponseHandler.java:52)
        at io.trino.server.remotetask.SimpleHttpResponseHandler.onSuccess(SimpleHttpResponseHandler.java:27)
        at com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1138)
        at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
        at java.base/java.lang.Thread.run(Thread.java:1570)
raunaqmorarka commented 4 months ago

Trino is designed to fully utilize CPU when there is enough parallelism in the workload, it usually doesn't lead to communication threads getting starved. Was there full GC or a crash of the worker ? Which release of Trino is this ? Could you check if disabling experimental.thread-per-driver-scheduler-enabled makes a difference ? Resource groups only provide admission control, it does not affect resource usage of running queries. If this happens under high concurrency, you can look at configuration a lower limit on query.max-concurrent-queries. If it's due to specific queries being resource heavy, then tuning task.max-drivers-per-task lower may help.

youngjinkim0706 commented 4 months ago

Thank you for your response, @raunaqmorarka. I'm going to check the parameters which you let me know.

FYI, I executed queries on both Trino 403 and 450 versions, the 503 error occured in both trino release. And these queries were run sequentially, meaning only a single query was executed in the Trino cluster at any given time.

There was a Major GC, not a Full GC, and the worker container was restarted. I suspect this occurred because an exception was thrown in the execute function of HiveSplitManager.java:

public void execute(Runnable command)
{
    try {
        delegate.execute(command);
    }
    catch (RejectedExecutionException e) {
        throw new TrinoException(SERVER_SHUTTING_DOWN, "Server is shutting down", e);
    }
}
youngjinkim0706 commented 4 months ago

Additionally, when I set the task.max-worker-threads parameter lower than the CPU resources allocated to the pod and executed the query, the query ran successfully.

What I am curious about is, if the value of max-worker-threads is high, it seems to occupy resources meant for communication, leading to the error that occurred earlier. Is there a way at the Trino configuration level to prioritize or allocate static resources specifically for communication threads? If not, can the CPU usage priority for threads be set in the JVM?

raunaqmorarka commented 4 months ago

There was a Major GC, not a Full GC, and the worker container was restarted

You should look into why the worker was restarted. You might need to reduce JVM heap size or increase memory.heap-headroom-per-node if the pod was oom killed by k8s or the JVM crashed on OOM.

akenO8 commented 4 months ago

Can't get trino.execution.executor:name=TaskExecutor.RunningSplits from http://worker:9300/v1/jmx/mbean image

okayhooni commented 4 months ago

I am not sure that your case is same as mine. but in my situation, the issue was linked to the default enablement of parquet.experimental.vectorized-decoding (introduced in v448) on worker containers running on AWS Graviton 2&3 (arm64) nodes.

If you are deploying Trino on these instance types, it could potentially be relevant to your scenario.

raunaqmorarka commented 4 months ago

I executed queries on both Trino 403 and 450 versions, the 503 error occured in both trino release

Based on that, it's probably not related to recent changes

okayhooni commented 4 months ago

I found this issue was reproduced even with lower task.max-worker-threads configuration.

In my case, this issue was just another side-effect of thread-per-driver scheduler, reported on the issue below.

Experimental config. experimental.thread-per-driver-scheduler-enabled was enabled by default since v438

When I disabled it, then this issue was removed..! I guess this is helpful also on your case :)

wwbbcjeyc commented 3 months ago

Can't get trino.execution.executor:name=TaskExecutor.RunningSplits from http://worker:9300/v1/jmx/mbean image

I found that when experimental.thread-per-driver-scheduler-enabled=true, it could not be obtained normally "Trino. Execution. Executor: name = TaskExecutor. RunningSplits" indicators, You need to set "experiment.thread-per-driver-scheduler-enabled" to "false"