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

Ray 1.3.0 CHECK fail during normal task scheduling #15990

Closed
1 of 2 tasks
gshimansky opened this issue May 21, 2021 · 20 comments · Fixed by #16063
Closed
1 of 2 tasks

Ray 1.3.0 CHECK fail during normal task scheduling #15990

gshimansky opened this issue May 21, 2021 · 20 comments · Fixed by #16063
Assignees
Labels
bug Something that is supposed to be working; but isn't P0 Issues that should be fixed in short order
Milestone

Comments

@gshimansky
Copy link

gshimansky commented May 21, 2021

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:

  1. You need to have virtualization settings enabled in your BIOS https://bce.berkeley.edu/enabling-virtualization-in-your-pc-bios.html
  2. Install vagrant from https://www.vagrantup.com/downloads
  3. Install VM provider if not installed. By default vagrant uses VirtualBox.
    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 by vagrant 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 an images pool which can be created and activated like this:
virsh pool-define-as images dir --target /localdisk/libvirt
virsh pool-start images
  1. Add your user to vboxusers or libvirt group and make sure that setting is effective.
  2. Run vagrant up in the same directory as Vagrantfile.
  3. To get to VM run vagrant ssh ubuntu2004-7gb.
  4. On VM activate conda environment and run tests command line.

This is a stack trace that I am getting in the crash:

Thread 37 "worker.io" received signal SIGABRT, Aborted.
[Switching to Thread 0x7fffd55fa700 (LWP 16109)]
__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  0x00007ffff7c6d859 in __GI_abort () at abort.c:79
#2  0x00007fffdd32bb05 in ray::SpdLogMessage::Flush() ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#3  0x00007fffdd32bb3d in ray::RayLog::~RayLog() () from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#4  0x00007fffdcf48b6c in ray::CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(std::tuple<int, std::vector<ray::ObjectID, std::allocator<ray::ObjectID> >, ray::ActorID> const&, ray::rpc::Address const*)::{lambda(ray::Status const&, ray::rpc::RequestWorkerLeaseReply const&)#1}::operator()(ray::Status const&, ray::rpc::RequestWorkerLeaseReply const&) const ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#5  0x00007fffdcf93dd5 in ray::rpc::ClientCallImpl<ray::rpc::RequestWorkerLeaseReply>::OnReplyReceived() ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#6  0x00007fffdce9dacb in std::_Function_handler<void (), ray::rpc::ClientCallManager::PollEventsFromCompletionQueue(int)::{lambda()#1}>::_M_invoke(std::_Any_data const&) () from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#7  0x00007fffdd2daa08 in boost::asio::detail::completion_handler<std::function<void ()> >::do_complete(void*, boost::asio::detail::scheduler_operation*, boost::system::error_code const&, unsigned long) ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#8  0x00007fffdd3e09a1 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/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#9  0x00007fffdd3e0ad1 in boost::asio::detail::scheduler::run(boost::system::error_code&) ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#10 0x00007fffdd3e25d0 in boost::asio::io_context::run() ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#11 0x00007fffdce9b895 in ray::CoreWorker::RunIOService() ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#12 0x00007fffdd685d10 in execute_native_thread_routine ()
   from /home/vagrant/miniconda3/envs/modin/lib/python3.8/site-packages/ray/_raylet.so
#13 0x00007ffff7fa8609 in start_thread (arg=<optimized out>) at pthread_create.c:477
#14 0x00007ffff7d6a293 in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95
  • I have verified my script runs in a clean environment and reproduces the issue.
  • I have verified the issue also occurs with the latest wheels.
@gshimansky gshimansky added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels May 21, 2021
@richardliaw
Copy link
Contributor

+1, yeah actually I get segfaults on Github actions often, and this has been the case in previous Ray versions too.

@devin-petersohn
Copy link
Member

Thanks @gshimansky for opening, and for creating a reproducer. I also want to mention that ray.init arg combinations do not change anything.

