Skip to content

Commit 26b1061

Browse files
authored
Remove redundant methods in P2PBarrierTask (#8924)
1 parent 9842ae9 commit 26b1061

File tree

2 files changed

+16
-44
lines changed

2 files changed

+16
-44
lines changed

distributed/shuffle/_core.py

Lines changed: 0 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import dask.config
2828
from dask._task_spec import Task, _inline_recursively
2929
from dask.core import flatten
30-
from dask.sizeof import sizeof
3130
from dask.typing import Key
3231
from dask.utils import parse_bytes, parse_timedelta
3332

@@ -601,41 +600,17 @@ def __init__(
601600
super().__init__(key, func, *args, **kwargs)
602601

603602
def copy(self) -> P2PBarrierTask:
604-
self.unpack()
605-
assert self.func is not None
606603
return P2PBarrierTask(
607604
self.key, self.func, *self.args, spec=self.spec, **self.kwargs
608605
)
609606

610-
def __sizeof__(self) -> int:
611-
return super().__sizeof__() + sizeof(self.spec)
612-
613607
def __repr__(self) -> str:
614608
return f"P2PBarrierTask({self.key!r})"
615609

616610
def inline(self, dsk: dict[Key, Any]) -> P2PBarrierTask:
617-
self.unpack()
618611
new_args = _inline_recursively(self.args, dsk)
619612
new_kwargs = _inline_recursively(self.kwargs, dsk)
620613
assert self.func is not None
621614
return P2PBarrierTask(
622615
self.key, self.func, *new_args, spec=self.spec, **new_kwargs
623616
)
624-
625-
def __getstate__(self) -> dict[str, Any]:
626-
state = super().__getstate__()
627-
state["spec"] = self.spec
628-
return state
629-
630-
def __setstate__(self, state: dict[str, Any]) -> None:
631-
super().__setstate__(state)
632-
self.spec = state["spec"]
633-
634-
def __eq__(self, value: object) -> bool:
635-
if not isinstance(value, P2PBarrierTask):
636-
return False
637-
if not super().__eq__(value):
638-
return False
639-
if self.spec != value.spec:
640-
return False
641-
return True

distributed/tests/test_client.py

Lines changed: 16 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@
4242
import dask
4343
import dask.bag as db
4444
from dask import delayed
45-
from dask._task_spec import no_function_cache
4645
from dask.optimization import SubgraphCallable
4746
from dask.tokenize import tokenize
4847
from dask.utils import get_default_shuffle_method, parse_timedelta, tmpfile
@@ -4934,29 +4933,27 @@ def __setstate__(self, state):
49344933

49354934
@gen_cluster(client=True)
49364935
async def test_robust_undeserializable_function(c, s, a, b, monkeypatch):
4937-
with no_function_cache():
4938-
4939-
class Foo:
4940-
def __getstate__(self):
4941-
return 1
4936+
class Foo:
4937+
def __getstate__(self):
4938+
return 1
49424939

4943-
def __setstate__(self, state):
4944-
raise MyException("hello")
4940+
def __setstate__(self, state):
4941+
raise MyException("hello")
49454942

4946-
def __call__(self, *args):
4947-
return 1
4943+
def __call__(self, *args):
4944+
return 1
49484945

4949-
future = c.submit(Foo(), 1)
4950-
await wait(future)
4951-
assert future.status == "error"
4952-
with raises_with_cause(RuntimeError, "deserialization", MyException, "hello"):
4953-
await future
4946+
future = c.submit(Foo(), 1)
4947+
await wait(future)
4948+
assert future.status == "error"
4949+
with raises_with_cause(RuntimeError, "deserialization", MyException, "hello"):
4950+
await future
49544951

4955-
futures = c.map(inc, range(10))
4956-
results = await c.gather(futures)
4952+
futures = c.map(inc, range(10))
4953+
results = await c.gather(futures)
49574954

4958-
assert results == list(map(inc, range(10)))
4959-
assert a.data and b.data
4955+
assert results == list(map(inc, range(10)))
4956+
assert a.data and b.data
49604957

49614958

49624959
@gen_cluster(client=True)

0 commit comments

Comments
 (0)