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

Redo lost changes

parent 8d4ead19
......@@ -97,7 +97,7 @@ paths_to_cats = (
def _determine_pf_parts(fs, paths, gather_statistics, **kwargs):
""" Determine how to access metadata and break read into ``parts``
"""Determine how to access metadata and break read into ``parts``
This logic is mostly to handle `gather_statistics=False` cases,
because this also means we should avoid scanning every file in the
......@@ -510,6 +510,7 @@ class FastParquetEngine(Engine):
partition_on=None,
ignore_divisions=False,
division_info=None,
schema=None,
**kwargs
):
if append and division_info is None:
......@@ -579,7 +580,8 @@ class FastParquetEngine(Engine):
)
i_offset = 0
return (fmd, i_offset)
schema = None # ArrowEngine compatibility
return (fmd, schema, i_offset)
@classmethod
def write_partition(
......
......@@ -15,7 +15,7 @@ class Engine:
filters=None,
**kwargs
):
""" Gather metadata about a Parquet Dataset to prepare for a read
"""Gather metadata about a Parquet Dataset to prepare for a read
This function is called once in the user's Python session to gather
important metadata about the parquet dataset.
......@@ -69,7 +69,7 @@ class Engine:
@classmethod
def read_partition(cls, fs, piece, columns, index, **kwargs):
""" Read a single piece of a Parquet dataset into a Pandas DataFrame
"""Read a single piece of a Parquet dataset into a Pandas DataFrame
This function is called many times in individual tasks
......
......@@ -6,6 +6,7 @@ from dask.dataframe.utils import PANDAS_GT_0240, PANDAS_VERSION
from dask.delayed import tokenize
from .io import from_delayed, from_pandas
from ... import delayed
from .. import methods
def read_sql_table(
......@@ -179,11 +180,13 @@ def read_sql_table(
or 1
)
if dtype.kind == "M":
divisions = pd.date_range(
start=mini,
end=maxi,
freq="%iS" % ((maxi - mini).total_seconds() / npartitions),
).tolist()
divisions = methods.tolist(
pd.date_range(
start=mini,
end=maxi,
freq="%iS" % ((maxi - mini).total_seconds() / npartitions),
)
)
divisions[0] = mini
divisions[-1] = maxi
elif dtype.kind in ["i", "u", "f"]:
......@@ -237,7 +240,7 @@ def to_sql(
compute=True,
parallel=False,
):
""" Store Dask Dataframe to a SQL table
"""Store Dask Dataframe to a SQL table
An empty table is created based on the "meta" DataFrame (and conforming to the caller's "if_exists" preference), and
then each block calls pd.DataFrame.to_sql (with `if_exists="append"`).
......
import math
import os
import sys
import warnings
......@@ -16,6 +17,7 @@ from dask.dataframe.optimize import optimize_read_parquet_getitem
from dask.dataframe.io.parquet.core import ParquetSubgraph
from dask.utils import natural_sort_key, parse_bytes
try:
import fastparquet
except ImportError:
......@@ -27,7 +29,6 @@ try:
except ImportError:
check_pa_divs = pa = False
try:
import pyarrow.parquet as pq
except ImportError:
......@@ -959,9 +960,22 @@ def test_to_parquet_pyarrow_w_inconsistent_schema_by_partition_fails_by_default(
)
ddf = dd.from_pandas(df, npartitions=2)
ddf.to_parquet(str(tmpdir), engine="pyarrow", partition_on=["partition_column"])
# In order to allow pyarrow to write an inconsistent schema,
# we need to avoid writing the _metadata file (will fail >0.17.1)
# and need to avoid schema inference (i.e. use `schema=None`)
ddf.to_parquet(
str(tmpdir),
engine="pyarrow",
partition_on=["partition_column"],
write_metadata_file=False,
schema=None,
)
# Test that schema is not validated by default
# (shouldn't raise error)
dd.read_parquet(str(tmpdir), engine="pyarrow", gather_statistics=False).compute()
# Test that read fails because of default behavior when schema not provided
# Test that read fails when validate_schema=True
with pytest.raises(ValueError) as e_info:
dd.read_parquet(
str(tmpdir),
......@@ -1062,6 +1076,53 @@ def test_to_parquet_pyarrow_w_inconsistent_schema_by_partition_succeeds_w_manual
assert np.array_equal(ddf_after_write.partition_column, df.partition_column)
@pytest.mark.parametrize("index", [False, True])
@pytest.mark.parametrize("schema", ["infer", "complex"])
def test_pyarrow_schema_inference(tmpdir, index, engine, schema):
check_pyarrow()
if pa.__version__ < LooseVersion("0.15.0"):
pytest.skip("PyArrow>=0.15 Required.")
if schema == "complex":
schema = {"index": pa.string(), "amount": pa.int64()}
tmpdir = str(tmpdir)
df = pd.DataFrame(
{
"index": ["1", "2", "3", "2", "3", "1", "4"],
"date": pd.to_datetime(
[
"2017-01-01",
"2017-01-01",
"2017-01-01",
"2017-01-02",
"2017-01-02",
"2017-01-06",
"2017-01-09",
]
),
"amount": [100, 200, 300, 400, 500, 600, 700],
},
index=range(7, 14),
)
if index:
df = dd.from_pandas(df, npartitions=2).set_index("index")
else:
df = dd.from_pandas(df, npartitions=2)
df.to_parquet(tmpdir, engine="pyarrow", schema=schema, compute=False).compute(
scheduler="synchronous"
)
df_out = dd.read_parquet(tmpdir, engine=engine)
if index and engine == "fastparquet":
# Fastparquet not handling divisions for
# pyarrow-written dataset with string index
assert_eq(df, df_out, check_divisions=False)
else:
assert_eq(df, df_out)
def test_partition_on(tmpdir, engine):
tmpdir = str(tmpdir)
df = pd.DataFrame(
......@@ -1089,6 +1150,34 @@ def test_partition_on(tmpdir, engine):
assert set(df.b[df.a2 == val]) == set(out.b[out.a2 == val])
def test_partition_on_duplicates(tmpdir, engine):
# https://github.com/dask/dask/issues/6445
tmpdir = str(tmpdir)
df = pd.DataFrame(
{
"a1": np.random.choice(["A", "B", "C"], size=100),
"a2": np.random.choice(["X", "Y", "Z"], size=100),
"data": np.random.random(size=100),
}
)
d = dd.from_pandas(df, npartitions=2)
for _ in range(2):
d.to_parquet(tmpdir, partition_on=["a1", "a2"], engine=engine)
out = dd.read_parquet(tmpdir, engine=engine).compute()
assert len(df) == len(out)
for root, dirs, files in os.walk(tmpdir):
for file in files:
assert file in (
"part.0.parquet",
"part.1.parquet",
"_common_metadata",
"_metadata",
)
@pytest.mark.parametrize("partition_on", ["aa", ["aa"]])
def test_partition_on_string(tmpdir, partition_on):
tmpdir = str(tmpdir)
......@@ -1211,6 +1300,29 @@ def test_filters_v0(tmpdir, write_engine, read_engine):
assert len(ddf2) > 0
def test_fiters_file_list(tmpdir, engine):
df = pd.DataFrame({"x": range(10), "y": list("aabbccddee")})
ddf = dd.from_pandas(df, npartitions=5)
ddf.to_parquet(str(tmpdir), engine=engine)
fils = str(tmpdir.join("*.parquet"))
ddf_out = dd.read_parquet(
fils, gather_statistics=True, engine=engine, filters=[("x", ">", 3)]
)
assert ddf_out.npartitions == 3
assert_eq(df[df["x"] > 3], ddf_out.compute(), check_index=False)
# Check that first parition gets filtered for single-path input
ddf2 = dd.read_parquet(
str(tmpdir.join("part.0.parquet")),
gather_statistics=True,
engine=engine,
filters=[("x", ">", 3)],
)
assert len(ddf2) == 0
def test_divisions_read_with_filters(tmpdir):
pytest.importorskip("fastparquet", minversion="0.3.1")
tmpdir = str(tmpdir)
......@@ -1420,6 +1532,20 @@ def test_writing_parquet_with_compression(tmpdir, compression, engine):
check_compression(engine, fn, compression)
@pytest.mark.parametrize("compression,", ["default", None, "gzip", "snappy"])
def test_writing_parquet_with_partition_on_and_compression(tmpdir, compression, engine):
fn = str(tmpdir)
if compression in ["snappy", "default"]:
pytest.importorskip("snappy")
df = pd.DataFrame({"x": ["a", "b", "c"] * 10, "y": [1, 2, 3] * 10})
df.index.name = "index"
ddf = dd.from_pandas(df, npartitions=3)
ddf.to_parquet(fn, compression=compression, engine=engine, partition_on=["x"])
check_compression(engine, fn, compression)
@pytest.fixture(
params=[
# fastparquet 0.1.3
......@@ -1817,6 +1943,13 @@ def test_arrow_partitioning(tmpdir):
def test_sorted_warnings(tmpdir, engine):
if engine == "pyarrow":
pytest.skip(
"ArrowEngine will only collect statistics for "
"known index columns and/or filtered columns."
)
tmpdir = str(tmpdir)
df = dd.from_pandas(
pd.DataFrame({"cola": range(10), "colb": range(10)}), npartitions=2
......@@ -1987,7 +2120,16 @@ def test_read_dir_nometa(tmpdir, write_engine, read_engine, statistics, remove_c
assert_eq(ddf, ddf2, check_divisions=False)
def test_timeseries_nulls_in_schema(tmpdir, engine):
@pytest.mark.parametrize("schema", ["infer", None])
def test_timeseries_nulls_in_schema(tmpdir, engine, schema):
if (
schema == "infer"
and engine == "pyarrow"
and pa.__version__ < LooseVersion("0.15.0")
):
pytest.skip("PyArrow>=0.15 Required.")
# GH#5608: relative path failing _metadata/_common_metadata detection.
tmp_path = str(tmpdir.mkdir("files"))
tmp_path = os.path.join(tmp_path, "../", "files")
......@@ -2000,14 +2142,18 @@ def test_timeseries_nulls_in_schema(tmpdir, engine):
ddf2 = ddf2.set_index("x").reset_index().persist()
ddf2.name = ddf2.name.where(ddf2.timestamp == "2000-01-01", None)
ddf2.to_parquet(tmp_path, engine=engine)
ddf_read = dd.read_parquet(tmp_path, engine=engine)
# Note: `append_row_groups` will fail with pyarrow>0.17.1 for _metadata write
ddf2.to_parquet(tmp_path, engine=engine, write_metadata_file=False, schema=schema)
ddf_read = dd.read_parquet(
tmp_path, engine=engine, dataset={"validate_schema": False}
)
assert_eq(ddf_read, ddf2, check_divisions=False, check_index=False)
# Can force schema validation on each partition in pyarrow
if engine == "pyarrow":
# The schema mismatch should raise an error
if engine == "pyarrow" and schema is None:
# The schema mismatch should raise an error if the
# dataset was written with `schema=None` (no inference)
with pytest.raises(ValueError):
ddf_read = dd.read_parquet(
tmp_path, dataset={"validate_schema": True}, engine=engine
......@@ -2215,6 +2361,41 @@ def test_split_row_groups_pyarrow(tmpdir):
assert ddf3.npartitions == 4
@pytest.mark.parametrize("split_row_groups", [1, 12])
@pytest.mark.parametrize("gather_statistics", [True, False])
def test_split_row_groups_int_pyarrow(tmpdir, split_row_groups, gather_statistics):
check_pyarrow()
tmp = str(tmpdir)
engine = "pyarrow"
row_group_size = 10
npartitions = 4
half_size = 400
df = pd.DataFrame(
{
"i32": np.arange(2 * half_size, dtype=np.int32),
"f": np.arange(2 * half_size, dtype=np.float64),
}
)
half = len(df) // 2
dd.from_pandas(df.iloc[:half], npartitions=npartitions).to_parquet(
tmp, engine=engine, row_group_size=row_group_size
)
dd.from_pandas(df.iloc[half:], npartitions=npartitions).to_parquet(
tmp, append=True, engine=engine, row_group_size=row_group_size
)
ddf2 = dd.read_parquet(
tmp,
engine=engine,
split_row_groups=split_row_groups,
gather_statistics=gather_statistics,
)
expected_rg_cout = int(half_size / row_group_size)
assert ddf2.npartitions == 2 * math.ceil(expected_rg_cout / split_row_groups)
def test_split_row_groups_filter_pyarrow(tmpdir):
check_pyarrow()
tmp = str(tmpdir)
......@@ -2234,7 +2415,7 @@ def test_split_row_groups_filter_pyarrow(tmpdir):
tmp,
engine="pyarrow",
gather_statistics=True,
split_row_groups=False,
split_row_groups=True,
filters=filters,
)
......@@ -2584,3 +2765,12 @@ def test_illegal_column_name(tmpdir, engine):
with pytest.raises(ValueError) as e:
ddf.to_parquet(fn, engine=engine)
assert null_name in str(e.value)
def test_divisions_with_null_partition(tmpdir, engine):
df = pd.DataFrame({"a": [1, 2, None, None], "b": [1, 2, 3, 4]})
ddf = dd.from_pandas(df, npartitions=2)
ddf.to_parquet(str(tmpdir), engine=engine, write_index=False)
ddf_read = dd.read_parquet(str(tmpdir), engine=engine, index="a")
assert ddf_read.divisions == (None, None, None)
......@@ -347,7 +347,12 @@ def tmp_db_uri():
@pytest.mark.parametrize("parallel", (False, True))
def test_to_sql(npartitions, parallel):
df_by_age = df.set_index("age")
df_appended = pd.concat([df, df,])
df_appended = pd.concat(
[
df,
df,
]
)
ddf = dd.from_pandas(df, npartitions)
ddf_by_age = ddf.set_index("age")
......
import pandas as pd
import json
from uuid import uuid4
def _get_pyarrow_dtypes(schema, categories):
......@@ -92,3 +93,8 @@ def _meta_from_dtypes(to_read_columns, file_dtypes, index_cols, column_index_nam
if len(column_index_names) > 1:
df.columns.names = column_index_names
return df
def _guid():
"""Simple utility function to get random hex string"""
return uuid4().hex
......@@ -167,7 +167,7 @@ def describe_numeric_aggregate(stats, name=None, is_timedelta_col=False):
part1 = typ([count, mean, std, min], index=["count", "mean", "std", "min"])
q.index = ["{0:g}%".format(l * 100) for l in q.index.tolist()]
q.index = ["{0:g}%".format(l * 100) for l in tolist(q.index)]
if is_series_like(q) and typ != type(q):
q = q.to_frame()
part3 = typ([max], index=["max"])
......@@ -231,7 +231,7 @@ def describe_nonnumeric_aggregate(stats, name):
def _cum_aggregate_apply(aggregate, x, y):
""" Apply aggregation function within a cumulative aggregation
"""Apply aggregation function within a cumulative aggregation
Parameters
----------
......@@ -540,6 +540,19 @@ def concat_pandas(
return out
tolist_dispatch = Dispatch("tolist")
def tolist(obj):
func = tolist_dispatch.dispatch(type(obj))
return func(obj)
@tolist_dispatch.register((pd.Series, pd.Index, pd.Categorical))
def tolist_pandas(obj):
return obj.tolist()
# cuDF may try to import old dispatch functions
hash_df = hash_object_dispatch
group_split = group_split_dispatch
......
......@@ -91,7 +91,7 @@ from ..utils import M
def align_partitions(*dfs):
""" Mutually partition and align DataFrame blocks
"""Mutually partition and align DataFrame blocks
This serves as precursor to multi-dataframe operations like join, concat,
or merge.
......@@ -168,7 +168,7 @@ def _maybe_align_partitions(args):
def require(divisions, parts, required=None):
""" Clear out divisions where required components are not present
"""Clear out divisions where required components are not present
In left, right, or inner joins we exclude portions of the dataset if one
side or the other is not present. We can achieve this at the partition
......@@ -276,7 +276,7 @@ def hash_join(
shuffle=None,
indicator=False,
):
""" Join two DataFrames on particular columns with hash join
"""Join two DataFrames on particular columns with hash join
This shuffles both datasets on the joined column and then performs an
embarrassingly parallel join partition-by-partition
......@@ -375,8 +375,7 @@ def single_partition_join(left, right, **kwargs):
def warn_dtype_mismatch(left, right, left_on, right_on):
""" Checks for merge column dtype mismatches and throws a warning (#4574)
"""
"""Checks for merge column dtype mismatches and throws a warning (#4574)"""
if not isinstance(left_on, list):
left_on = [left_on]
......@@ -591,7 +590,7 @@ def most_recent_tail_summary(left, right, by=None):
def compute_tails(ddf, by=None):
""" For each partition, returns the last row of the most recent nonempty
"""For each partition, returns the last row of the most recent nonempty
partition.
"""
empty = ddf._meta.iloc[0:0]
......@@ -614,7 +613,7 @@ def most_recent_head_summary(left, right, by=None):
def compute_heads(ddf, by=None):
""" For each partition, returns the first row of the next nonempty
"""For each partition, returns the first row of the next nonempty
partition.
"""
empty = ddf._meta.iloc[0:0]
......@@ -627,7 +626,7 @@ def compute_heads(ddf, by=None):
def pair_partitions(L, R):
""" Returns which partitions to pair for the merge_asof algorithm and the
"""Returns which partitions to pair for the merge_asof algorithm and the
bounds on which to split them up
"""
result = []
......@@ -976,7 +975,7 @@ def concat(
ignore_unknown_divisions=False,
**kwargs
):
""" Concatenate DataFrames along rows.
"""Concatenate DataFrames along rows.
- When axis=0 (default), concatenate DataFrames row-wise:
......
......@@ -45,7 +45,10 @@ def to_numeric(arg, errors="raise", meta=None):
)
if is_array:
return arg.map_blocks(
pd.to_numeric, name=arg._name + "-to_numeric", meta=meta, errors=errors,
pd.to_numeric,
name=arg._name + "-to_numeric",
meta=meta,
errors=errors,
)
if is_scalar:
return delayed(pd.to_numeric, pure=True)(arg, errors=errors)
......@@ -29,7 +29,10 @@ def optimize(dsk, keys, **kwargs):
if fuse_subgraphs is None:
fuse_subgraphs = True
dsk, dependencies = fuse(
dsk, keys, dependencies=dependencies, fuse_subgraphs=fuse_subgraphs,
dsk,
keys,
dependencies=dependencies,
fuse_subgraphs=fuse_subgraphs,
)
dsk, _ = cull(dsk, keys)
return dsk
......
......@@ -429,8 +429,7 @@ def dtype_info(df):
def partition_quantiles(df, npartitions, upsample=1.0, random_state=None):
""" Approximate quantiles of Series used for repartitioning
"""
"""Approximate quantiles of Series used for repartitioning"""
assert isinstance(df, Series)
# currently, only Series has quantile method
# Index.quantile(list-like) must be pd.Series, not pd.Index
......
Supports Markdown
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment