From fa8bbb21ad0545d1e6640fde7147bc9de6b16de7 Mon Sep 17 00:00:00 2001 From: Han Wang Date: Thu, 28 May 2020 18:07:42 -0700 Subject: [PATCH] ArrowDataFrame and DataFrame test suites (#17) * dask repartition * update docs * update * refactor extensions * add cotransform * fix lint * update * update * test spark * update * update * update * update * update * update * update * update * update * update * update * update * update * update * update * update * update * add spark * fix 3.8 * fix 3.8 * add spark support * remove modin * remove modin * update * improve repartition * improve test coverage * update file system * arrow dataframe and test suite * fix pandas df name * add IO * update version --- .gitignore | 3 + README.md | 2 +- docs/api/fugue.dataframe.rst | 14 +- docs/api/fugue.execution.rst | 6 +- docs/api/fugue.utils.rst | 12 +- fugue/__init__.py | 2 +- fugue/dag/tasks.py | 6 +- fugue/dag/workflow.py | 51 ++++- fugue/dataframe/__init__.py | 7 +- fugue/dataframe/array_dataframe.py | 55 ++--- fugue/dataframe/arrow_dataframe.py | 133 +++++++++++ fugue/dataframe/dataframe.py | 55 +++-- fugue/dataframe/iterable_dataframe.py | 60 ++--- fugue/dataframe/pandas_dataframe.py | 122 ++++++++++ fugue/dataframe/pandas_dataframes.py | 132 ----------- fugue/dataframe/utils.py | 16 +- fugue/exceptions.py | 32 +++ fugue/execution/__init__.py | 2 +- fugue/execution/execution_engine.py | 53 +++-- ...n_engine.py => native_execution_engine.py} | 50 ++++- fugue/extensions/builtins/__init__.py | 4 +- fugue/extensions/builtins/creators.py | 12 + fugue/extensions/builtins/outputters.py | 21 ++ fugue/extensions/builtins/processors.py | 4 +- fugue/extensions/creator/convert.py | 4 +- fugue/extensions/processor/convert.py | 4 +- fugue/extensions/transformer/convert.py | 8 +- fugue/utils/file.py | 77 ------- fugue/utils/interfaceless.py | 13 ++ fugue/utils/io.py | 180 +++++++++++++++ fugue_dask/dataframe.py | 97 ++++---- fugue_dask/execution_engine.py | 46 +++- fugue_dask/utils.py | 24 +- fugue_spark/dataframe.py | 82 ++++--- fugue_spark/execution_engine.py | 104 +++++++-- fugue_spark/utils/convert.py | 34 ++- fugue_spark/utils/io.py | 128 +++++++++++ fugue_test/builtin_suite.py | 50 ++++- fugue_test/dataframe_suite.py | 208 ++++++++++++++++++ fugue_test/execution_suite.py | 117 +++++++--- setup.py | 2 +- tests/fugue/collections/test_partition.py | 1 + tests/fugue/dag/test_workflow.py | 10 +- tests/fugue/dataframe/test_array_dataframe.py | 53 +---- tests/fugue/dataframe/test_arrow_dataframe.py | 50 +++++ tests/fugue/dataframe/test_dataframes.py | 2 +- .../dataframe/test_iterable_dataframe.py | 100 ++------- .../fugue/dataframe/test_pandas_dataframe.py | 47 ++-- tests/fugue/dataframe/test_utils.py | 2 +- .../execution/test_naive_execution_engine.py | 14 +- .../fugue/extensions/creator/test_convert.py | 7 +- .../extensions/processor/test_convert.py | 6 + .../transformer/test_convert_transformer.py | 9 + tests/fugue/utils/test_file.py | 36 --- tests/fugue/utils/test_interfaceless.py | 29 ++- tests/fugue/utils/test_io.py | 131 +++++++++++ tests/fugue_dask/test_dataframe.py | 44 +--- tests/fugue_dask/test_execution_engine.py | 16 +- tests/fugue_spark/test_dataframe.py | 127 +++-------- tests/fugue_spark/test_execution_engine.py | 32 +-- tests/fugue_spark/utils/test_io.py | 128 +++++++++++ tests/fugue_spark/utils/test_partition.py | 2 +- 62 files changed, 2021 insertions(+), 857 deletions(-) create mode 100644 fugue/dataframe/arrow_dataframe.py create mode 100644 fugue/dataframe/pandas_dataframe.py delete mode 100644 fugue/dataframe/pandas_dataframes.py rename fugue/execution/{naive_execution_engine.py => native_execution_engine.py} (85%) delete mode 100644 fugue/utils/file.py create mode 100644 fugue/utils/io.py create mode 100644 fugue_spark/utils/io.py create mode 100644 fugue_test/dataframe_suite.py create mode 100644 tests/fugue/dataframe/test_arrow_dataframe.py delete mode 100644 tests/fugue/utils/test_file.py create mode 100644 tests/fugue/utils/test_io.py create mode 100644 tests/fugue_spark/utils/test_io.py diff --git a/.gitignore b/.gitignore index 0060166a..e37272b4 100644 --- a/.gitignore +++ b/.gitignore @@ -136,3 +136,6 @@ tmp # dask dask-worker-space + +# spark +spark-warehourse diff --git a/README.md b/README.md index 2fe1fdf2..eec4c977 100644 --- a/README.md +++ b/README.md @@ -5,6 +5,6 @@ [![PyPI license](https://img.shields.io/pypi/l/fugue.svg)](https://pypi.python.org/pypi/fugue/) [![PyPI version](https://badge.fury.io/py/fugue.svg)](https://pypi.python.org/pypi/fugue/) [![Coverage Status](https://coveralls.io/repos/github/fugue-project/fugue/badge.svg)](https://coveralls.io/github/fugue-project/fugue) -[![Doc](https://readthedocs.org/projects/fugue/badge)](https://traid.readthedocs.org) +[![Doc](https://readthedocs.org/projects/fugue/badge)](https://triad.readthedocs.org) An abstraction layer for distributed computation diff --git a/docs/api/fugue.dataframe.rst b/docs/api/fugue.dataframe.rst index 912c8933..d95e665c 100644 --- a/docs/api/fugue.dataframe.rst +++ b/docs/api/fugue.dataframe.rst @@ -10,6 +10,14 @@ fugue.dataframe.array\_dataframe :undoc-members: :show-inheritance: +fugue.dataframe.arrow\_dataframe +-------------------------------- + +.. automodule:: fugue.dataframe.arrow_dataframe + :members: + :undoc-members: + :show-inheritance: + fugue.dataframe.dataframe ------------------------- @@ -34,10 +42,10 @@ fugue.dataframe.iterable\_dataframe :undoc-members: :show-inheritance: -fugue.dataframe.pandas\_dataframes ----------------------------------- +fugue.dataframe.pandas\_dataframe +--------------------------------- -.. automodule:: fugue.dataframe.pandas_dataframes +.. automodule:: fugue.dataframe.pandas_dataframe :members: :undoc-members: :show-inheritance: diff --git a/docs/api/fugue.execution.rst b/docs/api/fugue.execution.rst index f6526f9a..eb5d2706 100644 --- a/docs/api/fugue.execution.rst +++ b/docs/api/fugue.execution.rst @@ -10,10 +10,10 @@ fugue.execution.execution\_engine :undoc-members: :show-inheritance: -fugue.execution.naive\_execution\_engine ----------------------------------------- +fugue.execution.native\_execution\_engine +----------------------------------------- -.. automodule:: fugue.execution.naive_execution_engine +.. automodule:: fugue.execution.native_execution_engine :members: :undoc-members: :show-inheritance: diff --git a/docs/api/fugue.utils.rst b/docs/api/fugue.utils.rst index baeb8c0b..2d8d291f 100644 --- a/docs/api/fugue.utils.rst +++ b/docs/api/fugue.utils.rst @@ -2,18 +2,18 @@ fugue.utils ============ -fugue.utils.file ----------------- +fugue.utils.interfaceless +------------------------- -.. automodule:: fugue.utils.file +.. automodule:: fugue.utils.interfaceless :members: :undoc-members: :show-inheritance: -fugue.utils.interfaceless -------------------------- +fugue.utils.io +-------------- -.. automodule:: fugue.utils.interfaceless +.. automodule:: fugue.utils.io :members: :undoc-members: :show-inheritance: diff --git a/fugue/__init__.py b/fugue/__init__.py index f1380eed..d3ec452c 100644 --- a/fugue/__init__.py +++ b/fugue/__init__.py @@ -1 +1 @@ -__version__ = "0.1.7" +__version__ = "0.2.0" diff --git a/fugue/dag/tasks.py b/fugue/dag/tasks.py index 9cfc0af6..c815115c 100644 --- a/fugue/dag/tasks.py +++ b/fugue/dag/tasks.py @@ -108,9 +108,9 @@ def handle_broadcast(self, df: DataFrame) -> DataFrame: return df return self.execution_engine.broadcast(df) - def pre_partition(self, *args: Any, **kwargs: Any) -> "FugueTask": - self._pre_partition = PartitionSpec(*args, **kwargs) - return self + # def pre_partition(self, *args: Any, **kwargs: Any) -> "FugueTask": + # self._pre_partition = PartitionSpec(*args, **kwargs) + # return self class Create(FugueTask): diff --git a/fugue/dag/workflow.py b/fugue/dag/workflow.py index adda7d21..bd51364d 100644 --- a/fugue/dag/workflow.py +++ b/fugue/dag/workflow.py @@ -17,6 +17,8 @@ SelectColumns, Show, Zip, + Load, + Save, ) from triad.collections import Schema from triad.utils.assertion import assert_or_throw @@ -53,11 +55,20 @@ def process( params: Any = None, pre_partition: Any = None, ) -> TDF: + if pre_partition is None: + pre_partition = self._metadata.get("pre_partition", PartitionSpec()) df = self.workflow.process( self, using=using, schema=schema, params=params, pre_partition=pre_partition ) return self.to_self_type(df) + def output(self, using: Any, params: Any = None, pre_partition: Any = None) -> None: + if pre_partition is None: + pre_partition = self._metadata.get("pre_partition", PartitionSpec()) + self.workflow.output( + self, using=using, params=params, pre_partition=pre_partition + ) + def show( self, rows: int = 10, @@ -167,6 +178,24 @@ def __getitem__(self: TDF, columns: List[Any]) -> TDF: ) return self.to_self_type(df) + def save( + self, + path: str, + fmt: str = "", + mode: str = "overwrite", + partition: Any = None, + single: bool = False, + **kwargs: Any, + ) -> None: + if partition is None: + partition = self._metadata.get("pre_partition", PartitionSpec()) + self.workflow.output( + self, + using=Save, + pre_partition=partition, + params=dict(path=path, fmt=fmt, mode=mode, single=single, params=kwargs), + ) + @property def schema(self) -> Schema: # pragma: no cover raise NotImplementedError("WorkflowDataFrame does not support this method") @@ -206,6 +235,12 @@ def as_array_iterable( ) -> Iterable[Any]: # pragma: no cover raise NotImplementedError("WorkflowDataFrame does not support this method") + def _drop_cols(self: TDF, cols: List[str]) -> DataFrame: # pragma: no cover + raise NotImplementedError("WorkflowDataFrame does not support this method") + + def _select_cols(self, keys: List[Any]) -> DataFrame: # pragma: no cover + raise NotImplementedError("WorkflowDataFrame does not support this method") + class FugueWorkflow(object): def __init__(self, execution_engine: ExecutionEngine): @@ -224,6 +259,16 @@ def create( ) return self.add(task) + def load( + self, path: str, fmt: str = "", columns: Any = None, **kwargs: Any + ) -> WorkflowDataFrame: + task = Create( + self.execution_engine, + creator=Load, + params=dict(path=path, fmt=fmt, columns=columns, params=kwargs), + ) + return self.add(task) + def process( self, *dfs: Any, @@ -389,9 +434,9 @@ def __init__( self.dependency[k] = self._parse_single_dependency(v) def _parse_single_dependency(self, dep: Any) -> str: - if isinstance(dep, tuple): # (cursor_like_obj, output_name) - cursor = self._parse_cursor(dep[0]) - return cursor._task.name + "." + dep[1] + # if isinstance(dep, tuple): # (cursor_like_obj, output_name) + # cursor = self._parse_cursor(dep[0]) + # return cursor._task.name + "." + dep[1] return self._parse_cursor(dep)._task.single_output_expression def _parse_cursor(self, dep: Any) -> WorkflowDataFrame: diff --git a/fugue/dataframe/__init__.py b/fugue/dataframe/__init__.py index bf285e85..3e8d0f59 100644 --- a/fugue/dataframe/__init__.py +++ b/fugue/dataframe/__init__.py @@ -1,7 +1,8 @@ # flake8: noqa from fugue.dataframe.array_dataframe import ArrayDataFrame -from fugue.dataframe.dataframe import DataFrame, LocalDataFrame, LocalBoundedDataFrame +from fugue.dataframe.arrow_dataframe import ArrowDataFrame +from fugue.dataframe.dataframe import DataFrame, LocalBoundedDataFrame, LocalDataFrame from fugue.dataframe.dataframes import DataFrames from fugue.dataframe.iterable_dataframe import IterableDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame -from fugue.dataframe.utils import to_local_df, to_local_bounded_df +from fugue.dataframe.pandas_dataframe import PandasDataFrame +from fugue.dataframe.utils import to_local_bounded_df, to_local_df diff --git a/fugue/dataframe/array_dataframe.py b/fugue/dataframe/array_dataframe.py index 15078b5c..644758c2 100644 --- a/fugue/dataframe/array_dataframe.py +++ b/fugue/dataframe/array_dataframe.py @@ -1,10 +1,11 @@ from typing import Any, Dict, Iterable, List, Optional + from fugue.dataframe.dataframe import ( DataFrame, LocalBoundedDataFrame, _get_schema_change, ) -from triad.exceptions import InvalidOperationError +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError from triad.utils.assertion import assert_or_throw from triad.utils.pyarrow import apply_schema @@ -13,22 +14,25 @@ class ArrayDataFrame(LocalBoundedDataFrame): def __init__( # noqa: C901 self, df: Any = None, schema: Any = None, metadata: Any = None ): - if df is None: - super().__init__(schema, metadata) - self._native = [] - elif isinstance(df, DataFrame): - if schema is None: - super().__init__(df.schema, metadata) - self._native = df.as_array(type_safe=False) - else: - schema, _ = _get_schema_change(df.schema, schema) + try: + if df is None: super().__init__(schema, metadata) - self._native = df.as_array(schema.names, type_safe=False) - elif isinstance(df, Iterable): - super().__init__(schema, metadata) - self._native = df if isinstance(df, List) else list(df) - else: - raise ValueError(f"{df} is incompatible with ArrayDataFrame") + self._native = [] + elif isinstance(df, DataFrame): + if schema is None: + super().__init__(df.schema, metadata) + self._native = df.as_array(type_safe=False) + else: + schema, _ = _get_schema_change(df.schema, schema) + super().__init__(schema, metadata) + self._native = df.as_array(schema.names, type_safe=False) + elif isinstance(df, Iterable): + super().__init__(schema, metadata) + self._native = df if isinstance(df, List) else list(df) + else: + raise ValueError(f"{df} is incompatible with ArrayDataFrame") + except Exception as e: + raise FugueDataFrameInitError(e) @property def native(self) -> List[Any]: @@ -39,22 +43,23 @@ def empty(self) -> bool: return self.count() == 0 def peek_array(self) -> Any: + self.assert_not_empty() return list(self.native[0]) def count(self) -> int: return len(self.native) - def drop(self, cols: List[str]) -> DataFrame: - try: - schema = self.schema - cols - except Exception as e: - raise InvalidOperationError(str(e)) - if len(schema) == 0: - raise InvalidOperationError("Can't remove all columns of a dataframe") - return ArrayDataFrame(self, schema) + def _drop_cols(self, cols: List[str]) -> DataFrame: + return ArrayDataFrame(self, self.schema - cols) + + def _select_cols(self, keys: List[Any]) -> DataFrame: + return ArrayDataFrame(self, self.schema.extract(keys)) def rename(self, columns: Dict[str, str]) -> "DataFrame": - schema = self.schema.rename(columns) + try: + schema = self.schema.rename(columns) + except Exception as e: + raise FugueDataFrameOperationError(e) return ArrayDataFrame(self.native, schema) def as_array( diff --git a/fugue/dataframe/arrow_dataframe.py b/fugue/dataframe/arrow_dataframe.py new file mode 100644 index 00000000..7da210a9 --- /dev/null +++ b/fugue/dataframe/arrow_dataframe.py @@ -0,0 +1,133 @@ +from typing import Any, Dict, Iterable, List, Optional + +import pandas as pd +import pyarrow as pa +from fugue.dataframe.dataframe import DataFrame, LocalBoundedDataFrame, _input_schema +from triad.collections.schema import Schema +from triad.exceptions import InvalidOperationError +from triad.utils.assertion import assert_or_throw +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError + + +class ArrowDataFrame(LocalBoundedDataFrame): + def __init__( # noqa: C901 + self, + df: Any = None, + schema: Any = None, + metadata: Any = None, + pandas_df_wrapper: bool = False, + ): + try: + if df is None: + schema = _input_schema(schema).assert_not_empty() + arr = [pa.array([])] * len(schema) + self._native = pa.Table.from_arrays(arr, schema=schema.pa_schema) + super().__init__(schema, metadata) + return + elif isinstance(df, pa.Table): + assert_or_throw( + schema is None, + InvalidOperationError("can't reset schema for pa.Table"), + ) + self._native = df + super().__init__(Schema(df.schema), metadata) + return + elif isinstance(df, (pd.DataFrame, pd.Series)): + if isinstance(df, pd.Series): + df = df.to_frame() + pdf = df + if schema is None: + self._native = pa.Table.from_pandas( + pdf, + schema=Schema(pdf).pa_schema, + preserve_index=False, + safe=True, + ) + schema = Schema(self._native.schema) + else: + schema = _input_schema(schema).assert_not_empty() + self._native = pa.Table.from_pandas( + pdf, schema=schema.pa_schema, preserve_index=False, safe=True + ) + super().__init__(schema, metadata) + return + elif isinstance(df, Iterable): + schema = _input_schema(schema).assert_not_empty() + # n = len(schema) + # arr = [] + # for i in range(n): + # arr.append([]) + # for row in df: + # for i in range(n): + # arr[i].append(row[i]) + # cols = [pa.array(arr[i], type=schema.types[i]) for i in range(n)] + # self._native = pa.Table.from_arrays(cols, schema=schema.pa_schema) + pdf = pd.DataFrame(df, columns=schema.names) + schema = _input_schema(schema).assert_not_empty() + self._native = pa.Table.from_pandas( + pdf, schema=schema.pa_schema, preserve_index=False, safe=True + ) + super().__init__(schema, metadata) + return + else: + raise ValueError(f"{df} is incompatible with ArrowDataFrame") + except Exception as e: + raise FugueDataFrameInitError(e) + + @property + def native(self) -> pa.Table: + return self._native + + @property + def empty(self) -> bool: + return self.count() == 0 + + def peek_array(self) -> Any: + self.assert_not_empty() + it = iter(self.as_array_iterable(type_safe=True)) + return next(it) + + def count(self) -> int: + return self.native.shape[0] + + def as_pandas(self) -> pd.DataFrame: + return self.native.to_pandas() + + def _drop_cols(self, cols: List[str]) -> DataFrame: + return ArrowDataFrame(self.native.drop(cols)) + + def _select_cols(self, keys: List[Any]) -> DataFrame: + schema = self.schema.extract(keys) + cols = [self.native.columns[self.schema.index_of_key(k)] for k in keys] + table = pa.Table.from_arrays(cols, schema=schema.pa_schema) + return ArrowDataFrame(table) + + def rename(self, columns: Dict[str, str]) -> "DataFrame": + try: + schema = self.schema.rename(columns) + except Exception as e: + raise FugueDataFrameOperationError(e) + df = pa.Table.from_arrays(self.native.columns, schema=schema.pa_schema) + return ArrowDataFrame(df) + + def as_arrow(self, type_safe: bool = False) -> pa.Table: + return self.native + + def as_array( + self, columns: Optional[List[str]] = None, type_safe: bool = False + ) -> List[Any]: + return list(self.as_array_iterable(columns, type_safe=type_safe)) + + def as_array_iterable( + self, columns: Optional[List[str]] = None, type_safe: bool = False + ) -> Iterable[Any]: + if self.empty: + return + if columns is not None: + for x in self[columns].as_array_iterable(type_safe=type_safe): + yield x + else: + d = self.native.to_pydict() + cols = [d[n] for n in self.schema.names] + for arr in zip(*cols): + yield list(arr) diff --git a/fugue/dataframe/dataframe.py b/fugue/dataframe/dataframe.py index 3cf21e7d..185ebb5e 100644 --- a/fugue/dataframe/dataframe.py +++ b/fugue/dataframe/dataframe.py @@ -5,10 +5,11 @@ import pandas as pd import pyarrow as pa +from fugue.exceptions import FugueDataFrameEmptyError, FugueDataFrameOperationError from triad.collections.dict import ParamDict from triad.collections.schema import Schema from triad.exceptions import InvalidOperationError -from triad.utils.assertion import assert_arg_not_none +from triad.utils.assertion import assert_or_throw class DataFrame(ABC): @@ -62,14 +63,6 @@ def as_local(self) -> "LocalDataFrame": # pragma: no cover def is_bounded(self) -> bool: # pragma: no cover raise NotImplementedError - # @abstractmethod - # def as_local(self) -> "DataFrame": # pragma: no cover - # raise NotImplementedError - - # @abstractmethod - # def apply_schema(self, schema: Any) -> None: # pragma: no cover - # raise NotImplementedError - @property @abstractmethod def num_partitions(self) -> int: # pragma: no cover @@ -80,6 +73,9 @@ def num_partitions(self) -> int: # pragma: no cover def empty(self) -> bool: # pragma: no cover raise NotImplementedError + def assert_not_empty(self) -> None: + assert_or_throw(not self.empty, FugueDataFrameEmptyError("dataframe is empty")) + @abstractmethod def peek_array(self) -> Any: # pragma: no cover raise NotImplementedError @@ -96,9 +92,13 @@ def as_pandas(self) -> pd.DataFrame: pdf = pd.DataFrame(self.as_array(), columns=self.schema.names) return _enforce_type(pdf, self.schema) - # @abstractmethod - # def as_pyarrow(self) -> pa.Table: # pragma: no cover - # raise NotImplementedError + def as_arrow(self, type_safe: bool = False) -> pa.Table: + return pa.Table.from_pandas( + self.as_pandas().reset_index(drop=True), + preserve_index=False, + schema=self.schema.pa_schema, + safe=type_safe, + ) @abstractmethod def as_array( @@ -113,17 +113,36 @@ def as_array_iterable( raise NotImplementedError @abstractmethod - def drop(self, cols: List[str]) -> "DataFrame": # pragma: no cover + def _drop_cols(self, cols: List[str]) -> "DataFrame": # pragma: no cover raise NotImplementedError @abstractmethod def rename(self, columns: Dict[str, str]) -> "DataFrame": # pragma: no cover raise NotImplementedError - def __getitem__(self, keys: List[Any]) -> "DataFrame": - assert_arg_not_none(keys, "keys") - cols = list((self.schema - keys).keys()) - return self.drop(cols) + @abstractmethod + def _select_cols(self, cols: List[Any]) -> "DataFrame": # pragma: no cover + raise NotImplementedError + + def drop(self, cols: List[str]) -> "DataFrame": + try: + schema = self.schema - cols + except Exception as e: + raise FugueDataFrameOperationError(e) + if len(schema) == 0: + raise FugueDataFrameOperationError( + "can't remove all columns of a dataframe" + ) + return self._drop_cols(cols) + + def __getitem__(self, cols: List[Any]) -> "DataFrame": + try: + schema = self.schema.extract(cols) + except Exception as e: + raise FugueDataFrameOperationError(e) + if len(schema) == 0: + raise FugueDataFrameOperationError("must select at least one column") + return self._select_cols(cols) def show( self, @@ -350,7 +369,7 @@ def _enforce_type(df: pd.DataFrame, schema: Schema) -> pd.DataFrame: ns = s.isnull() s = s.fillna(0).astype(v.type.to_pandas_dtype()) s[ns] = None - else: + elif not pa.types.is_struct(v.type) and not pa.types.is_list(v.type): s = s.astype(v.type.to_pandas_dtype()) df[k] = s return df diff --git a/fugue/dataframe/iterable_dataframe.py b/fugue/dataframe/iterable_dataframe.py index ecb5177a..819fe8e9 100644 --- a/fugue/dataframe/iterable_dataframe.py +++ b/fugue/dataframe/iterable_dataframe.py @@ -6,33 +6,36 @@ _get_schema_change, ) from triad.collections.schema import Schema -from triad.exceptions import InvalidOperationError from triad.utils.iter import EmptyAwareIterable, make_empty_aware from triad.utils.pyarrow import apply_schema +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError class IterableDataFrame(LocalUnboundedDataFrame): def __init__( # noqa: C901 self, df: Any = None, schema: Any = None, metadata: Any = None ): - if df is None: - idf: Iterable[Any] = [] - orig_schema: Optional[Schema] = None - elif isinstance(df, IterableDataFrame): - idf = df.native - orig_schema = df.schema - elif isinstance(df, DataFrame): - idf = df.as_array_iterable(type_safe=False) - orig_schema = df.schema - elif isinstance(df, Iterable): - idf = df - orig_schema = None - else: - raise ValueError(f"{df} is incompatible with IterableDataFrame") - schema, pos = _get_schema_change(orig_schema, schema) - super().__init__(schema, metadata) - self._pos = pos - self._native = make_empty_aware(self._preprocess(idf)) + try: + if df is None: + idf: Iterable[Any] = [] + orig_schema: Optional[Schema] = None + elif isinstance(df, IterableDataFrame): + idf = df.native + orig_schema = df.schema + elif isinstance(df, DataFrame): + idf = df.as_array_iterable(type_safe=False) + orig_schema = df.schema + elif isinstance(df, Iterable): + idf = df + orig_schema = None + else: + raise ValueError(f"{df} is incompatible with IterableDataFrame") + schema, pos = _get_schema_change(orig_schema, schema) + super().__init__(schema, metadata) + self._pos = pos + self._native = make_empty_aware(self._preprocess(idf)) + except Exception as e: + raise FugueDataFrameInitError(e) @property def native(self) -> EmptyAwareIterable[Any]: @@ -43,19 +46,20 @@ def empty(self) -> bool: return self.native.empty def peek_array(self) -> Any: + self.assert_not_empty() return list(self.native.peek()) - def drop(self, cols: List[str]) -> DataFrame: - try: - schema = self.schema - cols - except Exception as e: - raise InvalidOperationError(str(e)) - if len(schema) == 0: - raise InvalidOperationError("Can't remove all columns of a dataframe") - return IterableDataFrame(self, schema) + def _drop_cols(self, cols: List[str]) -> DataFrame: + return IterableDataFrame(self, self.schema - cols) + + def _select_cols(self, keys: List[Any]) -> DataFrame: + return IterableDataFrame(self, self.schema.extract(keys)) def rename(self, columns: Dict[str, str]) -> "DataFrame": - schema = self.schema.rename(columns) + try: + schema = self.schema.rename(columns) + except Exception as e: + raise FugueDataFrameOperationError(e) return IterableDataFrame(self.native, schema) def as_array( diff --git a/fugue/dataframe/pandas_dataframe.py b/fugue/dataframe/pandas_dataframe.py new file mode 100644 index 00000000..54d13547 --- /dev/null +++ b/fugue/dataframe/pandas_dataframe.py @@ -0,0 +1,122 @@ +from typing import Any, Dict, Iterable, List, Optional, Tuple + +import pandas as pd +from fugue.dataframe.dataframe import ( + DataFrame, + LocalBoundedDataFrame, + _enforce_type, + _input_schema, +) +from triad.collections.schema import Schema +from triad.utils.assertion import assert_or_throw +from triad.utils.pandas_like import PD_UTILS +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError + + +class PandasDataFrame(LocalBoundedDataFrame): + def __init__( # noqa: C901 + self, + df: Any = None, + schema: Any = None, + metadata: Any = None, + pandas_df_wrapper: bool = False, + ): + try: + apply_schema = True + if df is None: + schema = _input_schema(schema).assert_not_empty() + df = [] + if isinstance(df, PandasDataFrame): + # TODO: This is useless if in this way and wrong + pdf = df.native + schema = None + elif isinstance(df, (pd.DataFrame, pd.Series)): + if isinstance(df, pd.Series): + df = df.to_frame() + pdf = df + schema = None if schema is None else _input_schema(schema) + if pandas_df_wrapper and schema is not None: + apply_schema = False + elif isinstance(df, Iterable): + schema = _input_schema(schema).assert_not_empty() + pdf = pd.DataFrame(df, columns=schema.names) + pdf = _enforce_type(pdf, schema) + apply_schema = False + else: + raise ValueError(f"{df} is incompatible with PandasDataFrame") + if apply_schema: + pdf, schema = self._apply_schema(pdf, schema) + super().__init__(schema, metadata) + self._native = pdf + except Exception as e: + raise FugueDataFrameInitError(e) + + @property + def native(self) -> pd.DataFrame: + return self._native + + @property + def empty(self) -> bool: + return self.native.empty + + def peek_array(self) -> Any: + self.assert_not_empty() + return self.native.iloc[0].values.tolist() + + def count(self) -> int: + return self.native.shape[0] + + def as_pandas(self) -> pd.DataFrame: + return self._native + + def _drop_cols(self, cols: List[str]) -> DataFrame: + cols = (self.schema - cols).names + return self._select_cols(cols) + + def _select_cols(self, cols: List[Any]) -> DataFrame: + schema = self.schema.extract(cols) + return PandasDataFrame( + self.native[schema.names], schema, pandas_df_wrapper=True + ) + + def rename(self, columns: Dict[str, str]) -> "DataFrame": + try: + schema = self.schema.rename(columns) + except Exception as e: + raise FugueDataFrameOperationError(e) + df = self.native.rename(columns=columns) + return PandasDataFrame(df, schema, pandas_df_wrapper=True) + + def as_array( + self, columns: Optional[List[str]] = None, type_safe: bool = False + ) -> List[Any]: + return list(self.as_array_iterable(columns, type_safe=type_safe)) + + def as_array_iterable( + self, columns: Optional[List[str]] = None, type_safe: bool = False + ) -> Iterable[Any]: + for row in PD_UTILS.as_array_iterable( + self.native, + schema=self.schema.pa_schema, + columns=columns, + type_safe=type_safe, + ): + yield row + + def _apply_schema( + self, pdf: pd.DataFrame, schema: Optional[Schema] + ) -> Tuple[pd.DataFrame, Schema]: + PD_UTILS.ensure_compatible(pdf) + if pdf.columns.dtype == "object": # pdf has named schema + pschema = _input_schema(pdf) + if schema is None or pschema == schema: + return pdf, pschema.assert_not_empty() + pdf = pdf[schema.assert_not_empty().names] + else: # pdf has no named schema + schema = _input_schema(schema).assert_not_empty() + assert_or_throw( + pdf.shape[1] == len(schema), + ValueError(f"Pandas datafame column count doesn't match {schema}"), + ) + pdf.columns = schema.names + return _enforce_type(pdf, schema), schema diff --git a/fugue/dataframe/pandas_dataframes.py b/fugue/dataframe/pandas_dataframes.py deleted file mode 100644 index 374f75b2..00000000 --- a/fugue/dataframe/pandas_dataframes.py +++ /dev/null @@ -1,132 +0,0 @@ -from typing import Any, Dict, Iterable, List, Optional, Tuple - -import pandas as pd -import pyarrow as pa -from fugue.dataframe.dataframe import ( - DataFrame, - LocalBoundedDataFrame, - _enforce_type, - _input_schema, -) -from triad.collections.schema import Schema -from triad.exceptions import InvalidOperationError -from triad.utils.assertion import assert_arg_not_none, assert_or_throw -from triad.utils.pyarrow import apply_schema - - -class PandasDataFrame(LocalBoundedDataFrame): - def __init__( # noqa: C901 - self, - df: Any = None, - schema: Any = None, - metadata: Any = None, - pandas_df_wrapper: bool = False, - ): - apply_schema = True - if df is None: - schema = _input_schema(schema).assert_not_empty() - df = [] - if isinstance(df, PandasDataFrame): - # TODO: This is useless if in this way and wrong - pdf = df.native - schema = None - elif isinstance(df, (pd.DataFrame, pd.Series)): - if isinstance(df, pd.Series): - df = df.to_frame() - pdf = df - schema = None if schema is None else _input_schema(schema) - if pandas_df_wrapper and schema is not None: - apply_schema = False - elif isinstance(df, Iterable): - assert_arg_not_none(schema, msg="schema can't be None for iterable input") - schema = _input_schema(schema).assert_not_empty() - pdf = pd.DataFrame(df, columns=schema.names) - pdf = _enforce_type(pdf, schema) - apply_schema = False - else: - raise ValueError(f"{df} is incompatible with PandasDataFrame") - if apply_schema: - pdf, schema = self._apply_schema(pdf, schema) - super().__init__(schema, metadata) - self._native = pdf - - @property - def native(self) -> pd.DataFrame: - return self._native - - @property - def empty(self) -> bool: - return self.native.empty - - def peek_array(self) -> Any: - return self.native.iloc[0].values.tolist() - - def count(self) -> int: - return self.native.shape[0] - - def as_pandas(self) -> pd.DataFrame: - return self._native - - def drop(self, cols: List[str]) -> DataFrame: - try: - schema = self.schema - cols - except Exception as e: - raise InvalidOperationError(str(e)) - if len(schema) == 0: - raise InvalidOperationError("Can't remove all columns of a dataframe") - return PandasDataFrame( - self.native.drop(cols, axis=1), schema, pandas_df_wrapper=True - ) - - def rename(self, columns: Dict[str, str]) -> "DataFrame": - df = self.native.rename(columns=columns) - schema = self.schema.rename(columns) - return PandasDataFrame(df, schema, pandas_df_wrapper=True) - - def as_array( - self, columns: Optional[List[str]] = None, type_safe: bool = False - ) -> List[Any]: - return list(self.as_array_iterable(columns, type_safe=type_safe)) - - def as_array_iterable( - self, columns: Optional[List[str]] = None, type_safe: bool = False - ) -> Iterable[Any]: - if self._native.shape[0] == 0: - return - sub = self.schema if columns is None else self.schema.extract(columns) - df = self._native[sub.names] - if not type_safe or all( - not isinstance(x, (pa.StructType, pa.ListType)) for x in self.schema.types - ): - for arr in df.itertuples(index=False, name=None): - yield list(arr) - else: # TODO: If schema has nested types, the conversion will be much slower - for arr in apply_schema( - self.schema.pa_schema, - df.itertuples(index=False, name=None), - copy=True, - deep=True, - str_as_json=True, - ): - yield arr - - def _apply_schema( - self, pdf: pd.DataFrame, schema: Optional[Schema] - ) -> Tuple[pd.DataFrame, Schema]: - assert_or_throw( - pdf.empty or type(pdf.index) == pd.RangeIndex, - ValueError("Pandas datafame must have default index"), - ) - if pdf.columns.dtype == "object": # pdf has named schema - pschema = _input_schema(pdf) - if schema is None or pschema == schema: - return pdf, pschema.assert_not_empty() - pdf = pdf[schema.assert_not_empty().names] - else: # pdf has no named schema - schema = _input_schema(schema).assert_not_empty() - assert_or_throw( - pdf.shape[1] == len(schema), - ValueError(f"Pandas datafame column count doesn't match {schema}"), - ) - pdf.columns = schema.names - return _enforce_type(pdf, schema), schema diff --git a/fugue/dataframe/utils.py b/fugue/dataframe/utils.py index d31c548c..9d23e676 100644 --- a/fugue/dataframe/utils.py +++ b/fugue/dataframe/utils.py @@ -6,16 +6,17 @@ import pandas as pd from fs import open_fs -from fs.base import FS as FileSystem from fugue.dataframe.array_dataframe import ArrayDataFrame from fugue.dataframe.dataframe import DataFrame, LocalBoundedDataFrame, LocalDataFrame from fugue.dataframe.iterable_dataframe import IterableDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from triad.collections import Schema from triad.exceptions import InvalidOperationError from triad.utils.assertion import assert_arg_not_none from triad.utils.assertion import assert_or_throw as aot +from triad.collections.fs import FileSystem + def _df_eq( df: DataFrame, @@ -27,6 +28,7 @@ def _df_eq( check_schema: bool = True, check_content: bool = True, check_metadata: bool = True, + no_pandas: bool = False, throw=False, ) -> bool: """_df_eq is for internal, local test purpose only. DO NOT use @@ -46,8 +48,14 @@ def _df_eq( ), f"metadata mismatch {df.metadata}, {df2.metadata}" if not check_content: return True - d1 = df1.as_pandas() - d2 = df2.as_pandas() + if no_pandas: + dd1 = [[x.__repr__()] for x in df1.as_array_iterable(type_safe=True)] + dd2 = [[x.__repr__()] for x in df2.as_array_iterable(type_safe=True)] + d1 = pd.DataFrame(dd1, columns=["data"]) + d2 = pd.DataFrame(dd2, columns=["data"]) + else: + d1 = df1.as_pandas() + d2 = df2.as_pandas() if not check_order: d1 = d1.sort_values(df1.schema.names) d2 = d2.sort_values(df1.schema.names) diff --git a/fugue/exceptions.py b/fugue/exceptions.py index 7fa09d53..4629ed08 100644 --- a/fugue/exceptions.py +++ b/fugue/exceptions.py @@ -17,6 +17,38 @@ def __init__(self, *args: Any): super().__init__(*args) +class FugueDataFrameError(FugueError): + """Fugue dataframe related error + """ + + def __init__(self, *args: Any): + super().__init__(*args) + + +class FugueDataFrameInitError(FugueDataFrameError): + """Fugue dataframe is empty + """ + + def __init__(self, *args: Any): + super().__init__(*args) + + +class FugueDataFrameEmptyError(FugueDataFrameError): + """Fugue dataframe is empty + """ + + def __init__(self, *args: Any): + super().__init__(*args) + + +class FugueDataFrameOperationError(FugueDataFrameError): + """Fugue dataframe is empty + """ + + def __init__(self, *args: Any): + super().__init__(*args) + + class FugueWorkflowError(FugueError): """Fugue workflow exceptions """ diff --git a/fugue/execution/__init__.py b/fugue/execution/__init__.py index 3a904d39..d6d38b18 100644 --- a/fugue/execution/__init__.py +++ b/fugue/execution/__init__.py @@ -1,3 +1,3 @@ # flake8: noqa from fugue.execution.execution_engine import ExecutionEngine, SQLEngine -from fugue.execution.naive_execution_engine import NaiveExecutionEngine, SqliteEngine +from fugue.execution.native_execution_engine import NativeExecutionEngine, SqliteEngine diff --git a/fugue/execution/execution_engine.py b/fugue/execution/execution_engine.py index 66f224ab..5e33b271 100644 --- a/fugue/execution/execution_engine.py +++ b/fugue/execution/execution_engine.py @@ -1,8 +1,7 @@ import logging from abc import ABC, abstractmethod -from typing import Any, Callable, Dict, Iterable, List, Optional +from typing import Any, Callable, Dict, Iterable, List, Optional, Union -from fs.base import FS as FileSystem from fugue.collections.partition import ( EMPTY_PARTITION_SPEC, PartitionCursor, @@ -14,6 +13,7 @@ from fugue.dataframe.utils import deserialize_df, serialize_df from fugue.exceptions import FugueBug from triad.collections import ParamDict, Schema +from triad.collections.fs import FileSystem from triad.exceptions import InvalidOperationError from triad.utils.assertion import assert_or_throw from triad.utils.convert import to_size @@ -29,9 +29,9 @@ def __init__(self, execution_engine: "ExecutionEngine") -> None: def execution_engine(self) -> "ExecutionEngine": return self._execution_engine - @property - def conf(self) -> ParamDict: - return self.execution_engine.conf + # @property + # def conf(self) -> ParamDict: + # return self.execution_engine.conf @abstractmethod def select(self, dfs: DataFrames, statement: str) -> DataFrame: # pragma: no cover @@ -77,6 +77,7 @@ def repartition( ) -> DataFrame: # pragma: no cover raise NotImplementedError + @abstractmethod def map( self, df: DataFrame, @@ -138,7 +139,7 @@ def serialize_by_partition( ) return self.map(df, s.run, output_schema, partition_spec, metadata) - def zip_dataframes( + def zip( self, df1: DataFrame, df2: DataFrame, @@ -151,7 +152,7 @@ def zip_dataframes( how = how.lower() assert_or_throw( "semi" not in how and "anti" not in how, - InvalidOperationError("zip_dataframes does not support semi or anti joins"), + InvalidOperationError("zip does not support semi or anti joins"), ) to_file_threshold = ( -1 if to_file_threshold == -1 else to_size(to_file_threshold) @@ -212,22 +213,28 @@ def comap( df, cs.run, output_schema, partition_spec, metadata, on_init=cs.on_init ) - # @abstractmethod - # def load_df( - # self, path: str, format_hint: Any = None, **kwargs: Any - # ) -> DataFrame: # pragma: no cover - # raise NotImplementedError - - # @abstractmethod - # def save_df( - # self, - # df: DataFrame, - # path: str, - # overwrite: bool, - # format_hint: Any = None, - # **kwargs: Any, - # ) -> None: # pragma: no cover - # raise NotImplementedError + @abstractmethod + def load_df( + self, + path: Union[str, List[str]], + format_hint: Any = None, + columns: Any = None, + **kwargs: Any, + ) -> DataFrame: # pragma: no cover + raise NotImplementedError + + @abstractmethod + def save_df( + self, + df: DataFrame, + path: str, + format_hint: Any = None, + mode: str = "overwrite", + partition_spec: PartitionSpec = EMPTY_PARTITION_SPEC, + force_single: bool = False, + **kwargs: Any, + ) -> None: # pragma: no cover + raise NotImplementedError def __copy__(self) -> "ExecutionEngine": return self diff --git a/fugue/execution/naive_execution_engine.py b/fugue/execution/native_execution_engine.py similarity index 85% rename from fugue/execution/naive_execution_engine.py rename to fugue/execution/native_execution_engine.py index ae767773..6a198b9b 100644 --- a/fugue/execution/naive_execution_engine.py +++ b/fugue/execution/native_execution_engine.py @@ -1,11 +1,13 @@ import logging -from typing import Any, Callable, Iterable, List, Optional +from typing import Any, Callable, Iterable, List, Optional, Union import pandas as pd import pyarrow as pa -from fs.base import FS as FileSystem -from fs.osfs import OSFS -from fugue.collections.partition import PartitionCursor, PartitionSpec +from fugue.collections.partition import ( + EMPTY_PARTITION_SPEC, + PartitionCursor, + PartitionSpec, +) from fugue.dataframe import ( DataFrame, DataFrames, @@ -20,11 +22,13 @@ ExecutionEngine, SQLEngine, ) +from fugue.utils.io import load_df, save_df from sqlalchemy import create_engine from triad.collections import Schema +from triad.collections.dict import ParamDict +from triad.collections.fs import FileSystem from triad.utils.assertion import assert_or_throw from triad.utils.pandas_like import PD_UTILS -from triad.collections.dict import ParamDict class SqliteEngine(SQLEngine): @@ -39,15 +43,15 @@ def select(self, dfs: DataFrames, statement: str) -> DataFrame: return PandasDataFrame(df) -class NaiveExecutionEngine(ExecutionEngine): +class NativeExecutionEngine(ExecutionEngine): def __init__(self, conf: Any = None): super().__init__(conf) - self._fs = OSFS("/") + self._fs = FileSystem() self._log = logging.getLogger() self._default_sql_engine = SqliteEngine(self) def __repr__(self) -> str: - return "NaiveExecutionEngine" + return "NativeExecutionEngine" @property def log(self) -> logging.Logger: @@ -190,6 +194,36 @@ def join( ) return PandasDataFrame(d.reset_index(drop=True), output_schema, metadata) + def load_df( + self, + path: Union[str, List[str]], + format_hint: Any = None, + columns: Any = None, + **kwargs: Any, + ) -> LocalBoundedDataFrame: + return self.to_df( + load_df( + path, format_hint=format_hint, columns=columns, fs=self.fs, **kwargs + ) + ) + + def save_df( + self, + df: DataFrame, + path: str, + format_hint: Any = None, + mode: str = "overwrite", + partition_spec: PartitionSpec = EMPTY_PARTITION_SPEC, + force_single: bool = False, + **kwargs: Any, + ) -> None: + if not partition_spec.empty: + self.log.warning( # pragma: no cover + f"partition_spec is not respected in {self}.save_df" + ) + df = self.to_df(df) + save_df(df, path, format_hint=format_hint, mode=mode, fs=self.fs, **kwargs) + def _validate_outer_joinable(self, schema: Schema, key_schema: Schema) -> None: # TODO: this is to prevent wrong behavior of pandas, we may not need it # s = schema - key_schema diff --git a/fugue/extensions/builtins/__init__.py b/fugue/extensions/builtins/__init__.py index 1d6c279f..e746824e 100644 --- a/fugue/extensions/builtins/__init__.py +++ b/fugue/extensions/builtins/__init__.py @@ -1,6 +1,6 @@ # flake8: noqa -from fugue.extensions.builtins.outputters import Show, AssertEqual -from fugue.extensions.builtins.creators import CreateData +from fugue.extensions.builtins.outputters import Show, AssertEqual, Save +from fugue.extensions.builtins.creators import CreateData, Load from fugue.extensions.builtins.processors import ( RunJoin, RunTransformer, diff --git a/fugue/extensions/builtins/creators.py b/fugue/extensions/builtins/creators.py index 56b5eac6..4ad2e811 100644 --- a/fugue/extensions/builtins/creators.py +++ b/fugue/extensions/builtins/creators.py @@ -9,3 +9,15 @@ def create(self) -> DataFrame: self.params.get_or_none("schema", object), self.params.get_or_none("metadata", object), ) + + +class Load(Creator): + def create(self) -> DataFrame: + kwargs = self.params.get("params", dict()) + path = self.params.get_or_throw("path", str) + format_hint = self.params.get("fmt", "") + columns = self.params.get_or_none("columns", object) + + return self.execution_engine.load_df( + path=path, format_hint=format_hint, columns=columns, **kwargs + ) diff --git a/fugue/extensions/builtins/outputters.py b/fugue/extensions/builtins/outputters.py index 8d7eee3b..49d82966 100644 --- a/fugue/extensions/builtins/outputters.py +++ b/fugue/extensions/builtins/outputters.py @@ -20,3 +20,24 @@ def process(self, dfs: DataFrames) -> None: expected = dfs[0] for i in range(1, len(dfs)): df_eq(expected, dfs[i], throw=True, **self.params) + + +class Save(Outputter): + def process(self, dfs: DataFrames) -> None: + assert len(dfs) == 1 + kwargs = self.params.get("params", dict()) + path = self.params.get_or_throw("path", str) + format_hint = self.params.get("fmt", "") + mode = self.params.get("mode", "overwrite") + partition_spec = self.partition_spec + force_single = self.params.get("single", False) + + self.execution_engine.save_df( + df=dfs[0], + path=path, + format_hint=format_hint, + mode=mode, + partition_spec=partition_spec, + force_single=force_single, + **kwargs + ) diff --git a/fugue/extensions/builtins/processors.py b/fugue/extensions/builtins/processors.py index a45d9640..a1d93924 100644 --- a/fugue/extensions/builtins/processors.py +++ b/fugue/extensions/builtins/processors.py @@ -106,7 +106,7 @@ def process(self, dfs: DataFrames) -> DataFrame: return self.execution_engine.serialize_by_partition( dfs[0], partition_spec, "_0", temp_path, to_file_threshold ) - df = self.execution_engine.zip_dataframes( + df = self.execution_engine.zip( dfs[0], dfs[1], how=how, @@ -115,7 +115,7 @@ def process(self, dfs: DataFrames) -> DataFrame: to_file_threshold=to_file_threshold, ) for i in range(2, len(dfs)): - df = self.execution_engine.zip_dataframes( + df = self.execution_engine.zip( df, dfs[i], how=how, diff --git a/fugue/extensions/creator/convert.py b/fugue/extensions/creator/convert.py index 6275a563..997cbdd5 100644 --- a/fugue/extensions/creator/convert.py +++ b/fugue/extensions/creator/convert.py @@ -4,7 +4,7 @@ from fugue.extensions.creator.creator import Creator from fugue.dataframe import DataFrame from fugue.exceptions import FugueInterfacelessError -from fugue.utils.interfaceless import FunctionWrapper +from fugue.utils.interfaceless import FunctionWrapper, parse_output_schema_from_comment from triad.collections import Schema from triad.utils.assertion import assert_or_throw from triad.utils.convert import to_function, to_instance @@ -56,6 +56,8 @@ def __call__(self, *args: Any, **kwargs: Any) -> Any: @no_type_check @staticmethod def from_func(func: Callable, schema: Any) -> "_FuncAsCreator": + if schema is None: + schema = parse_output_schema_from_comment(func) tr = _FuncAsCreator() tr._wrapper = FunctionWrapper(func, "^e?x*$", "^[dlsp]$") # type: ignore tr._need_engine = tr._wrapper.input_code.startswith("e") diff --git a/fugue/extensions/processor/convert.py b/fugue/extensions/processor/convert.py index 22f4767e..7ffcea88 100644 --- a/fugue/extensions/processor/convert.py +++ b/fugue/extensions/processor/convert.py @@ -4,7 +4,7 @@ from fugue.dataframe import DataFrame, DataFrames from fugue.exceptions import FugueInterfacelessError from fugue.extensions.processor.processor import Processor -from fugue.utils.interfaceless import FunctionWrapper +from fugue.utils.interfaceless import FunctionWrapper, parse_output_schema_from_comment from triad.collections import Schema from triad.utils.assertion import assert_or_throw from triad.utils.convert import to_function, to_instance @@ -63,6 +63,8 @@ def __call__(self, *args: Any, **kwargs: Any) -> Any: @no_type_check @staticmethod def from_func(func: Callable, schema: Any) -> "_FuncAsProcessor": + if schema is None: + schema = parse_output_schema_from_comment(func) tr = _FuncAsProcessor() tr._wrapper = FunctionWrapper( func, "^e?(c|[dlsp]+)x*$", "^[dlsp]$" diff --git a/fugue/extensions/transformer/convert.py b/fugue/extensions/transformer/convert.py index c2ae9738..4f115fdb 100644 --- a/fugue/extensions/transformer/convert.py +++ b/fugue/extensions/transformer/convert.py @@ -4,7 +4,7 @@ from fugue.dataframe import DataFrame, DataFrames, LocalDataFrame from fugue.exceptions import FugueInterfacelessError from fugue.extensions.transformer.transformer import CoTransformer, Transformer -from fugue.utils.interfaceless import FunctionWrapper +from fugue.utils.interfaceless import FunctionWrapper, parse_output_schema_from_comment from triad.collections.schema import Schema from triad.utils.assertion import assert_arg_not_none from triad.utils.convert import to_function, to_instance @@ -27,7 +27,7 @@ def deco(func: Callable) -> _FuncAsCoTransformer: def to_transformer( # noqa: C901 - obj: Any, schema: Any + obj: Any, schema: Any = None ) -> Union[Transformer, CoTransformer]: exp: Optional[Exception] = None try: @@ -87,6 +87,8 @@ def _parse_schema(self, obj: Any, df: DataFrame) -> Schema: @staticmethod def from_func(func: Callable, schema: Any) -> "_FuncAsTransformer": + if schema is None: + schema = parse_output_schema_from_comment(func) assert_arg_not_none(schema, "schema") tr = _FuncAsTransformer() tr._wrapper = FunctionWrapper(func, "^[lsp]x*$", "^[lsp]$") # type: ignore @@ -122,6 +124,8 @@ def _parse_schema(self, obj: Any) -> Schema: @staticmethod def from_func(func: Callable, schema: Any) -> "_FuncAsCoTransformer": + if schema is None: + schema = parse_output_schema_from_comment(func) assert_arg_not_none(schema, "schema") tr = _FuncAsCoTransformer() tr._wrapper = FunctionWrapper(func, "^(c|[lsp]+)x*$", "^[lsp]$") # type: ignore diff --git a/fugue/utils/file.py b/fugue/utils/file.py deleted file mode 100644 index 28461d9a..00000000 --- a/fugue/utils/file.py +++ /dev/null @@ -1,77 +0,0 @@ -import pathlib -from typing import Dict, Optional, Callable, Any -from urllib.parse import urlparse - -from triad.utils.assertion import assert_or_throw -import pandas as pd - -_FORMAT_MAP: Dict[str, str] = { - ".csv": "csv", - ".csv.gz": "csv", - ".parquet": "parquet", - ".json": "json", - ".json.gz": "json", -} - -_FORMAT_LOAD: Dict[str, Callable[..., pd.DataFrame]] = { - "csv": lambda p, **kwargs: pd.from_csv(p.uri, **kwargs), - "parquet": lambda p, **kwargs: pd.from_parquet(p.uri, **kwargs), - "json": lambda p, **kwargs: pd.from_json(p.uri, **kwargs), -} - -_FORMAT_SAVE: Dict[str, Callable] = { - "csv": lambda df, p, **kwargs: df.to_csv(p.uri, **kwargs), - "parquet": lambda df, p, **kwargs: df.to_parquet(p.uri, **kwargs), - "json": lambda df, p, **kwargs: df.to_json(p.uri, **kwargs), -} - - -class FileParser(object): - def __init__(self, uri: str, format_hint: Optional[str] = None): - self._uri = urlparse(uri) - if format_hint is None or format_hint == "": - assert_or_throw( - self.suffix in _FORMAT_MAP, - NotImplementedError(f"{self.suffix} is not supported"), - ) - self._format = _FORMAT_MAP[self.suffix] - else: - assert_or_throw( - format_hint in _FORMAT_MAP.values(), - NotImplementedError(f"{format_hint} is not supported"), - ) - self._format = format_hint - - @property - def uri(self) -> str: - return self._uri.geturl() - - @property - def scheme(self) -> str: - return self._uri.scheme - - @property - def path(self) -> str: - return self._uri.path - - @property - def suffix(self) -> str: - return "".join(pathlib.Path(self.path.lower()).suffixes) - - @property - def file_format(self) -> str: - return self._format - - -def load_pandas( - uri: str, format_hint: Optional[str] = None, **kwargs: Any -) -> pd.DataFrame: - p = FileParser(uri, format_hint) - return _FORMAT_LOAD[p.file_format](p, **kwargs) - - -def save_pandas( - df: pd.DataFrame, uri: str, format_hint: Optional[str] = None, **kwargs: Any -) -> None: - p = FileParser(uri, format_hint) - _FORMAT_SAVE[p.file_format](df, p, **kwargs) diff --git a/fugue/utils/interfaceless.py b/fugue/utils/interfaceless.py index 052273fd..5631ac88 100644 --- a/fugue/utils/interfaceless.py +++ b/fugue/utils/interfaceless.py @@ -18,6 +18,19 @@ from triad.utils.iter import make_empty_aware, EmptyAwareIterable from fugue.dataframe.dataframes import DataFrames +_COMMENT_SCHEMA_ANNOTATION = "schema:" + + +def parse_output_schema_from_comment(func: Callable) -> Optional[str]: + for comment in reversed((inspect.getcomments(func) or "").splitlines()): + comment = comment.replace(" ", "").replace("#", "") + if not comment.startswith(_COMMENT_SCHEMA_ANNOTATION): + continue + s = comment[len(_COMMENT_SCHEMA_ANNOTATION) :] + if s != "": + return s + return None + class FunctionWrapper(object): def __init__(self, func: Callable, params_re: str = ".*", return_re: str = ".*"): diff --git a/fugue/utils/io.py b/fugue/utils/io.py new file mode 100644 index 00000000..210c52f8 --- /dev/null +++ b/fugue/utils/io.py @@ -0,0 +1,180 @@ +import pathlib +from typing import Any, Callable, Dict, List, Optional, Tuple, Union +from urllib.parse import urlparse + +import pandas as pd +from fugue.dataframe import LocalBoundedDataFrame, LocalDataFrame, PandasDataFrame +from triad.collections.fs import FileSystem +from triad.collections.schema import Schema +from triad.exceptions import InvalidOperationError +from triad.utils.assertion import assert_or_throw + + +class FileParser(object): + def __init__(self, uri: str, format_hint: Optional[str] = None): + self._uri = urlparse(uri) + if format_hint is None or format_hint == "": + assert_or_throw( + self.suffix in _FORMAT_MAP, + NotImplementedError(f"{self.suffix} is not supported"), + ) + self._format = _FORMAT_MAP[self.suffix] + else: + assert_or_throw( + format_hint in _FORMAT_MAP.values(), + NotImplementedError(f"{format_hint} is not supported"), + ) + self._format = format_hint + + @property + def uri(self) -> str: + return self._uri.geturl() + + @property + def scheme(self) -> str: + return self._uri.scheme + + @property + def path(self) -> str: + return self._uri.path + + @property + def suffix(self) -> str: + return "".join(pathlib.Path(self.path.lower()).suffixes) + + @property + def file_format(self) -> str: + return self._format + + +def load_df( + uri: Union[str, List[str]], + format_hint: Optional[str] = None, + columns: Any = None, + fs: Optional[FileSystem] = None, + **kwargs: Any, +) -> LocalBoundedDataFrame: + if isinstance(uri, str): + fp = [FileParser(uri, format_hint)] + else: + fp = [FileParser(u, format_hint) for u in uri] + dfs: List[pd.DataFrame] = [] + schema: Any = None + for f in fp: + df, schema = _FORMAT_LOAD[f.file_format](f, columns, **kwargs) + dfs.append(df) + return PandasDataFrame(pd.concat(dfs), schema) + + +def save_df( + df: LocalDataFrame, + uri: str, + format_hint: Optional[str] = None, + mode: str = "overwrite", + fs: Optional[FileSystem] = None, + **kwargs: Any, +) -> None: + assert_or_throw( + mode in ["overwrite", "error"], NotImplementedError(f"{mode} is not supported") + ) + p = FileParser(uri, format_hint) + if fs is None: + fs = FileSystem() + if fs.exists(uri): + assert_or_throw(mode == "overwrite", FileExistsError(uri)) + _FORMAT_SAVE[p.file_format](df, p, **kwargs) + + +def _save_parquet(df: LocalDataFrame, p: FileParser, **kwargs: Any) -> None: + df.as_pandas().to_parquet( + p.uri, **{"engine": "pyarrow", "schema": df.schema.pa_schema, **kwargs} + ) + + +def _load_parquet( + p: FileParser, columns: Any = None, **kwargs: Any +) -> Tuple[pd.DataFrame, Any]: + if columns is None: + pdf = pd.read_parquet(p.uri, **{"engine": "pyarrow", **kwargs}) + return pdf, None + if isinstance(columns, list): # column names + pdf = pd.read_parquet(p.uri, columns=columns, **{"engine": "pyarrow", **kwargs}) + return pdf, None + schema = Schema(columns) + pdf = pd.read_parquet( + p.uri, columns=schema.names, **{"engine": "pyarrow", **kwargs} + ) + return pdf, schema + + +def _save_csv(df: LocalDataFrame, p: FileParser, **kwargs: Any) -> None: + df.as_pandas().to_csv(p.uri, **{"index": False, "header": False, **kwargs}) + + +def _load_csv( + p: FileParser, columns: Any = None, **kwargs: Any +) -> Tuple[pd.DataFrame, Any]: + kw = dict(kwargs) + header = kw.get("header", False) + if "header" in kw: + del kw["header"] + if str(header) in ["True", "0"]: + pdf = pd.read_csv(p.uri, **{"index_col": False, "header": 0, **kw}) + if columns is None: + return pdf, None + if isinstance(columns, list): # column names + return pdf[columns], None + schema = Schema(columns) + return pdf[schema.names], schema + if header is None or str(header) == "False": + if columns is None: + raise InvalidOperationError("columns must be set if without header") + if isinstance(columns, list): # column names + pdf = pd.read_csv( + p.uri, **{"index_col": False, "header": None, "names": columns, **kw} + ) + return pdf, None + schema = Schema(columns) + pdf = pd.read_csv( + p.uri, **{"index_col": False, "header": None, "names": schema.names, **kw} + ) + return pdf, schema + else: + raise NotImplementedError(f"{header} is not supported") + + +def _save_json(df: LocalDataFrame, p: FileParser, **kwargs: Any) -> None: + df.as_pandas().to_json(p.uri, **kwargs) + + +def _load_json( + p: FileParser, columns: Any = None, **kwargs: Any +) -> Tuple[pd.DataFrame, Any]: + pdf = pd.read_json(p.uri, **kwargs).reset_index(drop=True) + if columns is None: + return pdf, None + if isinstance(columns, list): # column names + return pdf[columns], None + schema = Schema(columns) + return pdf[schema.names], schema + + +_FORMAT_MAP: Dict[str, str] = { + ".csv": "csv", + ".csv.gz": "csv", + ".parquet": "parquet", + ".json": "json", + ".json.gz": "json", +} + +_FORMAT_LOAD: Dict[str, Callable[..., Tuple[pd.DataFrame, Any]]] = { + "csv": _load_csv, + "parquet": _load_parquet, + "json": _load_json, +} + +_FORMAT_SAVE: Dict[str, Callable] = { + "csv": _save_csv, + "parquet": _save_parquet, + "json": _save_json, +} diff --git a/fugue_dask/dataframe.py b/fugue_dask/dataframe.py index 6f21a9f9..8de55ac3 100644 --- a/fugue_dask/dataframe.py +++ b/fugue_dask/dataframe.py @@ -5,10 +5,10 @@ from fugue.dataframe import DataFrame, LocalDataFrame, PandasDataFrame from fugue.dataframe.dataframe import _input_schema from triad.collections.schema import Schema -from triad.exceptions import InvalidOperationError from triad.utils.assertion import assert_arg_not_none, assert_or_throw from fugue_dask.utils import DASK_UTILS from fugue_dask.constants import DEFAULT_CONFIG +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError class DaskDataFrame(DataFrame): @@ -20,38 +20,42 @@ def __init__( # noqa: C901 num_partitions: int = 0, type_safe=True, ): - if num_partitions <= 0: - num_partitions = DEFAULT_CONFIG.get_or_throw( - "fugue.dask.dataframe.default.partitions", int - ) - if df is None: - schema = _input_schema(schema).assert_not_empty() - df = [] - if isinstance(df, DaskDataFrame): - super().__init__(df.schema, df.metadata if metadata is None else metadata) - self._native: pd.DataFrame = df._native - return - elif isinstance(df, (pd.DataFrame, pd.Series)): - if isinstance(df, pd.Series): - df = df.to_frame() - pdf = df - schema = None if schema is None else _input_schema(schema) - elif isinstance(df, (pandas.DataFrame, pandas.Series)): - if isinstance(df, pandas.Series): - df = df.to_frame() - pdf = pd.from_pandas(df, npartitions=num_partitions) - schema = None if schema is None else _input_schema(schema) - elif isinstance(df, Iterable): - assert_arg_not_none(schema, msg="schema can't be None for iterable input") - schema = _input_schema(schema).assert_not_empty() - t = PandasDataFrame(df, schema) - pdf = pd.from_pandas(t.native, npartitions=num_partitions) - type_safe = False - else: - raise ValueError(f"{df} is incompatible with DaskDataFrame") - pdf, schema = self._apply_schema(pdf, schema, type_safe) - super().__init__(schema, metadata) - self._native = pdf + try: + if num_partitions <= 0: + num_partitions = DEFAULT_CONFIG.get_or_throw( + "fugue.dask.dataframe.default.partitions", int + ) + if df is None: + schema = _input_schema(schema).assert_not_empty() + df = [] + if isinstance(df, DaskDataFrame): + super().__init__( + df.schema, df.metadata if metadata is None else metadata + ) + self._native: pd.DataFrame = df._native + return + elif isinstance(df, (pd.DataFrame, pd.Series)): + if isinstance(df, pd.Series): + df = df.to_frame() + pdf = df + schema = None if schema is None else _input_schema(schema) + elif isinstance(df, (pandas.DataFrame, pandas.Series)): + if isinstance(df, pandas.Series): + df = df.to_frame() + pdf = pd.from_pandas(df, npartitions=num_partitions) + schema = None if schema is None else _input_schema(schema) + elif isinstance(df, Iterable): + schema = _input_schema(schema).assert_not_empty() + t = PandasDataFrame(df, schema) + pdf = pd.from_pandas(t.native, npartitions=num_partitions) + type_safe = False + else: + raise ValueError(f"{df} is incompatible with DaskDataFrame") + pdf, schema = self._apply_schema(pdf, schema, type_safe) + super().__init__(schema, metadata) + self._native = pdf + except Exception as e: + raise FugueDataFrameInitError(e) @property def native(self) -> pd.DataFrame: @@ -76,7 +80,16 @@ def empty(self) -> bool: def num_partitions(self) -> int: return self.native.npartitions + def _drop_cols(self, cols: List[str]) -> DataFrame: + cols = (self.schema - cols).names + return self._select_cols(cols) + + def _select_cols(self, cols: List[Any]) -> DataFrame: + schema = self.schema.extract(cols) + return DaskDataFrame(self.native[schema.names], schema, type_safe=False) + def peek_array(self) -> Any: + self.assert_not_empty() return self.as_pandas().iloc[0].values.tolist() def persist(self, **kwargs: Any) -> "DaskDataFrame": @@ -89,18 +102,12 @@ def count(self) -> int: def as_pandas(self) -> pandas.DataFrame: return self.native.compute().reset_index(drop=True) - def drop(self, cols: List[str]) -> DataFrame: + def rename(self, columns: Dict[str, str]) -> "DataFrame": try: - schema = self.schema - cols + schema = self.schema.rename(columns) except Exception as e: - raise InvalidOperationError(str(e)) - if len(schema) == 0: - raise InvalidOperationError("Can't remove all columns of a dataframe") - return DaskDataFrame(self.native.drop(cols, axis=1), schema, type_safe=False) - - def rename(self, columns: Dict[str, str]) -> "DataFrame": + raise FugueDataFrameOperationError(e) df = self.native.rename(columns=columns) - schema = self.schema.rename(columns) return DaskDataFrame(df, schema, type_safe=False) def as_array( @@ -111,9 +118,11 @@ def as_array( def as_array_iterable( self, columns: Optional[List[str]] = None, type_safe: bool = False ) -> Iterable[Any]: - sub = None if columns is None else self.schema.extract(columns).pa_schema return DASK_UTILS.as_array_iterable( - self.native.compute(), sub, type_safe=type_safe, null_safe=True + self.native, + schema=self.schema.pa_schema, + columns=columns, + type_safe=type_safe, ) def _apply_schema( diff --git a/fugue_dask/execution_engine.py b/fugue_dask/execution_engine.py index 70d6eabf..844c8d36 100644 --- a/fugue_dask/execution_engine.py +++ b/fugue_dask/execution_engine.py @@ -1,11 +1,13 @@ import logging -from typing import Any, Callable, Iterable, List, Optional +from typing import Any, Callable, Iterable, List, Optional, Union import dask.dataframe as pd import pyarrow as pa -from fs.base import FS as FileSystem -from fs.osfs import OSFS -from fugue.collections.partition import PartitionCursor, PartitionSpec +from fugue.collections.partition import ( + EMPTY_PARTITION_SPEC, + PartitionCursor, + PartitionSpec, +) from fugue.constants import KEYWORD_CORECOUNT, KEYWORD_ROWCOUNT from fugue.dataframe import DataFrame, LocalDataFrame, PandasDataFrame from fugue.dataframe.utils import get_join_schemas @@ -15,10 +17,12 @@ ExecutionEngine, SQLEngine, ) +from fugue.utils.io import load_df, save_df from fugue_dask.dataframe import DEFAULT_CONFIG, DaskDataFrame from fugue_dask.utils import DASK_UTILS from triad.collections import Schema from triad.collections.dict import ParamDict +from triad.collections.fs import FileSystem from triad.utils.assertion import assert_or_throw from triad.utils.hash import to_uuid from triad.utils.threading import RunOnce @@ -29,7 +33,7 @@ def __init__(self, conf: Any = None): p = ParamDict(DEFAULT_CONFIG) p.update(ParamDict(conf)) super().__init__(p) - self._fs = OSFS("/") + self._fs = FileSystem() self._log = logging.getLogger() self._default_sql_engine = SqliteEngine(self) @@ -115,6 +119,8 @@ def map( ) def _map(pdf: Any) -> pd.DataFrame: + if pdf.shape[0] == 0: + return PandasDataFrame([], output_schema).as_pandas() if len(presort_keys) > 0: pdf = pdf.sort_values(presort_keys, ascending=presort_asc) input_df = PandasDataFrame( @@ -211,6 +217,36 @@ def join( ) return DaskDataFrame(d.reset_index(drop=True), output_schema, metadata) + def load_df( + self, + path: Union[str, List[str]], + format_hint: Any = None, + columns: Any = None, + **kwargs: Any, + ) -> DaskDataFrame: + return self.to_df( + load_df( + path, format_hint=format_hint, columns=columns, fs=self.fs, **kwargs + ) + ) + + def save_df( + self, + df: DataFrame, + path: str, + format_hint: Any = None, + mode: str = "overwrite", + partition_spec: PartitionSpec = EMPTY_PARTITION_SPEC, + force_single: bool = False, + **kwargs: Any, + ) -> None: + if not partition_spec.empty: + self.log.warning( # pragma: no cover + f"partition_spec is not respected in {self}.save_df" + ) + df = self.to_df(df).as_local() + save_df(df, path, format_hint=format_hint, mode=mode, fs=self.fs, **kwargs) + def _validate_outer_joinable(self, schema: Schema, key_schema: Schema) -> None: # TODO: this is to prevent wrong behavior of pandas, we may not need it # s = schema - key_schema diff --git a/fugue_dask/utils.py b/fugue_dask/utils.py index a2757141..61a14709 100644 --- a/fugue_dask/utils.py +++ b/fugue_dask/utils.py @@ -1,3 +1,5 @@ +from typing import Optional + import dask.dataframe as pd import pandas import pyarrow as pa @@ -5,12 +7,28 @@ class DaskUtils(PandasLikeUtils[pd.DataFrame]): + def as_arrow( + self, df: pd.DataFrame, schema: Optional[pa.Schema] = None + ) -> pa.Table: + """Convert dask dataframe to pyarrow table + + :param df: dask dataframe + :param schema: if specified, it will be used to construct pyarrow table, + defaults to None + + :return: pyarrow table + """ + pdf = df.compute().reset_index(drop=True) + return pa.Table.from_pandas( + pdf, schema=schema, preserve_index=False, safe=False + ) + def enforce_type( self, df: pd.DataFrame, schema: pa.Schema, null_safe: bool = False ) -> pd.DataFrame: - """Enforce the pandas like dataframe to comply with `schema`. + """Enforce the dask dataframe to comply with `schema`. - :param df: pandas like dataframe + :param df: dask dataframe :param schema: pyarrow schema :param null_safe: whether to enforce None value for int, string and bool values :return: converted dataframe @@ -28,7 +46,7 @@ def is_compatile_index(self, df: pd.DataFrame) -> bool: """Check whether the datafame is compatible with the operations inside this utils collection - :param df: pandas like dataframe + :param df: dask dataframe :return: if it is compatible """ return isinstance( diff --git a/fugue_spark/dataframe.py b/fugue_spark/dataframe.py index 38832a61..b1f92ee8 100644 --- a/fugue_spark/dataframe.py +++ b/fugue_spark/dataframe.py @@ -11,10 +11,9 @@ LocalDataFrame, PandasDataFrame, ) -from fugue_spark.utils.convert import to_cast_expression, to_schema -from pyspark.sql import Row -from triad.collections import Schema -from triad.exceptions import InvalidOperationError +from fugue.exceptions import FugueDataFrameInitError, FugueDataFrameOperationError +from fugue_spark.utils.convert import to_cast_expression, to_schema, to_type_safe_input +from triad.collections.schema import SchemaError from triad.utils.assertion import assert_or_throw @@ -23,18 +22,23 @@ def __init__( # noqa: C901 self, df: Any = None, schema: Any = None, metadata: Any = None ): self._lock = RLock() - if isinstance(df, ps.DataFrame): - if schema is not None: - has_cast, expr = to_cast_expression(df, schema, True) - if has_cast: - df = df.selectExpr(*expr) - schema = to_schema(schema) - else: - schema = to_schema(df) - self._native = df - super().__init__(schema, metadata) - else: - raise ValueError(f"{df} is incompatible with PandasDataFrame") + try: + if isinstance(df, ps.DataFrame): + if schema is not None: + schema = to_schema(schema).assert_not_empty() + has_cast, expr = to_cast_expression(df, schema, True) + if has_cast: + df = df.selectExpr(*expr) + else: + schema = to_schema(df).assert_not_empty() + self._native = df + super().__init__(schema, metadata) + else: # pragma: no cover + assert_or_throw(schema is not None, SchemaError("schema is None")) + schema = to_schema(schema).assert_not_empty() + raise ValueError(f"{df} is incompatible with SparkDataFrame") + except Exception as e: + raise FugueDataFrameInitError(e) @property def native(self) -> ps.DataFrame: @@ -49,8 +53,9 @@ def is_bounded(self) -> bool: return True def as_local(self) -> LocalDataFrame: + # TODO: does it make sense to also include the metadata? if any(pa.types.is_nested(t) for t in self.schema.types): - data = list(self._convert(self.native.collect(), self.schema)) + data = list(to_type_safe_input(self.native.collect(), self.schema)) return ArrayDataFrame(data, self.schema) return PandasDataFrame(self.native.toPandas(), self.schema) @@ -63,9 +68,8 @@ def empty(self) -> bool: return self._first is None def peek_array(self) -> List[Any]: - if self._first is None: - raise InvalidOperationError("Dataframe is empty, can't peek_array") - return self._first + self.assert_not_empty() + return self._first # type: ignore def count(self) -> int: with self._lock: @@ -73,21 +77,22 @@ def count(self) -> int: self._df_count = self.native.count() return self._df_count + def _drop_cols(self, cols: List[str]) -> DataFrame: + cols = (self.schema - cols).names + return self._select_cols(cols) + + def _select_cols(self, cols: List[Any]) -> DataFrame: + schema = self.schema.extract(cols) + return SparkDataFrame(self.native[schema.names]) + def as_pandas(self) -> pd.DataFrame: return self.native.toPandas() - def drop(self, cols: List[str]) -> "SparkDataFrame": - assert_or_throw( - cols in self.schema, - InvalidOperationError(f"{cols} not all in {self.schema}"), - ) - assert_or_throw( - len(cols) < len(self.schema), - InvalidOperationError(f"can't drop all columns {self.schema}"), - ) - return SparkDataFrame(self.native.drop(*cols)) - def rename(self, columns: Dict[str, str]) -> "SparkDataFrame": + try: + self.schema.rename(columns) + except Exception as e: + raise FugueDataFrameOperationError(e) df = self.native for o, n in columns.items(): df = df.withColumnRenamed(o, n) @@ -104,7 +109,7 @@ def as_array_iterable( ) -> Iterable[Any]: sdf = self._withColumns(columns) if not type_safe: - for row in self._convert(sdf.native.rdd.toLocalIterator(), sdf.schema): + for row in to_type_safe_input(sdf.native.rdd.toLocalIterator(), sdf.schema): yield row else: df = IterableDataFrame(sdf.as_array_iterable(type_safe=False), sdf.schema) @@ -129,16 +134,3 @@ def _withColumns(self, columns: Optional[List[str]]) -> "SparkDataFrame": if columns is None: return self return SparkDataFrame(self.native.select(*columns)) - - def _convert(self, rows: Iterable[Row], schema: Schema) -> Iterable[List[Any]]: - idx = [p for p, t in enumerate(schema.types) if pa.types.is_struct(t)] - if len(idx) == 0: - for row in rows: - yield list(row) - else: - for row in rows: - r = list(row) - for i in idx: - if r[i] is not None: - r[i] = r[i].asDict() - yield r diff --git a/fugue_spark/execution_engine.py b/fugue_spark/execution_engine.py index 9a6fe3e4..04ad4067 100644 --- a/fugue_spark/execution_engine.py +++ b/fugue_spark/execution_engine.py @@ -1,10 +1,14 @@ import logging -from typing import Any, Callable, Dict, Iterable, List, Optional, Tuple +from typing import Any, Callable, Dict, Iterable, List, Optional, Tuple, Union +import pandas as pd +import pyarrow as pa import pyspark.sql as ps -from fs.base import FS as FileSystem -from fs.osfs import OSFS -from fugue.collections.partition import PartitionCursor, PartitionSpec +from fugue.collections.partition import ( + EMPTY_PARTITION_SPEC, + PartitionCursor, + PartitionSpec, +) from fugue.constants import KEYWORD_ROWCOUNT from fugue.dataframe import ( DataFrame, @@ -20,17 +24,19 @@ SQLEngine, ) from fugue_spark.dataframe import SparkDataFrame -from fugue_spark.utils.convert import to_schema, to_spark_schema +from fugue_spark.utils.convert import to_schema, to_spark_schema, to_type_safe_input from fugue_spark.utils.partition import ( even_repartition, hash_repartition, rand_repartition, ) +from fugue_spark.utils.io import SparkIO from pyspark import StorageLevel from pyspark.rdd import RDD from pyspark.sql import SparkSession from pyspark.sql.functions import broadcast, col -from triad.collections.dict import ParamDict +from triad.collections import ParamDict, Schema +from triad.collections.fs import FileSystem from triad.utils.assertion import assert_arg_not_none, assert_or_throw from triad.utils.iter import EmptyAwareIterable from triad.utils.threading import RunOnce @@ -47,8 +53,6 @@ "leftanti": "left_anti", } -_NO_PARTITION_SERIALIZATION_KEY = "__no_partition_serialization_key__" - class SparkSQLEngine(SQLEngine): def __init__(self, execution_engine: ExecutionEngine) -> None: @@ -72,7 +76,7 @@ def __init__(self, spark_session: SparkSession, conf: Any = None): cf = {x[0]: x[1] for x in spark_session.sparkContext.getConf().getAll()} cf.update(ParamDict(conf)) super().__init__(cf) - self._fs = OSFS("/") # TODO: this is not right + self._fs = FileSystem() self._log = logging.getLogger() self._default_sql_engine = SparkSQLEngine(self) self._broadcast_func = RunOnce( @@ -82,6 +86,7 @@ def __init__(self, spark_session: SparkSession, conf: Any = None): self._register_func = RunOnce( self._register, lambda *args, **kwargs: id(args[0]) ) + self._io = SparkIO(self.spark_session, self.fs) def __repr__(self) -> str: return "SparkExecutionEngine" @@ -117,10 +122,15 @@ def to_df( ) if isinstance(df, SparkDataFrame): return df - sdf = self.spark_session.createDataFrame( - df.as_pandas(), to_spark_schema(df.schema) - ) - return SparkDataFrame(sdf, df.schema, metadata) + if any(pa.types.is_struct(t) for t in df.schema.types): + sdf = self.spark_session.createDataFrame( + df.as_array(type_safe=True), to_spark_schema(df.schema) + ) + else: + sdf = self.spark_session.createDataFrame( + df.as_pandas(), to_spark_schema(df.schema) + ) + return SparkDataFrame(sdf, df.schema, df.metadata) if isinstance(df, ps.DataFrame): return SparkDataFrame( df, None if schema is None else to_schema(schema), metadata @@ -176,7 +186,7 @@ def map( on_init: Optional[Callable[[int, DataFrame], Any]] = None, ) -> DataFrame: df = self.to_df(self.repartition(df, partition_spec)) - mapper = _Mapper(df, map_func, partition_spec, on_init) + mapper = _Mapper(df, map_func, output_schema, partition_spec, on_init) sdf = df.native.rdd.mapPartitionsWithIndex(mapper.run, True) return self.to_df(sdf, output_schema, metadata) @@ -213,6 +223,38 @@ def join( res = d1.join(d2, on=key_schema.names, how=how).select(*cols) return self.to_df(res, output_schema, metadata) + def load_df( + self, + path: Union[str, List[str]], + format_hint: Any = None, + columns: Any = None, + **kwargs: Any, + ) -> DataFrame: + return self._io.load_df( + uri=path, format_hint=format_hint, columns=columns, **kwargs + ) + + def save_df( + self, + df: DataFrame, + path: str, + format_hint: Any = None, + mode: str = "overwrite", + partition_spec: PartitionSpec = EMPTY_PARTITION_SPEC, + force_single: bool = False, + **kwargs: Any, + ) -> None: + df = self.to_df(df) + self._io.save_df( + df, + uri=path, + format_hint=format_hint, + mode=mode, + partition_spec=partition_spec, + force_single=force_single, + **kwargs, + ) + def _broadcast(self, df: SparkDataFrame) -> SparkDataFrame: sdf = broadcast(df.native) return SparkDataFrame(sdf, df.schema, df.metadata) @@ -226,30 +268,40 @@ def _persist(self, df: SparkDataFrame, level: Any) -> SparkDataFrame: df.native.persist() self.log.info(f"Persist dataframe with {level}, count {df.count()}") return df - raise NotImplementedError(f"{level} is not supported persist type") + raise ValueError(f"{level} is not supported persist type") # pragma: no cover def _register(self, df: SparkDataFrame, name: str) -> SparkDataFrame: df.native.createOrReplaceTempView(name) return df -class _Mapper(object): +class _Mapper(object): # pragma: no cover + # pytest can't identify the coverage, but this part is fully tested def __init__( self, df: DataFrame, map_func: Callable[[PartitionCursor, LocalDataFrame], LocalDataFrame], + output_schema: Any, partition_spec: PartitionSpec, on_init: Optional[Callable[[int, DataFrame], Any]], ): super().__init__() self.schema = df.schema + self.output_schema = Schema(output_schema) self.metadata = df.metadata self.partition_spec = partition_spec self.map_func = map_func self.on_init = on_init + def _run(self, no: int, rows: Iterable[ps.Row]) -> Iterable[Any]: + it = self._run(no, rows) + for x in self._handle_special_values(it): + yield x + def run(self, no: int, rows: Iterable[ps.Row]) -> Iterable[Any]: - df = IterableDataFrame(rows, self.schema, self.metadata) + df = IterableDataFrame( + to_type_safe_input(rows, self.schema), self.schema, self.metadata + ) if df.empty: # pragma: no cover return cursor = self.partition_spec.get_cursor(self.schema, no) @@ -269,3 +321,21 @@ def run(self, no: int, rows: Iterable[ps.Row]) -> Iterable[Any]: res = self.map_func(cursor, sub_df) for r in res.as_array_iterable(type_safe=True): yield r + + def _handle_special_values(self, it: Iterable[Any]): + idx = [ + i + for i, t in enumerate(self.output_schema.types) + if pa.types.is_date(t) or pa.types.is_timestamp(t) + ] + if len(idx) == 0: + for x in it: + yield x + else: + for x in it: + for i in idx: + if x[i] is pd.NaT: + x[i] = None + elif isinstance(x[i], pd.Timestamp): + x[i] = x[i].to_pydatetime() + yield x diff --git a/fugue_spark/utils/convert.py b/fugue_spark/utils/convert.py index f64181c4..95dd4ca7 100644 --- a/fugue_spark/utils/convert.py +++ b/fugue_spark/utils/convert.py @@ -1,9 +1,9 @@ -from typing import Any, List, Tuple +from typing import Any, Iterable, List, Tuple import pyarrow as pa import pyspark.sql as ps import pyspark.sql.types as pt -from pyarrow.types import is_struct +from pyarrow.types import is_list, is_struct, is_timestamp from triad.collections import Schema from triad.utils.assertion import assert_arg_not_none, assert_or_throw from triad.utils.pyarrow import TRIAD_DEFAULT_TIMESTAMP @@ -67,6 +67,30 @@ def to_select_expression(schema_from: Any, schema_to: Any) -> List[str]: return expr +def to_type_safe_input(rows: Iterable[ps.Row], schema: Schema) -> Iterable[List[Any]]: + struct_idx = [p for p, t in enumerate(schema.types) if pa.types.is_struct(t)] + complex_list_idx = [ + p + for p, t in enumerate(schema.types) + if pa.types.is_list(t) and pa.types.is_nested(t.value_type) + ] + if len(struct_idx) == 0 and len(complex_list_idx) == 0: + for row in rows: + yield list(row) + elif len(complex_list_idx) == 0: + for row in rows: + r = list(row) + for i in struct_idx: + if r[i] is not None: + r[i] = r[i].asDict(recursive=True) + yield r + else: + for row in rows: + data = row.asDict(recursive=True) + r = [data[n] for n in schema.names] + yield r + + # TODO: the following function always set nullable to true, # but should we use field.nullable? def _to_arrow_type(dt: pt.DataType) -> pa.DataType: @@ -108,6 +132,12 @@ def _from_arrow_type(dt: pa.DataType) -> pt.DataType: for field in dt ] ) + elif is_list(dt): + if is_timestamp(dt.value_type): + raise TypeError( # pragma: no cover + "Spark: unsupported type in conversion from Arrow: " + str(dt) + ) + return pt.ArrayType(_from_arrow_type(dt.value_type)) return pt.from_arrow_type(dt) diff --git a/fugue_spark/utils/io.py b/fugue_spark/utils/io.py new file mode 100644 index 00000000..2f4eac2a --- /dev/null +++ b/fugue_spark/utils/io.py @@ -0,0 +1,128 @@ +from typing import Any, Callable, Dict, List, Optional, Union + +import pyspark.sql as ps +from fugue.collections.partition import EMPTY_PARTITION_SPEC, PartitionSpec +from fugue.dataframe import DataFrame +from fugue.utils.io import FileParser, save_df +from fugue_spark.dataframe import SparkDataFrame +from fugue_spark.utils.convert import to_schema, to_spark_schema +from pyspark.sql import SparkSession +from triad.collections import Schema +from triad.collections.fs import FileSystem +from triad.exceptions import InvalidOperationError +from triad.utils.assertion import assert_or_throw + + +class SparkIO(object): + def __init__(self, spark_session: SparkSession, fs: FileSystem): + self._session = spark_session + self._fs = fs + self._loads: Dict[str, Callable[..., DataFrame]] = { + "csv": self._load_csv, + "parquet": self._load_parquet, + "json": self._load_json, + } + + def load_df( + self, + uri: Union[str, List[str]], + format_hint: Optional[str] = None, + columns: Any = None, + **kwargs: Any, + ) -> DataFrame: + if isinstance(uri, str): + fp = [FileParser(uri, format_hint)] + else: + fp = [FileParser(u, format_hint) for u in uri] + fmts = list(set(f.file_format for f in fp)) # noqa: C401 + assert_or_throw( + len(fmts) == 1, NotImplementedError("can't support multiple formats") + ) + fmt = fmts[0] + files = [f.uri for f in fp] + return self._loads[fmt](files, columns, **kwargs) + + def save_df( + self, + df: SparkDataFrame, + uri: str, + format_hint: Optional[str] = None, + partition_spec: PartitionSpec = EMPTY_PARTITION_SPEC, + mode: str = "overwrite", + force_single: bool = False, + **kwargs: Any, + ) -> None: + if not force_single: + p = FileParser(uri, format_hint) + writer = self._get_writer(df.native, partition_spec) + writer.format(p.file_format).options(**kwargs).mode(mode) + writer.save(uri) + else: + ldf = df.as_local() + save_df(ldf, uri, format_hint=format_hint, mode=mode, fs=self._fs, **kwargs) + + def _get_writer( + self, sdf: ps.DataFrame, partition_spec: PartitionSpec + ) -> ps.DataFrameWriter: + # will not use bucket by because here we don't save as table + num = int(partition_spec.num_partitions) # TODO: this is hacky + by = partition_spec.partition_by + if num == 0 and len(by) == 0: + return sdf.write + elif num > 0 and len(by) == 0: + return sdf.repartition(num).write + elif num == 0 and len(by) > 0: + return sdf.write.partitionBy(*by) + return sdf.repartition(num, *by).write.partitionBy(*by) + + def _load_parquet( + self, p: List[str], columns: Any = None, **kwargs: Any + ) -> DataFrame: + sdf = self._session.read.parquet(*p, **kwargs) + if columns is None: + return SparkDataFrame(sdf) + if isinstance(columns, list): # column names + return SparkDataFrame(sdf)[columns] + schema = Schema(columns) + return SparkDataFrame(sdf[schema.names], schema) + + def _load_csv(self, p: List[str], columns: Any = None, **kwargs: Any) -> DataFrame: + kw = dict(kwargs) + header = kw.get("header", False) + header = str(header).lower() + if "header" in kw: + del kw["header"] + reader = self._session.read.format("csv") + reader.options(**kw) + if header == "true": + reader.option("header", "true") + if columns is None: + return SparkDataFrame(reader.load(p)) + if isinstance(columns, list): # column names + return SparkDataFrame(reader.load(p)[columns]) + schema = Schema(columns) + return SparkDataFrame(reader.load(p)[schema.names], schema) + if header in ["false", "none"]: + reader.option("header", "false") + if columns is None: + raise InvalidOperationError("columns must be set if without header") + if isinstance(columns, list): # column names + sdf = reader.load(p) + inferred = to_schema(sdf) + renames = [f"{k} AS {v}" for k, v in zip(inferred.names, columns)] + return SparkDataFrame(sdf.selectExpr(*renames)) + schema = Schema(columns) + sdf = reader.schema(to_spark_schema(schema)).load(p) + return SparkDataFrame(sdf, schema) + else: + raise NotImplementedError(f"{header} is not supported") + + def _load_json(self, p: List[str], columns: Any = None, **kwargs: Any) -> DataFrame: + reader = self._session.read.format("json") + reader.options(**kwargs) + if columns is None: + return SparkDataFrame(reader.load(p)) + if isinstance(columns, list): # column names + return SparkDataFrame(reader.load(p))[columns] + schema = Schema(columns) + return SparkDataFrame(reader.load(p)[schema.names], schema) diff --git a/fugue_test/builtin_suite.py b/fugue_test/builtin_suite.py index c421afe7..436c1a26 100644 --- a/fugue_test/builtin_suite.py +++ b/fugue_test/builtin_suite.py @@ -1,20 +1,24 @@ +import os from typing import Any, Dict, Iterable, List from unittest import TestCase import pandas as pd +import pytest from adagio.instances import WorkflowContext from fugue.dag.workflow import FugueWorkflow from fugue.dataframe import DataFrame, DataFrames, LocalDataFrame, PandasDataFrame from fugue.dataframe.array_dataframe import ArrayDataFrame from fugue.execution import ExecutionEngine -from fugue.execution.naive_execution_engine import SqliteEngine +from fugue.execution.native_execution_engine import SqliteEngine +from fugue.extensions.outputter import Outputter +from fugue.extensions.processor import Processor from fugue.extensions.transformer import ( CoTransformer, Transformer, - transformer, cotransformer, + transformer, ) -from fugue.extensions.processor import Processor +from triad.collections.fs import FileSystem class BuiltInTests(object): @@ -53,6 +57,10 @@ def test_create_process_output(self): b2.assert_eq(ArrayDataFrame([[3]], "a:int")) b2 = dag.process(a, a, a, using=MockProcessor3) b2.assert_eq(ArrayDataFrame([[3]], "a:int")) + a.process(mock_processor2).assert_eq(ArrayDataFrame([[1]], "a:int")) + a.output(mock_outputter2) + a.partition(num=3).output(MockOutputter3) + dag.output(dict(aa=a, bb=b), using=MockOutputter4) def test_zip(self): with self.dag() as dag: @@ -239,6 +247,24 @@ def test_col_ops(self): a[["x"]].rename(x="xx").assert_eq(ArrayDataFrame([[1], [2]], "xx:long")) + @pytest.fixture(autouse=True) + def init_tmpdir(self, tmpdir): + self.tmpdir = tmpdir + + def test_io(self): + path = os.path.join(self.tmpdir, "a") + path2 = os.path.join(self.tmpdir, "b.csv") + with self.dag() as dag: + b = dag.df([[6, 1], [2, 7]], "c:int,a:long") + b.partition(num=3).save(path, fmt="parquet", single=True) + b.save(path2, header=True) + assert FileSystem().isfile(path) + with self.dag() as dag: + a = dag.load(path, fmt="parquet", columns=["a", "c"]) + a.assert_eq(dag.df([[1, 6], [7, 2]], "a:long,c:int")) + a = dag.load(path2, header=True, columns="c:int,a:long") + a.assert_eq(dag.df([[6, 1], [2, 7]], "c:int,a:long")) + class DagTester(FugueWorkflow): def __init__(self, engine: ExecutionEngine): @@ -276,6 +302,22 @@ def mock_outputter(df1: List[List[Any]], df2: List[List[Any]]) -> None: assert len(df1) == len(df2) +def mock_outputter2(df: List[List[Any]]) -> None: + print(df) + + +class MockOutputter3(Outputter): + def process(self, dfs): + assert "3" == self.partition_spec.num_partitions + + +class MockOutputter4(Outputter): + def process(self, dfs): + for k, v in dfs.items(): + print(k) + v.show() + + class MockTransform1(Transformer): def get_output_schema(self, df: DataFrame) -> Any: assert "test" in self.workflow_conf @@ -308,7 +350,7 @@ def mock_tf0(df: pd.DataFrame, p=1) -> pd.DataFrame: return df -@transformer("*,ct:int,p:int") +# schema: *,ct:int,p:int def mock_tf1(df: pd.DataFrame, p=1) -> pd.DataFrame: df["ct"] = df.shape[0] df["p"] = p diff --git a/fugue_test/dataframe_suite.py b/fugue_test/dataframe_suite.py new file mode 100644 index 00000000..4b9db9d7 --- /dev/null +++ b/fugue_test/dataframe_suite.py @@ -0,0 +1,208 @@ +from datetime import datetime +from typing import Any +from unittest import TestCase + +import numpy as np +import pandas as pd +from fugue.dataframe import ArrowDataFrame, DataFrame +from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import ( + FugueDataFrameEmptyError, + FugueDataFrameInitError, + FugueDataFrameOperationError, +) +from pytest import raises +from triad.collections.schema import Schema + + +class DataFrameTests(object): + class Tests(TestCase): + @classmethod + def setUpClass(cls): + pass + + @classmethod + def tearDownClass(cls): + pass + + def df( + self, data: Any = None, schema: Any = None, metadata: Any = None + ) -> DataFrame: # pragma: no cover + raise NotImplementedError + + def test_init_basic(self): + raises(FugueDataFrameInitError, lambda: self.df()) + raises(FugueDataFrameInitError, lambda: self.df([])) + raises(FugueDataFrameInitError, lambda: self.df([[]], Schema())) + raises(FugueDataFrameInitError, lambda: self.df([[1]], Schema())) + # raises(SchemaError, lambda: self.df([[1]])) # schema can be inferred + + df = self.df([], "a:str,b:int") + assert df.empty + + def test_peek(self): + df = self.df([], "x:str,y:double") + raises(FugueDataFrameEmptyError, lambda: df.peek_array()) + raises(FugueDataFrameEmptyError, lambda: df.peek_dict()) + + df = self.df([["a", 1.0], ["b", 2.0]], "x:str,y:double") + assert not df.is_bounded or 2 == df.count() + assert not df.empty + assert ["a", 1.0] == df.peek_array() + assert dict(x="a", y=1.0) == df.peek_dict() + + def test_as_pandas(self): + df = self.df([["a", 1.0], ["b", 2.0]], "x:str,y:double") + pdf = df.as_pandas() + assert [["a", 1.0], ["b", 2.0]] == pdf.values.tolist() + + df = self.df([], "x:str,y:double") + pdf = df.as_pandas() + assert [] == pdf.values.tolist() + + def test_drop(self): + df = self.df([], "a:str,b:int").drop(["a"]) + assert df.schema == "b:int" + raises( + FugueDataFrameOperationError, lambda: df.drop(["b"]) + ) # can't be empty + raises( + FugueDataFrameOperationError, lambda: df.drop(["x"]) + ) # cols must exist + + df = self.df([["a", 1]], "a:str,b:int").drop(["a"]) + assert df.schema == "b:int" + raises( + FugueDataFrameOperationError, lambda: df.drop(["b"]) + ) # can't be empty + raises( + FugueDataFrameOperationError, lambda: df.drop(["x"]) + ) # cols must exist + assert [[1]] == df.as_array(type_safe=True) + + def test_select(self): + df = self.df([], "a:str,b:int")[["b"]] + assert df.schema == "b:int" + raises(FugueDataFrameOperationError, lambda: df[["a"]]) # not existed + raises(FugueDataFrameOperationError, lambda: df[[]]) # empty + + df = self.df([["a", 1]], "a:str,b:int")[["b"]] + assert df.schema == "b:int" + raises(FugueDataFrameOperationError, lambda: df[["a"]]) # not existed + raises(FugueDataFrameOperationError, lambda: df[[]]) # empty + assert [[1]] == df.as_array(type_safe=True) + + df = self.df([["a", 1, 2]], "a:str,b:int,c:int") + df_eq(df[["c", "a"]], [[2, "a"]], "a:str,c:int") + + def test_rename(self): + df = self.df([["a", 1]], "a:str,b:int") + df2 = df.rename(columns=dict(a="aa")) + assert df.schema == "a:str,b:int" + df_eq(df2, [["a", 1]], "aa:str,b:int", throw=True) + + def test_rename_invalid(self): + df = self.df([["a", 1]], "a:str,b:int") + raises( + FugueDataFrameOperationError, lambda: df.rename(columns=dict(aa="ab")) + ) + + def test_as_array(self): + for func in [ + lambda df, *args, **kwargs: df.as_array( + *args, **kwargs, type_safe=True + ), + lambda df, *args, **kwargs: list( + df.as_array_iterable(*args, **kwargs, type_safe=True) + ), + ]: + df = self.df([], "a:str,b:int") + assert [] == func(df) + + df = self.df([["a", 1]], "a:str,b:int") + assert [["a", 1]] == func(df) + df = self.df([["a", 1]], "a:str,b:int") + assert [["a", 1]] == func(df, ["a", "b"]) + df = self.df([["a", 1]], "a:str,b:int") + assert [[1, "a"]] == func(df, ["b", "a"]) + + for v in [1.0, np.float64(1.0)]: + df = self.df([[v, 1]], "a:double,b:int") + d = func(df) + assert [[1.0, 1]] == d + assert isinstance(d[0][0], float) + assert isinstance(d[0][1], int) + + # special values + df = self.df([[pd.Timestamp("2020-01-01"), 1]], "a:datetime,b:int") + data = func(df) + assert [[datetime(2020, 1, 1), 1]] == data + assert isinstance(data[0][0], datetime) + assert isinstance(data[0][1], int) + + df = self.df([[pd.NaT, 1]], "a:datetime,b:int") + assert [[None, 1]] == func(df) + + df = self.df([[float("nan"), 1]], "a:double,b:int") + assert [[None, 1]] == func(df) + + df = self.df([[float("inf"), 1]], "a:double,b:int") + assert [[float("inf"), 1]] == func(df) + + def test_as_dict_iterable(self): + df = self.df([[pd.NaT, 1]], "a:datetime,b:int") + assert [dict(a=None, b=1)] == list(df.as_dict_iterable()) + df = self.df([[pd.Timestamp("2020-01-01"), 1]], "a:datetime,b:int") + assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) + + def test_nested(self): + data = [[[30, 40]]] + df = self.df(data, "a:[int]") + a = df.as_array(type_safe=True) + assert data == a + + data = [[dict(a="1", b=[3, 4], d=1.0)], [dict(b=[30, 40])]] + df = self.df(data, "a:{a:str,b:[int]}") + a = df.as_array(type_safe=True) + assert [[dict(a="1", b=[3, 4])], [dict(a=None, b=[30, 40])]] == a + + data = [[[dict(b=[30, 40])]]] + df = self.df(data, "a:[{a:str,b:[int]}]") + a = df.as_array(type_safe=True) + assert [[[dict(a=None, b=[30, 40])]]] == a + + def test_as_arrow(self): + # pd.Nat + df = self.df([[pd.NaT, 1]], "a:datetime,b:int") + assert [dict(a=None, b=1)] == list( + ArrowDataFrame(df.as_arrow()).as_dict_iterable() + ) + # pandas timestamps + df = self.df([[pd.Timestamp("2020-01-01"), 1]], "a:datetime,b:int") + assert [dict(a=datetime(2020, 1, 1), b=1)] == list( + ArrowDataFrame(df.as_arrow()).as_dict_iterable() + ) + # float nan, list + data = [[[float("nan"), 2.0]]] + df = self.df(data, "a:[float]") + assert [[[None, 2.0]]] == ArrowDataFrame(df.as_arrow()).as_array() + # dict + data = [[dict(b="x")]] + df = self.df(data, "a:{b:str}") + assert data == ArrowDataFrame(df.as_arrow()).as_array() + # list[dict] + data = [[[dict(b=[30, 40])]]] + df = self.df(data, "a:[{b:[int]}]") + assert data == ArrowDataFrame(df.as_arrow()).as_array() + + def test_head(self): + df = self.df([], "a:str,b:int") + assert [] == df.head(1) + df = self.df([["a", 1]], "a:str,b:int") + if df.is_bounded: + assert [["a", 1]] == df.head(1) + assert [[1, "a"]] == df.head(1, ["b", "a"]) + + def test_show(self): + df = self.df([["a", 1]], "a:str,b:int") + df.show() diff --git a/fugue_test/execution_suite.py b/fugue_test/execution_suite.py index 00381cdf..b25cb243 100644 --- a/fugue_test/execution_suite.py +++ b/fugue_test/execution_suite.py @@ -1,8 +1,13 @@ import copy +import os +from datetime import datetime from unittest import TestCase +import pandas as pd +import pytest from fugue.collections.partition import PartitionSpec from fugue.dataframe import ArrayDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq from fugue.execution.execution_engine import ExecutionEngine from pytest import raises @@ -37,9 +42,6 @@ def test_map(self): def noop(cursor, data): return data - def select_top(cursor, data): - return ArrayDataFrame([cursor.row], cursor.row_schema) - def on_init(partition_no, data): # TODO: this test is not sufficient assert partition_no >= 0 @@ -88,6 +90,45 @@ def on_init(partition_no, data): ) df_eq(c, [[None, 4], [1, 2], [3, 4]], "a:double,b:int", throw=True) + def test_map_with_special_values(self): + def with_nat(cursor, data): + df = data.as_pandas() + df["nat"] = pd.NaT + schema = data.schema + "nat:datetime" + return PandasDataFrame(df, schema) + + e = self.engine + # test datetime with nat + dt = datetime.now() + o = ArrayDataFrame( + [[dt, 2], [None, 2], [None, 1], [dt, 5], [None, 4]], + "a:datetime,b:int", + dict(a=1), + ) + c = e.map( + o, select_top, o.schema, PartitionSpec(by=["a"], presort="b DESC") + ) + df_eq(c, [[None, 4], [dt, 5]], "a:datetime,b:int", throw=True) + d = e.map(c, with_nat, "a:datetime,b:int,nat:datetime", PartitionSpec()) + df_eq( + d, + [[None, 4, None], [dt, 5, None]], + "a:datetime,b:int,nat:datetime", + throw=True, + ) + # test list + o = ArrayDataFrame([[dt, [1, 2]]], "a:datetime,b:[int]") + c = e.map(o, select_top, o.schema, PartitionSpec(by=["a"])) + df_eq(c, o, check_order=True, throw=True) + + def test_map_with_dict_col(self): + e = self.engine + dt = datetime.now() + # test dict + o = ArrayDataFrame([[dt, dict(a=1)]], "a:datetime,b:{a:int}") + c = e.map(o, select_top, o.schema, PartitionSpec(by=["a"])) + df_eq(c, o, no_pandas=True, check_order=True, throw=True) + def test__join_cross(self): e = self.engine a = e.to_df([[1, 2], [3, 4]], "a:int,b:int") @@ -243,71 +284,65 @@ def test_serialize_by_partition(self): ) assert s.count() == 2 s = e.persist(e.serialize_by_partition(a, PartitionSpec(), df_name="_0")) - print(s.count()) assert s.count() == 1 s = e.persist( e.serialize_by_partition(a, PartitionSpec(by=["x"]), df_name="_0") ) assert s.count() == 1 - def test_zip_dataframes(self): + def test_zip(self): ps = PartitionSpec(by=["a"], presort="b DESC,c DESC") e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") sa = e.serialize_by_partition(a, ps, df_name="_0") sb = e.serialize_by_partition(b, ps, df_name="_1") - # test zip_dataframes with serialized dfs - z1 = e.persist(e.zip_dataframes(sa, sb, how="inner", partition_spec=ps)) + # test zip with serialized dfs + z1 = e.persist(e.zip(sa, sb, how="inner", partition_spec=ps)) assert 1 == z1.count() - z2 = e.persist( - e.zip_dataframes(sa, sb, how="left_outer", partition_spec=ps) - ) + z2 = e.persist(e.zip(sa, sb, how="left_outer", partition_spec=ps)) assert 2 == z2.count() # can't have duplicated keys - raises( - ValueError, - lambda: e.zip_dataframes(sa, sa, how="inner", partition_spec=ps), - ) + raises(ValueError, lambda: e.zip(sa, sa, how="inner", partition_spec=ps)) # not support semi or anti raises( InvalidOperationError, - lambda: e.zip_dataframes(sa, sa, how="anti", partition_spec=ps), + lambda: e.zip(sa, sa, how="anti", partition_spec=ps), ) raises( InvalidOperationError, - lambda: e.zip_dataframes(sa, sa, how="leftsemi", partition_spec=ps), + lambda: e.zip(sa, sa, how="leftsemi", partition_spec=ps), ) raises( InvalidOperationError, - lambda: e.zip_dataframes(sa, sa, how="LEFT SEMI", partition_spec=ps), + lambda: e.zip(sa, sa, how="LEFT SEMI", partition_spec=ps), ) # can't specify keys for cross join raises( InvalidOperationError, - lambda: e.zip_dataframes(sa, sa, how="cross", partition_spec=ps), + lambda: e.zip(sa, sa, how="cross", partition_spec=ps), ) - # test zip_dataframes with unserialized dfs - z3 = e.persist(e.zip_dataframes(a, b, partition_spec=ps)) + # test zip with unserialized dfs + z3 = e.persist(e.zip(a, b, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) - z3 = e.persist(e.zip_dataframes(a, sb, partition_spec=ps)) + z3 = e.persist(e.zip(a, sb, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) - z3 = e.persist(e.zip_dataframes(sa, b, partition_spec=ps)) + z3 = e.persist(e.zip(sa, b, partition_spec=ps)) df_eq(z1, z3, throw=True, check_metadata=False) - z4 = e.persist(e.zip_dataframes(a, b, how="left_outer", partition_spec=ps)) + z4 = e.persist(e.zip(a, b, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) - z4 = e.persist(e.zip_dataframes(a, sb, how="left_outer", partition_spec=ps)) + z4 = e.persist(e.zip(a, sb, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) - z4 = e.persist(e.zip_dataframes(sa, b, how="left_outer", partition_spec=ps)) + z4 = e.persist(e.zip(sa, b, how="left_outer", partition_spec=ps)) df_eq(z2, z4, throw=True, check_metadata=False) - z5 = e.persist(e.zip_dataframes(a, b, how="cross")) + z5 = e.persist(e.zip(a, b, how="cross")) assert z5.count() == 1 assert len(z5.schema) == 2 - z6 = e.persist(e.zip_dataframes(sa, b, how="cross")) + z6 = e.persist(e.zip(sa, b, how="cross")) assert z6.count() == 2 assert len(z6.schema) == 3 @@ -316,10 +351,10 @@ def test_comap(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [1, 5]], "a:int,b:int") b = e.to_df([[6, 1], [2, 7]], "c:int,a:int") - z1 = e.persist(e.zip_dataframes(a, b)) - z2 = e.persist(e.zip_dataframes(a, b, partition_spec=ps, how="left_outer")) + z1 = e.persist(e.zip(a, b)) + z2 = e.persist(e.zip(a, b, partition_spec=ps, how="left_outer")) z3 = e.persist(e.serialize_by_partition(a, partition_spec=ps, df_name="_x")) - z4 = e.persist(e.zip_dataframes(a, b, partition_spec=ps, how="cross")) + z4 = e.persist(e.zip(a, b, partition_spec=ps, how="cross")) def comap(cursor, dfs): assert not dfs.has_key @@ -358,3 +393,25 @@ def on_init(partition_no, dfs): res = e.comap(z4, comap, "v:str", PartitionSpec(), metadata=dict(a=1)) df_eq(res, [["_03,_12"]], "v:str", metadata=dict(a=1), throw=True) + + @pytest.fixture(autouse=True) + def init_tmpdir(self, tmpdir): + self.tmpdir = tmpdir + + def test_io(self): + e = self.engine + b = ArrayDataFrame([[6, 1], [2, 7]], "c:int,a:long") + path = os.path.join(self.tmpdir, "a") + e.save_df(b, path, format_hint="parquet", force_single=True) + assert e.fs.isfile(path) + c = e.load_df(path, format_hint="parquet", columns=["a", "c"]) + df_eq(c, [[1, 6], [7, 2]], "a:long,c:int", throw=True) + + path = os.path.join(self.tmpdir, "b.csv") + e.save_df(b, path, header=True) + c = e.load_df(path, header=True, columns="c:int,a:long") + df_eq(c, b, throw=True) + + +def select_top(cursor, data): + return ArrayDataFrame([cursor.row], cursor.row_schema) diff --git a/setup.py b/setup.py index e14c473b..0c5fa744 100644 --- a/setup.py +++ b/setup.py @@ -17,7 +17,7 @@ author_email="goodwanghan@gmail.com", keywords="distributed spark dask sql dsl domain specific language", url="http://github.com/fugue-project/fugue", - install_requires=["triad>=0.3.3", "adagio>=0.1.2", "fs", "sqlalchemy"], + install_requires=["triad>=0.3.5", "adagio>=0.1.2", "sqlalchemy"], extras_require={ "spark": ["pyspark"], "dask": ["dask[dataframe]", "cloudpickle>=1.4.0"], diff --git a/tests/fugue/collections/test_partition.py b/tests/fugue/collections/test_partition.py index e298254a..9cc75e97 100644 --- a/tests/fugue/collections/test_partition.py +++ b/tests/fugue/collections/test_partition.py @@ -93,6 +93,7 @@ def test_partition_cursor(): p = PartitionSpec(dict(partition_by=["b", "a"])) s = Schema("a:int,b:int,c:int,d:int") c = p.get_cursor(s, 2) + pt = p.get_partitioner(s) # this part is well covered in spark section assert c.row_schema == s assert c.key_schema == "b:int,a:int" diff --git a/tests/fugue/dag/test_workflow.py b/tests/fugue/dag/test_workflow.py index 274f7738..8e9b4f54 100644 --- a/tests/fugue/dag/test_workflow.py +++ b/tests/fugue/dag/test_workflow.py @@ -4,16 +4,22 @@ from fugue.collections.partition import PartitionSpec from fugue.dag.workflow import FugueWorkflow from fugue.dataframe.array_dataframe import ArrayDataFrame -from fugue.execution import NaiveExecutionEngine +from fugue.execution import NativeExecutionEngine from fugue.extensions.transformer.convert import transformer +from triad.exceptions import InvalidOperationError +from pytest import raises +import copy def test_builder(): - e = NaiveExecutionEngine() + e = NativeExecutionEngine() builder = FugueWorkflow(e) ctx = WorkflowContext() a = builder.create_data([[0], [0], [1]], "a:int") + raises(InvalidOperationError, lambda: a._task.copy()) + raises(InvalidOperationError, lambda: copy.copy(a._task)) + raises(InvalidOperationError, lambda: copy.deepcopy(a._task)) a.show() a.show() b = a.transform(mock_tf1, "*,b:int", partition=dict(by=["a"])) diff --git a/tests/fugue/dataframe/test_array_dataframe.py b/tests/fugue/dataframe/test_array_dataframe.py index b07331e5..f8ab13e9 100644 --- a/tests/fugue/dataframe/test_array_dataframe.py +++ b/tests/fugue/dataframe/test_array_dataframe.py @@ -1,19 +1,25 @@ import json from datetime import datetime +from typing import Any import numpy as np import pandas as pd from fugue.dataframe import ArrayDataFrame, PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue_test.dataframe_suite import DataFrameTests from pytest import raises from triad.collections.schema import Schema, SchemaError from triad.exceptions import InvalidOperationError -def test_init(): - raises(SchemaError, lambda: ArrayDataFrame()) - raises(SchemaError, lambda: ArrayDataFrame(schema=Schema())) +class ArrayDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> ArrayDataFrame: + return ArrayDataFrame(data, schema, metadata) + +def test_init(): df = ArrayDataFrame(schema="a:str,b:int") assert df.empty assert df.schema == "a:str,b:int" @@ -56,17 +62,10 @@ def test_init(): df = ArrayDataFrame([], "x:str,y:double") assert df.empty - raises(ValueError, lambda: ArrayDataFrame(123)) + raises(FugueDataFrameInitError, lambda: ArrayDataFrame(123)) def test_simple_methods(): - df = ArrayDataFrame([], "a:str,b:int") - assert 0 == df.count() - assert df.empty - raises(IndexError, lambda: df.peek_array()) - raises(IndexError, lambda: df.peek_dict()) - assert df.is_local - df = ArrayDataFrame([["a", 1], ["b", "2"]], "x:str,y:double") assert 2 == df.count() assert not df.empty @@ -95,34 +94,6 @@ def test_nested(): assert [[[dict(a=None, b=[30, 40])]]] == a -def test_drop(): - df = ArrayDataFrame([], "a:str,b:int").drop(["a"]) - assert df.empty - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - - df = ArrayDataFrame([["a", 1]], "a:str,b:int").drop(["a"]) - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - assert [[1]] == df.as_array(type_safe=True) - - df = ArrayDataFrame([["a", 1, 2]], "a:str,b:int,c:int") - df_eq(df[["a", "c"]], [["a", 2]], "a:str,c:int") - assert isinstance(df[["a", "c"]], ArrayDataFrame) - - with raises(SchemaError): - df[["a", "x"]] - - -def test_rename(): - df = ArrayDataFrame([["a", 1]], "a:str,b:int") - df2 = df.rename(columns=dict(a="aa")) - df_eq(df2, [["a", 1]], "aa:str,b:int", throw=True) - df_eq(df, [["a", 1]], "a:str,b:int", throw=True) - - def test_as_array(): df = ArrayDataFrame([], "a:str,b:int") assert [] == df.as_array() @@ -164,12 +135,10 @@ def test_as_array(): assert [[datetime(2020, 1, 1), 1]] == df.as_array(type_safe=True) df = ArrayDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [[pd.NaT, 1]] == df.as_array(type_safe=True) + assert [[None, 1]] == df.as_array(type_safe=True) def test_as_dict_iterable(): - df = ArrayDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [dict(a=pd.NaT, b=1)] == list(df.as_dict_iterable()) df = ArrayDataFrame([["2020-01-01", 1.1]], "a:datetime,b:int") assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) diff --git a/tests/fugue/dataframe/test_arrow_dataframe.py b/tests/fugue/dataframe/test_arrow_dataframe.py new file mode 100644 index 00000000..6c2882f7 --- /dev/null +++ b/tests/fugue/dataframe/test_arrow_dataframe.py @@ -0,0 +1,50 @@ +import json +from datetime import datetime +from typing import Any + +import numpy as np +import pandas as pd +from fugue.dataframe import ArrowDataFrame, PandasDataFrame +from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue_test.dataframe_suite import DataFrameTests +from pytest import raises +from triad.collections.schema import Schema, SchemaError +from triad.exceptions import InvalidOperationError + + +class ArrowDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> ArrowDataFrame: + return ArrowDataFrame(data, schema, metadata) + + +def test_init(): + df = ArrowDataFrame(schema="a:str,b:int") + assert df.empty + assert df.schema == "a:str,b:int" + assert df.is_bounded + + data = [["a", "1"], ["b", "2"]] + df = ArrowDataFrame(data, "a:str,b:str") + assert [["a", "1"], ["b", "2"]] == df.as_array(type_safe=True) + data = [["a", 1], ["b", 2]] + df = ArrowDataFrame(data, "a:str,b:int") + assert [["a", 1.0], ["b", 2.0]] == df.as_array(type_safe=True) + df = ArrowDataFrame(data, "a:str,b:double") + assert [["a", 1.0], ["b", 2.0]] == df.as_array(type_safe=True) + + ddf = ArrowDataFrame(df.native) + assert [["a", 1.0], ["b", 2.0]] == ddf.as_array(type_safe=True) + + df = ArrowDataFrame(df.as_pandas(), "a:str,b:double") + assert [["a", 1.0], ["b", 2.0]] == df.as_array(type_safe=True) + df = ArrowDataFrame(df.as_pandas()["b"]) + assert [[1.0], [2.0]] == df.as_array(type_safe=True) + + df = ArrowDataFrame([], "x:str,y:double") + assert df.empty + assert df.is_local + assert df.is_bounded + + raises(FugueDataFrameInitError, lambda: ArrowDataFrame(123)) diff --git a/tests/fugue/dataframe/test_dataframes.py b/tests/fugue/dataframe/test_dataframes.py index 9aa7f2df..4e07f060 100644 --- a/tests/fugue/dataframe/test_dataframes.py +++ b/tests/fugue/dataframe/test_dataframes.py @@ -1,6 +1,6 @@ from fugue.dataframe import DataFrames from fugue.dataframe.array_dataframe import ArrayDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from pytest import raises from triad.exceptions import InvalidOperationError diff --git a/tests/fugue/dataframe/test_iterable_dataframe.py b/tests/fugue/dataframe/test_iterable_dataframe.py index ce21c010..2cfb766e 100644 --- a/tests/fugue/dataframe/test_iterable_dataframe.py +++ b/tests/fugue/dataframe/test_iterable_dataframe.py @@ -1,19 +1,25 @@ import json from datetime import datetime +from typing import Any import numpy as np import pandas as pd -from pytest import raises from fugue.dataframe import IterableDataFrame, PandasDataFrame +from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue_test.dataframe_suite import DataFrameTests +from pytest import raises from triad.collections.schema import Schema, SchemaError from triad.exceptions import InvalidOperationError -from fugue.dataframe.utils import _df_eq as df_eq -def test_init(): - raises(SchemaError, lambda: IterableDataFrame()) - raises(SchemaError, lambda: IterableDataFrame(schema=Schema())) +class IterableDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> IterableDataFrame: + return IterableDataFrame(data, schema, metadata) + +def test_init(): df = IterableDataFrame(schema="a:str,b:int") assert df.empty assert df.schema == "a:str,b:int" @@ -56,17 +62,12 @@ def test_init(): df = IterableDataFrame([], "x:str,y:double") assert df.empty + assert df.is_local - raises(ValueError, lambda: IterableDataFrame(123)) + raises(FugueDataFrameInitError, lambda: IterableDataFrame(123)) def test_simple_methods(): - df = IterableDataFrame([], "a:str,b:int") - assert df.empty - raises(StopIteration, lambda: df.peek_array()) - raises(StopIteration, lambda: df.peek_dict()) - assert df.is_local - df = IterableDataFrame([["a", 1], ["b", "2"]], "x:str,y:double") raises(InvalidOperationError, lambda: df.count()) assert not df.empty @@ -95,81 +96,6 @@ def test_nested(): assert [[[dict(a=None, b=[30, 40])]]] == a -def test_drop(): - df = IterableDataFrame([], "a:str,b:int").drop(["a"]) - assert df.empty - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - - df = IterableDataFrame([["a", 1]], "a:str,b:int").drop(["a"]) - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - assert [[1]] == df.as_array(type_safe=True) - - -def test_rename(): - df = IterableDataFrame([["a", 1]], "a:str,b:int") - df2 = df.rename(columns=dict(a="aa")) - assert isinstance(df, IterableDataFrame) - df_eq(df2, [["a", 1]], "aa:str,b:int", throw=True) - - -def test_as_array(): - df = IterableDataFrame([], "a:str,b:int") - assert [] == df.as_array() - assert [] == df.as_array(type_safe=True) - assert [] == list(df.as_array_iterable()) - assert [] == list(df.as_array_iterable(type_safe=True)) - - df = IterableDataFrame([["a", 1]], "a:str,b:int") - assert [["a", 1]] == df.as_array() - df = IterableDataFrame([["a", 1]], "a:str,b:int") - assert [["a", 1]] == df.as_array(["a", "b"]) - df = IterableDataFrame([["a", 1]], "a:str,b:int") - assert [[1, "a"]] == df.as_array(["b", "a"]) - - df = IterableDataFrame([[1.0, 1.1]], "a:double,b:int") - d = df.as_array(type_safe=False) - assert [[1.0, 1]] != d - - df = IterableDataFrame([[1.0, 1.1]], "a:double,b:int") - d = df.as_array(type_safe=True) - assert [[1.0, 1]] == d - assert isinstance(d[0][0], float) - assert isinstance(d[0][1], int) - df = IterableDataFrame([[1.0, 1.1]], "a:double,b:int") - assert [[1.0, 1]] == df.as_array(["a", "b"], type_safe=True) - df = IterableDataFrame([[1.0, 1.1]], "a:double,b:int") - assert [[1, 1.0]] == df.as_array(["b", "a"], type_safe=True) - - df = IterableDataFrame([[np.float64(1.0), 1.1]], "a:double,b:int") - d = df.as_array(type_safe=True) - assert [[1.0, 1]] == d - assert isinstance(d[0][0], float) - assert isinstance(d[0][1], int) - - df = IterableDataFrame([[pd.Timestamp("2020-01-01"), 1.1]], "a:datetime,b:int") - assert [[datetime(2020, 1, 1), 1]] == df.as_array(type_safe=True) - - df = IterableDataFrame([["2020-01-01", 1.1]], "a:datetime,b:int") - assert [[datetime(2020, 1, 1), 1]] == df.as_array(type_safe=True) - - df = IterableDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [[pd.NaT, 1]] == df.as_array(type_safe=True) - - df = IterableDataFrame([[None, 1.1]], "a:datetime,b:int") - assert [[pd.NaT, 1]] == df.as_array(type_safe=True) - - -def test_as_dict_iterable(): - df = IterableDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [dict(a=pd.NaT, b=1)] == list(df.as_dict_iterable()) - df = IterableDataFrame([["2020-01-01", 1.1]], "a:datetime,b:int") - assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) - - def _test_as_array_perf(): s = Schema() arr = [] diff --git a/tests/fugue/dataframe/test_pandas_dataframe.py b/tests/fugue/dataframe/test_pandas_dataframe.py index 4e44bfb3..59c427be 100644 --- a/tests/fugue/dataframe/test_pandas_dataframe.py +++ b/tests/fugue/dataframe/test_pandas_dataframe.py @@ -1,21 +1,27 @@ import json import math from datetime import datetime +from typing import Any import numpy as np import pandas as pd from fugue.dataframe import PandasDataFrame from fugue.dataframe.array_dataframe import ArrayDataFrame from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue_test.dataframe_suite import DataFrameTests from pytest import raises from triad.collections.schema import Schema, SchemaError from triad.exceptions import InvalidOperationError -def test_init(): - raises(SchemaError, lambda: PandasDataFrame()) - raises(SchemaError, lambda: PandasDataFrame(schema=Schema())) +class PandasDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> PandasDataFrame: + return PandasDataFrame(data, schema, metadata) + +def test_init(): df = PandasDataFrame(schema="a:str,b:int") assert df.is_bounded assert df.count() == 0 @@ -23,7 +29,7 @@ def test_init(): assert Schema(df.native) == "a:str,b:int" pdf = pd.DataFrame([["a", 1], ["b", 2]]) - raises(SchemaError, lambda: PandasDataFrame(pdf)) + raises(FugueDataFrameInitError, lambda: PandasDataFrame(pdf)) df = PandasDataFrame(pdf, "a:str,b:str") assert [["a", "1"], ["b", "2"]] == df.native.values.tolist() df = PandasDataFrame(pdf, "a:str,b:int") @@ -53,7 +59,7 @@ def test_init(): df = PandasDataFrame([], "x:str,y:double") assert [] == df.native.values.tolist() - raises(ValueError, lambda: PandasDataFrame(123)) + raises(FugueDataFrameInitError, lambda: PandasDataFrame(123)) def test_simple_methods(): @@ -61,8 +67,6 @@ def test_simple_methods(): assert df.as_pandas() is df.native assert df.empty assert 0 == df.count() - raises(IndexError, lambda: df.peek_array()) - raises(IndexError, lambda: df.peek_dict()) assert df.is_local df = PandasDataFrame([["a", 1], ["b", "2"]], "x:str,y:double") @@ -85,20 +89,6 @@ def test_nested(): assert [[[dict(a=None, b=[30, 40])]]] == a -def test_drop(): - df = PandasDataFrame([], "a:str,b:int").drop(["a"]) - assert df.empty - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - - df = PandasDataFrame([["a", 1]], "a:str,b:int").drop(["a"]) - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - assert [[1]] == df.as_pandas().values.tolist() - - def test_rename(): df = PandasDataFrame([["a", 1]], "a:str,b:int") df2 = df.rename(columns=dict(a="aa")) @@ -139,11 +129,6 @@ def test_as_array(): # assert not isinstance(df.as_array()[0][0], pd.Timestamp) assert isinstance(df.as_array()[0][1], int) - df = PandasDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - df.native["a"] = pd.to_datetime(df.native["a"]) - assert isinstance(df.as_array()[0][0], datetime) - assert isinstance(df.as_array()[0][1], int) - df = PandasDataFrame([[1.0, 1.1]], "a:double,b:int") assert [[1.0, 1]] == df.as_array(type_safe=True) assert isinstance(df.as_array()[0][0], float) @@ -152,7 +137,7 @@ def test_as_array(): def test_as_dict_iterable(): df = PandasDataFrame([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [dict(a=pd.NaT, b=1)] == list(df.as_dict_iterable()) + assert [dict(a=None, b=1)] == list(df.as_dict_iterable()) df = PandasDataFrame([["2020-01-01", 1.1]], "a:datetime,b:int") assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) @@ -166,13 +151,13 @@ def test_nan_none(): df = ArrayDataFrame([[None, None]], "b:int,c:bool") arr = PandasDataFrame(df.as_pandas(), df.schema).as_array(type_safe=True)[0] - assert np.isnan(arr[0]) # TODO: this will cause inconsistent behavior cross engine - assert np.isnan(arr[1]) # TODO: this will cause inconsistent behavior cross engine + assert arr[0] is None + assert arr[1] is None df = ArrayDataFrame([["a", 1.1], [None, None]], "b:str,c:double") - arr = PandasDataFrame(df.as_pandas(), df.schema).as_array()[1] + arr = PandasDataFrame(df.as_pandas(), df.schema).as_array(type_safe=True)[1] assert arr[0] is None - assert math.isnan(arr[1]) + assert arr[1] is None def _test_as_array_perf(): diff --git a/tests/fugue/dataframe/test_utils.py b/tests/fugue/dataframe/test_utils.py index 6b3e1088..a3ae1650 100644 --- a/tests/fugue/dataframe/test_utils.py +++ b/tests/fugue/dataframe/test_utils.py @@ -7,7 +7,7 @@ from fugue.dataframe import to_local_bounded_df, to_local_df from fugue.dataframe.array_dataframe import ArrayDataFrame from fugue.dataframe.iterable_dataframe import IterableDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq from fugue.dataframe.utils import (deserialize_df, get_join_schemas, pickle_df, serialize_df, unpickle_df) diff --git a/tests/fugue/execution/test_naive_execution_engine.py b/tests/fugue/execution/test_naive_execution_engine.py index b550f3ee..c268800a 100644 --- a/tests/fugue/execution/test_naive_execution_engine.py +++ b/tests/fugue/execution/test_naive_execution_engine.py @@ -1,15 +1,19 @@ -from fugue.execution import NaiveExecutionEngine +from fugue.execution import NativeExecutionEngine from fugue_test.execution_suite import ExecutionEngineTests from fugue_test.builtin_suite import BuiltInTests -class NaiveExecutionEngineTests(ExecutionEngineTests.Tests): +class NativeExecutionEngineTests(ExecutionEngineTests.Tests): def make_engine(self): - e = NaiveExecutionEngine(dict(test=True)) + e = NativeExecutionEngine(dict(test=True)) return e + def test_map_with_dict_col(self): + # TODO: add back + return -class NaiveExecutionEngineBuiltInTests(BuiltInTests.Tests): + +class NativeExecutionEngineBuiltInTests(BuiltInTests.Tests): def make_engine(self): - e = NaiveExecutionEngine(dict(test=True)) + e = NativeExecutionEngine(dict(test=True)) return e diff --git a/tests/fugue/extensions/creator/test_convert.py b/tests/fugue/extensions/creator/test_convert.py index 34cba50a..15ed0750 100644 --- a/tests/fugue/extensions/creator/test_convert.py +++ b/tests/fugue/extensions/creator/test_convert.py @@ -43,7 +43,7 @@ def test_to_creator(): raises(FugueInterfacelessError, lambda: to_creator(t6, "a:int")) assert isinstance(to_creator(t7, "a:int"), Creator) raises(FugueInterfacelessError, lambda: to_creator(t7)) - + assert isinstance(to_creator(t8), Creator) def test_run_creator(): o1 = to_creator(t3) @@ -95,3 +95,8 @@ def t6() -> DataFrame: def t7() -> List[List[Any]]: pass + + +# schema: a:int +def t8(e: ExecutionEngine, a, b) -> List[List[Any]]: + pass diff --git a/tests/fugue/extensions/processor/test_convert.py b/tests/fugue/extensions/processor/test_convert.py index cdfaa37d..fdb5fc8b 100644 --- a/tests/fugue/extensions/processor/test_convert.py +++ b/tests/fugue/extensions/processor/test_convert.py @@ -43,6 +43,7 @@ def test_to_processor(): raises(FugueInterfacelessError, lambda: to_processor(t6, "a:int")) assert isinstance(to_processor(t7, "a:int"), Processor) raises(FugueInterfacelessError, lambda: to_processor(t7)) + assert isinstance(to_processor(t8), Processor) def test_run_processor(): @@ -109,3 +110,8 @@ def t6(dfs: DataFrames) -> DataFrame: def t7(dfs: DataFrames) -> List[List[Any]]: pass + + +# schema: a:int +def t8(e: ExecutionEngine, df1: DataFrame, df2: DataFrame) -> List[List[Any]]: + pass diff --git a/tests/fugue/extensions/transformer/test_convert_transformer.py b/tests/fugue/extensions/transformer/test_convert_transformer.py index fde20d7b..bf0fe047 100644 --- a/tests/fugue/extensions/transformer/test_convert_transformer.py +++ b/tests/fugue/extensions/transformer/test_convert_transformer.py @@ -37,6 +37,8 @@ def test_to_transformer(): raises(FugueInterfacelessError, lambda: to_transformer("t4", None)) e = to_transformer("t4", "*,b:int") assert isinstance(e, Transformer) + f = to_transformer("t5") + assert isinstance(f, Transformer) @transformer(["*", None, "b:int"]) @@ -64,3 +66,10 @@ def t4(df: Iterable[List[Any]]) -> Iterable[List[Any]]: for r in df: r += [1] yield r + + +# schema: *,b:int +def t5(df: Iterable[Dict[str, Any]]) -> Iterable[Dict[str, Any]]: + for r in df: + r["b"] = 1 + yield r diff --git a/tests/fugue/utils/test_file.py b/tests/fugue/utils/test_file.py deleted file mode 100644 index de83d57a..00000000 --- a/tests/fugue/utils/test_file.py +++ /dev/null @@ -1,36 +0,0 @@ -from fugue.utils.file import FileParser -from pytest import raises - - -def test_file_parser(): - f = FileParser("/a/b/c.parquet") - assert "/a/b/c.parquet" == f.uri - assert "" == f.scheme - assert "/a/b/c.parquet" == f.path - assert ".parquet" == f.suffix - assert "parquet" == f.file_format - - f = FileParser("s3://a/b/c.parquet") - assert "s3://a/b/c.parquet" == f.uri - assert "s3" == f.scheme - assert "/b/c.parquet" == f.path - assert ".parquet" == f.suffix - assert "parquet" == f.file_format - - f = FileParser("s3://a/b/c.ppp.gz", "csv") - assert "s3://a/b/c.ppp.gz" == f.uri - assert "s3" == f.scheme - assert "/b/c.ppp.gz" == f.path - assert ".ppp.gz" == f.suffix - assert "csv" == f.file_format - - f = FileParser("s3://a/b/c", "csv") - assert "s3://a/b/c" == f.uri - assert "s3" == f.scheme - assert "/b/c" == f.path - assert "" == f.suffix - assert "csv" == f.file_format - - raises(NotImplementedError, lambda: FileParser("s3://a/b/c.ppp")) - raises(NotImplementedError, lambda: FileParser("s3://a/b/c.parquet", "csvv")) - raises(NotImplementedError, lambda: FileParser("s3://a/b/c")) diff --git a/tests/fugue/utils/test_interfaceless.py b/tests/fugue/utils/test_interfaceless.py index 30b6356f..9de91728 100644 --- a/tests/fugue/utils/test_interfaceless.py +++ b/tests/fugue/utils/test_interfaceless.py @@ -3,14 +3,39 @@ import pandas as pd from fugue.dataframe import DataFrame, DataFrames, LocalDataFrame from fugue.dataframe.array_dataframe import ArrayDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq from fugue.execution import ExecutionEngine -from fugue.utils.interfaceless import FunctionWrapper, _parse_function +from fugue.utils.interfaceless import FunctionWrapper, _parse_function, parse_output_schema_from_comment from pytest import raises from triad.utils.iter import EmptyAwareIterable +def test_parse_output_schema_from_comment(): + def a(): + pass + + # asdfasdf + def b(): + pass + + # asdfasdf + # schema : s:int + # # # schema : a : int,b:str + # asdfasdf + def c(): + pass + + # schema: + def d(): + pass + + assert parse_output_schema_from_comment(a) is None + assert parse_output_schema_from_comment(b) is None + assert "a:int,b:str" == parse_output_schema_from_comment(c) + assert parse_output_schema_from_comment(d) is None + + def test_parse_function(): _parse_function(f1, "^edlp$", "n") _parse_function(f2, "^xxxx$", "n") diff --git a/tests/fugue/utils/test_io.py b/tests/fugue/utils/test_io.py new file mode 100644 index 00000000..17cc73a5 --- /dev/null +++ b/tests/fugue/utils/test_io.py @@ -0,0 +1,131 @@ +import os + +from fugue.dataframe.array_dataframe import ArrayDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame +from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue.utils.io import FileParser, load_df, save_df +from pytest import raises +from triad.collections.fs import FileSystem +from triad.exceptions import InvalidOperationError + + +def test_file_parser(): + f = FileParser("/a/b/c.parquet") + assert "/a/b/c.parquet" == f.uri + assert "" == f.scheme + assert "/a/b/c.parquet" == f.path + assert ".parquet" == f.suffix + assert "parquet" == f.file_format + + f = FileParser("s3://a/b/c.parquet") + assert "s3://a/b/c.parquet" == f.uri + assert "s3" == f.scheme + assert "/b/c.parquet" == f.path + assert ".parquet" == f.suffix + assert "parquet" == f.file_format + + f = FileParser("s3://a/b/c.ppp.gz", "csv") + assert "s3://a/b/c.ppp.gz" == f.uri + assert "s3" == f.scheme + assert "/b/c.ppp.gz" == f.path + assert ".ppp.gz" == f.suffix + assert "csv" == f.file_format + + f = FileParser("s3://a/b/c", "csv") + assert "s3://a/b/c" == f.uri + assert "s3" == f.scheme + assert "/b/c" == f.path + assert "" == f.suffix + assert "csv" == f.file_format + + raises(NotImplementedError, lambda: FileParser("s3://a/b/c.ppp")) + raises(NotImplementedError, lambda: FileParser("s3://a/b/c.parquet", "csvv")) + raises(NotImplementedError, lambda: FileParser("s3://a/b/c")) + + +def test_parquet_io(tmpdir): + df1 = PandasDataFrame([["1", 2, 3]], "a:str,b:int,c:long") + df2 = ArrayDataFrame([[[1, 2]]], "a:[int]") + # {a:int} will become {a:long} because pyarrow lib has issue + df3 = ArrayDataFrame([[dict(a=1)]], "a:{a:long}") + for df in [df1, df2, df3]: + path = os.path.join(tmpdir, "a.parquet") + save_df(df, path) + actual = load_df(path) + df_eq(df, actual, throw=True) + + save_df(df1, path) + actual = load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + actual = load_df(path, columns="b:str,a:int") + df_eq(actual, [["2", 1]], "b:str,a:int") + raises(FugueDataFrameInitError, lambda: load_df(path, columns="bb:str,a:int")) + + # load directory + fs = FileSystem() + folder = os.path.join(tmpdir, "folder") + fs.makedirs(folder) + f0 = os.path.join(folder, "_SUCCESS") + f1 = os.path.join(folder, "1.parquet") + f2 = os.path.join(folder, "3.parquet") + fs.touch(f0) + save_df(df1, f1) + save_df(df1, f2) + actual = load_df(folder, "parquet") + df_eq(actual, [["1", 2, 3], ["1", 2, 3]], "a:str,b:int,c:long") + + # load multiple paths + actual = load_df([f1, f2], "parquet") + df_eq(actual, [["1", 2, 3], ["1", 2, 3]], "a:str,b:int,c:long") + + # overwrite = False + raises(FileExistsError, lambda: save_df(df1, f1, mode="error")) + # can't overwrite directory + raises(IsADirectoryError, lambda: save_df( + df1, folder, format_hint="parquet", mode="overwrite")) + # wrong mode + raises(NotImplementedError, lambda: save_df(df1, f1, mode="dummy")) + + +def test_csv_io(tmpdir): + fs = FileSystem() + df1 = PandasDataFrame([["1", 2, 3]], "a:str,b:int,c:long") + path = os.path.join(tmpdir, "a.csv") + # without header + save_df(df1, path) + assert fs.readtext(path).startswith("1,2,3") + raises(InvalidOperationError, lambda: load_df(path, header=False)) + actual = load_df(path, columns=["a", "b", "c"], header=False) + assert [[1, 2, 3]] == actual.as_array() + assert actual.schema == "a:long,b:long,c:long" + actual = load_df(path, columns="a:double,b:str,c:str", header=False) + assert [[1.0, "2", "3"]] == actual.as_array() + assert actual.schema == "a:double,b:str,c:str" + # with header + save_df(df1, path, header=True) + assert fs.readtext(path).startswith("a,b,c") + actual = load_df(path, header=True) + assert [[1, 2, 3]] == actual.as_array() + actual = load_df(path, columns=["b", "a"], header=True) + assert [[2, 1]] == actual.as_array() + actual = load_df(path, columns="b:str,a:double", header=True) + assert [["2", 1.0]] == actual.as_array() + raises(KeyError, lambda: load_df(path, columns="b:str,x:double", header=True)) + + raises(NotImplementedError, lambda: load_df( + path, columns="b:str,x:double", header=2)) + + +def test_json(tmpdir): + fs = FileSystem() + df1 = PandasDataFrame([["1", 2, 3]], "a:str,b:int,c:long") + path = os.path.join(tmpdir, "a.json") + save_df(df1, path) + actual = load_df(path) + df_eq(actual, [[1, 2, 3]], "a:long,b:long,c:long") + actual = load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + actual = load_df(path, columns="b:str,a:int") + df_eq(actual, [["2", 1]], "b:str,a:int") + raises(KeyError, lambda: load_df(path, columns="bb:str,a:int")) diff --git a/tests/fugue_dask/test_dataframe.py b/tests/fugue_dask/test_dataframe.py index 4fbd617b..76d6f28b 100644 --- a/tests/fugue_dask/test_dataframe.py +++ b/tests/fugue_dask/test_dataframe.py @@ -1,30 +1,36 @@ import json import math from datetime import datetime +from typing import Any import dask.dataframe as pd import numpy as np import pandas from fugue.dataframe.array_dataframe import ArrayDataFrame -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError from fugue_dask.dataframe import DaskDataFrame +from fugue_test.dataframe_suite import DataFrameTests from pytest import raises from triad.collections.schema import Schema, SchemaError from triad.exceptions import InvalidOperationError -def test_init(): - raises(SchemaError, lambda: DaskDataFrame()) - raises(SchemaError, lambda: DaskDataFrame(schema=Schema())) +class DaskDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> DaskDataFrame: + return DaskDataFrame(data, schema, metadata) + +def test_init(): df = DaskDataFrame(schema="a:str,b:int") assert df.is_bounded assert df.count() == 0 assert df.schema == "a:str,b:int" pdf = pandas.DataFrame([["a", 1], ["b", 2]]) - raises(SchemaError, lambda: DaskDataFrame(pdf)) + raises(FugueDataFrameInitError, lambda: DaskDataFrame(pdf)) df = DaskDataFrame(pdf, "a:str,b:str") assert [["a", "1"], ["b", "2"]] == df.as_pandas().values.tolist() df = DaskDataFrame(pdf, "a:str,b:int") @@ -54,15 +60,13 @@ def test_init(): df = DaskDataFrame([], "x:str,y:double") assert [] == df.as_pandas().values.tolist() - raises(ValueError, lambda: DaskDataFrame(123)) + raises(FugueDataFrameInitError, lambda: DaskDataFrame(123)) def test_simple_methods(): df = DaskDataFrame([], "a:str,b:int") assert df.empty assert 0 == df.count() - raises(IndexError, lambda: df.peek_array()) - raises(IndexError, lambda: df.peek_dict()) assert not df.is_local df = DaskDataFrame([["a", 1], ["b", "2"]], "x:str,y:double") @@ -88,28 +92,6 @@ def _test_nested(): assert [[[dict(a=None, b=[30, 40])]]] == a -def test_drop(): - df = DaskDataFrame([], "a:str,b:int").drop(["a"]) - assert df.empty - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - - df = DaskDataFrame([["a", 1]], "a:str,b:int").drop(["a"]) - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - assert [[1]] == df.as_pandas().values.tolist() - - -def test_rename(): - df = DaskDataFrame([["a", 1]], "a:str,b:int") - df2 = df.rename(columns=dict(a="aa")) - assert isinstance(df, DaskDataFrame) - df_eq(df2, [["a", 1]], "aa:str,b:int", throw=True) - df_eq(df, [["a", 1]], "a:str,b:int", throw=True) - - def test_as_array(): df = DaskDataFrame([], "a:str,b:int") assert [] == df.as_array() @@ -153,8 +135,6 @@ def test_as_array(): def test_as_dict_iterable(): - df = DaskDataFrame([[pandas.NaT, 1.1]], "a:datetime,b:int") - assert [dict(a=pandas.NaT, b=1)] == list(df.as_dict_iterable()) df = DaskDataFrame([["2020-01-01", 1.1]], "a:datetime,b:int") assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) diff --git a/tests/fugue_dask/test_execution_engine.py b/tests/fugue_dask/test_execution_engine.py index cbdef6ff..630b3798 100644 --- a/tests/fugue_dask/test_execution_engine.py +++ b/tests/fugue_dask/test_execution_engine.py @@ -1,7 +1,9 @@ +from fugue.collections.partition import PartitionSpec +from fugue.dataframe.utils import _df_eq as df_eq from fugue_dask.execution_engine import DaskExecutionEngine from fugue_test.builtin_suite import BuiltInTests from fugue_test.execution_suite import ExecutionEngineTests -from fugue.collections.partition import PartitionSpec +from fugue.dataframe.pandas_dataframe import PandasDataFrame class DaskExecutionEngineTests(ExecutionEngineTests.Tests): @@ -12,6 +14,18 @@ def make_engine(self): def test__join_outer_pandas_incompatible(self): return + def test_map_with_dict_col(self): + # TODO: add back + return + + def test_to_df(self): + e = self.engine + a = e.to_df([[1, 2], [3, 4]], "a:int,b:int", dict(a=1)) + df_eq(a, [[1, 2], [3, 4]], "a:int,b:int", dict(a=1), throw=True) + a = e.to_df(PandasDataFrame([[1, 2], [3, 4]], "a:int,b:int", dict(a=1))) + df_eq(a, [[1, 2], [3, 4]], "a:int,b:int", dict(a=1), throw=True) + assert a is e.to_df(a) + def test_repartition(self): e = self.engine a = e.to_df([[1, 2], [3, 4], [5, 6], [7, 8], [9, 10]], "a:int,b:int") diff --git a/tests/fugue_spark/test_dataframe.py b/tests/fugue_spark/test_dataframe.py index 799f9c0b..cefed6e1 100644 --- a/tests/fugue_spark/test_dataframe.py +++ b/tests/fugue_spark/test_dataframe.py @@ -1,30 +1,50 @@ import json import math from datetime import datetime +from typing import Any import numpy as np import pandas as pd from fugue.dataframe.array_dataframe import ArrayDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue.dataframe.utils import _df_eq as df_eq from fugue.dataframe.utils import to_local_bounded_df +from fugue.exceptions import FugueDataFrameInitError from fugue_spark.dataframe import SparkDataFrame from fugue_spark.utils.convert import to_schema, to_spark_schema +from fugue_test.dataframe_suite import DataFrameTests from pyspark.sql import SparkSession from pytest import raises from triad.collections.schema import Schema, SchemaError from triad.exceptions import InvalidOperationError -from fugue.dataframe.pandas_dataframes import PandasDataFrame -def test_init(spark_session): - # spark_session = SparkSession.builder.config( - # "spark.master", "local[*]").getOrCreate() - raises(ValueError, lambda: SparkDataFrame()) +class SparkDataFrameTests(DataFrameTests.Tests): + def df(self, data: Any = None, schema: Any = None, + metadata: Any = None) -> SparkDataFrame: + session = SparkSession.builder.getOrCreate() + if data is None: + df = None + else: + if schema is not None: + pdf = PandasDataFrame(data, to_schema(schema), metadata) + df = session.createDataFrame(pdf.native, to_spark_schema(schema)) + else: + try: + df = session.createDataFrame(data) + except Exception: + raise FugueDataFrameInitError("schema error") + return SparkDataFrame(df, schema, metadata) + +def test_init(spark_session): sdf = spark_session.createDataFrame([["a", 1]]) df = SparkDataFrame(sdf, "a:str,b:double") assert [["a", 1.0]] == df.as_array() assert [["a", 1.0]] == df.as_pandas().values.tolist() + assert not df.is_local + assert df.is_bounded + assert df.num_partitions > 0 df = _df([["a", 1], ["b", 2]]) assert [["a", 1], ["b", 2]] == df.as_array() @@ -36,25 +56,6 @@ def test_init(spark_session): assert df.schema == "a:str,b:str" -def test_simple_methods(spark_session): - df = _df([], "a:int") - - assert df.empty - assert 0 == df.count() - raises(InvalidOperationError, lambda: df.peek_array()) - raises(InvalidOperationError, lambda: df.peek_dict()) - assert not df.is_local - assert df.is_bounded - - df = _df([["a", 1], ["b", 2]], "x:str,y:int") - assert not df.empty - assert df.is_bounded - assert df.num_partitions > 0 - assert 2 == df.count() - assert ["a", 1.0] == df.peek_array() - assert dict(x="a", y=1.0) == df.peek_dict() - - def test_nested(spark_session): # data = [[dict(a=1, b=[3, 4], d=1.0)], [json.dumps(dict(b=[30, "40"]))]] # df = SparkDataFrame(data, "a:{a:str,b:[int]}") @@ -82,84 +83,6 @@ def test_nested(spark_session): assert [[dict(a=None, b=[30, 40])]] == a -def test_drop(spark_session): - df = _df([], "a:str,b:int").drop(["a"]) - assert df.empty - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - - df = _df([["a", 1]], "a:str,b:int").drop(["a"]) - assert df.schema == "b:int" - raises(InvalidOperationError, lambda: df.drop(["b"])) # can't be empty - raises(InvalidOperationError, lambda: df.drop(["x"])) # cols must exist - assert [[1]] == df.as_pandas().values.tolist() - - -def test_rename(spark_session): - df = _df([["a", 1]], "a:str,b:int") - df2 = df.rename(columns=dict(a="aa")) - df_eq(df2, [["a", 1]], "aa:str,b:int", throw=True) - df_eq(df, [["a", 1]], "a:str,b:int", throw=True) - - -def test_as_array(spark_session): - for func in [lambda df, *args, **kwargs: df.as_array(*args, **kwargs), - lambda df, *args, **kwargs: list(df.as_array_iterable(*args, **kwargs))]: - df = _df([], "a:str,b:int") - - assert [] == func(df) - assert [] == func(df, type_safe=True) - - df = _df([["a", 1]], "a:str,b:int") - assert [["a", 1]] == func(df) - assert [["a", 1]] == func(df, ["a", "b"]) - assert [[1, "a"]] == func(df, ["b", "a"]) - - # prevent pandas auto type casting - df = _df([[1.0, 1]], "a:double,b:int") - assert [[1.0, 1]] == func(df) - assert isinstance(func(df)[0][0], float) - assert isinstance(func(df)[0][1], int) - assert [[1.0, 1]] == func(df, ["a", "b"]) - assert [[1, 1.0]] == func(df, ["b", "a"]) - - df = _df([[np.float64(1.1), 1]], "a:double,b:int") - assert [[1.1, 1]] == func(df) - assert isinstance(func(df)[0][0], float) - assert isinstance(func(df)[0][1], int) - - df = _df([[pd.Timestamp("2020-01-01"), 1.1]], "a:datetime,b:int") - assert [[datetime(2020, 1, 1), 1]] == func(df) - assert isinstance(func(df)[0][0], datetime) - assert isinstance(func(df)[0][1], int) - - df = _df([[pd.NaT, 1.1]], "a:datetime,b:int") - assert isinstance(func(df, type_safe=True)[0][0], datetime) - assert isinstance(func(df, type_safe=True)[0][1], int) - - df = _df([[1.0, 1]], "a:double,b:int") - assert [[1.0, 1]] == func(df, type_safe=True) - assert isinstance(func(df)[0][0], float) - assert isinstance(func(df)[0][1], int) - - -def test_as_dict_iterable(): - df = _df([[pd.NaT, 1.1]], "a:datetime,b:int") - assert [dict(a=pd.NaT, b=1)] == list(df.as_dict_iterable()) - df = _df([["2020-01-01", 1.1]], "a:datetime,b:int") - assert [dict(a=datetime(2020, 1, 1), b=1)] == list(df.as_dict_iterable()) - - -def test_head(): - df = _df([], "a:str,b:int") - assert [] == df.head(1) - df = _df([["a", 1]], "a:str,b:int") - assert [["a", 1]] == df.head(1) - assert [["a", 1]] == df.head(1) - assert [[1, "a"]] == df.head(1, ["b", "a"]) - - def _test_as_array_perf(): s = Schema() arr = [] diff --git a/tests/fugue_spark/test_execution_engine.py b/tests/fugue_spark/test_execution_engine.py index 56f9ae77..c01b10b6 100644 --- a/tests/fugue_spark/test_execution_engine.py +++ b/tests/fugue_spark/test_execution_engine.py @@ -3,11 +3,14 @@ import pytest from fugue.collections.partition import PartitionSpec from fugue.dataframe.array_dataframe import ArrayDataFrame +from fugue.extensions.builtins.outputters import df_eq from fugue.extensions.transformer import Transformer, transformer from fugue_spark.execution_engine import SparkExecutionEngine from fugue_test.builtin_suite import BuiltInTests from fugue_test.execution_suite import ExecutionEngineTests +from pyspark import StorageLevel from pyspark.sql import SparkSession +from pytest import raises class SparkExecutionEngineTests(ExecutionEngineTests.Tests): @@ -23,22 +26,21 @@ def init_session(self, spark_session): def test__join_outer_pandas_incompatible(self): return - def _test_repartition(self): - return + def test_persist(self): e = self.engine - a = e.to_df([[1, 2], [3, 4], [5, 6], [7, 8], [9, 10]], "a:int,b:int") - b = e.repartition(a, PartitionSpec()) - assert a is b - b = e.repartition(a, PartitionSpec(num=3)) - assert 3 == b.num_partitions - b = e.repartition(a, PartitionSpec(num="0")) - assert a is b - b = e.repartition(a, PartitionSpec(num="ROWCOUNT/2")) - assert 2 == b.num_partitions - b = e.repartition(a, PartitionSpec(num="ROWCOUNT-ROWCOUNT")) - assert a is b - b = e.repartition(a, PartitionSpec(by=["a"], num=3)) - assert a.num_partitions == b.num_partitions + o = ArrayDataFrame([[1, 2]], + "a:int,b:int", + dict(a=1), + ) + a = e.persist(o) + df_eq(a, o, throw=True) + a = e.persist(o, StorageLevel.MEMORY_ONLY) + df_eq(a, o, throw=True) + a = e.persist(o, "MEMORY_ONLY") + df_eq(a, o, throw=True) + # this passed because persist is run once on ths same object + e.persist(a, "xyz") + raises(ValueError, lambda: e.persist(o, "xyz")) class SparkExecutionEngineBuiltInTests(BuiltInTests.Tests): diff --git a/tests/fugue_spark/utils/test_io.py b/tests/fugue_spark/utils/test_io.py new file mode 100644 index 00000000..6b96b1a3 --- /dev/null +++ b/tests/fugue_spark/utils/test_io.py @@ -0,0 +1,128 @@ +import os + +from fugue.collections.partition import PartitionSpec +from fugue.dataframe.pandas_dataframe import PandasDataFrame +from fugue.dataframe.utils import _df_eq as df_eq +from fugue.exceptions import FugueDataFrameInitError +from fugue_spark.dataframe import SparkDataFrame +from fugue_spark.utils.convert import to_schema, to_spark_schema +from fugue_spark.utils.io import SparkIO +from pyspark.sql import SparkSession +from pyspark.sql.utils import AnalysisException +from pytest import raises +from triad.collections.fs import FileSystem +from triad.exceptions import InvalidOperationError + + +def test_parquet_io(tmpdir, spark_session): + si = SparkIO(spark_session, FileSystem()) + df1 = _df([["1", 2, 3]], "a:str,b:int,c:long") + df2 = _df([[[1, 2]]], "a:[int]") + # {a:int} will become {a:long} because pyarrow lib has issue + df3 = _df([[dict(a=1)]], "a:{a:long}") + for df in [df1, df2, df3]: + path = os.path.join(tmpdir, "a.parquet") + si.save_df(df, path) + actual = si.load_df(path) + df_eq(df, actual, throw=True) + + si.save_df(df1, path) + actual = si.load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + actual = si.load_df(path, columns="b:str,a:int") + df_eq(actual, [["2", 1]], "b:str,a:int") + raises(Exception, lambda: si.load_df(path, columns="bb:str,a:int")) + + # load directory + fs = FileSystem() + folder = os.path.join(tmpdir, "folder") + fs.makedirs(folder) + f0 = os.path.join(folder, "_SUCCESS") + f1 = os.path.join(folder, "1.parquet") + f2 = os.path.join(folder, "3.parquet") + fs.touch(f0) + si.save_df(df1, f1, force_single=True) + si.save_df(df1, f2, force_single=True) + assert fs.isfile(f1) + actual = si.load_df(folder, "parquet") + df_eq(actual, [["1", 2, 3], ["1", 2, 3]], "a:str,b:int,c:long") + + # load multiple paths + actual = si.load_df([f1, f2], "parquet") + df_eq(actual, [["1", 2, 3], ["1", 2, 3]], "a:str,b:int,c:long") + actual = si.load_df([f1, f2], "parquet", columns="b:str,a:str") + df_eq(actual, [["2", "1"], ["2", "1"]], "a:str,b:int,c:long") + + # overwrite = False + raises((FileExistsError, AnalysisException), + lambda: si.save_df(df1, f1, mode="error")) + # wrong mode + raises(Exception, lambda: si.save_df(df1, f1, mode="dummy")) + + +def test_csv_io(tmpdir, spark_session): + fs = FileSystem() + si = SparkIO(spark_session, fs) + df1 = _df([["1", 2, 3]], "a:str,b:int,c:long") + path = os.path.join(tmpdir, "a.csv") + # without header + si.save_df(df1, path) + raises(InvalidOperationError, lambda: si.load_df(path, header=False)) + actual = si.load_df(path, columns=["a", "b", "c"], header=False) + assert [["1", "2", "3"]] == actual.as_array() + assert actual.schema == "a:str,b:str,c:str" + actual = si.load_df(path, columns="a:double,b:str,c:str", header=False) + assert [[1.0, "2", "3"]] == actual.as_array() + assert actual.schema == "a:double,b:str,c:str" + # with header + si.save_df(df1, path, header=True) + actual = si.load_df(path, header=True) + assert [["1", "2", "3"]] == actual.as_array() + actual = si.load_df(path, columns=["b", "a"], header=True) + assert [["2", "1"]] == actual.as_array() + actual = si.load_df(path, columns="b:str,a:double", header=True) + assert [["2", 1.0]] == actual.as_array() + raises(Exception, lambda: si.load_df(path, columns="b:str,x:double", header=True)) + + raises(NotImplementedError, lambda: si.load_df( + path, columns="b:str,x:double", header=2)) + + +def test_json_io(tmpdir, spark_session): + fs = FileSystem() + si = SparkIO(spark_session, fs) + df1 = _df([["1", 2, 3]], "a:str,b:int,c:long") + path = os.path.join(tmpdir, "a.json") + si.save_df(df1, path) + actual = si.load_df(path) + df_eq(actual, [[1, 2, 3]], "a:long,b:long,c:long") + actual = si.load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + actual = si.load_df(path, columns="b:str,a:int") + df_eq(actual, [["2", 1]], "b:str,a:int") + raises(Exception, lambda: si.load_df(path, columns="bb:str,a:int")) + + +def test_save_with_partition(tmpdir, spark_session): + si = SparkIO(spark_session, FileSystem()) + df1 = _df([["1", 2, 3]], "a:str,b:int,c:long") + path = os.path.join(tmpdir, "a.parquet") + si.save_df(df1, path, partition_spec=PartitionSpec(num=2)) + actual = si.load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + si.save_df(df1, path, partition_spec=PartitionSpec(by=["a"])) + actual = si.load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + si.save_df(df1, path, partition_spec=PartitionSpec(by=["a"], num=2)) + actual = si.load_df(path, columns=["b", "a"]) + df_eq(actual, [[2, "1"]], "b:int,a:str") + + +def _df(data, schema=None, metadata=None): + session = SparkSession.builder.getOrCreate() + if schema is not None: + pdf = PandasDataFrame(data, to_schema(schema), metadata) + df = session.createDataFrame(pdf.native, to_spark_schema(schema)) + else: + df = session.createDataFrame(data) + return SparkDataFrame(df, schema, metadata) diff --git a/tests/fugue_spark/utils/test_partition.py b/tests/fugue_spark/utils/test_partition.py index 36742e8b..ea3e742c 100644 --- a/tests/fugue_spark/utils/test_partition.py +++ b/tests/fugue_spark/utils/test_partition.py @@ -1,4 +1,4 @@ -from fugue.dataframe.pandas_dataframes import PandasDataFrame +from fugue.dataframe.pandas_dataframe import PandasDataFrame from fugue_spark.utils.convert import to_schema, to_spark_schema from fugue_spark.utils.partition import (even_repartition, hash_repartition, rand_repartition)