-
Notifications
You must be signed in to change notification settings - Fork 6k
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
Ray 1.3.0 CHECK fail during normal task scheduling #15990
Comments
+1, yeah actually I get segfaults on Github actions often, and this has been the case in previous Ray versions too. |
Thanks @gshimansky for opening, and for creating a reproducer. I also want to mention that |
I suspect this is fixed with #14182 Can you reproduce the issue on the latest nightly wheels with RAY_PREALLOCATE_PLASMA_MEMORY=1? If so, I can take a look. |
Thanks @ericl for the response. I did as you suggest, output logs can be found here: https://github.com/modin-project/modin/pull/3011/checks?check_run_id=2667075822. The failure persists. I would like to understand why this was marked as P2. It is clearly a stability issue, and is preventing us from upgrading. |
@ericl Can you make this P1 priority? |
I ran my reproducer VM with ray 2.0.0.dev0 both enabling |
I built Ray with debug info, but it looks to me that abort happens because of crash in some different worker, so stack trace doesn't tell much. Anyway, with debug info it looks like this: [Switching to Thread 0x7fffc27fc700 (LWP 29973)] __GI_raise (sig=sig@entry=6) at ../sysdeps/unix/sysv/linux/raise.c:50 50 ../sysdeps/unix/sysv/linux/raise.c: No such file or directory. (gdb) bt #0 __GI_raise (sig=sig@entry=6) at ../sysdeps/unix/sysv/linux/raise.c:50 #1 0x00007ffff7c6c859 in __GI_abort () at abort.c:79 #2 0x00007fffdca1e5ca in ray::SpdLogMessage::Flush (this=0x7fff44018610) at src/ray/util/logging.cc:152 #3 0x00007fffdca1e676 in ray::SpdLogMessage::~SpdLogMessage (this=0x7fff44018610, __in_chrg=) at src/ray/util/logging.cc:156 #4 0x00007fffdca150e1 in ray::RayLog::~RayLog (this=0x7fffc27fb7b0, __in_chrg=) at src/ray/util/logging.cc:508 #5 0x00007fffdc422664 in ray::CoreWorkerDirectTaskSubmitter::::operator()(const ray::Status &, const ray::rpc::RequestWorkerLeaseReply &) const (__closure=0x7fff44027220, status=..., reply=...) at src/ray/core_worker/transport/direct_task_transport.cc:341 #6 0x00007fffdc425116 in std::_Function_handler >::_M_invoke(const std::_Any_data &, const ray::Status &, const ray::rpc::RequestWorkerLeaseReply &) (__functor=..., __args#0=..., __args#1=...) at /usr/include/c++/9/bits/std_function.h:300 #7 0x00007fffdc4f98b1 in std::function::operator()(ray::Status const&, ray::rpc::RequestWorkerLeaseReply const&) const (this=0x7fff4402b0c0, __args#0=..., __args#1=...) at /usr/include/c++/9/bits/std_function.h:688 #8 0x00007fffdc4f8811 in ray::rpc::ClientCallImpl::OnReplyReceived (this=0x7fff4402b070) at bazel-out/k8-dbg/bin/_virtual_includes/grpc_common_lib/ray/rpc/client_call.h:89 #9 0x00007fffdc316c31 in ray::rpc::ClientCallManager::PollEventsFromCompletionQueue(int)::{lambda()#1}::operator()() const ( __closure=0x7fffc27fbb30) at bazel-out/k8-dbg/bin/_virtual_includes/grpc_common_lib/ray/rpc/client_call.h:266 #10 0x00007fffdc33e858 in std::_Function_handler::_M_invoke(std::_Any_data const&) (__functor=...) at /usr/include/c++/9/bits/std_function.h:300 #11 0x00007fffdc273b8a in std::function::operator()() const (this=0x7fffc27fbb30) at /usr/include/c++/9/bits/std_function.h:688 #12 0x00007fffdc9b89d8 in boost::asio::asio_handler_invoke >(std::function&, ...) (function=...) at external/boost/boost/asio/handler_invoke_hook.hpp:69 #13 0x00007fffdc9b8153 in boost_asio_handler_invoke_helpers::invoke, std::function >(std::function&, std::function&) (function=..., context=...) at external/boost/boost/asio/detail/handler_invoke_helpers.hpp:37 #14 0x00007fffdc9b7829 in boost::asio::detail::handler_work, boost::asio::system_executor, boost::asio::system_executor>::complete >(std::function&, std::function&) (this=0x7fffc27fbb06, function=..., handler=...) at external/boost/boost/asio/detail/handler_work.hpp:100 #15 0x00007fffdc9b642f in boost::asio::detail::completion_handler >::do_complete(void*, boost::asio::detail::scheduler_operation*, boost::system::error_code const&, unsigned long) (owner=0x555556ab7ca0, base=0x7fffcc00d070) at external/boost/boost/asio/detail/completion_handler.hpp:70 #16 0x00007fffdcbb240e in boost::asio::detail::scheduler_operation::complete (this=0x7fffcc00d070, owner=0x555556ab7ca0, ec=..., bytes_transferred=0) at external/boost/boost/asio/detail/scheduler_operation.hpp:40 #17 0x00007fffdcba6727 in boost::asio::detail::scheduler::do_run_one (this=0x555556ab7ca0, lock=..., this_thread=..., ec=...) at external/boost/boost/asio/detail/impl/scheduler.ipp:447 #18 0x00007fffdcba5a14 in boost::asio::detail::scheduler::run (this=0x555556ab7ca0, ec=...) at external/boost/boost/asio/detail/impl/scheduler.ipp:200 #19 0x00007fffdcb9ed3f in boost::asio::io_context::run (this=0x555556d54c00) at external/boost/boost/asio/impl/io_context.ipp:63 #20 0x00007fffdc2d0ddd in ray::CoreWorker::RunIOService (this=0x555556d547b0) at src/ray/core_worker/core_worker.cc:832 #21 0x00007fffdc2cca1d in ray::CoreWorker::::operator()(void) const (__closure=0x5555572fbc68) at src/ray/core_worker/core_worker.cc:706 #22 0x00007fffdc2f94a3 in std::__invoke_impl >(std::__invoke_other, ray::CoreWorker:: &&) (__f=...) at /usr/include/c++/9/bits/invoke.h:60 #23 0x00007fffdc2f93ba in std::__invoke >(ray::CoreWorker:: &&) (__fn=...) at /usr/include/c++/9/bits/invoke.h:95 #24 0x00007fffdc2f92e8 in std::thread::_Invoker > >::_M_invoke<0>(std::_Index_tuple<0>) (this=0x5555572fbc68) at /usr/include/c++/9/thread:244 #25 0x00007fffdc2f9278 in std::thread::_Invoker > >::operator()(void) (this=0x5555572fbc68) at /usr/include/c++/9/thread:251 #26 0x00007fffdc2f921e in std::thread::_State_impl > > >::_M_run(void) (this=0x5555572fbc60) at /usr/include/c++/9/thread:195 #27 0x00007fffe9707067 in std::execute_native_thread_routine (__p=0x5555572fbc60) at /home/conda/feedstock_root/build_artifacts/ctng-compilers_1618239181388/work/.build/x86_64-conda-linux-gnu/src/gcc/libstdc++-v3/src/c++11/thread.cc:80 #28 0x00007ffff7fa7609 in start_thread (arg=) at pthread_create.c:477 #29 0x00007ffff7d69293 in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95 I ran session with RAY_LOG_LEVEL=DEBUG two times. |
Thanks for the debug logs! I found this in the raylet.out, investigating.
|
I just discovered a Ray debug page and ran it again with RAY_BACKEND_LOG_LEVEL=debug for logs: Thread 1 "raylet" received signal SIGABRT, Aborted. __GI_raise (sig=sig@entry=6) at ../sysdeps/unix/sysv/linux/raise.c:50 50 ../sysdeps/unix/sysv/linux/raise.c: No such file or directory. (gdb) bt #0 __GI_raise (sig=sig@entry=6) at ../sysdeps/unix/sysv/linux/raise.c:50 #1 0x00007ffff7a7b859 in __GI_abort () at abort.c:79 #2 0x0000555555dd0d3c in ray::SpdLogMessage::Flush (this=0x55555716bcf0) at src/ray/util/logging.cc:152 #3 0x0000555555dd0de8 in ray::SpdLogMessage::~SpdLogMessage (this=0x55555716bcf0, __in_chrg=) at src/ray/util/logging.cc:156 #4 0x0000555555dc7795 in ray::RayLog::~RayLog (this=0x7fffffffc490, __in_chrg=) at src/ray/util/logging.cc:508 #5 0x00005555558cb75a in ray::PullManager::CancelPull (this=0x555556e48930, request_id=3) at src/ray/object_manager/pull_manager.cc:338 #6 0x0000555555889afe in ray::ObjectManager::CancelPull (this=0x555556e46508, request_id=3) at src/ray/object_manager/object_manager.cc:237 #7 0x000055555572b6ca in ray::raylet::DependencyManager::RemoveTaskDependencies (this=0x555556e46eb0, task_id=...) at src/ray/raylet/dependency_manager.cc:196 #8 0x00005555557fd3b4 in ray::raylet::ClusterTaskManager::DispatchScheduledTasksToWorkers (this=0x555556e218b0, worker_pool=..., Python Exception No type named std::__detail::_Hash_node >, true>.: leased_workers=std::unordered_map with 2 elements) at src/ray/raylet/scheduling/cluster_task_manager.cc:249 #9 0x00005555558036ef in ray::raylet::ClusterTaskManager::ScheduleAndDispatchTasks (this=0x555556e218b0) at src/ray/raylet/scheduling/cluster_task_manager.cc:1028 #10 0x000055555574ea11 in ray::raylet::NodeManager::HandleWorkerAvailable (this=0x555556e464c8, worker=std::shared_ptr (use count 3, weak count 0) = {...}) at src/ray/raylet/node_manager.cc:1142 #11 0x00005555557539df in ray::raylet::NodeManager::HandleReturnWorker(ray::rpc::ReturnWorkerRequest const&, ray::rpc::ReturnWorkerReply*, std::function, std::function)>) (this=0x555556e464c8, request=..., reply=0x555557225668, send_reply_callback=...) at src/ray/raylet/node_manager.cc:1613 #12 0x00005555556c98d3 in ray::rpc::ServerCallImpl::HandleRequestImpl (this=0x555557224f40) at bazel-out/k8-dbg/bin/_virtual_includes/grpc_common_lib/ray/rpc/server_call.h:162 #13 0x00005555556bfecb in ray::rpc::ServerCallImpl::HandleRequest()::{lambda()#1}::operator()() const (this=0x555557224f40) at bazel-out/k8-dbg/bin/_virtual_includes/grpc_common_lib/ray/rpc/server_call.h:144 #14 0x00005555556cece3 in std::_Function_handler::HandleRequest()::{lambda()#1}>::_M_invoke(std::_Any_data const&) (__functor=...) at /usr/include/c++/9/bits/std_function.h:300 #15 0x0000555555694800 in std::function::operator()() const (this=0x7fffffffcda0) at /usr/include/c++/9/bits/std_function.h:688 #16 0x0000555555d6b8ee in boost::asio::asio_handler_invoke >(std::function&, ...) (function=...) at external/boost/boost/asio/handler_invoke_hook.hpp:69 #17 0x0000555555d6b069 in boost_asio_handler_invoke_helpers::invoke, std::function >(std::function&, std::function&) (function=..., context=...) at external/boost/boost/asio/detail/handler_invoke_helpers.hpp:37 #18 0x0000555555d6a73f in boost::asio::detail::handler_work, boost::asio::system_executor, boost::asio::system_executor>::complete >(std::function&, std::function&) (this=0x7fffffffcd76, function=..., handler=...) at external/boost/boost/asio/detail/handler_work.hpp:100 #19 0x0000555555d69345 in boost::asio::detail::completion_handler >::do_complete(void*, boost::asio::detail::scheduler_operation*, boost::system::error_code const&, unsigned long) (owner=0x555556e21b90, base=0x555557679e60) at external/boost/boost/asio/detail/completion_handler.hpp:70 #20 0x00005555564d2d0a in boost::asio::detail::scheduler_operation::complete (this=0x555557679e60, owner=0x555556e21b90, ec=..., bytes_transferred=0) at external/boost/boost/asio/detail/scheduler_operation.hpp:40 #21 0x00005555564c7023 in boost::asio::detail::scheduler::do_run_one (this=0x555556e21b90, lock=..., this_thread=..., ec=...) at external/boost/boost/asio/detail/impl/scheduler.ipp:447 #22 0x00005555564c6310 in boost::asio::detail::scheduler::run (this=0x555556e21b90, ec=...) at external/boost/boost/asio/detail/impl/scheduler.ipp:200 #23 0x00005555564bf63b in boost::asio::io_context::run (this=0x7fffffffd090) at external/boost/boost/asio/impl/io_context.ipp:63 #24 0x0000555555650539 in main (argc=1, argv=0x7fffffffd7d8) at src/ray/raylet/main.cc:280 |
Thanks! I'm also able to reproduce locally now with |
Thanks, turns out the issue was due to tasks requesting the same object id multiple times, which triggered the check failure. It only happened in a particular edge case where there were multiple tasks requesting the same object multiple times each. The smoking gun from the debug logs:
Note ffffffffffffffffffffffffffffffffffffffff0100000003000000 was removed twice. |
Did it happen because of constrained memory? |
It seems to be because ffffffffffffffffffffffffffffffffffffffff0100000003000000 was passed twice as an argument to the task (this is allowed in the Ray API, just a bit unusual so triggered this buggy edge case). I'm not sure how constrained memory triggers the bug exactly, in principle this can happen in any setting. |
I ran problematic Modin tests with your patch and results are mixed. While
|
Attached are logs for the run with raylet abort with this commit 45db24a |
@gshimansky I pushed a speculative fix to that branch which should fix this as well, could you try it out? |
With this fix I ran both tests 10 times and had no crashed any more. It looks like it works at last! |
Thanks for prioritizing @ericl! |
Thank you for a quick resolution of this problem. I checked merged master branch and problematic tests continue to pass normally on my test VM. |
Thanks, guys! |
What is the problem?
While transitioning Modin to Ray version 1.3.0 we have several tests crash in Github Actions CI. Crashes could not be reproduced in development environment until I tried to create a VM with the same specs as Github Actions run: https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners
Running such VM with 2 CPU cores and 7GB of RAM stably reproduces these crashes.
Ray version and other system information (Python version, TensorFlow version, OS):
Ray 1.3.0. Ubuntu 20.04.2 LTS.
Reproduction (REQUIRED)
I created a Vagrantfile to easily create and provision a reproducer VM:
Vagrantfile.gz
To use it follow these steps:
3.1. Install VirtualBox from https://www.virtualbox.org/wiki/Linux_Downloads
3.2. Alternatively you can use KVM. I checked that both VMs produce the same result. To use KVM you need to install
vagrant-libvirt
plugin byvagrant plugin install vagrant-libvirt
. It requires a bunch of dependencies which can be found here https://github.com/vagrant-libvirt/vagrant-libvirt. Also since my /var/lib filesystem is not large enough, I set up VMs to use animages
pool which can be created and activated like this:vboxusers
orlibvirt
group and make sure that setting is effective.vagrant up
in the same directory as Vagrantfile.vagrant ssh ubuntu2004-7gb
.This is a stack trace that I am getting in the crash:
The text was updated successfully, but these errors were encountered: