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

Remove deprecated code #5401

Merged
merged 4 commits into from Sep 13, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
1 change: 0 additions & 1 deletion dask/__init__.py
Expand Up @@ -2,7 +2,6 @@

from . import config, datasets
from .core import istask
from .context import set_options
from .local import get_sync as get

try:
Expand Down
1 change: 0 additions & 1 deletion dask/array/ghost.py

This file was deleted.

40 changes: 1 addition & 39 deletions dask/array/tests/test_atop.py
Expand Up @@ -8,13 +8,7 @@
import dask
import dask.array as da
from dask.highlevelgraph import HighLevelGraph
from dask.blockwise import (
Blockwise,
rewrite_blockwise,
optimize_blockwise,
index_subs,
blockwise,
)
from dask.blockwise import Blockwise, rewrite_blockwise, optimize_blockwise, index_subs
from dask.array.utils import assert_eq
from dask.array.numpy_compat import _numpy_116
from dask.utils_test import inc, dec
Expand Down Expand Up @@ -552,38 +546,6 @@ def test_validate_top_inputs():
assert "i" in str(info.value)


def test_gh_4176():
with warnings.catch_warnings():
warnings.simplefilter("ignore")
from dask.sharedict import ShareDict

def foo(A):
return A[None, ...]

A = da.ones(shape=(10, 20, 4), chunks=(2, 5, 4))

name = "D"

dsk = blockwise(
foo,
name,
("nsrc", "ntime", "nbl", "npol"),
A.name,
("ntime", "nbl", "npol"),
new_axes={"nsrc": 1},
numblocks={a.name: a.numblocks for a in (A,)},
)

array_dsk = ShareDict()
array_dsk.update(dsk)
array_dsk.update(A.__dask_graph__())

chunks = ((1,),) + A.chunks

D = da.Array(array_dsk, name, chunks, dtype=A.dtype)
D.sum(axis=0).compute()


def test_dont_merge_before_reductions():
x = da.ones(10, chunks=(5,))
y = da.blockwise(inc, "i", x, "i", dtype=x.dtype)
Expand Down
12 changes: 0 additions & 12 deletions dask/context.py
Expand Up @@ -13,18 +13,6 @@
thread_state = threading.local()


def set_options(*args, **kwargs):
""" Deprecated: see dask.config.set instead """
raise TypeError(
"The dask.set_options function has been deprecated.\n"
"Please use dask.config.set instead\n\n"
" Before: with dask.set_options(foo='bar'):\n"
" ...\n"
" After: with dask.config.set(foo='bar'):\n"
" ..."
)


def globalmethod(default=None, key=None, falsey=None):
""" Allow function to be taken over by globals

Expand Down
9 changes: 1 addition & 8 deletions dask/dataframe/core.py
Expand Up @@ -1339,9 +1339,7 @@ def _get_binary_operator(cls, op, inv=False):
else:
return lambda self, other: elemwise(op, self, other)

def rolling(
self, window, min_periods=None, freq=None, center=False, win_type=None, axis=0
):
def rolling(self, window, min_periods=None, center=False, win_type=None, axis=0):
"""Provides rolling transformations.

Parameters
Expand Down Expand Up @@ -1370,10 +1368,6 @@ def rolling(
Returns
-------
a Rolling object on which to call a method to compute a statistic

Notes
-----
The `freq` argument is not supported.
"""
from dask.dataframe.rolling import Rolling

Expand All @@ -1391,7 +1385,6 @@ def rolling(
self,
window=window,
min_periods=min_periods,
freq=freq,
center=center,
win_type=win_type,
axis=axis,
Expand Down
13 changes: 1 addition & 12 deletions dask/dataframe/rolling.py
Expand Up @@ -264,19 +264,8 @@ class Rolling(object):
"""Provides rolling window calculations."""

def __init__(
self,
obj,
window=None,
min_periods=None,
freq=None,
center=False,
win_type=None,
axis=0,
self, obj, window=None, min_periods=None, center=False, win_type=None, axis=0
):
if freq is not None:
msg = "The deprecated freq argument is not supported."
raise NotImplementedError(msg)

self.obj = obj # dataframe or series
self.window = window
self.min_periods = min_periods
Expand Down
121 changes: 0 additions & 121 deletions dask/sharedict.py

This file was deleted.

84 changes: 0 additions & 84 deletions dask/tests/test_sharedict.py

This file was deleted.