@@ -4704,36 +4704,74 @@ async def test_html_repr(c, s, a, b):
4704
4704
await f
4705
4705
4706
4706
4707
- @pytest .mark .parametrize ("add_deps " , [False , True ])
4707
+ @pytest .mark .parametrize ("deps " , ["same" , "less" , "more" ])
4708
4708
@gen_cluster (client = True , nthreads = [])
4709
- async def test_resubmit_different_task_same_key (c , s , add_deps ):
4709
+ async def test_resubmit_different_task_same_key_before_previous_is_done (c , s , deps ):
4710
4710
"""If an intermediate key has a different run_spec (either the callable function or
4711
4711
the dependencies / arguments) that will conflict with what was previously defined,
4712
4712
it should raise an error since this can otherwise break in many different places and
4713
4713
cause either spurious exceptions or even deadlocks.
4714
4714
4715
+ In this specific test, the previous run_spec has not been computed yet.
4716
+ See also test_resubmit_different_task_same_key_after_previous_is_done.
4717
+
4715
4718
For a real world example where this can trigger, see
4716
4719
https://github.com/dask/dask/issues/9888
4717
4720
"""
4718
- y1 = c .submit (inc , 1 , key = "y" )
4721
+ x1 = c .submit (inc , 1 , key = "x1" )
4722
+ y_old = c .submit (inc , x1 , key = "y" )
4719
4723
4720
- x = delayed (inc )(1 , dask_key_name = "x" ) if add_deps else 2
4721
- y2 = delayed (inc )(x , dask_key_name = "y" )
4722
- z = delayed (inc )(y2 , dask_key_name = "z" )
4723
-
4724
- if add_deps : # add_deps=True corrupts the state machine
4725
- s .validate = False
4724
+ x1b = x1 if deps != "less" else 2
4725
+ x2 = delayed (inc )(10 , dask_key_name = "x2" ) if deps == "more" else 11
4726
+ y_new = delayed (sum )([x1b , x2 ], dask_key_name = "y" )
4727
+ z = delayed (inc )(y_new , dask_key_name = "z" )
4726
4728
4727
4729
with captured_logger ("distributed.scheduler" , level = logging .WARNING ) as log :
4728
4730
fut = c .compute (z )
4729
4731
await wait_for_state ("z" , "waiting" , s )
4730
4732
4731
4733
assert "Detected different `run_spec` for key 'y'" in log .getvalue ()
4732
4734
4733
- if not add_deps : # add_deps=True hangs
4734
- async with Worker (s .address ):
4735
- assert await y1 == 2
4736
- assert await fut == 3
4735
+ async with Worker (s .address ):
4736
+ # Used old run_spec
4737
+ assert await y_old == 3
4738
+ assert await fut == 4
4739
+
4740
+
4741
+ @pytest .mark .parametrize ("deps" , ["same" , "less" , "more" ])
4742
+ @pytest .mark .parametrize ("release_previous" , [False , True ])
4743
+ @gen_cluster (client = True )
4744
+ async def test_resubmit_different_task_same_key_after_previous_is_done (
4745
+ c , s , a , b , deps , release_previous
4746
+ ):
4747
+ """Same as test_resubmit_different_task_same_key, but now the replaced task has
4748
+ already been computed and is either in memory or released, and so are its old
4749
+ dependencies, so they may need to be recomputed.
4750
+ """
4751
+ x1 = delayed (inc )(1 , dask_key_name = "x1" )
4752
+ x1fut = c .compute (x1 )
4753
+ y_old = c .submit (inc , x1fut , key = "y" )
4754
+ z1 = c .submit (inc , y_old , key = "z1" )
4755
+ await wait (z1 )
4756
+ if release_previous :
4757
+ del x1fut , y_old
4758
+ await wait_for_state ("x1" , "released" , s )
4759
+ await wait_for_state ("y" , "released" , s )
4760
+
4761
+ x1b = x1 if deps != "less" else 2
4762
+ x2 = delayed (inc )(10 , dask_key_name = "x2" ) if deps == "more" else 11
4763
+ y_new = delayed (sum )([x1b , x2 ], dask_key_name = "y" )
4764
+ z2 = delayed (inc )(y_new , dask_key_name = "z2" )
4765
+
4766
+ with captured_logger ("distributed.scheduler" , level = logging .WARNING ) as log :
4767
+ fut = c .compute (z2 )
4768
+ # Used old run_spec
4769
+ assert await fut == 4
4770
+ assert "x2" not in s .tasks
4771
+
4772
+ # _generate_taskstates won't run for a dependency that's already in memory
4773
+ has_warning = "Detected different `run_spec` for key 'y'" in log .getvalue ()
4774
+ assert has_warning is (release_previous or deps == "less" )
4737
4775
4738
4776
4739
4777
@gen_cluster (client = True , nthreads = [])
@@ -4801,21 +4839,17 @@ async def test_resubmit_nondeterministic_task_different_deps(c, s, add_deps):
4801
4839
y2 = delayed (lambda i , j : i )(x2 , o , dask_key_name = "y" )
4802
4840
z = delayed (inc )(y2 , dask_key_name = "z" )
4803
4841
4804
- if add_deps : # add_deps=True corrupts the state machine and hangs
4805
- s .validate = False
4806
-
4807
4842
with captured_logger ("distributed.scheduler" , level = logging .WARNING ) as log :
4808
4843
fut = c .compute (z )
4809
4844
await wait_for_state ("z" , "waiting" , s )
4810
4845
assert "Detected different `run_spec` for key 'y'" in log .getvalue ()
4811
4846
4812
- if not add_deps : # add_deps=True corrupts the state machine and hangs
4813
- async with Worker (s .address ):
4814
- assert await fut == 3
4847
+ async with Worker (s .address ):
4848
+ assert await fut == 3
4815
4849
4816
4850
4817
4851
@pytest .mark .parametrize (
4818
- "loglevel,expect_loglines" , [(logging .DEBUG , 3 ), (logging .WARNING , 1 )]
4852
+ "loglevel,expect_loglines" , [(logging .DEBUG , 2 ), (logging .WARNING , 1 )]
4819
4853
)
4820
4854
@gen_cluster (client = True , nthreads = [])
4821
4855
async def test_resubmit_different_task_same_key_warns_only_once (
@@ -4824,23 +4858,24 @@ async def test_resubmit_different_task_same_key_warns_only_once(
4824
4858
"""If all tasks of a layer are affected by the same run_spec collision, warn
4825
4859
only once.
4826
4860
"""
4827
- x1s = c .map (inc , [0 , 1 , 2 ], key = [("x " , 0 ), ("x " , 1 ), ("x " , 2 )])
4861
+ y1s = c .map (inc , [0 , 1 , 2 ], key = [("y " , 0 ), ("y " , 1 ), ("y " , 2 )])
4828
4862
dsk = {
4829
- ("x" , 0 ): 3 ,
4830
- ("x" , 1 ): 4 ,
4831
- ("x" , 2 ): 5 ,
4832
- ("y" , 0 ): (inc , ("x" , 0 )),
4833
- ("y" , 1 ): (inc , ("x" , 1 )),
4834
- ("y" , 2 ): (inc , ("x" , 2 )),
4863
+ "x" : 3 ,
4864
+ ("y" , 0 ): (inc , "x" ), # run_spec and dependencies change
4865
+ ("y" , 1 ): (inc , 4 ), # run_spec changes, dependencies don't
4866
+ ("y" , 2 ): (inc , 2 ), # Doesn't change
4867
+ ("z" , 0 ): (inc , ("y" , 0 )),
4868
+ ("z" , 1 ): (inc , ("y" , 1 )),
4869
+ ("z" , 2 ): (inc , ("y" , 2 )),
4835
4870
}
4836
4871
with captured_logger ("distributed.scheduler" , level = loglevel ) as log :
4837
- ys = c .get (dsk , [("y " , 0 ), ("y " , 1 ), ("y " , 2 )], sync = False )
4838
- await wait_for_state (("y " , 2 ), "waiting" , s )
4872
+ zs = c .get (dsk , [("z " , 0 ), ("z " , 1 ), ("z " , 2 )], sync = False )
4873
+ await wait_for_state (("z " , 2 ), "waiting" , s )
4839
4874
4840
4875
actual_loglines = len (
4841
4876
re .findall ("Detected different `run_spec` for key " , log .getvalue ())
4842
4877
)
4843
4878
assert actual_loglines == expect_loglines
4844
4879
4845
4880
async with Worker (s .address ):
4846
- assert await c .gather (ys ) == [2 , 3 , 4 ]
4881
+ assert await c .gather (zs ) == [2 , 3 , 4 ] # Kept old ys
0 commit comments