diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index d160fe2db4776..110172b6ce976 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -2362,6 +2362,10 @@ cdef CRayStatus check_signals() nogil: # The Python exceptions are not handled if it is raised from cdef, # so we have to handle it here. try: + if sys.is_finalizing(): + return CRayStatus.IntentionalSystemExit( + "Python is exiting.".encode("utf-8") + ) PyErr_CheckSignals() except KeyboardInterrupt: return CRayStatus.Interrupted(b"") diff --git a/python/ray/dag/tests/experimental/test_accelerated_dag.py b/python/ray/dag/tests/experimental/test_accelerated_dag.py index 00c476022b8a5..36e9101606cac 100644 --- a/python/ray/dag/tests/experimental/test_accelerated_dag.py +++ b/python/ray/dag/tests/experimental/test_accelerated_dag.py @@ -305,29 +305,25 @@ def test_multi_output_get_exception(ray_start_regular): ray.get(refs) -# TODO(wxdeng): Fix segfault. If this test is run, the following tests -# will segfault. -# def test_two_from_three_returns(ray_start_regular): -# a = Actor.remote(0) -# with InputNode() as i: -# o1, o2 = a.return_two_from_three.bind(i) -# dag = MultiOutputNode([o1, o2]) - -# compiled_dag = dag.experimental_compile() +def test_two_from_three_returns(ray_start_regular): + a = Actor.remote(0) + with InputNode() as i: + o1, o2 = a.return_two_from_three.bind(i) + dag = MultiOutputNode([o1, o2]) -# # A value error is raised because the number of returns is not equal to -# # the number of outputs. Since the value error is raised in the writer, -# # the reader fails to read the outputs and raises a channel error. + compiled_dag = dag.experimental_compile() -# # TODO(wxdeng): Fix exception type. The value error should be catched. -# # However, two exceptions are raised in the writer and reader respectively. + # A value error is raised because the number of returns is not equal to + # the number of outputs. Since the value error is raised in the writer, + # the reader fails to read the outputs and raises a channel error. -# # with pytest.raises(RayChannelError, match="Channel closed."): -# # with pytest.raises(ValueError, match="Expected 2 outputs, but got 3 outputs"): -# with pytest.raises(Exception): -# ray.get(compiled_dag.execute(1)) + # TODO(wxdeng): Fix exception type. The value error should be catched. + # However, two exceptions are raised in the writer and reader respectively. -# compiled_dag.teardown() + # with pytest.raises(RayChannelError, match="Channel closed."): + # with pytest.raises(ValueError, match="Expected 2 outputs, but got 3 outputs"): + with pytest.raises(Exception): + ray.get(compiled_dag.execute(1)) def test_kwargs_not_supported(ray_start_regular): @@ -1116,12 +1112,6 @@ def test_dag_exception_chained(ray_start_regular, capsys): # Can use the DAG after exceptions are thrown. assert ray.get(compiled_dag.execute(1)) == 2 - # Note: somehow the auto triggered teardown() from ray.shutdown() - # does not finish in time for this test, leading to a segfault - # of the following test (likely due to a dangling monitor thread - # upon the new Ray init). - compiled_dag.teardown() - @pytest.mark.parametrize("single_fetch", [True, False]) def test_dag_exception_multi_output(ray_start_regular, single_fetch, capsys):