From 5744e9366bc1b7f5d92274bf3edf9bcc4fbf1ff0 Mon Sep 17 00:00:00 2001 From: Jonathan Shi Date: Mon, 12 Dec 2022 15:43:45 -0800 Subject: [PATCH 1/4] tmp Signed-off-by: Jonathan Shi --- modin/core/dataframe/algebra/binary.py | 15 +++- .../dataframe/base/dataframe/dataframe.py | 16 ++-- .../dataframe/pandas/dataframe/dataframe.py | 54 +++++++++----- .../storage_formats/pandas/query_compiler.py | 74 +++++++++---------- 4 files changed, 94 insertions(+), 65 deletions(-) diff --git a/modin/core/dataframe/algebra/binary.py b/modin/core/dataframe/algebra/binary.py index 2d9645aeaca..5ddfa64290d 100644 --- a/modin/core/dataframe/algebra/binary.py +++ b/modin/core/dataframe/algebra/binary.py @@ -44,7 +44,7 @@ def register(cls, func, join_type="outer", labels="replace"): """ def caller( - query_compiler, other, broadcast=False, *args, dtypes=None, **kwargs + query_compiler, other, broadcast=False, *args, dtypes=None, copy_dtypes=False, **kwargs ): """ Apply binary `func` to passed operands. @@ -61,8 +61,14 @@ def caller( at the query compiler level, so this parameter is a hint that passed from a high level API. *args : args, Arguments that will be passed to `func`. - dtypes : "copy" or None, default: None - Whether to keep old dtypes or infer new dtypes from data. + dtypes : pandas.Series or scalar type, optional + The data types for the result. This is an optimization + because there are functions that always result in a particular data + type, and this allows us to avoid (re)computing it. + If the argument is a scalar type, then that type is assigned to each result column. + copy_dtypes : bool, default False + If True, the dtypes of the resulting dataframe are copied from the original, + and the ``dtypes`` argument is ignored. **kwargs : kwargs, Arguments that will be passed to `func`. @@ -93,6 +99,7 @@ def caller( join_type=join_type, labels=labels, dtypes=dtypes, + copy_dtypes=copy_dtypes, ) ) else: @@ -113,11 +120,13 @@ def caller( new_index=query_compiler.index, new_columns=query_compiler.columns, dtypes=dtypes, + copy_dtypes=copy_dtypes, ) else: new_modin_frame = query_compiler._modin_frame.map( lambda df: func(df, other, *args, **kwargs), dtypes=dtypes, + copy_dtypes=copy_dtypes, ) return query_compiler.__constructor__(new_modin_frame) diff --git a/modin/core/dataframe/base/dataframe/dataframe.py b/modin/core/dataframe/base/dataframe/dataframe.py index 536d43ceaea..e4803f6350f 100644 --- a/modin/core/dataframe/base/dataframe/dataframe.py +++ b/modin/core/dataframe/base/dataframe/dataframe.py @@ -92,7 +92,8 @@ def map( self, function: Callable, axis: Optional[Union[int, Axis]] = None, - dtypes: Optional[str] = None, + dtypes: Optional = None, + copy_dtypes: bool = False, ) -> "ModinDataframe": """ Apply a user-defined function row-wise if `axis`=0, column-wise if `axis`=1, and cell-wise if `axis` is None. @@ -103,10 +104,13 @@ def map( The function to map across the dataframe. axis : int or modin.core.dataframe.base.utils.Axis, optional The axis to map over. - dtypes : str, optional + dtypes : pandas.Series or scalar type, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. + copy_dtypes : bool, default: False + If True, the dtypes of the resulting dataframe are copied from the original, + and the ``dtypes`` argument is ignored. Returns ------- @@ -258,7 +262,7 @@ def reduce( self, axis: Union[int, Axis], function: Callable, - dtypes: Optional[str] = None, + dtypes: Optional = None, ) -> "ModinDataframe": """ Perform a user-defined aggregation on the specified axis, where the axis reduces down to a singleton. @@ -269,7 +273,7 @@ def reduce( The axis to perform the reduce over. function : callable(row|col) -> single value The reduce function to apply to each column. - dtypes : str, optional + dtypes : pandas.Series, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. @@ -291,7 +295,7 @@ def tree_reduce( axis: Union[int, Axis], map_func: Callable, reduce_func: Optional[Callable] = None, - dtypes: Optional[str] = None, + dtypes: Optional = None, ) -> "ModinDataframe": """ Perform a user-defined aggregation on the specified axis, where the axis reduces down to a singleton using a tree-reduce computation pattern. @@ -308,7 +312,7 @@ def tree_reduce( The map function to apply to each column. reduce_func : callable(row|col) -> single value, optional The reduce function to apply to the results of the map function. - dtypes : str, optional + dtypes : pandas.Series, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. diff --git a/modin/core/dataframe/pandas/dataframe/dataframe.py b/modin/core/dataframe/pandas/dataframe/dataframe.py index 07067ef83f4..c40cb667775 100644 --- a/modin/core/dataframe/pandas/dataframe/dataframe.py +++ b/modin/core/dataframe/pandas/dataframe/dataframe.py @@ -1599,7 +1599,7 @@ def _tree_reduce_func(df, *args, **kwargs): return _tree_reduce_func - def _compute_tree_reduce_metadata(self, axis, new_parts): + def _compute_tree_reduce_metadata(self, axis, new_parts, dtypes=None): """ Compute the metadata for the result of reduce function. @@ -1609,6 +1609,8 @@ def _compute_tree_reduce_metadata(self, axis, new_parts): The axis on which reduce function was applied. new_parts : NumPy 2D array Partitions with the result of applied function. + dtypes : pandas.Series, optional + The data types of the result. Returns ------- @@ -1623,12 +1625,11 @@ def _compute_tree_reduce_metadata(self, axis, new_parts): new_axes_lengths[axis] = [1] new_axes_lengths[axis ^ 1] = self._axes_lengths[axis ^ 1] - new_dtypes = None result = self.__constructor__( new_parts, *new_axes, *new_axes_lengths, - new_dtypes, + dtypes, ) return result @@ -1637,7 +1638,7 @@ def reduce( self, axis: Union[int, Axis], function: Callable, - dtypes: Optional[str] = None, + dtypes: Optional = None, ) -> "PandasDataframe": """ Perform a user-defined aggregation on the specified axis, where the axis reduces down to a singleton. Requires knowledge of the full axis for the reduction. @@ -1648,7 +1649,7 @@ def reduce( The axis to perform the reduce over. function : callable(row|col) -> single value The reduce function to apply to each column. - dtypes : str, optional + dtypes : pandas.Series, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. @@ -1667,7 +1668,7 @@ def reduce( new_parts = self._partition_mgr_cls.map_axis_partitions( axis.value, self._partitions, function ) - return self._compute_tree_reduce_metadata(axis.value, new_parts) + return self._compute_tree_reduce_metadata(axis.value, new_parts, dtypes) @lazy_metadata_decorator(apply_axis="opposite", axis_arg=0) def tree_reduce( @@ -1675,7 +1676,7 @@ def tree_reduce( axis: Union[int, Axis], map_func: Callable, reduce_func: Optional[Callable] = None, - dtypes: Optional[str] = None, + dtypes: Optional = None, ) -> "PandasDataframe": """ Apply function that will reduce the data to a pandas Series. @@ -1689,7 +1690,7 @@ def tree_reduce( reduce_func : callable(row|col) -> single value, optional Callable function to reduce the dataframe. If none, then apply map_func twice. - dtypes : str, optional + dtypes : pandas.Series, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. @@ -1710,10 +1711,10 @@ def tree_reduce( reduce_parts = self._partition_mgr_cls.map_axis_partitions( axis.value, map_parts, reduce_func ) - return self._compute_tree_reduce_metadata(axis.value, reduce_parts) + return self._compute_tree_reduce_metadata(axis.value, reduce_parts, dtypes) @lazy_metadata_decorator(apply_axis=None) - def map(self, func: Callable, dtypes: Optional[str] = None) -> "PandasDataframe": + def map(self, func: Callable, dtypes: Optional = None, copy_dtypes: bool = False) -> "PandasDataframe": """ Perform a function that maps across the entire dataset. @@ -1721,10 +1722,13 @@ def map(self, func: Callable, dtypes: Optional[str] = None) -> "PandasDataframe" ---------- func : callable(row|col|cell) -> row|col|cell The function to apply. - dtypes : dtypes of the result, optional + dtypes : pandas.Series or scalar type, optional The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. + copy_dtypes : bool, default: False + If True, the dtypes of the resulting dataframe are copied from the original, + and the ``dtypes`` argument is ignored. Returns ------- @@ -1732,7 +1736,7 @@ def map(self, func: Callable, dtypes: Optional[str] = None) -> "PandasDataframe" A new dataframe. """ new_partitions = self._partition_mgr_cls.map_partitions(self._partitions, func) - if dtypes == "copy": + if copy_dtypes: dtypes = self._dtypes elif dtypes is not None: dtypes = pandas.Series( @@ -2186,6 +2190,7 @@ def apply_full_axis( new_index=None, new_columns=None, dtypes=None, + copy_dtypes=False, keep_partitioning=True, ): """ @@ -2207,6 +2212,9 @@ def apply_full_axis( The data types of the result. This is an optimization because there are functions that always result in a particular data type, and allows us to avoid (re)computing it. + copy_dtypes : bool, default: False + If True, the dtypes of the resulting dataframe are copied from the original, + and the ``dtypes`` argument is ignored. keep_partitioning : boolean, default: True The flag to keep partition boundaries for Modin Frame. Setting it to True disables shuffling data from one partition to another. @@ -2226,6 +2234,7 @@ def apply_full_axis( new_index=new_index, new_columns=new_columns, dtypes=dtypes, + copy_dtypes=copy_dtypes, other=None, keep_partitioning=keep_partitioning, ) @@ -2401,7 +2410,7 @@ def apply_select_indices( @lazy_metadata_decorator(apply_axis="both") def broadcast_apply( - self, axis, func, other, join_type="left", labels="keep", dtypes=None + self, axis, func, other, join_type="left", labels="keep", dtypes=None, copy_dtypes=False, ): """ Broadcast axis partitions of `other` to partitions of `self` and apply a function. @@ -2419,8 +2428,12 @@ def broadcast_apply( labels : {"keep", "replace", "drop"}, default: "keep" Whether keep labels from `self` Modin DataFrame, replace them with labels from joined DataFrame or drop altogether to make them be computed lazily later. - dtypes : "copy" or None, default: None - Whether keep old dtypes or infer new dtypes from data. + dtypes : list-like, optional + The data types of the result. This is an optimization + because there are functions that always result in a particular data + type, and allows us to avoid (re)computing it. + copy_dtypes : bool, default: False + If True, the dtypes of the resulting dataframe are copied from the original. Returns ------- @@ -2441,8 +2454,7 @@ def broadcast_apply( new_frame = self._partition_mgr_cls.broadcast_apply( axis, func, left_parts, right_parts ) - if dtypes == "copy": - dtypes = self._dtypes + dtypes = self._dtypes if copy_dtypes else None def _pick_axis(get_axis, sizes_cache): if labels == "keep": @@ -2618,6 +2630,7 @@ def broadcast_apply_full_axis( apply_indices=None, enumerate_partitions=False, dtypes=None, + copy_dtypes=False, keep_partitioning=True, ): """ @@ -2642,10 +2655,13 @@ def broadcast_apply_full_axis( enumerate_partitions : bool, default: False Whether pass partition index into applied `func` or not. Note that `func` must be able to obtain `partition_idx` kwarg. - dtypes : list-like, default: None + dtypes : list-like, optional Data types of the result. This is an optimization because there are functions that always result in a particular data type, and allows us to avoid (re)computing it. + copy_dtypes : bool, default: False + If True, the dtypes of the resulting dataframe are copied from the original, + and the ``dtypes`` argument is ignored. keep_partitioning : boolean, default: True The flag to keep partition boundaries for Modin Frame. Setting it to True disables shuffling data from one partition to another. @@ -2677,7 +2693,7 @@ def broadcast_apply_full_axis( ) # Index objects for new object creation. This is shorter than if..else kw = self.__make_init_labels_args(new_partitions, new_index, new_columns) - if dtypes == "copy": + if copy_dtypes: kw["dtypes"] = self._dtypes elif dtypes is not None: kw["dtypes"] = pandas.Series( diff --git a/modin/core/storage_formats/pandas/query_compiler.py b/modin/core/storage_formats/pandas/query_compiler.py index 25e91401aa2..b4f5d1688ce 100644 --- a/modin/core/storage_formats/pandas/query_compiler.py +++ b/modin/core/storage_formats/pandas/query_compiler.py @@ -1339,7 +1339,7 @@ def stack(self, level, dropna): # Map partitions operations # These operations are operations that apply a function to every partition. - abs = Map.register(pandas.DataFrame.abs, dtypes="copy") + abs = Map.register(pandas.DataFrame.abs, copy_dtypes=True) applymap = Map.register(pandas.DataFrame.applymap) conj = Map.register(lambda df, *args, **kwargs: pandas.DataFrame(np.conj(df))) convert_dtypes = Map.register(pandas.DataFrame.convert_dtypes) @@ -1374,15 +1374,15 @@ def stack(self, level, dropna): # String map partitions operations - str_capitalize = Map.register(_str_map("capitalize"), dtypes="copy") - str_center = Map.register(_str_map("center"), dtypes="copy") + str_capitalize = Map.register(_str_map("capitalize"), copy_dtypes=True) + str_center = Map.register(_str_map("center"), copy_dtypes=True) str_contains = Map.register(_str_map("contains"), dtypes=np.bool_) str_count = Map.register(_str_map("count"), dtypes=int) str_endswith = Map.register(_str_map("endswith"), dtypes=np.bool_) - str_find = Map.register(_str_map("find"), dtypes="copy") - str_findall = Map.register(_str_map("findall"), dtypes="copy") - str_get = Map.register(_str_map("get"), dtypes="copy") - str_index = Map.register(_str_map("index"), dtypes="copy") + str_find = Map.register(_str_map("find"), copy_dtypes=True) + str_findall = Map.register(_str_map("findall"), copy_dtypes=True) + str_get = Map.register(_str_map("get"), copy_dtypes=True) + str_index = Map.register(_str_map("index"), copy_dtypes=True) str_isalnum = Map.register(_str_map("isalnum"), dtypes=np.bool_) str_isalpha = Map.register(_str_map("isalpha"), dtypes=np.bool_) str_isdecimal = Map.register(_str_map("isdecimal"), dtypes=np.bool_) @@ -1392,17 +1392,17 @@ def stack(self, level, dropna): str_isspace = Map.register(_str_map("isspace"), dtypes=np.bool_) str_istitle = Map.register(_str_map("istitle"), dtypes=np.bool_) str_isupper = Map.register(_str_map("isupper"), dtypes=np.bool_) - str_join = Map.register(_str_map("join"), dtypes="copy") + str_join = Map.register(_str_map("join"), copy_dtypes=True) str_len = Map.register(_str_map("len"), dtypes=int) - str_ljust = Map.register(_str_map("ljust"), dtypes="copy") - str_lower = Map.register(_str_map("lower"), dtypes="copy") - str_lstrip = Map.register(_str_map("lstrip"), dtypes="copy") - str_match = Map.register(_str_map("match"), dtypes="copy") - str_normalize = Map.register(_str_map("normalize"), dtypes="copy") - str_pad = Map.register(_str_map("pad"), dtypes="copy") - str_partition = Map.register(_str_map("partition"), dtypes="copy") - str_repeat = Map.register(_str_map("repeat"), dtypes="copy") - _str_extract = Map.register(_str_map("extract"), dtypes="copy") + str_ljust = Map.register(_str_map("ljust"), copy_dtypes=True) + str_lower = Map.register(_str_map("lower"), copy_dtypes=True) + str_lstrip = Map.register(_str_map("lstrip"), copy_dtypes=True) + str_match = Map.register(_str_map("match"), copy_dtypes=True) + str_normalize = Map.register(_str_map("normalize"), copy_dtypes=True) + str_pad = Map.register(_str_map("pad"), copy_dtypes=True) + str_partition = Map.register(_str_map("partition"), copy_dtypes=True) + str_repeat = Map.register(_str_map("repeat"), copy_dtypes=True) + _str_extract = Map.register(_str_map("extract"), copy_dtypes=True) def str_extract(self, pat, flags, expand): regex = re.compile(pat, flags=flags) @@ -1414,25 +1414,25 @@ def str_extract(self, pat, flags, expand): qc.columns = get_group_names(regex) return qc - str_replace = Map.register(_str_map("replace"), dtypes="copy") - str_rfind = Map.register(_str_map("rfind"), dtypes="copy") - str_rindex = Map.register(_str_map("rindex"), dtypes="copy") - str_rjust = Map.register(_str_map("rjust"), dtypes="copy") - str_rpartition = Map.register(_str_map("rpartition"), dtypes="copy") - str_rsplit = Map.register(_str_map("rsplit"), dtypes="copy") - str_rstrip = Map.register(_str_map("rstrip"), dtypes="copy") - str_slice = Map.register(_str_map("slice"), dtypes="copy") - str_slice_replace = Map.register(_str_map("slice_replace"), dtypes="copy") - str_split = Map.register(_str_map("split"), dtypes="copy") + str_replace = Map.register(_str_map("replace"), copy_dtypes=True) + str_rfind = Map.register(_str_map("rfind"), copy_dtypes=True) + str_rindex = Map.register(_str_map("rindex"), copy_dtypes=True) + str_rjust = Map.register(_str_map("rjust"), copy_dtypes=True) + str_rpartition = Map.register(_str_map("rpartition"), copy_dtypes=True) + str_rsplit = Map.register(_str_map("rsplit"), copy_dtypes=True) + str_rstrip = Map.register(_str_map("rstrip"), copy_dtypes=True) + str_slice = Map.register(_str_map("slice"), copy_dtypes=True) + str_slice_replace = Map.register(_str_map("slice_replace"), copy_dtypes=True) + str_split = Map.register(_str_map("split"), copy_dtypes=True) str_startswith = Map.register(_str_map("startswith"), dtypes=np.bool_) - str_strip = Map.register(_str_map("strip"), dtypes="copy") - str_swapcase = Map.register(_str_map("swapcase"), dtypes="copy") - str_title = Map.register(_str_map("title"), dtypes="copy") - str_translate = Map.register(_str_map("translate"), dtypes="copy") - str_upper = Map.register(_str_map("upper"), dtypes="copy") - str_wrap = Map.register(_str_map("wrap"), dtypes="copy") - str_zfill = Map.register(_str_map("zfill"), dtypes="copy") - str___getitem__ = Map.register(_str_map("__getitem__"), dtypes="copy") + str_strip = Map.register(_str_map("strip"), copy_dtypes=True) + str_swapcase = Map.register(_str_map("swapcase"), copy_dtypes=True) + str_title = Map.register(_str_map("title"), copy_dtypes=True) + str_translate = Map.register(_str_map("translate"), copy_dtypes=True) + str_upper = Map.register(_str_map("upper"), copy_dtypes=True) + str_wrap = Map.register(_str_map("wrap"), copy_dtypes=True) + str_zfill = Map.register(_str_map("zfill"), copy_dtypes=True) + str___getitem__ = Map.register(_str_map("__getitem__"), copy_dtypes=True) # END String map partitions operations @@ -2063,7 +2063,7 @@ def sort_index(self, **kwargs): ), new_index, new_columns, - dtypes="copy" if axis == 0 else None, + copy_dtypes=True if axis == 0 else None, ) return self.__constructor__(new_modin_frame) @@ -2195,7 +2195,7 @@ def getitem_array(self, key): # ones are just of bool dtype if len(key.dtypes) == 1 and is_bool_dtype(key.dtypes[0]): self.__validate_bool_indexer(key.index) - return self.__getitem_bool(key, broadcast=True, dtypes="copy") + return self.__getitem_bool(key, broadcast=True, copy_dtypes=True) key = key.to_pandas().squeeze(axis=1) From 58d9990143baccd83f90ccec506cb8f6b65822af Mon Sep 17 00:00:00 2001 From: Jonathan Shi Date: Mon, 12 Dec 2022 15:47:24 -0800 Subject: [PATCH 2/4] lint Signed-off-by: Jonathan Shi --- modin/core/dataframe/algebra/binary.py | 8 +++++++- modin/core/dataframe/base/dataframe/dataframe.py | 9 ++++++--- modin/core/dataframe/pandas/dataframe/dataframe.py | 13 +++++++++++-- 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/modin/core/dataframe/algebra/binary.py b/modin/core/dataframe/algebra/binary.py index 5ddfa64290d..b55d0a2395e 100644 --- a/modin/core/dataframe/algebra/binary.py +++ b/modin/core/dataframe/algebra/binary.py @@ -44,7 +44,13 @@ def register(cls, func, join_type="outer", labels="replace"): """ def caller( - query_compiler, other, broadcast=False, *args, dtypes=None, copy_dtypes=False, **kwargs + query_compiler, + other, + broadcast=False, + *args, + dtypes=None, + copy_dtypes=False, + **kwargs ): """ Apply binary `func` to passed operands. diff --git a/modin/core/dataframe/base/dataframe/dataframe.py b/modin/core/dataframe/base/dataframe/dataframe.py index e4803f6350f..4c756d50191 100644 --- a/modin/core/dataframe/base/dataframe/dataframe.py +++ b/modin/core/dataframe/base/dataframe/dataframe.py @@ -19,6 +19,9 @@ from abc import ABC, abstractmethod from typing import List, Hashable, Optional, Callable, Union, Dict + +import pandas + from modin.core.dataframe.base.dataframe.utils import Axis, JoinType @@ -92,7 +95,7 @@ def map( self, function: Callable, axis: Optional[Union[int, Axis]] = None, - dtypes: Optional = None, + dtypes: Optional[Union[pandas.Series, type]] = None, copy_dtypes: bool = False, ) -> "ModinDataframe": """ @@ -262,7 +265,7 @@ def reduce( self, axis: Union[int, Axis], function: Callable, - dtypes: Optional = None, + dtypes: Optional[pandas.Series] = None, ) -> "ModinDataframe": """ Perform a user-defined aggregation on the specified axis, where the axis reduces down to a singleton. @@ -295,7 +298,7 @@ def tree_reduce( axis: Union[int, Axis], map_func: Callable, reduce_func: Optional[Callable] = None, - dtypes: Optional = None, + dtypes: Optional[pandas.Series] = None, ) -> "ModinDataframe": """ Perform a user-defined aggregation on the specified axis, where the axis reduces down to a singleton using a tree-reduce computation pattern. diff --git a/modin/core/dataframe/pandas/dataframe/dataframe.py b/modin/core/dataframe/pandas/dataframe/dataframe.py index c40cb667775..93bddc9555c 100644 --- a/modin/core/dataframe/pandas/dataframe/dataframe.py +++ b/modin/core/dataframe/pandas/dataframe/dataframe.py @@ -1714,7 +1714,9 @@ def tree_reduce( return self._compute_tree_reduce_metadata(axis.value, reduce_parts, dtypes) @lazy_metadata_decorator(apply_axis=None) - def map(self, func: Callable, dtypes: Optional = None, copy_dtypes: bool = False) -> "PandasDataframe": + def map( + self, func: Callable, dtypes: Optional = None, copy_dtypes: bool = False + ) -> "PandasDataframe": """ Perform a function that maps across the entire dataset. @@ -2410,7 +2412,14 @@ def apply_select_indices( @lazy_metadata_decorator(apply_axis="both") def broadcast_apply( - self, axis, func, other, join_type="left", labels="keep", dtypes=None, copy_dtypes=False, + self, + axis, + func, + other, + join_type="left", + labels="keep", + dtypes=None, + copy_dtypes=False, ): """ Broadcast axis partitions of `other` to partitions of `self` and apply a function. From a89d5fec0dbe5c2fc9ec3674795f288afd4df3fa Mon Sep 17 00:00:00 2001 From: Jonathan Shi Date: Tue, 3 Jan 2023 13:47:52 -0800 Subject: [PATCH 3/4] Update `df.map` docstring Co-authored-by: Anatoly Myachev --- modin/core/dataframe/base/dataframe/dataframe.py | 1 + 1 file changed, 1 insertion(+) diff --git a/modin/core/dataframe/base/dataframe/dataframe.py b/modin/core/dataframe/base/dataframe/dataframe.py index 4c756d50191..c3b7e5524f7 100644 --- a/modin/core/dataframe/base/dataframe/dataframe.py +++ b/modin/core/dataframe/base/dataframe/dataframe.py @@ -111,6 +111,7 @@ def map( The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. + If the argument is a scalar type, then that type is assigned to each result column. copy_dtypes : bool, default: False If True, the dtypes of the resulting dataframe are copied from the original, and the ``dtypes`` argument is ignored. From d9e95abe53edecc3a6586734d301d80cc8fd922c Mon Sep 17 00:00:00 2001 From: Jonathan Shi Date: Tue, 3 Jan 2023 13:50:33 -0800 Subject: [PATCH 4/4] replace list-like with pandas.Series Signed-off-by: Jonathan Shi --- modin/core/dataframe/pandas/dataframe/dataframe.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/modin/core/dataframe/pandas/dataframe/dataframe.py b/modin/core/dataframe/pandas/dataframe/dataframe.py index 93bddc9555c..81d4b3e9618 100644 --- a/modin/core/dataframe/pandas/dataframe/dataframe.py +++ b/modin/core/dataframe/pandas/dataframe/dataframe.py @@ -1728,6 +1728,7 @@ def map( The data types for the result. This is an optimization because there are functions that always result in a particular data type, and this allows us to avoid (re)computing it. + If the argument is a scalar type, then that type is assigned to each result column. copy_dtypes : bool, default: False If True, the dtypes of the resulting dataframe are copied from the original, and the ``dtypes`` argument is ignored. @@ -2210,7 +2211,7 @@ def apply_full_axis( new_columns : list-like, optional The columns of the result. We may know this in advance, and if not provided it must be computed. - dtypes : list-like, optional + dtypes : pandas.Series, optional The data types of the result. This is an optimization because there are functions that always result in a particular data type, and allows us to avoid (re)computing it. @@ -2437,7 +2438,7 @@ def broadcast_apply( labels : {"keep", "replace", "drop"}, default: "keep" Whether keep labels from `self` Modin DataFrame, replace them with labels from joined DataFrame or drop altogether to make them be computed lazily later. - dtypes : list-like, optional + dtypes : pandas.Series, optional The data types of the result. This is an optimization because there are functions that always result in a particular data type, and allows us to avoid (re)computing it. @@ -2664,7 +2665,7 @@ def broadcast_apply_full_axis( enumerate_partitions : bool, default: False Whether pass partition index into applied `func` or not. Note that `func` must be able to obtain `partition_idx` kwarg. - dtypes : list-like, optional + dtypes : pandas.Series, optional Data types of the result. This is an optimization because there are functions that always result in a particular data type, and allows us to avoid (re)computing it.