@rkooo567 rkooo567 added this to the Core Bugs milestone May 21, 2021
@rkooo567 rkooo567 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 May 21, 2021
@ericl ericl added P2 Important issue, but not time-critical and removed P1 Issue that should be fixed within a few weeks labels May 25, 2021
@ericl
Copy link
Contributor

ericl commented May 25, 2021

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.

@ericl ericl self-assigned this May 25, 2021
@devin-petersohn
Copy link
Member

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.

@aregm
Copy link

aregm commented May 25, 2021

@ericl Can you make this P1 priority?

@gshimansky
Copy link
Author

I ran my reproducer VM with ray 2.0.0.dev0 both enabling RAY_PREALLOCATE_PLASMA_MEMORY=1 and without it. Crash still happens and stack trace looks identical to that of version 1.3.0.
I also tried enabling swapon -a which turns on 2GB of swap (default setting for this ubuntu VM), but it doesn't make a difference either and crash still happens.

@ericl ericl added P1 Issue that should be fixed within a few weeks and removed P2 Important issue, but not time-critical labels May 25, 2021
@gshimansky
Copy link
Author

gshimansky commented May 25, 2021

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.
First time without RAY_PREALLOCATE_PLASMA_MEMORY=1:
session_2021-05-25_17-17-08_857253_30048.tar.gz
and second time with RAY_PREALLOCATE_PLASMA_MEMORY=1:
session_2021-05-25_17-19-09_608357_30240.tar.gz

@ericl
Copy link
Contributor

ericl commented May 25, 2021

Thanks for the debug logs! I found this in the raylet.out, investigating.

[2021-05-25 17:19:12,506 C 30268 30268] pull_manager.cc:338:  Check failed: it->second.bundle_request_ids.erase(bundle_it->first) 
[2021-05-25 17:19:12,506 E 30268 30268] logging.cc:441: *** Aborted at 1621963152 (unix time) try "date -d @1621963152" if you are using GNU date ***
[2021-05-25 17:19:12,506 E 30268 30268] logging.cc:441: PC: @                0x0 (unknown)
[2021-05-25 17:19:12,507 E 30268 30268] logging.cc:441: *** SIGABRT (@0x3e80000763c) received by PID 30268 (TID 0x7f79c0265840) from PID 30268; stack trace: ***

@gshimansky
Copy link
Author

I just discovered a Ray debug page and ran it again with RAY_BACKEND_LOG_LEVEL=debug for logs:
session_2021-05-25_18-21-46_798873_30958.tar.gz
and RAY_RAYLET_GDB=1 RAY_RAYLET_TMUX=1 for worker stack trace:

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

@ericl
Copy link
Contributor

ericl commented May 25, 2021

Thanks! I'm also able to reproduce locally now with ray.init(object_store_memory=1e9).

@ericl ericl added P0 Issues that should be fixed in short order and removed P1 Issue that should be fixed within a few weeks labels May 25, 2021
@ericl ericl changed the title Ray 1.3.0 crashes (aborts) when ran in constrained resources Ray 1.3.0 CHECK fail during normal task scheduling May 25, 2021
@ericl
Copy link
Contributor

ericl commented May 25, 2021

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:

[2021-05-25 18:21:49,655 D 30986 30986] pull_manager.cc:301: Cancel pull request 3
[2021-05-25 18:21:49,655 D 30986 30986] pull_manager.cc:331: Removing an object pull request of id: ffffffffffffffffffffffffffffffffffffffff0100000003000000
[2021-05-25 18:21:49,655 D 30986 30986] pull_manager.cc:331: Removing an object pull request of id: ffffffffffffffffffffffffffffffffffffffff0100000009000000
[2021-05-25 18:21:49,655 D 30986 30986] pull_manager.cc:331: Removing an object pull request of id: ffffffffffffffffffffffffffffffffffffffff0100000003000000

Note ffffffffffffffffffffffffffffffffffffffff0100000003000000 was removed twice.

