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

Worker crashes on ray.get while spilling objects #15808

Closed
kfstorm opened this issue May 14, 2021 · 5 comments · Fixed by #15903
Closed

Worker crashes on ray.get while spilling objects #15808

kfstorm opened this issue May 14, 2021 · 5 comments · Fixed by #15903
Assignees
Labels
bug Something that is supposed to be working; but isn't P1 Issue that should be fixed within a few weeks
Milestone

Comments

@kfstorm
Copy link
Member

kfstorm commented May 14, 2021

What is the problem?

Ray version and other system information (Python version, TensorFlow version, OS): Ant internal repo synced to the community commit 6e19fac.

We found an object spilling bug in our internal code repo and I’m wondering if this has been fixed in the community. I suspect CoreWorker::PlasmaCallback may crash if spilling happens between Contains and Get.

Worker log:

14793[2021-05-13 16:15:55,359 D 106959 106992] core_worker.cc:1267: b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 in plasma, doing fetch-and-get
14794[2021-05-13 16:15:55,360 D 106959 106992] core_worker.cc:1285: Plasma GET timeout 0
14800[2021-05-13 16:15:55,450 C 106959 106992] core_worker.cc:2848:  Check failed: _s.ok() Bad status: TimedOut: Get timed out: some object(s) not ready.
14801[2021-05-13 16:15:55,450 E 106959 106992] logging.cc:419: *** Aborted at 1620893755 (unix time) try "date -d @1620893755" if you are using GNU date ***
14806[2021-05-13 16:15:55,532 E 106959 106992] logging.cc:419: PC: @                0x0 (unknown)
14807[2021-05-13 16:15:55,595 E 106959 106992] logging.cc:419: *** SIGABRT (@0x1f40001a1cf) received by PID 106959 (TID 0x7f51c9ffb700) from PID 106959; stack trace: ***
14808[2021-05-13 16:15:55,666 E 106959 106992] logging.cc:419:     @     0x7f520bbf35d0 (unknown)
14809[2021-05-13 16:15:55,741 E 106959 106992] logging.cc:419:     @     0x7f520b1418af __GI_raise
14810[2021-05-13 16:15:55,820 E 106959 106992] logging.cc:419:     @     0x7f520b1434aa __GI_abort
14811[2021-05-13 16:15:55,883 E 106959 106992] logging.cc:419:     @     0x7f5203fbf9cb ray::RayLog::~RayLog()
14812[2021-05-13 16:15:55,960 E 106959 106992] logging.cc:419:     @     0x7f52038ec284 ray::CoreWorker::PlasmaCallback()
14813[2021-05-13 16:15:56,036 E 106959 106992] logging.cc:419:     @     0x7f5203873e17 _ZNSt17_Function_handlerIFvSt10shared_ptrIN3ray9RayObjectEEEZNS1_10CoreWorker8GetAsyncERKNS1_8ObjectIDESt8functionIFvS3_S6_PvEESA_EUlS3_E_E9_M_invokeERKSt9_Any_dataOS3_
14823[2021-05-13 16:15:56,106 E 106959 106992] logging.cc:419:     @     0x7f520395223a ray::CoreWorkerMemoryStore::Put()
14824[2021-05-13 16:15:56,182 E 106959 106992] logging.cc:419:     @     0x7f52039735bb _ZNSt17_Function_handlerIFvRKN3ray6StatusERKNS0_3rpc20GetObjectStatusReplyEEZNS0_14FutureResolver18ResolveFutureAsyncERKNS0_8ObjectIDERKNS4_7AddressEEUlS3_S7_E_E9_M_invokeERKSt9_Any_dataS3_S7_
14826[2021-05-13 16:15:56,257 E 106959 106992] logging.cc:419:     @     0x7f52038e0b02 _ZN5boost4asio6detail18completion_handlerISt5_BindIFZZN3ray3rpc16BrpcStreamClient10CallMethodINS5_22GetObjectStatusRequestENS5_20GetObjectStatusReplyENS5_28CoreWorkerServiceMessageTypeEEEvT1_SB_RKT_RSt8functionIFvRKNS4_6StatusERKT0_EERKSsENKUlSI_RKS9_E_clESI_SS_EUlRSF_IFvSI_SS_EEE_SV_EEE11do_completeEPvPNS1_19scheduler_operationERKNS_6system10error_codeEm
14834[2021-05-13 16:15:56,326 E 106959 106992] logging.cc:419:     @     0x7f52042c7089 boost::asio::detail::scheduler::do_run_one()
14835[2021-05-13 16:15:56,406 E 106959 106992] logging.cc:419:     @     0x7f52042c8271 boost::asio::detail::scheduler::run()
14836[2021-05-13 16:15:56,492 E 106959 106992] logging.cc:419:     @     0x7f52042c8c4f boost::asio::io_context::run()
14837[2021-05-13 16:15:56,563 E 106959 106992] logging.cc:419:     @     0x7f520385fd90 ray::CoreWorker::RunIOService()
14838[2021-05-13 16:15:56,627 E 106959 106992] logging.cc:419:     @     0x7f5202dae79f (unknown)
14848[2021-05-13 16:15:56,690 E 106959 106992] logging.cc:419:     @     0x7f520bbe96ca start_thread
14849[2021-05-13 16:15:56,764 E 106959 106992] logging.cc:419:     @     0x7f520b213edf __GI___clone

