Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Core] Too many threads in ray worker #36936

Open
messense opened this issue Jun 29, 2023 · 16 comments
Open

[Core] Too many threads in ray worker #36936

messense opened this issue Jun 29, 2023 · 16 comments
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P1 Issue that should be fixed within a few weeks performance

Comments

@messense
Copy link

What happened + What you expected to happen

On a 8 core Linux server, in local mode after calling ray.init(), ray creates 8 (idle) workers (looks reasonable), but in each of them it also creates many threads, in this case total of 33 threads in each worker:

  • 3 worker.io
  • 4 grpc_global_tim
  • 1 task_event_buff
  • 4 client.poll0
  • 1 server.poll0
  • 1 grpcpp_sync_ser
  • 1 grpc_health_che
  • 2 resolver-execut
  • 2 default-execut
  • 4 ray :: IDLE
  • 1 timer_manager
  • 8 event_engine

On a 128 core Linux server:

  • 33 worker.io
  • 4 grpc_global_tim
  • 1 task_event_buff
  • 4 client.poll0
  • 1 server.poll0
  • 1 grpcpp_sync_ser
  • 1 grpc_health_che
  • 2 resolver-execut
  • 2 default-execut
  • 4 ray :: IDLE
  • 1 timer_manager
  • 32 event_engine

total of 86 threads in one ray IDLE process. And there are 128 ray IDLE processes, a total of over 10000 threads spawned just by calling ray.init().

See also https://discuss.ray.io/t/too-many-threads-in-ray-worker/10881

Versions / Dependencies

$ ray --version
ray, version 2.4.0

$ cat /etc/os-release
PRETTY_NAME="Ubuntu 22.04.2 LTS"
NAME="Ubuntu"
VERSION_ID="22.04"
VERSION="22.04.2 LTS (Jammy Jellyfish)"
VERSION_CODENAME=jammy
ID=ubuntu
ID_LIKE=debian
HOME_URL="https://www.ubuntu.com/"
SUPPORT_URL="https://help.ubuntu.com/"
BUG_REPORT_URL="https://bugs.launchpad.net/ubuntu/"
PRIVACY_POLICY_URL="https://www.ubuntu.com/legal/terms-and-policies/privacy-policy"
UBUNTU_CODENAME=jammy

Reproduction script

import ray

ray.init()

Issue Severity

Medium: It is a significant difficulty but I can work around it.

@messense messense added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jun 29, 2023
@rkooo567 rkooo567 added the core Issues that should be addressed in Ray Core label Jun 29, 2023
@xieus xieus added P1 Issue that should be fixed within a few weeks and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jul 3, 2023
@rkooo567 rkooo567 removed their assignment Jul 7, 2023
@heliar-k
Copy link

heliar-k commented Sep 5, 2023

Any progress of this?

@rkooo567
Copy link
Contributor

rkooo567 commented Sep 5, 2023

cc @rynewang is it the same issue you fixed recently?

@beantowel
Copy link

I thought this problem led to the "pthread_create resource temporarily unavailable..." error when there are too many parallel tasks, say 32 tasks. The logging part in Ray itself and the application code (in my case, some OpenMP code) suffer from it.

When the task is running, every ray::xxx process gets a 100+ nTH count.

@rynewang
Copy link
Contributor

Since it's Ray 2.4.0 maybe it's #33957 . Could you try on Ray 2.7.1?

@messense
Copy link
Author

@rynewang There is no actor involved in this repro script

import ray

ray.init()

@rkooo567
Copy link
Contributor

rkooo567 commented Nov 2, 2023

There were issues we create a thread per caller for async actor, and thats' been fixed. I think this issue is still not fixed.

33 worker.io
4 grpc_global_tim
1 task_event_buff
4 client.poll0
1 server.poll0
1 grpcpp_sync_ser
1 grpc_health_che
2 resolver-execut
2 default-execut
4 ray :: IDLE
1 timer_manager
32 event_engine

event_engine is from grpc, and there's currently no way to control (unless we patch grpc). I think in the reality most of them is idle.

it is mysterious why we have so many worker.io threads per proc (we are expected to have only 1 per proc).

In terms of fix timeline, as https://discuss.ray.io/t/too-many-threads-in-ray-worker/10881/12?u=sangcho says, we may not prioritize the fix at least in a while unless there's concrete proof in performance impact. Regarding the system resources limit, we in general recommend to set high ulimit.