@gshimansky
Copy link
Author

Note ffffffffffffffffffffffffffffffffffffffff0100000003000000 was removed twice.

Did it happen because of constrained memory?

@ericl
Copy link
Contributor

ericl commented May 25, 2021

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.

@gshimansky
Copy link
Author

I ran problematic Modin tests with your patch and results are mixed. While modin/pandas/test/dataframe/test_binary.py::test_math_functions_fill_value[add-None-check_different_index] seems to be always passing now, modin/pandas/test/test_groupby.py::test_agg_func_None_rename[True-by_and_agg_dict2] passes, but at the same time outputs some abort log about 80% of times that I run it:

modin/pandas/test/test_groupby.py .                                                                                                  [100%](raylet) [2021-05-25 19:50:58,648 C 33291 33291] pull_manager.cc:413:  Check failed: it != object_pull_requests_.end() 
(raylet) [2021-05-25 19:50:58,648 E 33291 33291] logging.cc:441: *** Aborted at 1621972258 (unix time) try "date -d @1621972258" if you are using GNU date ***
(raylet) [2021-05-25 19:50:58,648 E 33291 33291] logging.cc:441: PC: @                0x0 (unknown)
(raylet) [2021-05-25 19:50:58,648 E 33291 33291] logging.cc:441: *** SIGABRT (@0x3e80000820b) received by PID 33291 (TID 0x7f2e78a0c840) from PID 33291; stack trace: ***
(raylet) [2021-05-25 19:50:58,649 E 33291 33291] logging.cc:441:     @     0x7f2e78c3e3c0 (unknown)
(raylet) [2021-05-25 19:50:58,649 E 33291 33291] logging.cc:441:     @     0x7f2e78a5718b gsignal
(raylet) [2021-05-25 19:50:58,649 E 33291 33291] logging.cc:441:     @     0x7f2e78a36859 abort
(raylet) [2021-05-25 19:50:58,651 E 33291 33291] logging.cc:441:     @     0x55f901ea4c9e ray::SpdLogMessage::Flush()
(raylet) [2021-05-25 19:50:58,654 E 33291 33291] logging.cc:441:     @     0x55f901ea4d4a ray::SpdLogMessage::~SpdLogMessage()
(raylet) [2021-05-25 19:50:58,656 E 33291 33291] logging.cc:441:     @     0x55f901e9b6f7 ray::RayLog::~RayLog()
(raylet) [2021-05-25 19:50:58,660 E 33291 33291] logging.cc:441:     @     0x55f9019a00d2 ray::PullManager::TryToMakeObjectLocal()
(raylet) [2021-05-25 19:50:58,664 E 33291 33291] logging.cc:441:     @     0x55f9019a0d11 ray::PullManager::Tick()
(raylet) [2021-05-25 19:50:58,667 E 33291 33291] logging.cc:441:     @     0x55f90196534d ray::ObjectManager::Tick()
(raylet) [2021-05-25 19:50:58,668 E 33291 33291] logging.cc:441:     @     0x55f90196521a _ZZN3ray13ObjectManager4TickERKN5boost6system10error_codeEENKUlS5_E0_clES5_
(raylet) [2021-05-25 19:50:58,668 E 33291 33291] logging.cc:441:     @     0x55f90196ead5 _ZN5boost4asio6detail7binder1IZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlS8_E0_S6_EclEv
(raylet) [2021-05-25 19:50:58,668 E 33291 33291] logging.cc:441:     @     0x55f90196e928 _ZN5boost4asio6detail17executor_functionINS1_7binder1IZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlS9_E0_S7_EESaIvEE11do_completeEPNS1_22executor_function_baseEb
(raylet) [2021-05-25 19:50:58,672 E 33291 33291] logging.cc:441:     @     0x55f90172d6e9 boost::asio::detail::executor_function_base::complete()
(raylet) [2021-05-25 19:50:58,678 E 33291 33291] logging.cc:441:     @     0x55f90172d7da boost::asio::executor::function::operator()()
(raylet) [2021-05-25 19:50:58,682 E 33291 33291] logging.cc:441:     @     0x55f90175b11e boost::asio::asio_handler_invoke<>()
(raylet) [2021-05-25 19:50:58,686 E 33291 33291] logging.cc:441:     @     0x55f90175afe6 boost_asio_handler_invoke_helpers::invoke<>()
(raylet) [2021-05-25 19:50:58,692 E 33291 33291] logging.cc:441:     @     0x55f90175ab37 boost::asio::io_context::executor_type::dispatch<>()
(raylet) [2021-05-25 19:50:58,697 E 33291 33291] logging.cc:441:     @     0x55f90175a76b boost::asio::executor::impl<>::dispatch()
(raylet) [2021-05-25 19:50:58,697 E 33291 33291] logging.cc:441:     @     0x55f90196d7e4 _ZNK5boost4asio8executor8dispatchINS0_6detail7binder1IZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlSA_E0_S8_EESaIvEEEvOT_RKT0_
(raylet) [2021-05-25 19:50:58,697 E 33291 33291] logging.cc:441:     @     0x55f90196d196 _ZNK5boost4asio6detail18io_object_executorINS0_8executorEE8dispatchINS1_7binder1IZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlSC_E0_SA_EESaIvEEEvOT_RKT0_
(raylet) [2021-05-25 19:50:58,697 E 33291 33291] logging.cc:441:     @     0x55f90196ce0e _ZN5boost4asio6detail12handler_workIZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlS8_E0_NS1_18io_object_executorINS0_8executorEEESC_E8completeINS1_7binder1IS9_S6_EEEEvRT_RS9_
(raylet) [2021-05-25 19:50:58,697 E 33291 33291] logging.cc:441:     @     0x55f90196c526 _ZN5boost4asio6detail12wait_handlerIZN3ray13ObjectManager4TickERKNS_6system10error_codeEEUlS8_E0_NS1_18io_object_executorINS0_8executorEEEE11do_completeEPvPNS1_19scheduler_operationES8_m
(raylet) [2021-05-25 19:50:58,698 E 33291 33291] logging.cc:441:     @     0x55f9025a6c4a boost::asio::detail::scheduler_operation::complete()
(raylet) [2021-05-25 19:50:58,699 E 33291 33291] logging.cc:441:     @     0x55f90259af63 boost::asio::detail::scheduler::do_run_one()
(raylet) [2021-05-25 19:50:58,700 E 33291 33291] logging.cc:441:     @     0x55f90259a250 boost::asio::detail::scheduler::run()
(raylet) [2021-05-25 19:50:58,705 E 33291 33291] logging.cc:441:     @     0x55f90259357b boost::asio::io_context::run()
(raylet) [2021-05-25 19:50:58,709 E 33291 33291] logging.cc:441:     @     0x55f901724539 main
(raylet) [2021-05-25 19:50:58,709 E 33291 33291] logging.cc:441:     @     0x7f2e78a380b3 __libc_start_main
(raylet) [2021-05-25 19:50:58,714 E 33291 33291] logging.cc:441:     @     0x55f901721aae _start

@gshimansky
Copy link
Author

Attached are logs for the run with raylet abort with this commit 45db24a
session_2021-05-25_19-50-54_942726_33263.tar.gz

@ericl
Copy link
Contributor

ericl commented May 25, 2021

@gshimansky I pushed a speculative fix to that branch which should fix this as well, could you try it out?

@gshimansky
Copy link
Author

With this fix I ran both tests 10 times and had no crashed any more. It looks like it works at last!

@devin-petersohn
Copy link
Member

Thanks for prioritizing @ericl!

@gshimansky
Copy link
Author

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.

@aregm
Copy link

aregm commented May 27, 2021

Thanks, guys!

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 P0 Issues that should be fixed in short order
Projects
None yet
Development

Successfully merging a pull request may close this issue.

6 participants