Raylet log greped by b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000:

208300[2021-05-13 16:15:54,898 D 1747 1817] store.cc:1006: Received create request for object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 assigned request ID 2867
208301[2021-05-13 16:15:54,898 D 1747 1817] store.cc:300: creating object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 size 974
208302[2021-05-13 16:15:54,898 D 1747 1817] store.cc:1156: Finishing create object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 request ID 2867
208303[2021-05-13 16:15:54,899 D 1747 1817] store.cc:681: sealing 1 objects b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, 
208304[2021-05-13 16:15:54,899 D 1747 1817] store.cc:684: sealing object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208305[2021-05-13 16:15:54,899 D 1747 1747] object_manager.cc:154: Object added b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208306[2021-05-13 16:15:54,899 D 1747 1747] object_directory.cc:76: Reporting object added to GCS b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208307[2021-05-13 16:15:54,899 D 1747 1747] service_based_accessor.cc:1213: Adding object location, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, node id = f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, job id = 3a107fe9
208308[2021-05-13 16:15:54,900 D 1747 1747] node_manager.cc:3137: Object local b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000,  on f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, 0 tasks ready
208309[2021-05-13 16:15:54,900 D 1747 1747] local_object_manager.cc:34: Pinning object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208313[2021-05-13 16:15:54,901 D 1747 1747] service_based_accessor.cc:1229: Finished adding object location, status = OK, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, node id = f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, job id = 3a107fe9
208314[2021-05-13 16:15:54,902 D 1747 1817] store.cc:656: releasing object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208405[2021-05-13 16:15:55,098 D 1747 1747] local_object_manager.cc:201: Spilling object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208419[2021-05-13 16:15:55,099 D 1747 1747] local_object_manager.cc:219: Sending spill request for object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208631[2021-05-13 16:15:55,247 D 1747 1817] store.cc:656: releasing object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208690[2021-05-13 16:15:55,264 D 1747 1747] local_object_manager.cc:257: Object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 spilled at /home/admin/ray-pack/tmp/spill/ray_spilled_object-36fc554dc1083c4142f15f3382855d2d605941e4696c88b42b000000-multi-14?offset=64007806&size=996
208691[2021-05-13 16:15:55,264 D 1747 1747] service_based_accessor.cc:1243: Adding object spilled location, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, spilled_url = /home/admin/ray-pack/tmp/spill/ray_spilled_object-36fc554dc1083c4142f15f3382855d2d605941e4696c88b42b000000-multi-14?offset=64007806&size=996, job id = 3a107fe9
208722[2021-05-13 16:15:55,269 D 1747 1747] client.cc:606: called plasma_release on conn 38 with object id b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208754[2021-05-13 16:15:55,277 D 1747 1817] store.cc:656: releasing object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208755[2021-05-13 16:15:55,278 D 1747 1817] store.cc:622: Releasing object no longer in use b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208886[2021-05-13 16:15:55,360 D 1747 1817] store.cc:787: evicting 6 objects b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, 36fc554dc1083c4142f15f3382855d2d605941e4696c88b422000000, 999a0b110aa7ed5d2dce8164b7db13ddc62fa5593710fe7929000000, 999a0b110aa7ed5d2dce8164b7db13ddc62fa5593710fe7922000000, 787d69af0692d1dbba0063c16d3e77fab6802c1d0adc225e1e000000, 36fc554dc1083c4142f15f3382855d2d605941e4696c88b424000000, 
208887[2021-05-13 16:15:55,360 D 1747 1817] store.cc:792: evicting object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208891[2021-05-13 16:15:55,360 D 1747 1747] object_directory.cc:85: Reporting object removed to GCS b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208892[2021-05-13 16:15:55,360 D 1747 1747] service_based_accessor.cc:1268: Removing object location, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, node id = f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, job id = 3a107fe9
208895[2021-05-13 16:15:55,361 D 1747 1747] node_manager.cc:3195: Object missing b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000,  on f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, 0 tasks waiting
208913[2021-05-13 16:15:55,364 D 1747 1747] service_based_accessor.cc:1283: Finished removing object location, status = OK, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, node id = f9b9d102f3806eeb62ec286fcf2bd0aca3c6157b8ebe375127d58427, job id = 3a107fe9
208919[2021-05-13 16:15:55,365 D 1747 1747] task_dependency_manager.cc:185: Task ffffffffffffffffd9c1a74367c096f0242961ee24000080 blocked on object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208920[2021-05-13 16:15:55,365 D 1747 1747] pull_manager.cc:19: Pull  of object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208921[2021-05-13 16:15:55,366 D 1747 1747] reconstruction_policy.cc:216: Listening and maybe reconstructing object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000
208925[2021-05-13 16:15:55,367 D 1747 1747] service_based_accessor.cc:1378: Unsubscribing object location, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, job id = 3a107fe9
208927[2021-05-13 16:15:55,368 D 1747 1747] service_based_accessor.cc:1384: Finished unsubscribing object location, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, job id = 3a107fe9
208929[2021-05-13 16:15:55,368 D 1747 1747] reconstruction_policy.cc:232: Reconstruction for object b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000 canceled
208938[2021-05-13 16:15:55,371 D 1747 1747] service_based_accessor.cc:1177: Getting object locations, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, job id = 3a107fe9
208941[2021-05-13 16:15:55,372 D 1747 1747] service_based_accessor.cc:1185: Finished getting object locations, status = OK, object id = b03458cf901098282ca68956c65eedcd9c2d89d43a107fe92b000000, job id = 3a107fe9