@rkooo567 rkooo567 removed their assignment Nov 8, 2023
@baonq-me
Copy link

In my case, around ~4000 threads were created. Sometimes I can not scale up ray instance like below

2|generation0  | (pid=84957) [2024-05-28 01:37:16,897 E 84957 85191] logging.cc:101: Unhandled exception: N5boost10wrapexceptINS_6system12system_errorEEE. what(): thread: Resource temporarily unavailable [system:11]
2|generation0  | (raylet) E0528 01:37:16.954933046   84803 thd.cc:157]                           pthread_create failed: Resource temporarily unavailable
2|generation0  | (pid=84964) /lib/x86_64-linux-gnu/libc.so.6(+0x94ac3) [0x7fc7d1627ac3]
2|generation0  | (pid=84964) /lib/x86_64-linux-gnu/libc.so.6(clone+0x44) [0x7fc7d16b8bf4] __clone
2|generation0  | (pid=84964)
2|generation0  | (pid=84964) *** SIGABRT received at time=1716860238 on cpu 141 ***
2|generation0  | (pid=84964) PC: @     0x7fc7d16299fc  (unknown)  pthread_kill
2|generation0  | (pid=84964)     @     0x7fc7d15d5520  (unknown)  (unknown)
2|generation0  | (pid=84964) [2024-05-28 01:37:18,532 E 84964 86575] logging.cc:365: *** SIGABRT received at time=1716860238 on cpu 141 ***
2|generation0  | (pid=84964) [2024-05-28 01:37:18,532 E 84964 86575] logging.cc:365: PC: @     0x7fc7d16299fc  (unknown)  pthread_kill
2|generation0  | (pid=84964) [2024-05-28 01:37:18,532 E 84964 86575] logging.cc:365:     @     0x7fc7d15d5520  (unknown)  (unknown)
2|generation0  | (pid=84964) Fatal Python error: Aborted

@messense
Copy link
Author

messense commented May 28, 2024

I've done another investigation today, and finally found the source of these worker.io threads

std::unique_ptr<boost::asio::thread_pool> &_GetServerCallExecutor() {
static auto thread_pool = std::make_unique<boost::asio::thread_pool>(
::RayConfig::instance().num_server_call_thread());
return thread_pool;
}

by default the num_server_call_thread is cpu_count / 4:

/// The pool size for grpc server call.
RAY_CONFIG(int64_t,
num_server_call_thread,
std::max((int64_t)1, (int64_t)(std::thread::hardware_concurrency() / 4U)))

the backtrace of the thread creation is like this

