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

Redo lost changes

parent 8d4ead19
......@@ -120,7 +120,7 @@ def pandas_read_text(
enforce=False,
path=None,
):
""" Convert a block of bytes to a Pandas DataFrame
"""Convert a block of bytes to a Pandas DataFrame
Parameters
----------
......@@ -164,7 +164,7 @@ def pandas_read_text(
def coerce_dtypes(df, dtypes):
""" Coerce dataframe to dtypes safely
"""Coerce dataframe to dtypes safely
Operates in place
......@@ -265,7 +265,7 @@ def text_blocks_to_pandas(
specified_dtypes=None,
path=None,
):
""" Convert blocks of bytes to a dask.dataframe
"""Convert blocks of bytes to a dask.dataframe
This accepts a list of lists of values of bytes where each list corresponds
to one file, and the value of bytes concatenate to comprise the entire
......@@ -726,13 +726,10 @@ def to_csv(
Parameters
----------
df : dask.DataFrame
Data to save
filename : string
Path glob indicating the naming scheme for the output files
name_function : callable, default None
Function accepting an integer (partition index) and producing a
string to replace the asterisk in the given filename globstring.
Should preserve the lexicographic order of partitions. Not
supported when `single_file` is `True`.
single_file : bool, default False
Whether to save everything into a single CSV file. Under the
single file mode, each partition is appended at the end of the
......@@ -740,74 +737,35 @@ def to_csv(
append mode and thus the single file mode, especially on cloud
storage systems such as S3 or GCS. A warning will be issued when
writing to a file that is not backed by a local filesystem.
compression : string or None
String like 'gzip' or 'xz'. Must support efficient random access.
Filenames with extensions corresponding to known compression
algorithms (gz, bz2) will be compressed accordingly automatically
compute: bool
If true, immediately executes. If False, returns a set of delayed
objects, which can be computed at a later time.
sep : character, default ','
Field delimiter for the output file
na_rep : string, default ''
Missing data representation
float_format : string, default None
Format string for floating point numbers
columns : sequence, optional
Columns to write
header : boolean or list of string, default True
Write out column names. If a list of string is given it is assumed
to be aliases for the column names
header_first_partition_only : boolean, default None
If set to `True`, only write the header row in the first output
file. By default, headers are written to all partitions under
the multiple file mode (`single_file` is `False`) and written
only once under the single file mode (`single_file` is `True`).
It must not be `False` under the single file mode.
index : boolean, default True
Write row names (index)
index_label : string or sequence, or False, default None
Column label for index column(s) if desired. If None is given, and
`header` and `index` are True, then the index names are used. A
sequence should be given if the DataFrame uses MultiIndex. If
False do not print fields for index names. Use index_label=False
for easier importing in R
nanRep : None
deprecated, use na_rep
mode : str
Python write mode, default 'w'
encoding : string, optional
A string representing the encoding to use in the output file,
defaults to 'ascii' on Python 2 and 'utf-8' on Python 3.
mode : str
Python write mode, default 'w'
name_function : callable, default None
Function accepting an integer (partition index) and producing a
string to replace the asterisk in the given filename globstring.
Should preserve the lexicographic order of partitions. Not
supported when `single_file` is `True`.
compression : string, optional
a string representing the compression to use in the output file,
allowed values are 'gzip', 'bz2', 'xz',
only used when the first argument is a filename
line_terminator : string, default '\\n'
The newline character or character sequence to use in the output
file
quoting : optional constant from csv module
defaults to csv.QUOTE_MINIMAL
quotechar : string (length 1), default '\"'
character used to quote fields
doublequote : boolean, default True
Control quoting of `quotechar` inside a field
escapechar : string (length 1), default None
character used to escape `sep` and `quotechar` when appropriate
chunksize : int or None
rows to write at a time
tupleize_cols : boolean, default False
write multi_index columns as a list of tuples (if True)
or new (expanded format) if False)
date_format : string, default None
Format string for datetime objects
decimal: string, default '.'
Character recognized as decimal separator. E.g. use ',' for
European data
storage_options: dict
compute : bool
If true, immediately executes. If False, returns a set of delayed
objects, which can be computed at a later time.
storage_options : dict
Parameters passed on to the backend filesystem class.
header_first_partition_only : boolean, default None
If set to `True`, only write the header row in the first output
file. By default, headers are written to all partitions under
the multiple file mode (`single_file` is `False`) and written
only once under the single file mode (`single_file` is `True`).
It must not be `False` under the single file mode.
compute_kwargs : dict, optional
Options to be passed in to the compute method
kwargs : dict, optional
Additional parameters to pass to `pd.DataFrame.to_csv()`
Returns
-------
......
......@@ -90,7 +90,7 @@ def make_timeseries(
seed=None,
**kwargs
):
""" Create timeseries dataframe with random data
"""Create timeseries dataframe with random data
Parameters
----------
......@@ -206,7 +206,7 @@ def daily_stock(
data_source="yahoo",
random_state=None,
):
""" Create artificial stock data
"""Create artificial stock data
This data matches daily open/high/low/close values from Yahoo! Finance, but
interpolates values within each day with random values. This makes the
......
......@@ -47,7 +47,7 @@ def to_hdf(
dask_kwargs={},
**kwargs
):
""" Store Dask Dataframe to Hierarchical Data Format (HDF) files
"""Store Dask Dataframe to Hierarchical Data Format (HDF) files
This is a parallel version of the Pandas function of the same name. Please
see the Pandas docstring for more detailed information about shared keyword
......
......@@ -78,7 +78,7 @@ def _meta_from_array(x, columns=None, index=None, meta=None):
def from_array(x, chunksize=50000, columns=None, meta=None):
""" Read any sliceable array into a Dask Dataframe
"""Read any sliceable array into a Dask Dataframe
Uses getitem syntax to pull slices out of the array. The array need not be
a NumPy array but must support slicing syntax
......@@ -228,7 +228,7 @@ def from_pandas(data, npartitions=None, chunksize=None, sort=True, name=None):
def from_bcolz(x, chunksize=None, categorize=True, index=None, lock=lock, **kwargs):
""" Read BColz CTable into a Dask Dataframe
"""Read BColz CTable into a Dask Dataframe
BColz is a fast on-disk compressed column store with careful attention
given to compression. https://bcolz.readthedocs.io/en/latest/
......@@ -318,7 +318,7 @@ def from_bcolz(x, chunksize=None, categorize=True, index=None, lock=lock, **kwar
def dataframe_from_ctable(x, slc, columns=None, categories=None, lock=lock):
""" Get DataFrame from bcolz.ctable
"""Get DataFrame from bcolz.ctable
Parameters
----------
......@@ -394,7 +394,7 @@ def dataframe_from_ctable(x, slc, columns=None, categories=None, lock=lock):
def from_dask_array(x, columns=None, index=None, meta=None):
""" Create a Dask DataFrame from a Dask Array.
"""Create a Dask DataFrame from a Dask Array.
Converts a 2d array into a DataFrame and a 1d array into a Series.
......@@ -485,7 +485,7 @@ def from_dask_array(x, columns=None, index=None, meta=None):
def _link(token, result):
""" A dummy function to link results together in a graph
"""A dummy function to link results together in a graph
We use this to enforce an artificial sequential ordering on tasks that
don't explicitly pass around a shared resource
......@@ -527,7 +527,7 @@ def to_bag(df, index=False):
def to_records(df):
""" Create Dask Array from a Dask Dataframe
"""Create Dask Array from a Dask Dataframe
Warning: This creates a dask.array without precise shape information.
Operations that depend on shape information, like slicing or reshaping,
......@@ -550,7 +550,7 @@ def to_records(df):
def from_delayed(
dfs, meta=None, divisions=None, prefix="from-delayed", verify_meta=True
):
""" Create Dask DataFrame from many Dask Delayed objects
"""Create Dask DataFrame from many Dask Delayed objects
Parameters
----------
......@@ -614,7 +614,7 @@ def from_delayed(
def sorted_division_locations(seq, npartitions=None, chunksize=None):
""" Find division locations and values in sorted list
"""Find division locations and values in sorted list
Examples
--------
......
from functools import partial
from collections import OrderedDict
from collections import defaultdict
import json
import warnings
from distutils.version import LooseVersion
......@@ -7,10 +7,11 @@ from distutils.version import LooseVersion
import pandas as pd
import pyarrow as pa
import pyarrow.parquet as pq
from pyarrow.compat import guid
from ....utils import natural_sort_key, getargspec
from ....utils import getargspec
from ..utils import _get_pyarrow_dtypes, _meta_from_dtypes
from ...utils import clear_known_categories
from ....core import flatten
from dask import delayed
from .utils import (
_parse_pandas_metadata,
......@@ -20,196 +21,38 @@ from .utils import (
)
preserve_ind_supported = pa.__version__ >= LooseVersion("0.15.0")
schema_field_supported = pa.__version__ >= LooseVersion("0.15.0")
def _get_md_row_groups(pieces):
""" Read file-footer metadata from each individual piece.
#
# Private Helper Functions
#
Since this operation can be painfully slow in some cases, abort
if any metadata or statistics are missing
"""
row_groups = []
row_groups_per_piece = []
for piece in pieces:
num_row_groups = piece.get_metadata().num_row_groups
for rg in range(num_row_groups):
row_group = piece.get_metadata().row_group(rg)
for c in range(row_group.num_columns):
if not row_group.column(c).statistics:
return (None, None)
row_groups.append(row_group)
row_groups_per_piece.append(num_row_groups)
if len(row_groups) == len(pieces):
row_groups_per_piece = None
# TODO: Skip row_groups_per_piece after ARROW-2801
return row_groups, row_groups_per_piece
def _get_row_groups_per_piece(pieces, metadata, path, fs):
""" Determine number of row groups in each dataset piece.
This function requires access to ParquetDataset.metadata
"""
# TODO: Remove this function after ARROW-2801
if metadata.num_row_groups == len(pieces):
return None # pieces already map to row-groups
result = OrderedDict()
for piece in pieces:
result[piece.path] = 0
for rg in range(metadata.num_row_groups):
filename = metadata.row_group(rg).column(0).file_path
if filename:
result[fs.sep.join([path, filename])] += 1
else:
return None # File path is missing, abort
return tuple(result.values())
def _merge_statistics(stats, s):
""" Update `stats` with vaules in `s`
"""
stats[-1]["total_byte_size"] += s["total_byte_size"]
stats[-1]["num-rows"] += s["num-rows"]
ncols = len(stats[-1]["columns"])
ncols_n = len(s["columns"])
if ncols != ncols_n:
raise ValueError(f"Column count not equal ({ncols} vs {ncols_n})")
for i in range(ncols):
name = stats[-1]["columns"][i]["name"]
j = i
for ii in range(ncols):
if name == s["columns"][j]["name"]:
break
if ii == ncols - 1:
raise KeyError(f"Column statistics missing for {name}")
j = (j + 1) % ncols
min_n = s["columns"][j]["min"]
max_n = s["columns"][j]["max"]
null_count_n = s["columns"][j]["null_count"]
min_i = stats[-1]["columns"][i]["min"]
max_i = stats[-1]["columns"][i]["max"]
stats[-1]["columns"][i]["min"] = min(min_i, min_n)
stats[-1]["columns"][i]["max"] = max(max_i, max_n)
stats[-1]["columns"][i]["null_count"] += null_count_n
return True
class SimplePiece:
""" SimplePiece
Surrogate class for PyArrow ParquetDatasetPiece.
Only used for flat datasets (not partitioned) where
a "_metadata" file is available.
"""
def __init__(self, path):
self.path = path
self.partition_keys = None
self.row_group = None
def _determine_dataset_parts(fs, paths, gather_statistics, filters, dataset_kwargs):
""" 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
dataset.
"""
parts = []
if len(paths) > 1:
base, fns = _analyze_paths(paths, fs)
if "_metadata" in fns:
# We have a _metadata file
# PyArrow cannot handle "_metadata"
# when `paths` is a list.
paths.remove(base + fs.sep + "_metadata")
fns.remove("_metadata")
if gather_statistics is not False:
# If we are allowed to gather statistics,
# lets use "_metadata" instead of opening
# every file. Note that we don't need to check if
# the dataset is flat here, because PyArrow cannot
# properly handle partitioning in this case anyway.
dataset = pq.ParquetDataset(
base + fs.sep + "_metadata",
filesystem=fs,
filters=filters,
**dataset_kwargs,
)
dataset.metadata = dataset.pieces[0].get_metadata()
dataset.pieces = [SimplePiece(path) for path in paths]
dataset.partitions = None
return parts, dataset
if gather_statistics is not False:
# This scans all the files
dataset = pq.ParquetDataset(
paths, filesystem=fs, filters=filters, **dataset_kwargs
def _append_row_groups(metadata, md):
try:
metadata.append_row_groups(md)
except RuntimeError as err:
if "requires equal schemas" in str(err):
raise RuntimeError(
"Schemas are inconsistent, try using "
'`to_parquet(..., schema="infer")`, or pass an explicit '
"pyarrow schema."
)
if dataset.schema is None:
# The dataset may have inconsistent schemas between files.
# If so, we should try to use a "_common_metadata" file
proxy_path = (
base + fs.sep + "_common_metadata"
if "_common_metadata" in fns
else paths[0]
)
dataset.schema = pq.ParquetDataset(proxy_path, filesystem=fs).schema
else:
# Rely on schema for 0th file.
# Will need to pass a list of paths to read_partition
dataset = pq.ParquetDataset(paths[0], filesystem=fs, **dataset_kwargs)
parts = [base + fs.sep + fn for fn in fns]
elif fs.isdir(paths[0]):
# This is a directory, check for _metadata, then _common_metadata
allpaths = fs.glob(paths[0] + fs.sep + "*")
base, fns = _analyze_paths(allpaths, fs)
# Check if dataset is "not flat" (partitioned into directories).
# If so, we will need to let pyarrow generate the `dataset` object.
not_flat = any([fs.isdir(p) for p in fs.glob(fs.sep.join([base, "*"]))])
if "_metadata" in fns and "validate_schema" not in dataset_kwargs:
dataset_kwargs["validate_schema"] = False
if not_flat or "_metadata" in fns or gather_statistics is not False:
# Let arrow do its thing (use _metadata or scan files)
dataset = pq.ParquetDataset(
paths, filesystem=fs, filters=filters, **dataset_kwargs
)
if dataset.schema is None:
# The dataset may have inconsistent schemas between files.
# If so, we should try to use a "_common_metadata" file
proxy_path = (
base + fs.sep + "_common_metadata"
if "_common_metadata" in fns
else allpaths[0]
)
dataset.schema = pq.ParquetDataset(proxy_path, filesystem=fs).schema
else:
# Use _common_metadata file if it is available.
# Otherwise, just use 0th file
if "_common_metadata" in fns:
dataset = pq.ParquetDataset(
base + fs.sep + "_common_metadata", filesystem=fs, **dataset_kwargs
)
else:
dataset = pq.ParquetDataset(
allpaths[0], filesystem=fs, **dataset_kwargs
)
parts = [base + fs.sep + fn for fn in fns if fn != "_common_metadata"]
else:
# There is only one file to read
dataset = pq.ParquetDataset(paths, filesystem=fs, **dataset_kwargs)
return parts, dataset
raise err
def _write_partitioned(table, root_path, partition_cols, fs, index_cols=(), **kwargs):
""" Write table to a partitioned dataset with pyarrow.
def _write_partitioned(
table, root_path, filename, partition_cols, fs, index_cols=(), **kwargs
):
"""Write table to a partitioned dataset with pyarrow.
Logic copied from pyarrow.parquet.
(arrow/python/pyarrow/parquet.py::write_to_dataset)
Logic copied from pyarrow.parquet.
(arrow/python/pyarrow/parquet.py::write_to_dataset)
TODO: Remove this in favor of pyarrow's `write_to_dataset`
once ARROW-8244 is addressed.
TODO: Remove this in favor of pyarrow's `write_to_dataset`
once ARROW-8244 is addressed.
"""
fs.mkdirs(root_path, exist_ok=True)
......@@ -245,12 +88,11 @@ def _write_partitioned(table, root_path, partition_cols, fs, index_cols=(), **kw
subgroup, preserve_index=preserve_index, schema=subschema, safe=False
)
prefix = fs.sep.join([root_path, subdir])
fs.mkdir(prefix, exists_ok=True)
outfile = guid() + ".parquet"
full_path = fs.sep.join([prefix, outfile])
fs.mkdirs(prefix, exist_ok=True)
full_path = fs.sep.join([prefix, filename])
with fs.open(full_path, "wb") as f:
pq.write_table(subtable, f, metadata_collector=md_list, **kwargs)
md_list[-1].set_file_path(fs.sep.join([subdir, outfile]))
md_list[-1].set_file_path(fs.sep.join([subdir, filename]))
return md_list
......@@ -265,149 +107,575 @@ def _index_in_schema(index, schema):
return False # No index to check
class ArrowEngine(Engine):
@classmethod
def read_metadata(
cls,
fs,
paths,
categories=None,
index=None,
gather_statistics=None,
filters=None,
split_row_groups=True,
**kwargs,
):
# Define the dataset object to use for metadata,
# Also, initialize `parts`. If `parts` is populated here,
# then each part will correspond to a file. Otherwise, each part will
# correspond to a row group (populated below)
parts, dataset = _determine_dataset_parts(
fs, paths, gather_statistics, filters, kwargs.get("dataset", {})
def _get_dataset_object(paths, fs, filters, dataset_kwargs):
"""Generate a ParquetDataset object"""
kwargs = dataset_kwargs.copy()
if "validate_schema" not in kwargs:
kwargs["validate_schema"] = False
if len(paths) > 1:
# This is a list of files
base, fns = _analyze_paths(paths, fs)
proxy_metadata = None
if "_metadata" in fns:
# We have a _metadata file. PyArrow cannot handle
# "_metadata" when `paths` is a list. So, we shuld
# open "_metadata" separately.
paths.remove(fs.sep.join([base, "_metadata"]))
fns.remove("_metadata")
with fs.open(fs.sep.join([base, "_metadata"]), mode="rb") as fil:
proxy_metadata = pq.ParquetFile(fil).metadata
# Create our dataset from the list of data files.
# Note #1: that this will not parse all the files (yet)
# Note #2: Cannot pass filters for legacy pyarrow API (see issue#6512).
# We can handle partitions + filtering for list input after
# adopting new pyarrow.dataset API.
dataset = pq.ParquetDataset(paths, filesystem=fs, **kwargs)
if proxy_metadata:
dataset.metadata = proxy_metadata
elif fs.isdir(paths[0]):
# This is a directory. We can let pyarrow do its thing.
# Note: In the future, it may be best to avoid listing the
# directory if we can get away with checking for the
# existence of _metadata. Listing may be much more
# expensive in storage systems like S3.
allpaths = fs.glob(paths[0] + fs.sep + "*")
base, fns = _analyze_paths(allpaths, fs)
dataset = pq.ParquetDataset(paths[0], filesystem=fs, filters=filters, **kwargs)
else:
# This is a single file. No danger in gathering statistics
# and/or splitting row-groups without a "_metadata" file
base = paths[0]
fns = [None]
dataset = pq.ParquetDataset(paths[0], filesystem=fs, **kwargs)
return dataset, base, fns
def _gather_metadata(
paths, fs, split_row_groups, gather_statistics, filters, dataset_kwargs
):
"""Gather parquet metadata into a single data structure.
Use _metadata or aggregate footer metadata into a single
object. Also, collect other information necessary for
parquet-to-ddf mapping (e.g. schema, partition_info).