Commit b41c932e authored by Kruyff,D.L.W. (Dylan)'s avatar Kruyff,D.L.W. (Dylan)
Browse files

Redo lost changes

parent 8d4ead19
......@@ -12,6 +12,10 @@ from dask.utils import tmpdir
files = ["a", "b"]
requests = pytest.importorskip("requests")
errs = (requests.exceptions.RequestException,)
if LooseVersion(fsspec.__version__) > "0.7.4":
aiohttp = pytest.importorskip("aiohttp")
errs = errs + (aiohttp.client_exceptions.ClientResponseError,)
@pytest.fixture(scope="module")
......@@ -112,7 +116,7 @@ def test_ops_blocksize(dir_server):
def test_errors(dir_server):
f = open_files("http://localhost:8999/doesnotexist")[0]
with pytest.raises(requests.exceptions.RequestException):
with pytest.raises(errs):
with f as f:
f.read()
f = open_files("http://nohost/")[0]
......
......@@ -3,15 +3,19 @@ import os
from contextlib import contextmanager
from functools import partial
from distutils.version import LooseVersion
import shlex
import subprocess
import sys
import time
import pytest
import numpy as np
s3fs = pytest.importorskip("s3fs")
boto3 = pytest.importorskip("boto3")
moto = pytest.importorskip("moto")
httpretty = pytest.importorskip("httpretty")
moto = pytest.importorskip("moto", minversion="1.3.14")
pytest.importorskip("flask") # server mode needs flask too
requests = pytest.importorskip("requests")
from tlz import concat, valmap
......@@ -23,6 +27,9 @@ from fsspec.compression import compr
compute = partial(compute, scheduler="sync")
numpy_120_mark = pytest.mark.xfail(
LooseVersion(np.__version__) >= "1.20.0", reason="Upstream incompatibility"
)
test_bucket_name = "test"
......@@ -57,56 +64,71 @@ def ensure_safe_environment_variables():
os.environ.update(saved_environ)
@pytest.fixture()
def s3():
@pytest.fixture
def s3so():
return dict(client_kwargs={"endpoint_url": "http://127.0.0.1:5555/"})
endpoint_uri = "http://127.0.0.1:5555/"
@pytest.fixture(scope="module")
def s3_base():
with ensure_safe_environment_variables():
# temporary workaround as moto fails for botocore >= 1.11 otherwise,
# see https://github.com/spulec/moto/issues/1924 & 1952
os.environ.setdefault("AWS_ACCESS_KEY_ID", "foobar_key")
os.environ.setdefault("AWS_SECRET_ACCESS_KEY", "foobar_secret")
os.environ["AWS_ACCESS_KEY_ID"] = "foobar_key"
os.environ["AWS_SECRET_ACCESS_KEY"] = "foobar_secret"
# pipe to null to avoid logging in terminal
proc = subprocess.Popen(
shlex.split("moto_server s3 -p 5555"), stdout=subprocess.DEVNULL
)
# writable local S3 system
import moto
timeout = 8
while True:
try:
# OK to go once server is accepting connections
r = requests.get(endpoint_uri)
if r.ok:
break
except Exception:
pass
timeout -= 0.1
time.sleep(0.1)
assert timeout > 0, "Timed out waiting for moto server"
yield
# shut down external process
proc.terminate()
try:
proc.wait(timeout=3)
except subprocess.TimeoutExpired:
proc.kill()
if sys.platform == "win32":
# belt & braces
subprocess.call("TASKKILL /F /PID {pid} /T".format(pid=proc.pid))
with moto.mock_s3():
client = boto3.client("s3")
client.create_bucket(Bucket=test_bucket_name, ACL="public-read-write")
for f, data in files.items():
client.put_object(Bucket=test_bucket_name, Key=f, Body=data)
fs = s3fs.S3FileSystem(anon=True)
fs.invalidate_cache
yield fs
httpretty.HTTPretty.disable()
httpretty.HTTPretty.reset()
@pytest.fixture
def s3(s3_base):
with s3_context() as fs:
yield fs
@contextmanager
def s3_context(bucket, files):
with ensure_safe_environment_variables():
# temporary workaround as moto fails for botocore >= 1.11 otherwise,
# see https://github.com/spulec/moto/issues/1924 & 1952
os.environ.setdefault("AWS_ACCESS_KEY_ID", "foobar_key")
os.environ.setdefault("AWS_SECRET_ACCESS_KEY", "foobar_secret")
with moto.mock_s3():
client = boto3.client("s3")
client.create_bucket(Bucket=bucket, ACL="public-read-write")
for f, data in files.items():
client.put_object(Bucket=bucket, Key=f, Body=data)
fs = DaskS3FileSystem(anon=True)
fs.invalidate_cache()
yield fs
for f, data in files.items():
try:
client.delete_object(Bucket=bucket, Key=f, Body=data)
except Exception:
pass
finally:
httpretty.HTTPretty.disable()
httpretty.HTTPretty.reset()
def s3_context(bucket=test_bucket_name, files=files):
client = boto3.client("s3", endpoint_url=endpoint_uri)
client.create_bucket(Bucket=bucket, ACL="public-read-write")
for f, data in files.items():
client.put_object(Bucket=bucket, Key=f, Body=data)
fs = s3fs.S3FileSystem(
anon=True, client_kwargs={"endpoint_url": "http://127.0.0.1:5555/"}
)
s3fs.S3FileSystem.clear_instance_cache()
fs.invalidate_cache()
try:
yield fs
finally:
fs.rm(bucket, recursive=True)
@pytest.fixture()
......@@ -187,7 +209,7 @@ def s3_with_yellow_tripdata(s3):
df.to_csv(sfile, index=False)
key = "nyc-taxi/2015/yellow_tripdata_2015-01.csv"
client = boto3.client("s3")
client = boto3.client("s3", endpoint_url="http://127.0.0.1:5555/")
client.put_object(Bucket=test_bucket_name, Key=key, Body=file)
key = "nyc-taxi/2014/yellow_tripdata_2014-{:0>2d}.csv"
......@@ -212,19 +234,21 @@ def test_get_s3():
DaskS3FileSystem(secret="key", password="key")
def test_open_files_write(s3):
def test_open_files_write(s3, s3so):
paths = ["s3://" + test_bucket_name + "/more/" + f for f in files]
fils = open_files(paths, mode="wb")
fils = open_files(paths, mode="wb", **s3so)
for fil, data in zip(fils, files.values()):
with fil as f:
f.write(data)
sample, values = read_bytes("s3://" + test_bucket_name + "/more/test/accounts.*")
sample, values = read_bytes(
"s3://" + test_bucket_name + "/more/test/accounts.*", **s3so
)
results = compute(*concat(values))
assert set(list(files.values())) == set(results)
def test_read_bytes(s3):
sample, values = read_bytes("s3://" + test_bucket_name + "/test/accounts.*")
def test_read_bytes(s3, s3so):
sample, values = read_bytes("s3://" + test_bucket_name + "/test/accounts.*", **s3so)
assert isinstance(sample, bytes)
assert sample[:5] == files[sorted(files)[0]][:5]
assert sample.endswith(b"\n")
......@@ -238,38 +262,48 @@ def test_read_bytes(s3):
assert set(results) == set(files.values())
def test_read_bytes_sample_delimiter(s3):
def test_read_bytes_sample_delimiter(s3, s3so):
sample, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts.*", sample=80, delimiter=b"\n"
"s3://" + test_bucket_name + "/test/accounts.*",
sample=80,
delimiter=b"\n",
**s3so
)
assert sample.endswith(b"\n")
sample, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts.1.json", sample=80, delimiter=b"\n"
"s3://" + test_bucket_name + "/test/accounts.1.json",
sample=80,
delimiter=b"\n",
**s3so
)
assert sample.endswith(b"\n")
sample, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts.1.json", sample=2, delimiter=b"\n"
"s3://" + test_bucket_name + "/test/accounts.1.json",
sample=2,
delimiter=b"\n",
**s3so
)
assert sample.endswith(b"\n")
def test_read_bytes_non_existing_glob(s3):
def test_read_bytes_non_existing_glob(s3, s3so):
with pytest.raises(IOError):
read_bytes("s3://" + test_bucket_name + "/non-existing/*")
read_bytes("s3://" + test_bucket_name + "/non-existing/*", **s3so)
def test_read_bytes_blocksize_none(s3):
def test_read_bytes_blocksize_none(s3, s3so):
_, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts.*", blocksize=None
"s3://" + test_bucket_name + "/test/accounts.*", blocksize=None, **s3so
)
assert sum(map(len, values)) == len(files)
def test_read_bytes_blocksize_on_large_data(s3_with_yellow_tripdata):
def test_read_bytes_blocksize_on_large_data(s3_with_yellow_tripdata, s3so):
_, L = read_bytes(
"s3://{}/nyc-taxi/2015/yellow_tripdata_2015-01.csv".format(test_bucket_name),
blocksize=None,
anon=True,
**s3so
)
assert len(L) == 1
......@@ -277,14 +311,15 @@ def test_read_bytes_blocksize_on_large_data(s3_with_yellow_tripdata):
"s3://{}/nyc-taxi/2014/*.csv".format(test_bucket_name),
blocksize=None,
anon=True,
**s3so
)
assert len(L) == 12
@pytest.mark.parametrize("blocksize", [5, 15, 45, 1500])
def test_read_bytes_block(s3, blocksize):
def test_read_bytes_block(s3, blocksize, s3so):
_, vals = read_bytes(
"s3://" + test_bucket_name + "/test/account*", blocksize=blocksize
"s3://" + test_bucket_name + "/test/account*", blocksize=blocksize, **s3so
)
assert list(map(len, vals)) == [(len(v) // blocksize + 1) for v in files.values()]
......@@ -297,16 +332,18 @@ def test_read_bytes_block(s3, blocksize):
@pytest.mark.parametrize("blocksize", [5, 15, 45, 1500])
def test_read_bytes_delimited(s3, blocksize):
def test_read_bytes_delimited(s3, blocksize, s3so):
_, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts*",
blocksize=blocksize,
delimiter=b"\n",
**s3so
)
_, values2 = read_bytes(
"s3://" + test_bucket_name + "/test/accounts*",
blocksize=blocksize,
delimiter=b"foo",
**s3so
)
assert [a.key for a in concat(values)] != [b.key for b in concat(values2)]
......@@ -320,7 +357,10 @@ def test_read_bytes_delimited(s3, blocksize):
# delimiter not at the end
d = b"}"
_, values = read_bytes(
"s3://" + test_bucket_name + "/test/accounts*", blocksize=blocksize, delimiter=d
"s3://" + test_bucket_name + "/test/accounts*",
blocksize=blocksize,
delimiter=d,
**s3so
)
results = compute(*concat(values))
res = [r for r in results if r]
......@@ -334,7 +374,7 @@ def test_read_bytes_delimited(s3, blocksize):
@pytest.mark.parametrize(
"fmt,blocksize", [(fmt, None) for fmt in compr] + [(fmt, 10) for fmt in compr]
)
def test_compression(s3, fmt, blocksize):
def test_compression(s3, fmt, blocksize, s3so):
if fmt not in compress:
pytest.skip("compression function not provided")
s3._cache.clear()
......@@ -345,10 +385,14 @@ def test_compression(s3, fmt, blocksize):
"s3://compress/test/accounts.*",
compression=fmt,
blocksize=blocksize,
**s3so
)
return
sample, values = read_bytes(
"s3://compress/test/accounts.*", compression=fmt, blocksize=blocksize
"s3://compress/test/accounts.*",
compression=fmt,
blocksize=blocksize,
**s3so
)
assert sample.startswith(files[sorted(files)[0]][:10])
assert sample.endswith(b"\n")
......@@ -358,8 +402,10 @@ def test_compression(s3, fmt, blocksize):
@pytest.mark.parametrize("mode", ["rt", "rb"])
def test_open_files(s3, mode):
myfiles = open_files("s3://" + test_bucket_name + "/test/accounts.*", mode=mode)
def test_open_files(s3, mode, s3so):
myfiles = open_files(
"s3://" + test_bucket_name + "/test/accounts.*", mode=mode, **s3so
)
assert len(myfiles) == len(files)
for lazy_file, path in zip(myfiles, sorted(files)):
with lazy_file as f:
......@@ -371,21 +417,22 @@ def test_open_files(s3, mode):
double = lambda x: x * 2
def test_modification_time_read_bytes():
def test_modification_time_read_bytes(s3, s3so):
with s3_context("compress", files):
_, a = read_bytes("s3://compress/test/accounts.*", anon=True)
_, b = read_bytes("s3://compress/test/accounts.*", anon=True)
_, a = read_bytes("s3://compress/test/accounts.*", anon=True, **s3so)
_, b = read_bytes("s3://compress/test/accounts.*", anon=True, **s3so)
assert [aa._key for aa in concat(a)] == [bb._key for bb in concat(b)]
with s3_context("compress", valmap(double, files)):
_, c = read_bytes("s3://compress/test/accounts.*", anon=True)
_, c = read_bytes("s3://compress/test/accounts.*", anon=True, **s3so)
assert [aa._key for aa in concat(a)] != [cc._key for cc in concat(c)]
@pytest.mark.parametrize("engine", ["pyarrow", "fastparquet"])
def test_parquet(s3, engine):
@numpy_120_mark
def test_parquet(s3, engine, s3so):
dd = pytest.importorskip("dask.dataframe")
from dask.dataframe._compat import tm
......@@ -409,19 +456,20 @@ def test_parquet(s3, engine):
index=pd.Index(np.arange(1000), name="foo"),
)
df = dd.from_pandas(data, chunksize=500)
df.to_parquet(url, engine=engine)
df.to_parquet(url, engine=engine, storage_options=s3so)
files = [f.split("/")[-1] for f in s3.ls(url)]
assert "_common_metadata" in files
assert "part.0.parquet" in files
df2 = dd.read_parquet(url, index="foo", engine=engine)
df2 = dd.read_parquet(url, index="foo", engine=engine, storage_options=s3so)
assert len(df2.divisions) > 1
tm.assert_frame_equal(data, df2.compute())
def test_parquet_wstoragepars(s3):
@numpy_120_mark
def test_parquet_wstoragepars(s3, s3so):
dd = pytest.importorskip("dask.dataframe")
pytest.importorskip("fastparquet")
......@@ -432,14 +480,16 @@ def test_parquet_wstoragepars(s3):
data = pd.DataFrame({"i32": np.array([0, 5, 2, 5])})
df = dd.from_pandas(data, chunksize=500)
df.to_parquet(url, write_index=False)
df.to_parquet(url, write_index=False, storage_options=s3so)
dd.read_parquet(url, storage_options={"default_fill_cache": False})
dd.read_parquet(url, storage_options=dict(**s3so, **{"default_fill_cache": False}))
assert s3.current().default_fill_cache is False
dd.read_parquet(url, storage_options={"default_fill_cache": True})
dd.read_parquet(url, storage_options=dict(**s3so, **{"default_fill_cache": True}))
assert s3.current().default_fill_cache is True
dd.read_parquet(url, storage_options={"default_block_size": 2 ** 20})
dd.read_parquet(
url, storage_options=dict(**s3so, **{"default_block_size": 2 ** 20})
)
assert s3.current().default_block_size == 2 ** 20
with s3.current().open(url + "/_metadata") as f:
assert f.blocksize == 2 ** 20
......
......@@ -7,7 +7,7 @@ overhead = sys.getsizeof(1.23) * 4 + sys.getsizeof(()) * 4
class Cache(Callback):
""" Use cache for computation
"""Use cache for computation
Examples
--------
......
......@@ -4,7 +4,7 @@ __all__ = ["Callback", "add_callbacks"]
class Callback(object):
""" Base class for using the callback mechanism
"""Base class for using the callback mechanism
Create a callback with functions of the following signatures:
......
......@@ -59,7 +59,7 @@ def canonical_name(k, config):
def update(old, new, priority="new"):
""" Update a nested dictionary with values from another
"""Update a nested dictionary with values from another
This is like dict.update except that it smoothly merges nested values
......@@ -102,7 +102,7 @@ def update(old, new, priority="new"):
def merge(*dicts):
""" Update a sequence of nested dictionaries
"""Update a sequence of nested dictionaries
This prefers the values in the latter dictionaries to those in the former
......@@ -124,7 +124,7 @@ def merge(*dicts):
def collect_yaml(paths=paths):
""" Collect configuration from yaml files
"""Collect configuration from yaml files
This searches through a list of paths, expands to find all yaml or json
files, and then parses each file.
......@@ -167,7 +167,7 @@ def collect_yaml(paths=paths):
def collect_env(env=None):
""" Collect config from environment variables
"""Collect config from environment variables
This grabs environment variables of the form "DASK_FOO__BAR_BAZ=123" and
turns these into config variables of the form ``{"foo": {"bar-baz": 123}}``
......@@ -256,7 +256,7 @@ def ensure_file(source, destination=None, comment=True):
class set(object):
""" Temporarily set configuration values within a context manager
"""Temporarily set configuration values within a context manager
Parameters
----------
......@@ -460,7 +460,7 @@ def get(key, default=no_default, config=config):
def rename(aliases, config=config):
""" Rename old keys to new keys
"""Rename old keys to new keys
This helps migrate older configuration versions over time
"""
......@@ -479,7 +479,7 @@ def rename(aliases, config=config):
def update_defaults(new, config=config, defaults=defaults):
""" Add a new set of defaults to the configuration
"""Add a new set of defaults to the configuration
It does two things:
......@@ -492,7 +492,7 @@ def update_defaults(new, config=config, defaults=defaults):
def expand_environment_variables(config):
""" Expand environment variables in a nested config dictionary
"""Expand environment variables in a nested config dictionary
This function will recursively search through any nested dictionaries
and/or lists.
......@@ -532,7 +532,7 @@ deprecations = {
def check_deprecations(key: str, deprecations: dict = deprecations):
""" Check if the provided value has been renamed or removed
"""Check if the provided value has been renamed or removed
Parameters
----------
......
......@@ -12,7 +12,7 @@ thread_state = threading.local()
def globalmethod(default=None, key=None, falsey=None):
""" Allow function to be taken over by globals
"""Allow function to be taken over by globals
This modifies a method so that occurrences of it may be taken over by
functions registered in the global options. Can be used as a decorator or a
......
......@@ -6,7 +6,7 @@ no_default = "__no_default__"
def ishashable(x):
""" Is x hashable?
"""Is x hashable?
Examples
--------
......@@ -24,7 +24,7 @@ def ishashable(x):
def istask(x):
""" Is x a runnable task?
"""Is x a runnable task?
A task is a tuple with a callable first argument
......@@ -84,7 +84,7 @@ def lists_to_tuples(res, keys):
def _execute_task(arg, cache, dsk=None):
""" Do the actual work of collecting data and executing a function
"""Do the actual work of collecting data and executing a function
Examples
--------
......@@ -128,7 +128,7 @@ def _execute_task(arg, cache, dsk=None):
def get(dsk, out, cache=None):
""" Get value from Dask
"""Get value from Dask
Examples
--------
......@@ -157,7 +157,7 @@ def get(dsk, out, cache=None):
def get_dependencies(dsk, key=None, task=no_default, as_list=False):
""" Get the immediate tasks on which this task depends
"""Get the immediate tasks on which this task depends
Examples
--------
......@@ -217,7 +217,7 @@ def get_dependencies(dsk, key=None, task=no_default, as_list=False):
def get_deps(dsk):
""" Get dependencies and dependents from dask dask graph
"""Get dependencies and dependents from dask dask graph
>>> dsk = {'a': 1, 'b': (inc, 'a'), 'c': (inc, 'b')}
>>> dependencies, dependents = get_deps(dsk)
......@@ -279,7 +279,7 @@ def reverse_dict(d):