#0  __pthread_create_2_1 (newthread=0x555556cb26a0, attr=0x0, start_routine=0x7ffff6cc9c00 <boost_asio_detail_posix_thread_function>, arg=0x7fff70004570) at ./nptl/pthread_create.c:621
#1  0x00007ffff6cd355e in boost::asio::detail::posix_thread::start_thread(boost::asio::detail::posix_thread::func_base*) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#2  0x00007ffff6cd3a1c in boost::asio::thread_pool::thread_pool(unsigned long) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#3  0x00007ffff66157d4 in ray::rpc::(anonymous namespace)::_GetServerCallExecutor() () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#4  0x00007ffff6615869 in ray::rpc::GetServerCallExecutor() () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#5  0x00007ffff6338382 in std::_Function_handler<void (ray::Status, std::function<void ()>, std::function<void ()>), ray::rpc::ServerCallImpl<ray::rpc::CoreWorkerServiceHandler, ray::rpc::GetCoreWorkerStatsRequest, ray::rpc::GetCoreWorkerStatsReply, (ray::rpc::AuthType)0>::HandleRequestImpl(bool)::{lambda(ray::Status, std::function<void ()>, std::function<void ()>)#2}>::_M_invoke(std::_Any_data const&, ray::Status&&, std::function<void ()>&&, std::function<void ()>&&) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#6  0x00007ffff637a8b9 in ray::core::CoreWorker::HandleGetCoreWorkerStats(ray::rpc::GetCoreWorkerStatsRequest, ray::rpc::GetCoreWorkerStatsReply*, std::function<void (ray::Status, std::function<void ()>, std::function<void ()>)>) ()
   from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#7  0x00007ffff636fcf4 in ray::rpc::ServerCallImpl<ray::rpc::CoreWorkerServiceHandler, ray::rpc::GetCoreWorkerStatsRequest, ray::rpc::GetCoreWorkerStatsReply, (ray::rpc::AuthType)0>::HandleRequestImpl(bool) ()
   from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#8  0x00007ffff6626f5e in EventTracker::RecordExecution(std::function<void ()> const&, std::shared_ptr<StatsHandle>) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#9  0x00007ffff662034e in std::_Function_handler<void (), instrumented_io_context::post(std::function<void ()>, std::string, long)::{lambda()#1}>::_M_invoke(std::_Any_data const&) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#10 0x00007ffff66207c6 in boost::asio::detail::completion_handler<std::function<void ()>, boost::asio::io_context::basic_executor_type<std::allocator<void>, 0ul> >::do_complete(void*, boost::asio::detail::scheduler_operation*, boost::system::error_code const&, unsigned long)
    () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#11 0x00007ffff6cd022b in boost::asio::detail::scheduler::do_run_one(boost::asio::detail::conditionally_enabled_mutex::scoped_lock&, boost::asio::detail::scheduler_thread_info&, boost::system::error_code const&) ()
   from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#12 0x00007ffff6cd1ba9 in boost::asio::detail::scheduler::run(boost::system::error_code&) () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#13 0x00007ffff6cd22b2 in boost::asio::io_context::run() () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#14 0x00007ffff6351889 in ray::core::CoreWorker::RunIOService() () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#15 0x00007ffff670fb80 in thread_proxy () from /home/ubuntu/example/.venv/lib/python3.10/site-packages/ray/_raylet.so
#16 0x00007ffff7c94ac3 in start_thread (arg=<optimized out>) at ./nptl/pthread_create.c:442
#17 0x00007ffff7d26850 in clone3 () at ../sysdeps/unix/sysv/linux/x86_64/clone3.S:81

seems like these threads inherit the worker.io thread name but they are not the real worker.io thread.

To workaround it, you can set the env var RAY_num_server_call_thread to a lower value.

I'm not sure if std::max((int64_t)1, (int64_t)(std::thread::hardware_concurrency() / 4U))) is a sensible default value for it, maybe it should be capped to avoid creating lots of useless threads on large servers.

@anyscalesam
Copy link
Contributor

@jjyao should we close this and open another GH issue to track the addition of a cap?

@BrunoBelucci
Copy link

I recently encountered the same issue. While this may not directly impact performance, there is a problem when the system reaches the user thread limit, which prevents the creation of new threads. This can be especially problematic in environments like a SLURM cluster. In my current setup, I am hitting the thread limit on the nodes of my SLURM cluster, resulting in messages like "bash: fork: retry: Resource temporarily unavailable" when trying to submit a simple job or SSH into a node where my Ray processes are running.

@rynewang
Copy link
Contributor

Thanks for the investigation in #36936 (comment) ! I think for now the users can set RAY_num_server_call_thread environment variables on their end to unblock. Note this only limits worker.io thread counts; gRPC will still create a lot of threads.

Note to future self: understand if using image_uri runtime env can save this

@anyscalesam
Copy link
Contributor

🤔 > @rynewang maybe an enhancement for the future?

@jjyao jjyao added P2 Important issue, but not time-critical and removed P1 Issue that should be fixed within a few weeks labels Oct 30, 2024
@fersarr
Copy link

fersarr commented Nov 19, 2024

Yes please! an enhancement would be welcome. We still get thousands of threads in our kubernetes pod even with RAY_num_server_call_thread=1

@putdanil
Copy link

putdanil commented Dec 2, 2024

I agree, creates issues with runpod

@jjyao jjyao added P1 Issue that should be fixed within a few weeks and removed P2 Important issue, but not time-critical labels Jan 14, 2025
@FengLi666
Copy link

Any progress? Based on our experience, excessive threads (even with less schedule) could lead to significant performance loss for compute-intensive calls...

@messense
Copy link
Author

@FengLi666 Please consider do a performance comparsion somehow to show the significant performance loss to help prioritize this, otherwise they will just keep telling us to set a high ulimit.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P1 Issue that should be fixed within a few weeks performance
Projects
None yet
Development

No branches or pull requests