Worker crashed here:

RAY_CHECK_OK(Get(std::vector<ObjectID>{object_id}, 0, &vec));

Reproduction (REQUIRED)

Unfortunately I can't find a way to reproduce this issue.

@kfstorm kfstorm 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 14, 2021
@kfstorm kfstorm changed the title Worker crashes on ray.get while spilling Worker crashes on ray.get while spilling objects May 14, 2021
@rkooo567 rkooo567 added this to the Core Bugs milestone May 14, 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 14, 2021
@rkooo567
Copy link
Contributor

Btw, do you know how to reproduce it? I assume if we run object spilling workload with async actor, it can be pretty easy to invoke, but it might be great if we can have a script (so that we can have unit tests)

@kfstorm
Copy link
Member Author

kfstorm commented May 14, 2021

Ah, I didn't realize that I need to use an async actor. Now I can reproduce it with the community version. Here is the repro script. I just modified test_spill_during_get to use async APIs.

@pytest.mark.skipif(
    platform.system() == "Windows", reason="Failing on Windows.")
@pytest.mark.asyncio
async def test_spill_during_get(object_spilling_config, shutdown_only):
    object_spilling_config, _ = object_spilling_config
    address = ray.init(
        num_cpus=4,
        object_store_memory=100 * 1024 * 1024,
        _system_config={
            "automatic_object_spilling_enabled": True,
            "object_store_full_delay_ms": 100,
            "max_io_workers": 1,
            "object_spilling_config": object_spilling_config,
            "min_spilling_size": 0,
        },
    )

    @ray.remote
    class Actor:
        async def f(self):
            return np.zeros(10 * 1024 * 1024)

    a = Actor.remote()
    ids = []
    for i in range(10):
        x = a.f.remote()
        print(i, x)
        ids.append(x)

    # Concurrent gets, which require restoring from external storage, while
    # objects are being created.
    for x in ids:
        print((await x).shape)
    assert_no_thrashing(address["redis_address"])

Note that we need to add a sleep statement to easily reproduce the issue. I don't know how to mock this gap in test.

diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc
index dcd1135c91..657bdc3338 100644
--- a/src/ray/core_worker/core_worker.cc
+++ b/src/ray/core_worker/core_worker.cc
@@ -2832,6 +2832,7 @@ void CoreWorker::PlasmaCallback(SetResultCallback success,
   bool object_is_local = false;
   if (Contains(object_id, &object_is_local).ok() && object_is_local) {
     std::vector<std::shared_ptr<RayObject>> vec;
+    std::this_thread::sleep_for(std::chrono::seconds(10));
     RAY_CHECK_OK(Get(std::vector<ObjectID>{object_id}, 0, &vec));
     RAY_CHECK(vec.size() > 0)
         << "Failed to get local object but Raylet notified object is local.";

@rkooo567
Copy link
Contributor

Hmm I see. This seems to be pretty hard to test it... maybe just adding async actor-based object spilling + manual verification is the only thing we can do.

@ericl ericl assigned rkooo567 and unassigned rkooo567 May 17, 2021
@ericl
Copy link
Contributor

ericl commented May 17, 2021

@kfstorm could you take a look at this first?

@kfstorm
Copy link
Member Author

kfstorm commented May 18, 2021

@Catch-Bull and I are working on this.

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 P1 Issue that should be fixed within a few weeks
Projects
None yet
3 participants