From ed93e6b518c5bc69f2597abbe1743a88693c9347 Mon Sep 17 00:00:00 2001 From: Devin Petersohn Date: Mon, 30 Jul 2018 08:30:37 -0700 Subject: [PATCH] Rewriting the partitioning Adding some partitioning updates Updating remote Continuing backend rewrite progress Adding idxmax/min Adding head/tail/repr for data_manager Fixing transpose Updating remote with bugfixes for transpose/repr Fixing a number of operations Fix quantile Adding more functionality, __getitem__ Fixing more tests Fixing drop, passing tests Add insert to new structure Cleaning up unneeded imports Updating remote Fix minor bug Updating remote Add sort_index Minor refactor of code. Cleaning some up Continuing logic migration Add some docs Adding more docs Add more method level documentation. Adding documentation and cleaning up Retructuring partitioning files for simplicity Adding more docs, cleaning up docs Fix performance bug, more cleanup More cleanup/renaming Adding factory skeleton Adding from_pandas code path and update constructor Removing debugging code Cleaning up dead code Added type checking and changed how variables were read in from kwargs (#1) Removing IndexMetadata and code_gen Adding preliminary apply method Updated sample and eval to the new backend (#2) * Added type checking and changed how variables were read in from kwargs * Updated sample to new architecture * Made test_sample more rigourous * Removed 'default=' from kwargs.get's * Updated eval to the new backend * Added two more tests for eval Finalizing apply and start agg Fixing some broken stuff with apply, update remote Starting dictionary apply and fillna Fixed dictionary apply and fillna Moves Inter DataFrame Operations Logic to Data Manager (#3) * Moving multi dataframe operation logic to data_manager * Remove Unused Functions from dataframe.py * removed unnecessary isScalar arg * minor code cleanup * changed _operator_handler name * removing hasattr from data_manager * cleaning up dataframe.py code for add function * changing name to _validate_other * cleaned up kwargs parsing in data_manager function * updated all inter df functions * commenting out old helper functions in dataframe.py * cleaned up unused code * fixed type error for functions using map_across_axis Updated info and memory_usage to new backend (#4) * Added type checking and changed how variables were read in from kwargs * Updated sample to new architecture * Made test_sample more rigourous * Removed 'default=' from kwargs.get's * Updated eval to the new backend * Added two more tests for eval * Updated memory_usage to new backend * Updated info and memory_usage to the new backend * Updated info and memory_usage to be standalone tests and updated the tests * Updated info to do only one pass * Updated info to do everything in one run with DataFrame * Update info to do everything in one run with Series * Updated info to do everything in one run with DataFrame * Updated to get everything working and moved appropriate parts to DataManager Adding first where implementation Adding sort_values and update implementations Cleaning up dead code Adding manual_shuffle abstraction Starting merge Add merge Cleaning up Add dtype (#6) * Added type checking and changed how variables were read in from kwargs * Updated sample to new architecture * Made test_sample more rigourous * Removed 'default=' from kwargs.get's * Updated eval to the new backend * Added two more tests for eval * Updated memory_usage to new backend * Updated info and memory_usage to the new backend * Updated info and memory_usage to be standalone tests and updated the tests * Updated info to do only one pass * Updated info to do everything in one run with DataFrame * Update info to do everything in one run with Series * Updated info to do everything in one run with DataFrame * Updated to get everything working and moved appropriate parts to DataManager * Removed extraneous print statement * Moved dtypes stuff to data manager * Fixed calculating dtypes to only doing a full_reduce instead of map_full_axis * Updated astype to new backend * Updated astype to new backend * Updated ftypes to new backend * Added dtypes argument to map_partitions * Fixing dtypes * Cleaning up dtype and merge issues Fix isin bug Cleaning up Cleaning up more unused code Updated iterables and to_datetime to new backend and improved astype runtime (#7) * Added type checking and changed how variables were read in from kwargs * Updated sample to new architecture * Made test_sample more rigourous * Removed 'default=' from kwargs.get's * Updated eval to the new backend * Added two more tests for eval * Updated memory_usage to new backend * Updated info and memory_usage to the new backend * Updated info and memory_usage to be standalone tests and updated the tests * Updated info to do only one pass * Updated info to do everything in one run with DataFrame * Update info to do everything in one run with Series * Updated info to do everything in one run with DataFrame * Updated to get everything working and moved appropriate parts to DataManager * Removed extraneous print statement * Moved dtypes stuff to data manager * Fixed calculating dtypes to only doing a full_reduce instead of map_full_axis * Updated astype to new backend * Updated astype to new backend * Updated ftypes to new backend * Added dtypes argument to map_partitions * Updated astype and added dtypes option to _from_old_block_partitions in RayPandasDataManager * Undid unnecessary change * Updated iterables to new backend * Updated to_datetime to new backend * Reverted some changes for PR * Replaced pd with pandas * Made additional changes mentioned in (#7) Cleaning up Cleaning up imports Fix minor bug from getting kwargs Concat now working with new architecture (#9) * concat now working with new architecture * fixing functionality for pandas Series * updated append_list_of_data_managers function for concat * minor stylistic fix * remove unused append_data_manager function * fixed join * removed axis arg from join function read_csv changes and improvements in performance (#10) * Test changes to io * Update io changes * Fix performance bug * Debugging performance * Debugging performance on large IO * Making some performance tuning changes * Cleaning up and adding performance improvements * Cleaning up * Addressing comments * Addressing comments Fix bug Formatting fix fillna bug updated rdiv, rpow, rsub methods (#12) * updated rdiv, rpow, rsub methods * spelled dataframe wrong Fixed eval and astype (#11) * Updated to_datetime docstring * Updated astype tests * Commented out loc and iloc tests * Updated eval * removed empty space and uncommented test_loc and test_iloc Passes test_mixed_dtype_dataframe and test_nan_dataframe (#15) * Fixed describe and quantiles and cleaned up code * Updated numeric functions and handles empty dataframes * Fixed dtypes and ftypes * Imported is_numeric_dtype from pandas * Cleaned up print statements in test_dataframe.py Cleaning up and enabling tests. Fix __repr__ Removing dead code Fix where bugy Fix append error checking Fix read_csv args bug Fix read_parquet Groupby implementation Adding groupby final fix Adding docs Fix for info (#16) * Quick fix for info * Removed extraneous print statement * Restructured to use count and memory_usage instead Minor optimization change get_dummies implementation (#19) * intial code for get_dummies * Starting help on get_dummies * Fix get_dummies * Removing dead code * bug fix for get_dummies Rewrite loc (#20) * Rewrite the rewrite Finish implement loc/iloc Remove debug lines, fix typo Removing unused imports * Removing dead code * Changing naming of clone * Formatting and removing dead code * Moving imports for matching pandas --- modin/__init__.py | 20 + modin/data_management/__init__.py | 0 modin/data_management/data_manager.py | 1680 +++++++++ modin/data_management/factories.py | 38 + modin/data_management/partitioning/README.md | 13 + .../data_management/partitioning/__init__.py | 0 .../partitioning/axis_partition.py | 211 ++ .../partitioning/partition_collections.py | 836 +++++ .../partitioning/remote_partition.py | 284 ++ modin/data_management/partitioning/utils.py | 35 + modin/pandas/__init__.py | 29 +- modin/pandas/concat.py | 106 +- modin/pandas/dataframe.py | 3219 ++++------------- modin/pandas/datetimes.py | 52 +- modin/pandas/groupby.py | 446 +-- modin/pandas/index/__init__.py | 0 modin/pandas/index/partitioned_index.py | 29 + modin/pandas/index_metadata.py | 436 --- modin/pandas/indexing.py | 346 +- modin/pandas/io.py | 215 +- modin/pandas/iterator.py | 24 +- modin/pandas/pandas_code_gen.py | 93 - modin/pandas/reshape.py | 109 +- modin/pandas/test/test_concat.py | 28 +- modin/pandas/test/test_dataframe.py | 216 +- modin/pandas/test/test_groupby.py | 44 +- modin/pandas/utils.py | 720 +--- 27 files changed, 4368 insertions(+), 4861 deletions(-) create mode 100644 modin/data_management/__init__.py create mode 100644 modin/data_management/data_manager.py create mode 100644 modin/data_management/factories.py create mode 100644 modin/data_management/partitioning/README.md create mode 100644 modin/data_management/partitioning/__init__.py create mode 100644 modin/data_management/partitioning/axis_partition.py create mode 100644 modin/data_management/partitioning/partition_collections.py create mode 100644 modin/data_management/partitioning/remote_partition.py create mode 100644 modin/data_management/partitioning/utils.py create mode 100644 modin/pandas/index/__init__.py create mode 100644 modin/pandas/index/partitioned_index.py delete mode 100644 modin/pandas/index_metadata.py delete mode 100644 modin/pandas/pandas_code_gen.py diff --git a/modin/__init__.py b/modin/__init__.py index 936376e08a9..79acd0d8dd6 100644 --- a/modin/__init__.py +++ b/modin/__init__.py @@ -24,5 +24,25 @@ def _execute_cmd_in_temp_env(cmd): return "Unknown" +def get_execution_engine(): + # In the future, when there are multiple engines and different ways of + # backing the DataFrame, there will have to be some changed logic here to + # decide these things. In the meantime, we will use the currently supported + # execution engine + backing (Pandas + Ray). + return "Ray" + + +def get_partition_format(): + # See note above about engine + backing. + return "Pandas" + + __git_revision__ = git_version() __version__ = "0.1.1" +__execution_engine__ = get_execution_engine() +__partition_format__ = get_partition_format() + +# We don't want these used outside of this file. +del git_version +del get_execution_engine +del get_partition_format diff --git a/modin/data_management/__init__.py b/modin/data_management/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/modin/data_management/data_manager.py b/modin/data_management/data_manager.py new file mode 100644 index 00000000000..6adebf7da08 --- /dev/null +++ b/modin/data_management/data_manager.py @@ -0,0 +1,1680 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import numpy as np +import pandas + +from pandas.compat import string_types +from pandas.core.dtypes.cast import find_common_type +from pandas.core.dtypes.common import (_get_dtype_from_object, is_list_like, is_numeric_dtype) +from pandas.core.index import _ensure_index + +from .partitioning.partition_collections import BlockPartitions + + +class PandasDataManager(object): + """This class implements the logic necessary for operating on partitions + with a Pandas backend. This logic is specific to Pandas. + """ + + def __init__(self, block_partitions_object: BlockPartitions, + index: pandas.Index, columns: pandas.Index, dtypes=None): + assert isinstance(block_partitions_object, BlockPartitions) + self.data = block_partitions_object + self.index = index + self.columns = columns + if dtypes is not None: + self._dtype_cache = dtypes + + def __constructor__(self, block_paritions_object, index, columns, dtypes=None): + """By default, clone method will invoke an init""" + return type(self)(block_paritions_object, index, columns, dtypes) + + # Index, columns and dtypes objects + _dtype_cache = None + + def _get_dtype(self): + if self._dtype_cache is None: + map_func = lambda df: df.dtypes + + def func(row): + return find_common_type(row.values) + + self._dtype_cache = self.data.full_reduce(map_func, lambda df: df.apply(func, axis=0), 0) + self._dtype_cache.index = self.columns + return self._dtype_cache + + def _set_dtype(self, dtypes): + self._dtype_cache = dtypes + + dtypes = property(_get_dtype, _set_dtype) + + # These objects are currently not distributed. + _index_cache = None + _columns_cache = None + + def _get_index(self): + return self._index_cache + + def _get_columns(self): + return self._columns_cache + + def _validate_set_axis(self, new_labels, old_labels): + new_labels = _ensure_index(new_labels) + old_len = len(old_labels) + new_len = len(new_labels) + if old_len != new_len: + raise ValueError('Length mismatch: Expected axis has %d elements, ' + 'new values have %d elements' % (old_len, new_len)) + return new_labels + + def _set_index(self, new_index): + if self._index_cache is None: + self._index_cache = _ensure_index(new_index) + else: + new_index = self._validate_set_axis(new_index, self._index_cache) + self._index_cache = new_index + + def _set_columns(self, new_columns): + if self._columns_cache is None: + self._columns_cache = _ensure_index(new_columns) + else: + new_columns = self._validate_set_axis(new_columns, self._columns_cache) + self._columns_cache = new_columns + + columns = property(_get_columns, _set_columns) + index = property(_get_index, _set_index) + # END Index, columns, and dtypes objects + + def compute_index(self, axis, data_object, compute_diff=True): + """Computes the index after a number of rows have been removed. + + Note: In order for this to be used properly, the indexes must not be + changed before you compute this. + + Args: + axis: The axis to extract the index from. + data_object: The new data object to extract the index from. + compute_diff: True to use `self` to compute the index from self + rather than data_object. This is used when the dimension of the + index may have changed, but the deleted rows/columns are + unknown + + Returns: + A new pandas.Index object. + """ + def pandas_index_extraction(df, axis): + if not axis: + return df.index + else: + try: + return df.columns + except AttributeError: + return pandas.Index([]) + + index_obj = self.index if not axis else self.columns + old_blocks = self.data if compute_diff else None + new_indices = data_object.get_indices(axis=axis, index_func=lambda df: pandas_index_extraction(df, axis), old_blocks=old_blocks) + + return index_obj[new_indices] if compute_diff else new_indices + # END Index and columns objects + + # Internal methods + # These methods are for building the correct answer in a modular way. + # Please be careful when changing these! + def _prepare_method(self, pandas_func, **kwargs): + """Prepares methods given various metadata. + + :param pandas_func: + :param kwargs: + :return: + """ + + if self._is_transposed: + def helper(df, internal_indices=[]): + return pandas_func(df.T, **kwargs) + else: + def helper(df, internal_indices=[]): + return pandas_func(df, **kwargs) + return helper + + def numeric_indices(self): + """Returns the numeric indices + + Args: + axis: The axis to extract the indices from. + + Returns: + List of index names + """ + columns = list() + for col, dtype in zip(self.columns, self.dtypes): + if is_numeric_dtype(dtype): + columns.append(col) + return columns + # END Internal methods + + # Metadata modification methods + def add_prefix(self, prefix): + new_column_names = self.columns.map(lambda x: str(prefix) + str(x)) + return self.__constructor__(self.data, self.index, new_column_names, self._dtype_cache) + + def add_suffix(self, suffix): + new_column_names = self.columns.map(lambda x: str(x) + str(suffix)) + return self.__constructor__(self.data, self.index, new_column_names, self._dtype_cache) + # END Metadata modification methods + + # Copy + # For copy, we don't want a situation where we modify the metadata of the + # copies if we end up modifying something here. We copy all of the metadata + # to prevent that. + def copy(self): + return self.__constructor__(self.data.copy(), self.index.copy(), self.columns.copy(), self._dtype_cache) + + # Append/Concat/Join (Not Merge) + # The append/concat/join operations should ideally never trigger remote + # compute. These operations should only ever be manipulations of the + # metadata of the resulting object. It should just be a simple matter of + # appending the other object's blocks and adding np.nan columns for the new + # columns, if needed. If new columns are added, some compute may be + # required, though it can be delayed. + # + # Currently this computation is not delayed, and it may make a copy of the + # DataFrame in memory. This can be problematic and should be fixed in the + # future. TODO: Delay reindexing + def _join_index_objects(self, axis, other_index, how, sort=True): + """Joins a pair of index objects (columns or rows) by a given strategy. + + :param other_index: + :param axis: The axis index object to join (0 for columns, 1 for index) + :param how: + :return: + """ + if isinstance(other_index, list): + joined_obj = self.columns if not axis else self.index + # TODO: revisit for performance + for obj in other_index: + joined_obj = joined_obj.join(obj, how=how) + + return joined_obj + if not axis: + return self.columns.join(other_index, how=how, sort=sort) + else: + return self.index.join(other_index, how=how, sort=sort) + + def join(self, other, **kwargs): + if isinstance(other, list): + return self._join_list_of_managers(other, **kwargs) + else: + return self._join_data_manager(other, **kwargs) + + def concat(self, axis, other, **kwargs): + return self._append_list_of_managers(other, axis, **kwargs) + + def _append_list_of_managers(self, others, axis, **kwargs): + if not isinstance(others, list): + others = [others] + assert all(isinstance(other, type(self)) for other in others), \ + "Different Manager objects are being used. This is not allowed" + + + sort = kwargs.get("sort", None) + join = kwargs.get("join", "outer") + ignore_index = kwargs.get("ignore_index", False) + + # Concatenating two managers requires aligning their indices. After the + # indices are aligned, it should just be a simple concatenation of the + # `BlockPartitions` objects. This should not require remote compute. + joined_axis = self._join_index_objects(axis, [other.columns if axis == 0 + else other.index for other in others], join, sort=sort) + + # Since we are concatenating a list of managers, we will align all of + # the indices based on the `joined_axis` computed above. + to_append = [other.reindex(axis ^ 1, joined_axis).data for other in others] + new_self = self.reindex(axis ^ 1, joined_axis).data + new_data = new_self.concat(axis, to_append) + + if axis == 0: + # The indices will be appended to form the final index. + # If `ignore_index` is true, we create a RangeIndex that is the + # length of all of the index objects combined. This is the same + # behavior as pandas. + new_index = self.index.append([other.index for other in others]) if not ignore_index else pandas.RangeIndex(len(self.index) + sum([len(other.index) for other in others])) + return self.__constructor__(new_data, new_index, joined_axis) + else: + # The columns will be appended to form the final columns. + new_columns = self.columns.append([other.columns for other in others]) + return self.__constructor__(new_data, joined_axis, new_columns) + + def _join_data_manager(self, other, **kwargs): + assert isinstance(other, type(self)), \ + "This method is for data manager objects only" + + + # Uses join's default value (though should not revert to default) + how = kwargs.get("how", "left") + sort = kwargs.get("sort", False) + lsuffix = kwargs.get("lsuffix", "") + rsuffix = kwargs.get("rsuffix", "") + + joined_index = self._join_index_objects(1, other.index, how, sort=sort) + + to_join = other.reindex(0, joined_index).data + new_self = self.reindex(0, joined_index).data + + new_data = new_self.concat(1, to_join) + + # We are using proxy DataFrame objects to build the columns based on + # the `lsuffix` and `rsuffix`. + self_proxy = pandas.DataFrame(columns=self.columns) + other_proxy = pandas.DataFrame(columns=other.columns) + new_columns = self_proxy.join(other_proxy, lsuffix=lsuffix, rsuffix=rsuffix).columns + + return self.__constructor__(new_data, joined_index, new_columns) + + def _join_list_of_managers(self, others, **kwargs): + assert isinstance(others, list), \ + "This method is for lists of DataManager objects only" + assert all(isinstance(other, type(self)) for other in others), \ + "Different Manager objects are being used. This is not allowed" + + + # Uses join's default value (though should not revert to default) + how = kwargs.get("how", "left") + sort = kwargs.get("sort", False) + lsuffix = kwargs.get("lsuffix", "") + rsuffix = kwargs.get("rsuffix", "") + + joined_index = self._join_index_objects(1, [other.index for other in others], how, sort=sort) + + to_join = [other.reindex(0, joined_index).data for other in others] + new_self = self.reindex(0, joined_index).data + + new_data = new_self.concat(1, to_join) + + # This stage is to efficiently get the resulting columns, including the + # suffixes. + self_proxy = pandas.DataFrame(columns=self.columns) + others_proxy = [pandas.DataFrame(columns=other.columns) for other in others] + new_columns = self_proxy.join(others_proxy, lsuffix=lsuffix, rsuffix=rsuffix).columns + + return self.__constructor__(new_data, joined_index, new_columns) + # END Append/Concat/Join + + # Inter-Data operations (e.g. add, sub) + # These operations require two DataFrames and will change the shape of the + # data if the index objects don't match. An outer join + op is performed, + # such that columns/rows that don't have an index on the other DataFrame + # result in NaN values. + def inter_manager_operations(self, other, how_to_join, func): + + + assert isinstance(other, type(self)), \ + "Must have the same DataManager subclass to perform this operation" + + joined_index = self._join_index_objects(1, other.index, how_to_join, sort=False) + new_columns = self._join_index_objects(0, other.columns, how_to_join, sort=False) + + reindexed_other = other.reindex(0, joined_index).data + reindexed_self = self.reindex(0, joined_index).data + + # THere is an interesting serialization anomaly that happens if we do + # not use the columns in `inter_data_op_builder` from here (e.g. if we + # pass them in). Passing them in can cause problems, so we will just + # use them from here. + self_cols = self.columns + other_cols = other.columns + + def inter_data_op_builder(left, right, self_cols, other_cols, func): + left.columns = self_cols + right.columns = other_cols + result = func(left, right) + result.columns = pandas.RangeIndex(len(result.columns)) + return result + + new_data = reindexed_self.inter_data_operation(1, lambda l, r: inter_data_op_builder(l, r, self_cols, other_cols, func), reindexed_other) + + return self.__constructor__(new_data, joined_index, new_columns) + + def _inter_df_op_handler(self, func, other, **kwargs): + """Helper method for inter-DataFrame and scalar operations""" + axis = kwargs.get("axis", 0) + + if isinstance(other, type(self)): + return self.inter_manager_operations(other, "outer", lambda x, y: func(x, y, **kwargs)) + else: + return self.scalar_operations(axis, other, lambda df: func(df, other, **kwargs)) + + def add(self, other, **kwargs): + # TODO: need to write a prepare_function for inter_df operations + func = pandas.DataFrame.add + return self._inter_df_op_handler(func, other, **kwargs) + + def div(self, other, **kwargs): + func = pandas.DataFrame.div + return self._inter_df_op_handler(func, other, **kwargs) + + def eq(self, other, **kwargs): + func = pandas.DataFrame.eq + return self._inter_df_op_handler(func, other, **kwargs) + + def floordiv(self, other, **kwargs): + func = pandas.DataFrame.floordiv + return self._inter_df_op_handler(func, other, **kwargs) + + def ge(self, other, **kwargs): + func = pandas.DataFrame.ge + return self._inter_df_op_handler(func, other, **kwargs) + + def gt(self, other, **kwargs): + func = pandas.DataFrame.gt + return self._inter_df_op_handler(func, other, **kwargs) + + def le(self, other, **kwargs): + func = pandas.DataFrame.le + return self._inter_df_op_handler(func, other, **kwargs) + + def lt(self, other, **kwargs): + func = pandas.DataFrame.lt + return self._inter_df_op_handler(func, other, **kwargs) + + def mod(self, other, **kwargs): + func = pandas.DataFrame.mod + return self._inter_df_op_handler(func, other, **kwargs) + + def mul(self, other, **kwargs): + func = pandas.DataFrame.mul + return self._inter_df_op_handler(func, other, **kwargs) + + def ne(self, other, **kwargs): + func = pandas.DataFrame.ne + return self._inter_df_op_handler(func, other, **kwargs) + + def pow(self, other, **kwargs): + func = pandas.DataFrame.pow + return self._inter_df_op_handler(func, other, **kwargs) + + def rdiv(self, other, **kwargs): + func = pandas.DataFrame.rdiv + return self._inter_df_op_handler(func, other, **kwargs) + + def rpow(self, other, **kwargs): + func = pandas.DataFrame.rpow + return self._inter_df_op_handler(func, other, **kwargs) + + def rsub(self, other, **kwargs): + func = pandas.DataFrame.rsub + return self._inter_df_op_handler(func, other, **kwargs) + + def sub(self, other, **kwargs): + func = pandas.DataFrame.sub + return self._inter_df_op_handler(func, other, **kwargs) + + def truediv(self, other, **kwargs): + func = pandas.DataFrame.truediv + return self._inter_df_op_handler(func, other, **kwargs) + + def update(self, other, **kwargs): + assert isinstance(other, type(self)), \ + "Must have the same DataManager subclass to perform this operation" + + def update_builder(df, other, **kwargs): + df.update(other, **kwargs) + return df + + return self._inter_df_op_handler(update_builder, other, **kwargs) + + def where(self, cond, other, **kwargs): + + + assert isinstance(cond, type(self)), \ + "Must have the same DataManager subclass to perform this operation" + + if isinstance(other, type(self)): + # Note: Currently we are doing this with two maps across the entire + # data. This can be done with a single map, but it will take a + # modification in the `BlockPartition` class. + # If this were in one pass it would be ~2x faster. + # TODO rewrite this to take one pass. + def where_builder_first_pass(cond, other, **kwargs): + return cond.where(cond, other, **kwargs) + + def where_builder_second_pass(df, new_other, **kwargs): + return df.where(new_other == True, new_other, **kwargs) + + # We are required to perform this reindexing on everything to + # shuffle the data together + reindexed_cond = cond.reindex(0, self.index).data + reindexed_other = other.reindex(0, self.index).data + reindexed_self = self.reindex(0, self.index).data + + first_pass = reindexed_cond.inter_data_operation(1, lambda l, r: where_builder_first_pass(l, r, **kwargs), reindexed_other) + final_pass = reindexed_self.inter_data_operation(1, lambda l, r: where_builder_second_pass(l, r, **kwargs), first_pass) + return self.__constructor__(final_pass, self.index, self.columns) + else: + axis = kwargs.get("axis", 0) + # Rather than serializing and passing in the index/columns, we will + # just change this index to match the internal index. + if isinstance(other, pandas.Series): + other.index = [i for i in range(len(other))] + + def where_builder_series(df, cond, other, **kwargs): + return df.where(cond, other, **kwargs) + + reindexed_self = self.reindex(axis, self.index if not axis else self.columns).data + reindexed_cond = cond.reindex(axis, self.index if not axis else self.columns).data + + new_data = reindexed_self.inter_data_operation(axis, lambda l, r: where_builder_series(l, r, other, **kwargs), reindexed_cond) + return self.__constructor__(new_data, self.index, self.columns) + # END Inter-Data operations + + # Single Manager scalar operations (e.g. add to scalar, list of scalars) + def scalar_operations(self, axis, scalar, func): + if isinstance(scalar, list): + + new_data = self.map_across_full_axis(axis, func) + return self.__constructor__(new_data, self.index, self.columns) + else: + return self.map_partitions(func) + # END Single Manager scalar operations + + # Reindex/reset_index (may shuffle data) + def reindex(self, axis, labels, **kwargs): + # To reindex, we need a function that will be shipped to each of the + # partitions. + def reindex_builer(df, axis, old_labels, new_labels, **kwargs): + if axis: + df.columns = old_labels + new_df = df.reindex(columns=new_labels, **kwargs) + # reset the internal columns back to a RangeIndex + new_df.columns = pandas.RangeIndex(len(new_df.columns)) + return new_df + else: + df.index = old_labels + new_df = df.reindex(index=new_labels, **kwargs) + # reset the internal index back to a RangeIndex + new_df.reset_index(inplace=True, drop=True) + return new_df + + old_labels = self.columns if axis else self.index + + new_index = self.index if axis else labels + new_columns = labels if axis else self.columns + + func = self._prepare_method(lambda df: reindex_builer(df, axis, old_labels, labels, **kwargs)) + + # The reindex can just be mapped over the axis we are modifying. This + # is for simplicity in implementation. We specify num_splits here + # because if we are repartitioning we should (in the future). + # Additionally this operation is often followed by an operation that + # assumes identical partitioning. Internally, we *may* change the + # partitioning during a map across a full axis. + new_data = self.map_across_full_axis(axis, func) + return self.__constructor__(new_data, new_index, new_columns) + + def reset_index(self, **kwargs): + + drop = kwargs.get("drop", False) + new_index = pandas.RangeIndex(len(self.index)) + + if not drop: + new_column_name = "index" if "index" not in self.columns else "level_0" + new_columns = self.columns.insert(0, new_column_name) + result = self.insert(0, new_column_name, self.index) + return self.__constructor__(result.data, new_index, new_columns) + else: + # The copies here are to ensure that we do not give references to + # this object for the purposes of updates. + return self.__constructor__(self.data.copy(), new_index, self.columns.copy(), self._dtype_cache) + + # END Reindex/reset_index + + # Transpose + # For transpose, we aren't going to immediately copy everything. Since the + # actual transpose operation is very fast, we will just do it before any + # operation that gets called on the transposed data. See _prepare_method + # for how the transpose is applied. + # + # Our invariants assume that the blocks are transposed, but not the + # data inside. Sometimes we have to reverse this transposition of blocks + # for simplicity of implementation. + # + # _is_transposed, 0 for False or non-transposed, 1 for True or transposed. + _is_transposed = 0 + + def transpose(self, *args, **kwargs): + + new_data = self.data.transpose(*args, **kwargs) + # Switch the index and columns and transpose the + new_manager = self.__constructor__(new_data, self.columns, self.index) + # It is possible that this is already transposed + new_manager._is_transposed = self._is_transposed ^ 1 + return new_manager + # END Transpose + + # Full Reduce operations + # + # These operations result in a reduced dimensionality of data. + # Currently, this means a Pandas Series will be returned, but in the future + # we will implement a Distributed Series, and this will be returned + # instead. + def full_reduce(self, axis, map_func, reduce_func=None, numeric_only=False): + if numeric_only: + index = self.numeric_indices() + if len(index) == 0: + return pandas.Series(dtype=np.float64) + nonnumeric = [col for col, dtype in zip(self.columns, self.dtypes) if not is_numeric_dtype(dtype)] + if axis: + return self.drop(columns=nonnumeric).full_reduce(axis, map_func) + else: + if not axis: + index = self.columns + else: + index = self.index + + if reduce_func is None: + reduce_func = map_func + + # The XOR here will ensure that we reduce over the correct axis that + # exists on the internal partitions. We flip the axis + result = self.data.full_reduce(map_func, reduce_func, axis ^ self._is_transposed) + result.index = index + return result + + def count(self, **kwargs): + axis = kwargs.get("axis", 0) + numeric_only = kwargs.get("numeric_only", False) + map_func = self._prepare_method(pandas.DataFrame.count, **kwargs) + reduce_func = self._prepare_method(pandas.DataFrame.sum, **kwargs) + return self.full_reduce(axis, map_func, reduce_func, numeric_only) + + def max(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + numeric_only = True if axis else kwargs.get("numeric_only", False) + func = self._prepare_method(pandas.DataFrame.max, **kwargs) + return self.full_reduce(axis, func, numeric_only=numeric_only) + + def mean(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def mean_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.mean(df, **kwargs) + + func = self._prepare_method(mean_builder, **kwargs) + return self.full_reduce(axis, func, numeric_only=True) + + def min(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + numeric_only = True if axis else kwargs.get("numeric_only", False) + func = self._prepare_method(pandas.DataFrame.min, **kwargs) + return self.full_reduce(axis, func, numeric_only=numeric_only) + + def prod(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + index = self.index if axis else self.columns + func = self._prepare_method(pandas.DataFrame.prod, **kwargs) + return self.full_reduce(axis, func, numeric_only=True) + + def sum(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + numeric_only = True if axis else kwargs.get("numeric_only", False) + func = self._prepare_method(pandas.DataFrame.sum, **kwargs) + return self.full_reduce(axis, func, numeric_only=numeric_only) + # END Full Reduce operations + + # Map partitions operations + # These operations are operations that apply a function to every partition. + def map_partitions(self, func, new_dtypes=None): + + return self.__constructor__(self.data.map_across_blocks(func), self.index, self.columns, new_dtypes) + + def abs(self): + func = self._prepare_method(pandas.DataFrame.abs) + new_dtypes = pandas.Series([np.dtype('float64') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def applymap(self, func): + remote_func = self._prepare_method(pandas.DataFrame.applymap, func=func) + return self.map_partitions(remote_func) + + def isin(self, **kwargs): + func = self._prepare_method(pandas.DataFrame.isin, **kwargs) + new_dtypes = pandas.Series([np.dtype('bool') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def isna(self): + func = self._prepare_method(pandas.DataFrame.isna) + new_dtypes = pandas.Series([np.dtype('bool') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def isnull(self): + func = self._prepare_method(pandas.DataFrame.isnull) + new_dtypes = pandas.Series([np.dtype('bool') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def negative(self, **kwargs): + func = self._prepare_method(pandas.DataFrame.__neg__, **kwargs) + return self.map_partitions(func) + + def notna(self): + func = self._prepare_method(pandas.DataFrame.notna) + new_dtypes = pandas.Series([np.dtype('bool') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def notnull(self): + func = self._prepare_method(pandas.DataFrame.notnull) + new_dtypes = pandas.Series([np.dtype('bool') for _ in self.columns], index=self.columns) + return self.map_partitions(func, new_dtypes=new_dtypes) + + def round(self, **kwargs): + func = self._prepare_method(pandas.DataFrame.round, **kwargs) + return self.map_partitions(func, new_dtypes=self._dtype_cache) + # END Map partitions operations + + # Column/Row partitions reduce operations + # + # These operations result in a reduced dimensionality of data. + # Currently, this means a Pandas Series will be returned, but in the future + # we will implement a Distributed Series, and this will be returned + # instead. + def full_axis_reduce(self, func, axis): + result = self.data.map_across_full_axis(axis, func).to_pandas(self._is_transposed) + + if not axis: + result.index = self.columns + else: + result.index = self.index + + return result + + def all(self, **kwargs): + axis = kwargs.get("axis", 0) + func = self._prepare_method(pandas.DataFrame.all, **kwargs) + return self.full_axis_reduce(func, axis) + + def any(self, **kwargs): + axis = kwargs.get("axis", 0) + func = self._prepare_method(pandas.DataFrame.any, **kwargs) + return self.full_axis_reduce(func, axis) + + def first_valid_index(self): + + # It may be possible to incrementally check each partition, but this + # computation is fairly cheap. + def first_valid_index_builder(df): + df.index = pandas.RangeIndex(len(df.index)) + return df.apply(lambda df: df.first_valid_index()) + + func = self._prepare_method(first_valid_index_builder) + # We get the minimum from each column, then take the min of that to get + # first_valid_index. + first_result = self.full_axis_reduce(func, 0) + + return self.index[first_result.min()] + + def _post_process_idx_ops(self, axis, intermediate_result): + index = self.index if not axis else self.columns + result = intermediate_result.apply(lambda x: index[x]) + return result + + def idxmax(self, **kwargs): + + # The reason for the special treatment with idxmax/min is because we + # need to communicate the row number back here. + def idxmax_builder(df, **kwargs): + df.index = pandas.RangeIndex(len(df.index)) + return df.idxmax(**kwargs) + + axis = kwargs.get("axis", 0) + func = self._prepare_method(idxmax_builder, **kwargs) + max_result = self.full_axis_reduce(func, axis) + # Because our internal partitions don't track the external index, we + # have to do a conversion. + return self._post_process_idx_ops(axis, max_result) + + def idxmin(self, **kwargs): + + # The reason for the special treatment with idxmax/min is because we + # need to communicate the row number back here. + def idxmin_builder(df, **kwargs): + df.index = pandas.RangeIndex(len(df.index)) + return df.idxmin(**kwargs) + + axis = kwargs.get("axis", 0) + func = self._prepare_method(idxmin_builder, **kwargs) + min_result = self.full_axis_reduce(func, axis) + # Because our internal partitions don't track the external index, we + # have to do a conversion. + return self._post_process_idx_ops(axis, min_result) + + def last_valid_index(self): + + def last_valid_index_builder(df): + df.index = pandas.RangeIndex(len(df.index)) + return df.apply(lambda df: df.last_valid_index()) + + func = self._prepare_method(last_valid_index_builder) + # We get the maximum from each column, then take the max of that to get + # last_valid_index. + first_result = self.full_axis_reduce(func, 0) + + return self.index[first_result.max()] + + def memory_usage(self, **kwargs): + def memory_usage_builder(df, **kwargs): + return df.memory_usage(index=False, deep=deep) + + deep = kwargs.get('deep', False) + func = self._prepare_method(memory_usage_builder, **kwargs) + return self.full_axis_reduce(func, 0) + + def nunique(self, **kwargs): + axis = kwargs.get("axis", 0) + func = self._prepare_method(pandas.DataFrame.nunique, **kwargs) + return self.full_axis_reduce(func, axis) + + def to_datetime(self, **kwargs): + columns = self.columns + def to_datetime_builder(df, **kwargs): + df.columns = columns + return pandas.to_datetime(df, **kwargs) + func = self._prepare_method(to_datetime_builder, **kwargs) + return self.full_axis_reduce(func, 1) + # END Column/Row partitions reduce operations + + # Column/Row partitions reduce operations over select indices + # + # These operations result in a reduced dimensionality of data. + # Currently, this means a Pandas Series will be returned, but in the future + # we will implement a Distributed Series, and this will be returned + # instead. + def full_axis_reduce_along_select_indices(self, func, axis, index, pandas_result=True): + # Convert indices to numeric indices + old_index = self.index if axis else self.columns + numeric_indices = [i for i, name in enumerate(old_index) if name in index] + result = self.data.apply_func_to_select_indices_along_full_axis(axis, func, numeric_indices) + + if pandas_result: + result = result.to_pandas(self._is_transposed) + if not axis: + result.index = index + else: + result.index = index + + return result + + def describe(self, **kwargs): + + axis = 0 + + new_index = self.numeric_indices() + if len(new_index) != 0: + numeric = True + else: + numeric = False + # If no numeric dtypes, then do all + new_index = self.columns + + def describe_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.describe(df, **kwargs) + + func = self._prepare_method(describe_builder, **kwargs) + new_data = self.full_axis_reduce_along_select_indices(func, 0, new_index, False) + new_index = self.compute_index(0, new_data, False) + new_columns = self.compute_index(1, new_data, True) + if numeric: + new_dtypes = pandas.Series([np.float64 for _ in new_columns], index=new_columns) + else: + new_dtypes = pandas.Series([np.object for _ in new_columns], index=new_columns) + + return self.__constructor__(new_data, new_index, new_columns, new_dtypes) + + def median(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def median_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.median(df, **kwargs) + + func = self._prepare_method(median_builder, **kwargs) + return self.full_axis_reduce_along_select_indices(func, axis, new_index) + + def skew(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def skew_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.skew(df, **kwargs) + + func = self._prepare_method(skew_builder, internal_indices=[], **kwargs) + return self.full_axis_reduce_along_select_indices(func, axis, new_index) + + def std(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def std_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.std(df, **kwargs) + + func = self._prepare_method(std_builder, **kwargs) + return self.full_axis_reduce_along_select_indices(func, axis, new_index) + + def var(self, **kwargs): + # Pandas default is 0 (though not mentioned in docs) + axis = kwargs.get("axis", 0) + + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def var_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.var(df, **kwargs) + + func = self._prepare_method(var_builder, **kwargs) + return self.full_axis_reduce_along_select_indices(func, axis, new_index) + + def quantile_for_single_value(self, **kwargs): + axis = kwargs.get("axis", 0) + q = kwargs.get("q", 0.5) + assert type(q) is float + + new_index = self.numeric_indices() + if len(new_index) == 0: + return pandas.Series(dtype=np.float64) + + def quantile_builder(df, internal_indices=[], **kwargs): + return pandas.DataFrame.quantile(df, **kwargs) + + func = self._prepare_method(quantile_builder, **kwargs) + result = self.full_axis_reduce_along_select_indices(func, axis, new_index) + result.name = q + return result + # END Column/Row partitions reduce operations over select indices + + # Map across rows/columns + # These operations require some global knowledge of the full column/row + # that is being operated on. This means that we have to put all of that + # data in the same place. + def map_across_full_axis(self, axis, func): + return self.data.map_across_full_axis(axis, func) + + def _cumulative_builder(self, func, **kwargs): + axis = kwargs.get("axis", 0) + func = self._prepare_method(func, **kwargs) + new_data = self.map_across_full_axis(axis, func) + return self.__constructor__(new_data, self.index, self.columns, self._dtype_cache) + + def cumsum(self, **kwargs): + return self._cumulative_builder(pandas.DataFrame.cumsum, **kwargs) + + def cummax(self, **kwargs): + return self._cumulative_builder(pandas.DataFrame.cummax, **kwargs) + + def cummin(self, **kwargs): + return self._cumulative_builder(pandas.DataFrame.cummin, **kwargs) + + def cumprod(self, **kwargs): + return self._cumulative_builder(pandas.DataFrame.cumprod, **kwargs) + + def diff(self, **kwargs): + + + axis = kwargs.get("axis", 0) + + func = self._prepare_method(pandas.DataFrame.diff, **kwargs) + new_data = self.map_across_full_axis(axis, func) + + return self.__constructor__(new_data, self.index, self.columns) + + def dropna(self, **kwargs): + axis = kwargs.get("axis", 0) + subset = kwargs.get("subset") + thresh = kwargs.get("thresh") + how = kwargs.get("how", "any") + # We need to subset the axis that we care about with `subset`. This + # will be used to determine the number of values that are NA. + if subset is not None: + if not axis: + compute_na = self.getitem_column_array(subset) + else: + compute_na = self.getitem_row_array(subset) + else: + compute_na = self + + if not isinstance(axis, list): + axis = [axis] + # We are building this dictionary first to determine which columns + # and rows to drop. This way we do not drop some columns before we + # know which rows need to be dropped. + if thresh is not None: + # Count the number of NA values and specify which are higher than + # thresh. + drop_values = {ax ^ 1: compute_na.isna().sum(axis=ax ^ 1) > thresh for ax in axis} + else: + drop_values = {ax ^ 1: getattr(compute_na.isna(), how)(axis=ax ^ 1) for ax in axis} + + if 0 not in drop_values: + drop_values[0] = None + + if 1 not in drop_values: + drop_values[1] = None + + rm_from_index = [obj for obj in compute_na.index[drop_values[1]]] if drop_values[1] is not None else None + rm_from_columns = [obj for obj in compute_na.columns[drop_values[0]]] if drop_values[0] is not None else None + else: + rm_from_index = compute_na.index[drop_values[1]] if drop_values[1] is not None else None + rm_from_columns = compute_na.columns[drop_values[0]] if drop_values[0] is not None else None + + return self.drop(index=rm_from_index, columns=rm_from_columns) + + def eval(self, expr, **kwargs): + + inplace = kwargs.get("inplace", False) + + columns = self.index if self._is_transposed else self.columns + index = self.columns if self._is_transposed else self.index + + # Dun eval on columns to determine result type + columns_copy = pandas.DataFrame(columns=self.columns) + columns_copy = columns_copy.eval(expr, inplace=False, **kwargs) + expect_series = isinstance(columns_copy, pandas.Series) + + # if there is no assignment, then we simply save the results + # in the first column + if expect_series: + if inplace: + raise ValueError("Cannot operate inplace if there is no assignment") + else: + expr = "{0} = {1}".format(columns[0], expr) + + def eval_builder(df, **kwargs): + df.columns = columns + result = df.eval(expr, inplace=False, **kwargs) + result.columns = pandas.RangeIndex(0, len(result.columns)) + return result + + func = self._prepare_method(eval_builder, **kwargs) + new_data = self.map_across_full_axis(1, func) + + if expect_series: + result = new_data.to_pandas()[0] + result.name = columns_copy.name + result.index = index + return result + else: + columns = columns_copy.columns + return self.__constructor__(new_data, self.index, columns) + + def mode(self, **kwargs): + axis = kwargs.get("axis", 0) + func = self._prepare_method(pandas.DataFrame.mode, **kwargs) + new_data = self.map_across_full_axis(axis, func) + + counts = self.__constructor__(new_data, self.index, self.columns).notnull().sum(axis=axis) + max_count = counts.max() + + new_index = pandas.RangeIndex(max_count) if not axis else self.index + new_columns = self.columns if not axis else pandas.RangeIndex(max_count) + + # We have to reindex the DataFrame so that all of the partitions are + # matching in shape. The next steps ensure this happens. + final_labels = new_index if not axis else new_columns + # We build these intermediate objects to avoid depending directly on + # the underlying implementation. + final_data = self.__constructor__(new_data, new_index, new_columns).map_across_full_axis(axis, lambda df: df.reindex(axis=axis, labels=final_labels)) + return self.__constructor__(final_data, new_index, new_columns, self._dtype_cache) + + def fillna(self, **kwargs): + + + axis = kwargs.get("axis", 0) + value = kwargs.get("value") + + if isinstance(value, dict): + value = kwargs.pop("value") + + if axis == 0: + index = self.columns + else: + index = self.index + value = {idx: value[key] for key in value for idx in index.get_indexer_for([key])} + + def fillna_dict_builder(df, func_dict={}): + return df.fillna(value=func_dict, **kwargs) + + new_data = self.data.apply_func_to_select_indices(axis, fillna_dict_builder, value, keep_remaining=True) + return self.__constructor__(new_data, self.index, self.columns) + else: + func = self._prepare_method(pandas.DataFrame.fillna, **kwargs) + new_data = self.map_across_full_axis(axis, func) + return self.__constructor__(new_data, self.index, self.columns) + + def quantile_for_list_of_values(self, **kwargs): + + axis = kwargs.get("axis", 0) + q = kwargs.get("q", 0.5) + assert isinstance(q, (pandas.Series, np.ndarray, pandas.Index, list)) + + index = self.index if axis else self.columns + new_columns = list() + for i, dtype in enumerate(self.dtypes): + if is_numeric_dtype(dtype): + new_columns.append(index[i]) + + func = self._prepare_method(pandas.DataFrame.quantile, **kwargs) + + q_index = pandas.Float64Index(q) + + new_data = self.map_across_full_axis(axis, func) + return self.__constructor__(new_data, q_index, new_columns) + + def query(self, expr, **kwargs): + + columns = self.columns + + def query_builder(df, **kwargs): + # This is required because of an Arrow limitation + # TODO revisit for Arrow error + df = df.copy() + df.index = pandas.RangeIndex(len(df)) + df.columns = columns + df.query(expr, inplace=True, **kwargs) + df.columns = pandas.RangeIndex(len(df.columns)) + return df + + func = self._prepare_method(query_builder, **kwargs) + new_data = self.map_across_full_axis(1, func) + # Query removes rows, so we need to update the index + new_index = self.compute_index(0, new_data, True) + + return self.__constructor__(new_data, new_index, self.columns, self.dtypes) + + def rank(self, **kwargs): + + + axis = kwargs.get("axis", 0) + numeric_only = True if axis else kwargs.get("numeric_only", False) + + func = self._prepare_method(pandas.DataFrame.rank, **kwargs) + new_data = self.map_across_full_axis(axis, func) + + # Since we assume no knowledge of internal state, we get the columns + # from the internal partitions. + if numeric_only: + new_columns = self.compute_index(1, new_data, True) + else: + new_columns = self.columns + new_dtypes = pandas.Series([np.float64 for _ in new_columns], index=new_columns) + return self.__constructor__(new_data, self.index, new_columns, new_dtypes) + # END Map across rows/columns + + # Map across select rows/columns + # These operations require some global knowledge of the full column/row + # that is being operated on. This means that we have to put all of that + # data in the same place. + def astype(self, col_dtypes, errors='raise', **kwargs): + + + # Group the indicies to update together and create new dtypes series + dtype_indices = dict() + columns = col_dtypes.keys() + new_dtypes = self.dtypes.copy() + + numeric_indices = list(self.columns.get_indexer_for(columns)) + + for i, column in enumerate(columns): + dtype = col_dtypes[column] + if dtype != self.dtypes[column]: + if dtype in dtype_indices.keys(): + dtype_indices[dtype].append(numeric_indices[i]) + else: + dtype_indices[dtype] = [numeric_indices[i]] + new_dtype = np.dtype(dtype) + if dtype != np.int32 and new_dtype == np.int32: + new_dtype = np.dtype('int64') + elif dtype != np.float32 and new_dtype == np.float32: + new_dtype = np.dtype('float64') + new_dtypes[column] = new_dtype + + new_data = self.data + for dtype in dtype_indices.keys(): + resulting_dtype = None + + def astype(df, internal_indices=[]): + block_dtypes = dict() + for ind in internal_indices: + block_dtypes[df.columns[ind]] = dtype + return df.astype(block_dtypes) + + new_data = new_data.apply_func_to_select_indices(0, astype, dtype_indices[dtype], keep_remaining=True) + + return self.__constructor__(new_data, self.index, self.columns, new_dtypes) + # END Map across rows/columns + + # Head/Tail/Front/Back + def head(self, n): + + # We grab the front if it is transposed and flag as transposed so that + # we are not physically updating the data from this manager. This + # allows the implementation to stay modular and reduces data copying. + if self._is_transposed: + # Transpose the blocks back to their original orientation first to + # ensure that we extract the correct data on each node. The index + # on a transposed manager is already set to the correct value, so + # we need to only take the head of that instead of re-transposing. + result = self.__constructor__(self.data.transpose().take(1, n).transpose(), self.index[:n], self.columns, self._dtype_cache) + result._is_transposed = True + else: + result = self.__constructor__(self.data.take(0, n), self.index[:n], self.columns, self._dtype_cache) + return result + + def tail(self, n): + + # See head for an explanation of the transposed behavior + if self._is_transposed: + result = self.__constructor__(self.data.transpose().take(1, -n).transpose(), self.index[-n:], self.columns, self._dtype_cache) + result._is_transposed = True + else: + result = self.__constructor__(self.data.take(0, -n), self.index[-n:], self.columns, self._dtype_cache) + + return result + + def front(self, n): + + # See head for an explanation of the transposed behavior + if self._is_transposed: + result = self.__constructor__(self.data.transpose().take(0, n).transpose(), self.index, self.columns[:n], self.dtypes[:n]) + result._is_transposed = True + else: + result = self.__constructor__(self.data.take(1, n), self.index, self.columns[:n], self.dtypes[:n]) + return result + + def back(self, n): + + # See head for an explanation of the transposed behavior + if self._is_transposed: + result = self.__constructor__(self.data.transpose().take(0, -n).transpose(), self.index, self.columns[-n:], self.dtypes[-n:]) + result._is_transposed = True + else: + result = self.__constructor__(self.data.take(1, -n), self.index, self.columns[-n:], self.dtypes[-n:]) + return result + # End Head/Tail/Front/Back + + # Data Management Methods + def free(self): + """In the future, this will hopefully trigger a cleanup of this object. + """ + # TODO create a way to clean up this object. + return + # END Data Management Methods + + # To/From Pandas + def to_pandas(self): + df = self.data.to_pandas(is_transposed=self._is_transposed) + df.index = self.index + df.columns = self.columns + return df + + @classmethod + def from_pandas(cls, df, block_partitions_cls): + new_index = df.index + new_columns = df.columns + new_dtypes = df.dtypes + + new_data = block_partitions_cls.from_pandas(df) + + return cls(new_data, new_index, new_columns, dtypes=new_dtypes) + + # __getitem__ methods + def getitem_single_key(self, key): + numeric_index = self.columns.get_indexer_for([key]) + + new_data = self.getitem_column_array([key]) + if len(numeric_index) > 1: + return new_data + else: + # This is the case that we are returning a single Series. + # We do this post processing because everything is treated a a list + # from here on, and that will result in a DataFrame. + return new_data.to_pandas()[key] + + def getitem_column_array(self, key): + + # Convert to list for type checking + numeric_indices = list(self.columns.get_indexer_for(key)) + + # Internal indices is left blank and the internal + # `apply_func_to_select_indices` will do the conversion and pass it in. + def getitem(df, internal_indices=[]): + return df.iloc[:, internal_indices] + + result = self.data.apply_func_to_select_indices(0, getitem, numeric_indices, keep_remaining=False) + + # We can't just set the columns to key here because there may be + # multiple instances of a key. + new_columns = self.columns[numeric_indices] + new_dtypes = self.dtypes[numeric_indices] + return self.__constructor__(result, self.index, new_columns, new_dtypes) + + def getitem_row_array(self, key): + # Convert to list for type checking + numeric_indices = list(self.index.get_indexer_for(key)) + + def getitem(df, internal_indices=[]): + return df.iloc[internal_indices] + + result = self.data.apply_func_to_select_indices(1, getitem, numeric_indices, keep_remaining=False) + # We can't just set the index to key here because there may be multiple + # instances of a key. + new_index = self.index[numeric_indices] + return self.__constructor__(result, new_index, self.columns, self._dtype_cache) + + # END __getitem__ methods + + # __delitem__ and drop + # These will change the shape of the resulting data. + def delitem(self, key): + return self.drop(columns=[key]) + + def drop(self, index=None, columns=None): + + + if index is None: + new_data = self.data + new_index = self.index + else: + def delitem(df, internal_indices=[]): + return df.drop(index=df.index[internal_indices]) + + numeric_indices = list(self.index.get_indexer_for(index)) + new_data = self.data.apply_func_to_select_indices(1, delitem, numeric_indices, keep_remaining=True) + # We can't use self.index.drop with duplicate keys because in Pandas + # it throws an error. + new_index = [self.index[i] for i in range(len(self.index)) if i not in numeric_indices] + + if columns is None: + new_columns = self.columns + new_dtypes = self.dtypes + else: + def delitem(df, internal_indices=[]): + return df.drop(columns=df.columns[internal_indices]) + + numeric_indices = list(self.columns.get_indexer_for(columns)) + new_data = new_data.apply_func_to_select_indices(0, delitem, numeric_indices, keep_remaining=True) + # We can't use self.columns.drop with duplicate keys because in Pandas + # it throws an error. + new_columns = [self.columns[i] for i in range(len(self.columns)) if i not in numeric_indices] + new_dtypes = self.dtypes.drop(columns) + return self.__constructor__(new_data, new_index, new_columns, new_dtypes) + # END __delitem__ and drop + + # Insert + # This method changes the shape of the resulting data. In Pandas, this + # operation is always inplace, but this object is immutable, so we just + # return a new one from here and let the front end handle the inplace + # update. + def insert(self, loc, column, value): + + + def insert(df, internal_indices=[]): + internal_idx = internal_indices[0] + df.insert(internal_idx, internal_idx, value, allow_duplicates=True) + return df + + new_data = self.data.apply_func_to_select_indices_along_full_axis(0, insert, loc, keep_remaining=True) + new_columns = self.columns.insert(loc, column) + + # Because a Pandas Series does not allow insert, we make a DataFrame + # and insert the new dtype that way. + temp_dtypes = pandas.DataFrame(self.dtypes).T + temp_dtypes.insert(loc, column, _get_dtype_from_object(value)) + new_dtypes = temp_dtypes.iloc[0] + + return self.__constructor__(new_data, self.index, new_columns, new_dtypes) + # END Insert + + # UDF (apply and agg) methods + # There is a wide range of behaviors that are supported, so a lot of the + # logic can get a bit convoluted. + def apply(self, func, axis, *args, **kwargs): + if callable(func): + return self._callable_func(func, axis, *args, **kwargs) + elif isinstance(func, dict): + return self._dict_func(func, axis, *args, **kwargs) + elif is_list_like(func): + return self._list_like_func(func, axis, *args, **kwargs) + else: + pass + + def _post_process_apply(self, result_data, axis, try_scale=True): + if try_scale: + try: + index = self.compute_index(0, result_data, True) + except IndexError: + index = self.compute_index(0, result_data, False) + try: + columns = self.compute_index(1, result_data, True) + except IndexError: + columns = self.compute_index(1, result_data, False) + else: + if not axis: + index = self.compute_index(0, result_data, False) + columns = self.columns + else: + index = self.index + columns = self.compute_index(1, result_data, False) + # `apply` and `aggregate` can return a Series or a DataFrame object, + # and since we need to handle each of those differently, we have to add + # this logic here. + if len(columns) == 0: + series_result = result_data.to_pandas(False) + if not axis and len(series_result) == len(self.columns) and len(index) != len(series_result): + index = self.columns + elif axis and len(series_result) == len(self.index) and len(index) != len(series_result): + index = self.index + + series_result.index = index + return series_result + + return self.__constructor__(result_data, index, columns) + + def _dict_func(self, func, axis, *args, **kwargs): + if "axis" not in kwargs: + kwargs["axis"] = axis + + if axis == 0: + index = self.columns + else: + index = self.index + + func = {idx: func[key] for key in func for idx in index.get_indexer_for([key])} + + def dict_apply_builder(df, func_dict={}): + return df.apply(func_dict, *args, **kwargs) + + result_data = self.data.apply_func_to_select_indices_along_full_axis(axis, dict_apply_builder, func, keep_remaining=False) + + full_result = self._post_process_apply(result_data, axis) + + # The columns can get weird because we did not broadcast them to the + # partitions and we do not have any guarantee that they are correct + # until here. Fortunately, the keys of the function will tell us what + # the columns are. + if isinstance(full_result, pandas.Series): + full_result.index = [self.columns[idx] for idx in func] + return full_result + + def _list_like_func(self, func, axis, *args, **kwargs): + + func_prepared = self._prepare_method(lambda df: df.apply(func, *args, **kwargs)) + new_data = self.map_across_full_axis(axis, func_prepared) + + # When the function is list-like, the function names become the index + new_index = [f if isinstance(f, string_types) else f.__name__ for f in func] + return self.__constructor__(new_data, new_index, self.columns) + + def _callable_func(self, func, axis, *args, **kwargs): + + def callable_apply_builder(df, func, axis, index, *args, **kwargs): + if not axis: + df.index = index + df.columns = pandas.RangeIndex(len(df.columns)) + else: + df.columns = index + df.index = pandas.RangeIndex(len(df.index)) + + result = df.apply(func, axis=axis, *args, **kwargs) + return result + + index = self.index if not axis else self.columns + + func_prepared = self._prepare_method(lambda df: callable_apply_builder(df, func, axis, index, *args, **kwargs)) + result_data = self.map_across_full_axis(axis, func_prepared) + return self._post_process_apply(result_data, axis) + # END UDF + + # Manual Partitioning methods (e.g. merge, groupby) + # These methods require some sort of manual partitioning due to their + # nature. They require certain data to exist on the same partition, and + # after the shuffle, there should be only a local map required. + def _manual_repartition(self, axis, repartition_func, **kwargs): + """This method applies all manual partitioning functions. + + :param axis: + :param repartition_func: + + Returns: + A `BlockPartitions` object. + """ + func = self._prepare_method(repartition_func, **kwargs) + return self.data.manual_shuffle(axis, func) + + def groupby_agg(self, by, axis, agg_func, groupby_args={}, agg_args={}): + remote_index = self.index if not axis else self.columns + + def groupby_agg_builder(df): + if not axis: + df.index = remote_index + else: + df.columns = remote_index + return agg_func(df.groupby(by=by, axis=axis, **groupby_args), **agg_args) + func_prepared = self._prepare_method(lambda df: groupby_agg_builder(df)) + result_data = self.map_across_full_axis(axis, func_prepared) + return self._post_process_apply(result_data, axis, try_scale=False) + # END Manual Partitioning methods + + def get_dummies(self, columns, **kwargs): + cls = type(self) + + # `columns` as None does not mean all columns, by default it means only + # non-numeric columns. + if columns is None: + columns = [c for c in self.columns if not is_numeric_dtype(self.dtypes[c])] + + # If we aren't computing any dummies, there is no need for any + # remote compute. + if len(columns) == 0: + return self.copy() + elif not is_list_like(columns): + columns = [columns] + + # We have to do one of two things in order to ensure the final columns + # are correct. Our first option is to map over the data and assign the + # columns in a separate pass. That is what we have chosen to do here. + # This is not as efficient, but it requires less information from the + # lower layers and does not break any of our internal requirements. The + # second option is that we assign the columns as a part of the + # `get_dummies` call. This requires knowledge of the length of each + # partition, and breaks some of our assumptions and separation of + # concerns. + def set_columns(df, columns): + df.columns = columns + return df + + set_cols = self.columns + columns_applied = self.map_across_full_axis(1, lambda df: set_columns(df, set_cols)) + + # In some cases, we are mapping across all of the data. It is more + # efficient if we are mapping over all of the data to do it this way + # than it would be to reuse the code for specific columns. + if len(columns) == len(self.columns): + def get_dummies_builder(df): + if df is not None: + if not df.empty: + return pandas.get_dummies(df, **kwargs) + else: + return pandas.DataFrame([]) + + func = self._prepare_method(lambda df: get_dummies_builder(df)) + new_data = columns_applied.map_across_full_axis(0, func) + untouched_data = None + else: + def get_dummies_builder(df, internal_indices=[]): + return pandas.get_dummies(df.iloc[:, internal_indices], columns=None, **kwargs) + + numeric_indices = list(self.columns.get_indexer_for(columns)) + new_data = columns_applied.apply_func_to_select_indices_along_full_axis(0, get_dummies_builder, numeric_indices, keep_remaining=False) + untouched_data = self.drop(columns=columns) + + # Since we set the columns in the beginning, we can just extract them + # here. There is fortunately no required extra steps for a correct + # column index. + final_columns = self.compute_index(1, new_data, False) + + # If we mapped over all the data we are done. If not, we need to + # prepend the `new_data` with the raw data from the columns that were + # not selected. + if len(columns) != len(self.columns): + new_data = untouched_data.data.concat(1, new_data) + final_columns = untouched_data.columns.append(pandas.Index(final_columns)) + + return cls(new_data, self.index, final_columns) + + # Indexing + def view(self, index=None, columns=None): + index_map_series = pandas.Series(np.arange(len(self.index)), index=self.index) + column_map_series = pandas.Series(np.arange(len(self.columns)), index=self.columns) + + if index is not None: + index_map_series = index_map_series.reindex(index) + if columns is not None: + column_map_series = column_map_series.reindex(columns) + + return PandasDataManagerView(self.data, index_map_series.index, column_map_series.index, self.dtypes, + index_map_series, column_map_series) + + def squeeze(self, ndim=0, axis=None): + squeezed = self.data.to_pandas().squeeze() + + if ndim == 1: + squeezed = pandas.Series(squeezed) + scaler_axis = self.index if axis == 0 else self.columns + non_scaler_axis = self.index if axis == 1 else self.columns + + squeezed.name = scaler_axis[0] + squeezed.index = non_scaler_axis + + return squeezed + + def write_items(self, row_numeric_index, col_numeric_index, broadcasted_items): + + def iloc_mut(partition, row_internal_indices, col_internal_indices, item): + partition = partition.copy() + partition.iloc[row_internal_indices, col_internal_indices] = item + return partition + + mutated_blk_partitions = self.data.apply_func_to_indices_both_axis( + func=iloc_mut, + row_indices=row_numeric_index, + col_indices=col_numeric_index, + mutate=True, + item_to_distribute=broadcasted_items + ) + self.data = mutated_blk_partitions + + def global_idx_to_numeric_idx(self, axis, indices): + """ + Note: this function involves making copies of the index in memory. + + :param axis: + :param indices: + :return: + """ + assert axis in ['row', 'col', 'columns'] + if axis == 'row': + return pandas.Index(pandas.Series(np.arange(len(self.index)), index=self.index).loc[indices].values) + elif axis in ['col', 'columns']: + return pandas.Index(pandas.Series(np.arange(len(self.columns)), index=self.columns).loc[indices].values) + + def enlarge_partitions(self, new_row_labels=None, new_col_labels=None): + new_data = self.data.enlarge_partitions(len(new_row_labels), len(new_col_labels)) + concated_index = self.index.append(type(self.index)(new_row_labels)) if new_row_labels else self.index + concated_columns = self.columns.append(type(self.columns)(new_col_labels)) if new_col_labels else self.columns + return self.__constructor__(new_data, concated_index, concated_columns) + + +class PandasDataManagerView(PandasDataManager): + """ + This class represent a view of the PandasDataManager + + In particular, the following constraints are broken: + - (len(self.index), len(self.columns)) != self.data.shape + """ + + def __init__(self, + block_partitions_object: BlockPartitions, + index: pandas.Index, + columns: pandas.Index, + dtypes=None, + index_map_series: pandas.Series=None, + columns_map_series: pandas.Series=None): + """ + :param index_map_series: a Pandas Series Object mapping user-facing index to numeric index. + :param columns_map_series: a Pandas Series Object mapping user-facing index to numeric index. + """ + assert index_map_series is not None + assert columns_map_series is not None + assert index.equals(index_map_series.index) + assert columns.equals(columns_map_series.index) + + self.index_map = index_map_series + self.columns_map = columns_map_series + self.is_view = True + + PandasDataManager.__init__(self, block_partitions_object, index, columns, dtypes) + + def __constructor__(self, block_partitions_object: BlockPartitions, index: pandas.Index, + columns: pandas.Index, dtypes=None): + new_index_map = self.index_map.reindex(index) + new_columns_map = self.columns_map.reindex(columns) + + return type(self)(block_partitions_object, index, columns, dtypes, new_index_map, new_columns_map) + + def _get_data(self) -> BlockPartitions: + """ + Perform the map step + :return: + """ + def iloc(partition, row_internal_indices, col_internal_indices): + return partition.iloc[row_internal_indices, col_internal_indices] + + masked_data = self.parent_data.apply_func_to_indices_both_axis(func=iloc, + row_indices=self.index_map.values, + col_indices=self.columns_map.values, + lazy=True, + keep_remaining=False) + return masked_data + + def _set_data(self, new_data): + """Note this setter will be called by the `super(PandasDataManagerView).__init__` function""" + self.parent_data = new_data + + data = property(_get_data, _set_data) + + def global_idx_to_numeric_idx(self, axis, indices): + assert axis in ['row', 'col', 'columns'] + if axis == 'row': + return self.index_map.loc[indices].index + elif axis in ['col', 'columns']: + return self.columns_map.loc[indices].index diff --git a/modin/data_management/factories.py b/modin/data_management/factories.py new file mode 100644 index 00000000000..6030d080417 --- /dev/null +++ b/modin/data_management/factories.py @@ -0,0 +1,38 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import sys + +from .. import __execution_engine__ as execution_engine +from .. import __partition_format__ as partition_format +from .data_manager import PandasDataManager +from .partitioning.partition_collections import RayBlockPartitions + + +class BaseFactory(object): + + @classmethod + def _determine_engine(cls): + factory_name = \ + partition_format + "Backed" + execution_engine + "Factory" + + return getattr(sys.modules[__name__], factory_name) + + @classmethod + def build_manager(cls): + return cls._determine_engine().build_manager() + + @classmethod + def from_pandas(cls, df): + return cls._determine_engine()._from_pandas(df) + + @classmethod + def _from_pandas(cls, df): + return cls.data_mgr_cls.from_pandas(df, cls.block_partitions_cls) + + +class PandasBackedRayFactory(BaseFactory): + + data_mgr_cls = PandasDataManager + block_partitions_cls = RayBlockPartitions diff --git a/modin/data_management/partitioning/README.md b/modin/data_management/partitioning/README.md new file mode 100644 index 00000000000..38d8523b54f --- /dev/null +++ b/modin/data_management/partitioning/README.md @@ -0,0 +1,13 @@ +## Implementation Note + +### Object Hierarchy + +- `remote_partition.py` contains `RemotePartition` interface and its implementations. +- `partition_collections.py` contains `BlockPartitions` interface and its implementations. + - `BlockPartitions` manages 2D-array of `RemotePartition` object +- `axis_partition.py` contains `AxisPartition` and with the following hierarchy: + ``` + AxisPartition -> RayAxisPartition -> {RayColumnPartition, RayRowPartition} + ``` + - `AxisPartition` is a high level view onto BlockPartitions' data. It is more + convient to operate on `AxisPartition` sometimes. \ No newline at end of file diff --git a/modin/data_management/partitioning/__init__.py b/modin/data_management/partitioning/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/modin/data_management/partitioning/axis_partition.py b/modin/data_management/partitioning/axis_partition.py new file mode 100644 index 00000000000..fd8f76dd6d5 --- /dev/null +++ b/modin/data_management/partitioning/axis_partition.py @@ -0,0 +1,211 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import pandas +import ray + +from .remote_partition import RayRemotePartition +from .utils import compute_chunksize + + +class AxisPartition(object): + """This abstract class represents the Parent class for any + `ColumnPartition` or `RowPartition` class. This class is intended to + simplify the way that operations are performed + + Note 0: The procedures that use this class and its methods assume that + they have some global knowledge about the entire axis. This may + require the implementation to use concatenation or append on the + list of block partitions in this object. + + Note 1: The `BlockPartitions` object that controls these objects + (through the API exposed here) has an invariant that requires that + this object is never returned from a function. It assumes that + there will always be `RemotePartition` object stored and structures + itself accordingly. + + The only abstract method needed to implement is the `apply` method. + """ + def apply(self, func, num_splits=None, other_axis_partition=None, **kwargs): + """Applies a function to a full axis. + + Note: The procedures that invoke this method assume full axis + knowledge. Implement this method accordingly. + + Important: You must return a list of `RemotePartition` objects from + this method. See Note 1 for this class above for more information. + + Args: + func: The function to apply. This will be preprocessed according to + the corresponding `RemotePartitions` object. + num_splits: The number of objects to return, the number of splits + for the resulting object. It is up to this method to choose the + splitting at this time. + other_axis_partition: Another `AxisPartition` object to be applied + to func. This is for operations that are between datasets. + + Returns: + A list of `RemotePartition` objects. + """ + raise NotImplementedError("Must be implemented in children classes") + + def shuffle(self, func, num_splits=None, **kwargs): + """Shuffle the order of the data in this axis based on the `func`. + + Args: + func: + num_splits: + kwargs: + + Returns: + A list of `RemotePartition` objects. + """ + raise NotImplementedError("Must be implemented in children classes") + + +class RayAxisPartition(AxisPartition): + + def __init__(self, list_of_blocks): + # Unwrap from RemotePartition object for ease of use + self.list_of_blocks = [obj.oid for obj in list_of_blocks] + + def apply(self, func, num_splits=None, other_axis_partition=None, **kwargs): + """Applies func to the object in the plasma store. + + See notes in Parent class about this method. + + Args: + func: The function to apply. + num_splits: The number of times to split the result object. + other_axis_partition: Another `RayAxisPartition` object to apply to + func with this one. + + Returns: + A list of `RayRemotePartition` objects. + """ + if num_splits is None: + num_splits = len(self.list_of_blocks) + + if other_axis_partition is not None: + return [RayRemotePartition(obj) for obj in deploy_ray_func_between_two_axis_partitions._submit(args=(self.axis, func, num_splits, len(self.list_of_blocks), kwargs) + tuple(self.list_of_blocks + other_axis_partition.list_of_blocks), num_return_vals=num_splits)] + + return [RayRemotePartition(obj) for obj in deploy_ray_axis_func._submit(args=(self.axis, func, num_splits, kwargs, *self.list_of_blocks), num_return_vals=num_splits)] + + def shuffle(self, func, num_splits=None, **kwargs): + """Shuffle the order of the data in this axis based on the `func`. + + Extends `AxisPartition.shuffle`. + + :param func: + :param num_splits: + :param kwargs: + :return: + """ + if num_splits is None: + num_splits = len(self.list_of_blocks) + + return [RayRemotePartition(obj) for obj in deploy_ray_shuffle_func._submit(args=(self.axis, func, num_splits, kwargs, *self.list_of_blocks), num_return_vals=num_splits)] + + +class RayColumnPartition(RayAxisPartition): + """The column partition implementation for Ray. All of the implementation + for this class is in the parent class, and this class defines the axis + to perform the computation over. + """ + axis = 0 + + +class RayRowPartition(RayAxisPartition): + """The row partition implementation for Ray. All of the implementation + for this class is in the parent class, and this class defines the axis + to perform the computation over. + """ + axis = 1 + + +def split_result_of_axis_func_pandas(axis, num_splits, result): + """Split the Pandas result evenly based on the provided number of splits. + + Args: + axis: The axis to split across. + num_splits: The number of even splits to create. + result: The result of the computation. This should be a Pandas + DataFrame. + + Returns: + A list of Pandas DataFrames. + """ + # We do this to restore block partitioning + if axis == 0 or type(result) is pandas.Series: + chunksize = compute_chunksize(len(result), num_splits) + return [result.iloc[chunksize * i: chunksize * (i + 1)] for i in range(num_splits)] + else: + chunksize = compute_chunksize(len(result.columns), num_splits) + return [result.iloc[:, chunksize * i: chunksize * (i + 1)] for i in range(num_splits)] + + +@ray.remote +def deploy_ray_axis_func(axis, func, num_splits, kwargs, *partitions): + """Deploy a function along a full axis in Ray. + + Args: + axis: The axis to perform the function along. + func: The function to perform. + num_splits: The number of splits to return + (see `split_result_of_axis_func_pandas`) + kwargs: A dictionary of keyword arguments. + partitions: All partitions that make up the full axis (row or column) + + Returns: + A list of Pandas DataFrames. + """ + dataframe = pandas.concat(partitions, axis=axis, copy=False) + result = func(dataframe, **kwargs) + return split_result_of_axis_func_pandas(axis, num_splits, result) + + +@ray.remote +def deploy_ray_func_between_two_axis_partitions(axis, func, num_splits, len_of_left, kwargs, *partitions): + """Deploy a function along a full axis between two data sets in Ray. + + Args: + axis: The axis to perform the function along. + func: The function to perform. + num_splits: The number of splits to return + (see `split_result_of_axis_func_pandas`). + len_of_left: The number of values in `partitions` that belong to the + left data set. + kwargs: A dictionary of keyword arguments. + partitions: All partitions that make up the full axis (row or column) + for both data sets. + + Returns: + A list of Pandas DataFrames. + """ + lt_frame = pandas.concat(list(partitions[:len_of_left]), axis=axis, copy=False) + rt_frame = pandas.concat(list(partitions[len_of_left:]), axis=axis, copy=False) + + result = func(lt_frame, rt_frame, **kwargs) + return split_result_of_axis_func_pandas(axis, num_splits, result) + + +@ray.remote +def deploy_ray_shuffle_func(axis, func, numsplits, kwargs, *partitions): + """Deploy a function that defines the partitions along this axis. + + Args: + axis: + func: + numsplits: + kwargs: + partitions: + + Returns: + A list of Pandas DataFrames. + """ + dataframe = pandas.concat(partitions, axis=axis, copy=False) + result = func(dataframe, numsplits=numsplits, **kwargs) + + assert isinstance(result, list) + return result diff --git a/modin/data_management/partitioning/partition_collections.py b/modin/data_management/partitioning/partition_collections.py new file mode 100644 index 00000000000..e79e142b1b8 --- /dev/null +++ b/modin/data_management/partitioning/partition_collections.py @@ -0,0 +1,836 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +from typing import Tuple + +import numpy as np +import ray +import pandas + +from .remote_partition import RayRemotePartition +from .axis_partition import RayColumnPartition, RayRowPartition +from .utils import compute_chunksize, _get_nan_block_id + + +class BlockPartitions(object): + """Abstract Class that manages a set of `RemotePartition` objects, and + structures them into a 2D numpy array. This object will interact with + each of these objects through the `RemotePartition` API. + + Note: See the Abstract Methods and Fields section immediately below this + for a list of requirements for subclassing this object. + """ + + # Abstract Methods and Fields: Must implement in children classes + # In some cases, there you may be able to use the same implementation for + # some of these abstract methods, but for the sake of generality they are + # treated differently. + def __init__(self, partitions): + """Init must accept a parameter `partitions` that is a 2D numpy array + of type `_partition_class` (defined below). This method will be + called from a factory. + + Args: + partitions: A 2D numpy array of the type defined in + `_partition_class`. + """ + raise NotImplementedError("Must be implemented in children classes") + + # Partition class is the class to use for storing each partition. It must + # extend the `RemotePartition` class. + _partition_class = None + + def preprocess_func(self, map_func): + """Preprocess a function to be applied to `RemotePartition` objects. + + Note: If your `RemotePartition` objects assume that a function provided + is serialized or wrapped or in some other format, this is the place + to add that logic. It is possible that this can also just return + `map_func` if the `apply` method of the `RemotePartition` object + you are using does not require any modification to a given + function. + + Args: + map_func: The function to be preprocessed. + + Returns + The preprocessed version of the `map_func` provided. Note: This + does not require any specific format, only that the + `RemotePartition.apply` method will recognize it (For the subclass + being used). + """ + return self._partition_class.preprocess_func(map_func) + + @property + def column_partitions(self): + """A list of `AxisPartition` objects, represents column partitions. + + Note: Each value in this list will an `AxisPartition` object. + `AxisPartition` is located in the `remote_partition.py` file. + + Returns a list of `AxisPartition` objects. + """ + raise NotImplementedError("Must be implemented in children classes") + + @property + def row_partitions(self): + """A list of `AxisPartition` objects. + + Note: Each value in this list will be an `AxisPartition` object. + `AxisPartition` is located in the `remote_partition.py` file. + + Returns a list of `AxisPartition` objects. + """ + raise NotImplementedError("Must be implemented in children classes") + + # END Abstract Methods + + # Lengths of the blocks + _lengths_cache = None + + # These are set up as properties so that we only use them when we need + # them. We also do not want to trigger this computation on object creation. + @property + def block_lengths(self): + """Gets the lengths of the blocks. + + Note: This works with the property structure `_lengths_cache` to avoid + having to recompute these values each time they are needed. + """ + if self._lengths_cache is None: + # The first column will have the correct lengths. We have an + # invariant that requires that all blocks be the same length in a + # row of blocks. + self._lengths_cache = [obj.length().get() for obj in self.partitions.T[0]] + return self._lengths_cache + + # Widths of the blocks + _widths_cache = None + + @property + def block_widths(self): + """Gets the widths of the blocks. + + Note: This works with the property structure `_widths_cache` to avoid + having to recompute these values each time they are needed. + """ + if self._widths_cache is None: + # The first column will have the correct lengths. We have an + # invariant that requires that all blocks be the same width in a + # column of blocks. + self._widths_cache = [obj.width().get() for obj in self.partitions[0]] + return self._widths_cache + + @property + def shape(self) -> Tuple[int, int]: + return int(np.sum(self.block_lengths)), int(np.sum(self.block_widths)) + + def full_reduce(self, map_func, reduce_func, axis): + """Perform a full reduce on the data. + + Note: This follows the 2-phase reduce paradigm, where each partition + performs a local reduction (map_func), then partitions are brought + together and the final reduction occurs. + Args: + map_func: The function that will be performed on all partitions. + This is the local reduction on each partition. + reduce_func: The final reduction function. This can differ from the + `map_func` + axis: The axis to perform this operation along + (0 - index, 1 - columns) + Returns: + A Pandas Series + """ + mapped_parts = self.map_across_blocks(map_func).partitions + + if reduce_func is None: + reduce_func = map_func + # For now we return a pandas.Series until ours gets implemented. + # We have to build the intermediate frame based on the axis passed, + # thus axis=axis and axis=axis ^ 1 + # + # This currently requires special treatment because of the intermediate + # DataFrame. The individual partitions return Series objects, and those + # cannot be concatenated the correct way without casting them as + # DataFrames. + full_frame = pandas.concat([pandas.concat([pandas.DataFrame(part.get()).T for part in row_of_parts], axis=axis ^ 1) for row_of_parts in mapped_parts], axis=axis) + + # Transpose because operations where axis == 1 assume that the + # operation is performed across the other axis + if axis == 1: + full_frame = full_frame.T + + return reduce_func(full_frame) + + def map_across_blocks(self, map_func): + """Applies `map_func` to every partition. + + Args: + map_func: The function to apply. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + # For the subclasses, because we never return this abstract type + cls = type(self) + + preprocessed_map_func = self.preprocess_func(map_func) + new_partitions = np.array([[part.apply(preprocessed_map_func) for part in row_of_parts] for row_of_parts in self.partitions]) + return cls(new_partitions) + + def lazy_map_across_blocks(self, map_func, kwargs): + cls = type(self) + preprocessed_map_func = self.preprocess_func(map_func) + new_partitions = np.array( + [[part.add_to_apply_calls(preprocessed_map_func, kwargs) for part in row_of_parts] + for row_of_parts in self.partitions]) + return cls(new_partitions) + + + def map_across_full_axis(self, axis, map_func): + """Applies `map_func` to every partition. + + Note: This method should be used in the case that `map_func` relies on + some global information about the axis. + + Args: + axis: The axis to perform the map across (0 - index, 1 - columns). + map_func: The function to apply. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + # Since we are already splitting the DataFrame back up after an + # operation, we will just use this time to compute the number of + # partitions as best we can right now. + num_splits = cls._compute_num_partitions() + + preprocessed_map_func = self.preprocess_func(map_func) + partitions = self.column_partitions if not axis else self.row_partitions + result_blocks = np.array([part.apply(preprocessed_map_func, num_splits) for part in partitions]) + # If we are mapping over columns, they are returned to use the same as + # rows, so we need to transpose the returned 2D numpy array to return + # the structure to the correct order. + return cls(result_blocks.T) if not axis else cls(result_blocks) + + def take(self, axis, n): + """Take the first (or last) n rows or columns from the blocks + + Note: Axis = 0 will be equivalent to `head` or `tail` + Axis = 1 will be equivalent to `front` or `back` + + Args: + axis: The axis to extract (0 for extracting rows, 1 for extracting columns) + n: The number of rows or columns to extract, negative denotes to extract from the bottom of the object + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + + # These are the partitions that we will extract over + if not axis: + partitions = self.partitions + bin_lengths = self.block_lengths + else: + partitions = self.partitions.T + bin_lengths = self.block_widths + + if n < 0: + reversed_bins = bin_lengths + reversed_bins.reverse() + length_bins = np.cumsum(reversed_bins) + n *= -1 + idx = int(np.digitize(n, length_bins)) + if idx > 0: + remaining = int(n - length_bins[idx - 1]) + else: + remaining = n + # In this case, we require no remote compute. This is much faster. + if remaining == 0: + result = partitions[-idx:] + else: + # Reverse for ease of iteration and then re-reverse at the end + partitions = partitions[::-1] + # We build this iloc to avoid creating a bunch of helper methods. + # This code creates slice objects to be passed to `iloc` to grab + # the last n rows or columns depending on axis. + slice_obj = slice(-remaining, None) if axis == 0 else (slice(None), slice(-remaining, None)) + func = self.preprocess_func(lambda df: df.iloc[slice_obj]) + # We use idx + 1 here because the loop is not inclusive, and we + # need to iterate through idx. + result = np.array([partitions[i] if i != idx else [obj.apply(func) for obj in partitions[i]] for i in range(idx + 1)])[::-1] + else: + length_bins = np.cumsum(bin_lengths) + idx = int(np.digitize(n, length_bins)) + if idx > 0: + remaining = int(n - length_bins[idx - 1]) + else: + remaining = n + # In this case, we require no remote compute. This is much faster. + if remaining == 0: + result = partitions[:idx] + else: + # We build this iloc to avoid creating a bunch of helper methods. + # This code creates slice objects to be passed to `iloc` to grab + # the first n rows or columns depending on axis. + slice_obj = slice(remaining) if axis == 0 else (slice(None), slice(remaining)) + func = self.preprocess_func(lambda df: df.iloc[slice_obj]) + # See note above about idx + 1 + result = np.array([partitions[i] if i != idx else [obj.apply(func) for obj in partitions[i]] for i in range(idx + 1)]) + + return cls(result.T) if axis else cls(result) + + def concat(self, axis, other_blocks): + """Concatenate the blocks with another set of blocks. + + Note: Assumes that the blocks are already the same shape on the + dimension being concatenated. A ValueError will be thrown if this + condition is not met. + + Args: + axis: The axis to concatenate to. + other_blocks: the other blocks to be concatenated. This is a + BlockPartitions object. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + if type(other_blocks) is list: + other_blocks = [blocks.partitions for blocks in other_blocks] + return cls(np.concatenate([self.partitions] + other_blocks, axis=axis)) + else: + return cls(np.append(self.partitions, other_blocks.partitions, axis=axis)) + + def copy(self): + """Create a copy of this object. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + return cls(self.partitions.copy()) + + def transpose(self, *args, **kwargs): + """Transpose the blocks stored in this object. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + return cls(self.partitions.T) + + def to_pandas(self, is_transposed=False): + """Convert this object into a Pandas DataFrame from the partitions. + + Args: + is_transposed: A flag for telling this object that the external + representation is transposed, but not the internal. + + Returns: + A Pandas DataFrame + """ + # In the case this is transposed, it is easier to just temporarily + # transpose back then transpose after the conversion. The performance + # is the same as if we individually transposed the blocks and + # concatenated them, but the code is much smaller. + if is_transposed: + return self.transpose().to_pandas(False).T + else: + retrieved_objects = [[obj.to_pandas() for obj in part] for part in self.partitions] + if all(isinstance(part, pandas.Series) for row in retrieved_objects for part in row): + axis = 0 + retrieved_objects = np.array(retrieved_objects).T + elif all(isinstance(part, pandas.DataFrame) for row in retrieved_objects for part in row): + axis = 1 + else: + raise ValueError("Some partitions contain Series and some contain DataFrames") + + df_rows = [pandas.concat([part for part in row], axis=axis) for row in retrieved_objects] + + if len(df_rows) == 0: + return pandas.DataFrame() + else: + return pandas.concat(df_rows) + + @classmethod + def from_pandas(cls, df): + num_splits = cls._compute_num_partitions() + put_func = cls._partition_class.put + + row_chunksize = max(1, compute_chunksize(len(df), num_splits)) + col_chunksize = max(1, compute_chunksize(len(df.columns), num_splits)) + + # Each chunk must have a RangeIndex that spans its length and width + # according to our invariant. + def chunk_builder(i, j): + chunk = df.iloc[i: i + row_chunksize, j: j + col_chunksize] + chunk.index = pandas.RangeIndex(len(chunk.index)) + chunk.columns = pandas.RangeIndex(len(chunk.columns)) + return put_func(chunk) + + parts = [[chunk_builder(i, j) for j in range(0, len(df.columns), col_chunksize)] for i in range(0, len(df), row_chunksize)] + + return cls(np.array(parts)) + + def get_indices(self, axis=0, index_func=None, old_blocks=None): + """This gets the internal indices stored in the partitions. + + Note: These are the global indices of the object. This is mostly useful + when you have deleted rows/columns internally, but do not know + which ones were deleted. + + Args: + axis: This axis to extract the labels. (0 - index, 1 - columns). + index_func: The function to be used to extract the function. + scale_index: True if we need to add the lengths of previous blocks, + False otherwise. This should be True if each block had the + index reset based on the length of that particular block, + otherwise it should be False. + old_blocks: An optional previous object that this object was + created from. This is used to compute the correct offsets. + + Returns: + A Pandas Index object. + """ + assert callable(index_func), \ + "Must tell this function how to extract index" + + if axis == 0: + func = self.preprocess_func(index_func) + # We grab the first column of blocks and extract the indices + new_indices = [idx.apply(func).get() for idx in self.partitions.T[0]] + # This is important because sometimes we have resized the data. The new + # sizes will not be valid if we are trying to compute the index on a + # new object that has a different length. + if old_blocks is not None: + cumulative_block_lengths = np.array(old_blocks.block_lengths).cumsum() + else: + cumulative_block_lengths = np.array(self.block_lengths).cumsum() + else: + func = self.preprocess_func(index_func) + new_indices = [idx.apply(func).get() for idx in self.partitions[0]] + + if old_blocks is not None: + cumulative_block_lengths = np.array(old_blocks.block_widths).cumsum() + else: + cumulative_block_lengths = np.array(self.block_widths).cumsum() + + full_indices = new_indices[0] + + if old_blocks is not None: + for i in range(len(new_indices)): + # If the length is 0 there is nothing to append. + if i == 0 or len(new_indices[i]) == 0: + continue + # The try-except here is intended to catch issues where we are + # trying to get a string index out of the internal index. + try: + append_val = new_indices[i] + cumulative_block_lengths[i - 1] + except TypeError: + append_val = new_indices[i] + + full_indices = full_indices.append(append_val) + else: + full_indices = full_indices.append(new_indices[1:]) + + return full_indices + + @classmethod + def _compute_num_partitions(cls): + """Currently, this method returns the default. In the future it will + estimate the optimal number of partitions. + + :return: + """ + from ...pandas import DEFAULT_NPARTITIONS + return DEFAULT_NPARTITIONS + + # Extracting rows/columns + def _get_blocks_containing_index(self, axis, index): + """Convert a global index to a block index and local index. + + Note: This method is primarily used to convert a global index into a + partition index (along the axis provided) and local index (useful + for `iloc` or similar operations. + + Args: + axis: The axis along which to get the indices + (0 - columns, 1 - rows) + index: The global index to convert. + + Returns: + A tuple containing (block index and internal index). + """ + if not axis: + cumulative_column_widths = np.array(self.block_widths).cumsum() + block_idx = int(np.digitize(index, cumulative_column_widths)) + # Compute the internal index based on the previous lengths. This + # is a global index, so we must subtract the lengths first. + internal_idx = index if not block_idx else index - cumulative_column_widths[block_idx - 1] + return block_idx, internal_idx + else: + cumulative_row_lengths = np.array(self.block_lengths).cumsum() + block_idx = int(np.digitize(index, cumulative_row_lengths)) + # See note above about internal index + internal_idx = index if not block_idx else index - cumulative_row_lengths[block_idx - 1] + return block_idx, internal_idx + + def _get_dict_of_block_index(self, axis, indices): + """Convert indices to a dict of block index to internal index mapping. + + Note: See `_get_blocks_containing_index` for primary usage. This method + accepts a list of indices rather than just a single value, and uses + `_get_blocks_containing_index`. + + Args: + axis: The axis along which to get the indices + (0 - columns, 1 - rows) + indices: A list of global indices to convert. + + Returns + A dictionary of {block index: list of local indices}. + """ + # Get the internal index and create a dictionary so we only have to + # travel to each partition once. + all_partitions_and_idx = [self._get_blocks_containing_index(axis, i) for i in indices] + partitions_dict = {} + + for part_idx, internal_idx in all_partitions_and_idx: + if part_idx not in partitions_dict: + partitions_dict[part_idx] = [internal_idx] + else: + partitions_dict[part_idx].append(internal_idx) + + return partitions_dict + + def _apply_func_to_list_of_partitions(self, func, partitions, **kwargs): + """Applies a function to a list of remote partitions. + + Note: The main use for this is to preprocess the func. + + Args: + func: The func to apply + partitions: The list of partitions + + Returns: + A list of RemotePartition objects. + """ + preprocessed_func = self.preprocess_func(func) + return [obj.apply(preprocessed_func, **kwargs) for obj in partitions] + + def apply_func_to_select_indices(self, axis, func, indices, keep_remaining=False): + """Applies a function to select indices. + + Note: Your internal function must take a kwarg `internal_indices` for + this to work correctly. This prevents information leakage of the + internal index to the external representation. + + Args: + axis: The axis to apply the func over. + func: The function to apply to these indices. + indices: The indices to apply the function to. + keep_remaining: Whether or not to keep the other partitions. + Some operations may want to drop the remaining partitions and + keep only the results. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + + # Handling dictionaries has to be done differently, but we still want + # to figure out the partitions that need to be applied to, so we will + # store the dictionary in a separate variable and assign `indices` to + # the keys to handle it the same as we normally would. + if isinstance(indices, dict): + dict_indices = indices + indices = list(indices.keys()) + else: + dict_indices = None + + if not isinstance(indices, list): + indices = [indices] + + partitions_dict = self._get_dict_of_block_index(axis, indices) + + if not axis: + partitions_for_apply = self.partitions.T + else: + partitions_for_apply = self.partitions + + # We may have a command to perform different functions on different + # columns at the same time. We attempt to handle this as efficiently as + # possible here. Functions that use this in the dictionary format must + # accept a keyword argument `func_dict`. + if dict_indices is not None: + if not keep_remaining: + result = np.array([self._apply_func_to_list_of_partitions(func, partitions_for_apply[i], func_dict={idx: dict_indices[idx] for idx in partitions_dict[i]}) for i in partitions_dict]) + else: + result = np.array([partitions_for_apply[i] if i not in partitions_dict else self._apply_func_to_list_of_partitions(func, partitions_for_apply[i], func_dict={idx: dict_indices[i] for idx in partitions_dict[i]}) for i in range(len(partitions_for_apply))]) + else: + if not keep_remaining: + # We are passing internal indices in here. In order for func to + # actually be able to use this information, it must be able to take in + # the internal indices. This might mean an iloc in the case of Pandas + # or some other way to index into the internal representation. + result = np.array([self._apply_func_to_list_of_partitions(func, partitions_for_apply[i], internal_indices=partitions_dict[i]) for i in partitions_dict]) + else: + # The difference here is that we modify a subset and return the + # remaining (non-updated) blocks in their original position. + result = np.array([partitions_for_apply[i] if i not in partitions_dict else self._apply_func_to_list_of_partitions(func, partitions_for_apply[i], internal_indices=partitions_dict[i]) for i in range(len(partitions_for_apply))]) + + return cls(result.T) if not axis else cls(result) + + def apply_func_to_select_indices_along_full_axis(self, axis, func, indices, keep_remaining=False): + """Applies a function to a select subset of full columns/rows. + + Note: This should be used when you need to apply a function that relies + on some global information for the entire column/row, but only need + to apply a function to a subset. + + Important: For your func to operate directly on the indices provided, + it must use `internal_indices` as a keyword argument. + + Args: + axis: The axis to apply the function over (0 - rows, 1 - columns) + func: The function to apply. + indices: The global indices to apply the func to. + keep_remaining: Whether or not to keep the other partitions. + Some operations may want to drop the remaining partitions and + keep only the results. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + if isinstance(indices, dict): + dict_indices = indices + indices = list(indices.keys()) + else: + dict_indices = None + + if not isinstance(indices, list): + indices = [indices] + + partitions_dict = self._get_dict_of_block_index(axis, indices) + preprocessed_func = self.preprocess_func(func) + + # Since we might be keeping the remaining blocks that are not modified, + # we have to also keep the block_partitions object in the correct + # direction (transpose for columns). + if not axis: + partitions_for_apply = self.column_partitions + partitions_for_remaining = self.partitions.T + else: + partitions_for_apply = self.row_partitions + partitions_for_remaining = self.partitions + + # We may have a command to perform different functions on different + # columns at the same time. We attempt to handle this as efficiently as + # possible here. Functions that use this in the dictionary format must + # accept a keyword argument `func_dict`. + if dict_indices is not None: + if not keep_remaining: + result = np.array([partitions_for_apply[i].apply(preprocessed_func, func_dict={idx: dict_indices[idx] for idx in partitions_dict[i]}) for i in partitions_dict]) + else: + result = np.array([partitions_for_remaining[i] if i not in partitions_dict else self._apply_func_to_list_of_partitions(preprocessed_func, partitions_for_apply[i], func_dict={idx: dict_indices[idx] for idx in partitions_dict[i]}) for i in range(len(partitions_for_apply))]) + else: + if not keep_remaining: + # See notes in `apply_func_to_select_indices` + result = np.array([partitions_for_apply[i].apply(preprocessed_func, internal_indices=partitions_dict[i]) for i in partitions_dict]) + else: + # See notes in `apply_func_to_select_indices` + result = np.array([partitions_for_remaining[i] if i not in partitions_dict else partitions_for_apply[i].apply(preprocessed_func, internal_indices=partitions_dict[i]) for i in range(len(partitions_for_remaining))]) + + return cls(result.T) if not axis else cls(result) + + + def apply_func_to_indices_both_axis(self, func, row_indices, col_indices, + lazy=False, keep_remaining=True, mutate=False, + item_to_distribute=None): + """ + Apply a function to along both axis + + Important: For your func to operate directly on the indices provided, + it must use `row_internal_indices, col_internal_indices` as keyword arguments. + """ + cls = type(self) + + if not mutate: + partition_copy = self.partitions.copy() + else: + partition_copy = self.partitions + + operation_mask = np.full(self.partitions.shape, False) + + row_position_counter = 0 + for row_blk_idx, row_internal_idx in self._get_dict_of_block_index(1, row_indices).items(): + col_position_counter = 0 + for col_blk_idx, col_internal_idx in self._get_dict_of_block_index(0, col_indices).items(): + remote_part = partition_copy[row_blk_idx, col_blk_idx] + + if item_to_distribute is not None: + item = item_to_distribute[ + row_position_counter:row_position_counter+len(row_internal_idx), + col_position_counter:col_position_counter+len(col_internal_idx) + ] + item = {'item': item} + else: + item = dict() + + if lazy: + result = remote_part.add_to_apply_calls(func, + row_internal_indices=row_internal_idx, + col_internal_indices=col_internal_idx, + **item) + else: + result = remote_part.apply(func, + row_internal_indices=row_internal_idx, + col_internal_indices=col_internal_idx, + **item) + + partition_copy[row_blk_idx, col_blk_idx] = result + operation_mask[row_blk_idx, col_blk_idx] = True + + col_position_counter += len(col_internal_idx) + row_position_counter += len(row_internal_idx) + + column_idx = np.where(np.any(operation_mask, axis=0))[0] + row_idx = np.where(np.any(operation_mask, axis=1))[0] + if not keep_remaining: + partition_copy = partition_copy[row_idx][:, column_idx] + + return cls(partition_copy) + + + def inter_data_operation(self, axis, func, other): + """Apply a function that requires two BlockPartitions objects. + + Args: + axis: The axis to apply the function over (0 - rows, 1 - columns) + func: The function to apply + other: The other BlockPartitions object to apply func to. + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + + if axis: + partitions = self.row_partitions + other_partitions = other.row_partitions + else: + partitions = self.column_partitions + other_partitions = other.column_partitions + + func = self.preprocess_func(func) + + result = np.array([partitions[i].apply(func, num_splits=cls._compute_num_partitions(), other_axis_partition=other_partitions[i]) for i in range(len(partitions))]) + return cls(result) if axis else cls(result.T) + + def manual_shuffle(self, axis, shuffle_func): + """Shuffle the partitions based on the `shuffle_func`. + + Args: + axis: + shuffle_func: + + Returns: + A new BlockPartitions object, the type of object that called this. + """ + cls = type(self) + + if axis: + partitions = self.row_partitions + else: + partitions = self.column_partitions + + func = self.preprocess_func(shuffle_func) + result = np.array([part.shuffle(func, num_splits=cls._compute_num_partitions()) for part in partitions]) + return cls(result) if axis else cls(result.T) + + def __getitem__(self, key): + cls = type(self) + return cls(self.partitions[key]) + + def __len__(self): + return sum(self.block_lengths) + + def enlarge_partitions(self, n_rows=None, n_cols=None): + data = self.partitions + block_partitions_cls = type(self) + + if n_rows: + n_cols_lst = self.block_widths + nan_oids_lst = [self._partition_class(_get_nan_block_id(self._partition_class, n_rows, n_cols_)) for n_cols_ in n_cols_lst] + new_chunk = block_partitions_cls(np.array([nan_oids_lst])) + data = self.concat(axis=0, other_blocks=new_chunk) + + if n_cols: + n_rows_lst = self.block_lengths + nan_oids_lst = [self._partition_class(_get_nan_block_id(self._partition_class, n_rows_, n_cols)) for n_rows_ in n_rows_lst] + new_chunk = block_partitions_cls(np.array([nan_oids_lst]).T) + data = self.concat(axis=1, other_blocks=new_chunk) + + return data + + +class RayBlockPartitions(BlockPartitions): + """This method implements the interface in `BlockPartitions`.""" + + # This object uses RayRemotePartition objects as the underlying store. + _partition_class = RayRemotePartition + + def __init__(self, partitions): + self.partitions = partitions + + # We override these for performance reasons. + # Lengths of the blocks + _lengths_cache = None + + # These are set up as properties so that we only use them when we need + # them. We also do not want to trigger this computation on object creation. + @property + def block_lengths(self): + """Gets the lengths of the blocks. + + Note: This works with the property structure `_lengths_cache` to avoid + having to recompute these values each time they are needed. + """ + if self._lengths_cache is None: + # The first column will have the correct lengths. We have an + # invariant that requires that all blocks be the same length in a + # row of blocks. + self._lengths_cache = ray.get([obj.length().oid for obj in self.partitions.T[0]]) + return self._lengths_cache + + # Widths of the blocks + _widths_cache = None + + @property + def block_widths(self): + """Gets the widths of the blocks. + + Note: This works with the property structure `_widths_cache` to avoid + having to recompute these values each time they are needed. + """ + if self._widths_cache is None: + # The first column will have the correct lengths. We have an + # invariant that requires that all blocks be the same width in a + # column of blocks. + self._widths_cache = ray.get([obj.width().oid for obj in self.partitions[0]]) + return self._widths_cache + + @property + def column_partitions(self): + """A list of `RayColumnPartition` objects.""" + return [RayColumnPartition(col) for col in self.partitions.T] + + @property + def row_partitions(self): + """A list of `RayRowPartition` objects.""" + return [RayRowPartition(row) for row in self.partitions] diff --git a/modin/data_management/partitioning/remote_partition.py b/modin/data_management/partitioning/remote_partition.py new file mode 100644 index 00000000000..4f57882a46c --- /dev/null +++ b/modin/data_management/partitioning/remote_partition.py @@ -0,0 +1,284 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import pandas +import ray + + +class RemotePartition(object): + """This abstract class holds the data and metadata for a single partition. + The methods required for implementing this abstract class are listed in + the section immediately following this. + + The API exposed by the children of this object is used in + `BlockPartitions`. + + Note: These objects are treated as immutable by `BlockPartitions` + subclasses. There is no logic for updating inplace. + """ + + # Abstract methods and fields. These must be implemented in order to + # properly subclass this object. There are also some abstract classmethods + # to implement. + def get(self): + """Return the object wrapped by this one to the original format. + + Note: This is the opposite of the classmethod `put`. + E.g. if you assign `x = RemotePartition.put(1)`, `x.get()` should + always return 1. + + Returns: + The object that was `put`. + """ + raise NotImplementedError("Must be implemented in child class") + + def apply(self, func, **kwargs): + """Apply some callable function to the data in this partition. + + Note: It is up to the implementation how kwargs are handled. They are + an important part of many implementations. As of right now, they + are not serialized. + + Args: + func: The lambda to apply (may already be correctly formatted) + + Returns: + A new `RemotePartition` containing the object that has had `func` + applied to it. + """ + raise NotImplementedError("Must be implemented in child class") + + def add_to_apply_calls(self, func, **kwargs): + """Add the function to the apply function call stack. + + This function will be executed when apply is called. It will be executed + in the order inserted; apply's func operates the last and return + """ + raise NotImplementedError("Must be implemented in child class") + + def to_pandas(self): + """Convert the object stored in this partition to a Pandas DataFrame. + + Note: If the underlying object is a Pandas DataFrame, this will likely + only need to call `get` + + Returns: + A Pandas DataFrame. + """ + raise NotImplementedError("Must be implemented in child class") + + @classmethod + def put(cls, obj): + """A factory classmethod to format a given object. + + Args: + obj: An object. + + Returns: + A `RemotePartitions` object. + """ + raise NotImplementedError("Must be implemented in child class") + + @classmethod + def preprocess_func(cls, func): + """Preprocess a function before an `apply` call. + + Note: This is a classmethod because the definition of how to preprocess + should be class-wide. Also, we may want to use this before we + deploy a preprocessed function to multiple `RemotePartition` + objects. + + Args: + func: The function to preprocess. + + Returns: + An object that can be accepted by `apply`. + """ + raise NotImplementedError("Must be implemented in child class") + + @classmethod + def length_extraction_fn(cls): + """The function to compute the length of the object in this partition. + + Returns: + A callable function. + """ + raise NotImplementedError("Must be implemented in child class") + + @classmethod + def width_extraction_fn(cls): + """The function to compute the width of the object in this partition. + + Returns: + A callable function. + """ + raise NotImplementedError("Must be implemented in child class") + + _length_cache = None + _width_cache = None + + def length(self): + if self._length_cache is None: + cls = type(self) + func = cls.length_extraction_fn() + preprocessed_func = cls.preprocess_func(func) + + self._length_cache = self.apply(preprocessed_func) + return self._length_cache + + def width(self): + if self._width_cache is None: + cls = type(self) + func = cls.width_extraction_fn() + preprocessed_func = cls.preprocess_func(func) + + self._width_cache = self.apply(preprocessed_func) + return self._width_cache + + @classmethod + def empty(cls): + raise NotImplementedError("To be implemented in the child class!") + + +class RayRemotePartition(RemotePartition): + + def __init__(self, object_id): + assert type(object_id) is ray.ObjectID + + self.oid = object_id + self.call_queue = [] + + def get(self): + """Gets the object out of the plasma store. + + Returns: + The object from the plasma store. + """ + if len(self.call_queue): + return self.apply(lambda x: x).get() + + return ray.get(self.oid) + + def apply(self, func, **kwargs): + """Apply a function to the object stored in this partition. + + Note: It does not matter if func is callable or an ObjectID. Ray will + handle it correctly either way. The keyword arguments are sent as a + dictionary. + + Args: + func: The function to apply. + + Returns: + A RayRemotePartition object. + """ + oid = self.oid + self.call_queue.append((func, kwargs)) + + def call_queue_closure(oid_obj, call_queues): + + for func, kwargs in call_queues: + if isinstance(func, ray.ObjectID): + func = ray.get(func) + if isinstance(kwargs, ray.ObjectID): + kwargs = ray.get(kwargs) + + oid_obj = func(oid_obj, **kwargs) + + return oid_obj + + oid = deploy_ray_func.remote(call_queue_closure, oid, kwargs={'call_queues': self.call_queue}) + self.call_queue = [] + + return RayRemotePartition(oid) + + + def add_to_apply_calls(self, func, **kwargs): + self.call_queue.append((func, kwargs)) + return self + + + def __copy__(self): + return RayRemotePartition(object_id=self.oid) + + def to_pandas(self): + """Convert the object stored in this partition to a Pandas DataFrame. + + Returns: + A Pandas DataFrame. + """ + dataframe = self.get() + assert type(dataframe) is pandas.DataFrame or type(dataframe) is pandas.Series + + return dataframe + + @classmethod + def put(cls, obj): + """Put an object in the Plasma store and wrap it in this object. + + Args: + obj: The object to be put. + + Returns: + A `RayRemotePartition` object. + """ + return RayRemotePartition(ray.put(obj)) + + @classmethod + def preprocess_func(cls, func): + """Put a callable function into the plasma store for use in `apply`. + + Args: + func: The function to preprocess. + + Returns: + A ray.ObjectID. + """ + return ray.put(func) + + @classmethod + def length_extraction_fn(cls): + return length_fn_pandas + + @classmethod + def width_extraction_fn(cls): + return width_fn_pandas + + @classmethod + def empty(cls): + return cls.put(pandas.DataFrame()) + + +def length_fn_pandas(df): + assert isinstance(df, (pandas.DataFrame, pandas.Series)) + return len(df) + + +def width_fn_pandas(df): + assert isinstance(df, (pandas.DataFrame, pandas.Series)) + if isinstance(df, pandas.DataFrame): + return len(df.columns) + else: + return 1 + + +@ray.remote +def deploy_ray_func(func, partition, kwargs): + """Deploy a function to a partition in Ray. + + Args: + func: The function to apply. + partition: The partition to apply the function to. + kwargs: A dictionary of keyword arguments for the function. + + Returns: + The result of the function. + """ + try: + return func(partition, **kwargs) + # Sometimes Arrow forces us to make a copy of an object before we operate + # on it. We don't want the error to propagate to the user, and we want to + # avoid copying unless we absolutely have to. + except ValueError: + return func(partition.copy(), **kwargs) diff --git a/modin/data_management/partitioning/utils.py b/modin/data_management/partitioning/utils.py new file mode 100644 index 00000000000..c47be187193 --- /dev/null +++ b/modin/data_management/partitioning/utils.py @@ -0,0 +1,35 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + +import numpy as np +import pandas + + +def compute_chunksize(length, num_splits): + # We do this to avoid zeros and having an extremely large last partition + return length // num_splits if length % num_splits == 0 \ + else length // num_splits + 1 + + +def _get_nan_block_id(partition_class, n_row=1, n_col=1, transpose=False): + """A memory efficient way to get a block of NaNs. + + Args: + partition_class (RemotePartition): The class to use to put the object + in the remote format. + n_row(int): The number of rows. + n_col(int): The number of columns. + transpose(bool): If true, swap rows and columns. + Returns: + ObjectID of the NaN block. + """ + global _NAN_BLOCKS + if transpose: + n_row, n_col = n_col, n_row + shape = (n_row, n_col) + if shape not in _NAN_BLOCKS: + arr = np.tile(np.array(np.NaN), shape) + # TODO Not use pandas.DataFrame here, but something more general. + _NAN_BLOCKS[shape] = partition_class.put(pandas.DataFrame(data=arr)) + return _NAN_BLOCKS[shape] diff --git a/modin/pandas/__init__.py b/modin/pandas/__init__.py index 5f7b6758556..badf5661d7d 100644 --- a/modin/pandas/__init__.py +++ b/modin/pandas/__init__.py @@ -14,6 +14,14 @@ import ray from .. import __git_revision__, __version__ +from .concat import concat +from .dataframe import DataFrame +from .datetimes import to_datetime +from .io import ( + read_csv, read_parquet, read_json, read_html, read_clipboard, read_excel, + read_hdf, read_feather, read_msgpack, read_stata, read_sas, read_pickle, + read_sql) +from .reshape import get_dummies try: if threading.current_thread().name == "MainThread": @@ -30,27 +38,6 @@ num_cpus = ray.global_state.cluster_resources()['CPU'] DEFAULT_NPARTITIONS = int(num_cpus) - -def set_npartition_default(n): - global DEFAULT_NPARTITIONS - DEFAULT_NPARTITIONS = n - - -def get_npartitions(): - return DEFAULT_NPARTITIONS - - -# We import these file after above two function -# because they depend on npartitions. -from .concat import concat # noqa: 402 -from .dataframe import DataFrame # noqa: 402 -from .datetimes import to_datetime # noqa: 402 -from .io import ( # noqa: 402 - read_csv, read_parquet, read_json, read_html, read_clipboard, read_excel, - read_hdf, read_feather, read_msgpack, read_stata, read_sas, read_pickle, - read_sql) -from .reshape import get_dummies # noqa: 402 - __all__ = [ "DataFrame", "Series", "read_csv", "read_parquet", "read_json", "read_html", "read_clipboard", "read_excel", "read_hdf", "read_feather", diff --git a/modin/pandas/concat.py b/modin/pandas/concat.py index 0781e34257e..d8e957f566f 100644 --- a/modin/pandas/concat.py +++ b/modin/pandas/concat.py @@ -4,10 +4,7 @@ import pandas -import numpy as np - from .dataframe import DataFrame -from .utils import _reindex_helper def concat(objs, @@ -65,89 +62,20 @@ def concat(objs, raise ValueError("Only can inner (intersect) or outer (union) join the" " other axis") - # We need this in a list because we use it later. - all_index, all_columns = list( - zip(*[(obj.index, obj.columns) for obj in objs])) - - def series_to_df(series, columns): - df = pandas.DataFrame(series) - df.columns = columns - return DataFrame(df) - - # Pandas puts all of the Series in a single column named 0. This is - # true regardless of the existence of another column named 0 in the - # concat. - if axis == 0: - objs = [ - series_to_df(obj, [0]) if isinstance(obj, pandas.Series) else obj - for obj in objs - ] - else: - # Pandas starts the count at 0 so this will increment the names as - # long as there's a new nameless Series being added. - def name_incrementer(i): - val = i[0] - i[0] += 1 - return val - - i = [0] - objs = [ - series_to_df( - obj, obj.name if obj.name is not None else name_incrementer(i)) - if isinstance(obj, pandas.Series) else obj for obj in objs - ] - - # Using concat on the columns and index is fast because they're empty, - # and it forces the error checking. It also puts the columns in the - # correct order for us. - final_index = \ - pandas.concat([pandas.DataFrame(index=idx) for idx in all_index], - axis=axis, join=join, join_axes=join_axes, - ignore_index=ignore_index, keys=keys, levels=levels, - names=names, verify_integrity=verify_integrity, - copy=False).index - final_columns = \ - pandas.concat([pandas.DataFrame(columns=col) - for col in all_columns], - axis=axis, join=join, join_axes=join_axes, - ignore_index=ignore_index, keys=keys, levels=levels, - names=names, verify_integrity=verify_integrity, - copy=False).columns - - # Put all of the DataFrames into Ray format - # TODO just partition the DataFrames instead of building a new Ray DF. - objs = [ - DataFrame(obj) if isinstance(obj, (pandas.DataFrame, - pandas.Series)) else obj - for obj in objs - ] - - # Here we reuse all_columns/index so we don't have to materialize objects - # from remote memory built in the previous line. In the future, we won't be - # building new DataFrames, rather just partitioning the DataFrames. - if axis == 0: - new_blocks = np.array([ - _reindex_helper._submit( - args=tuple([ - all_columns[i], final_columns, axis, - len(objs[0]._block_partitions) - ] + part.tolist()), - num_return_vals=len(objs[0]._block_partitions)) - for i in range(len(objs)) for part in objs[i]._block_partitions - ]) - else: - # Transposing the columns is necessary because the remote task treats - # everything like rows and returns in row-major format. Luckily, this - # operation is cheap in numpy. - new_blocks = np.array([ - _reindex_helper._submit( - args=tuple([ - all_index[i], final_index, axis, - len(objs[0]._block_partitions.T) - ] + part.tolist()), - num_return_vals=len(objs[0]._block_partitions.T)) - for i in range(len(objs)) for part in objs[i]._block_partitions.T - ]).T - - return DataFrame( - block_partitions=new_blocks, columns=final_columns, index=final_index) + # We have the weird Series and axis check because, when concatenating a + # dataframe to a series on axis=0, pandas ignores the name of the series, + # and this check aims to mirror that (possibly buggy) functionality + objs = [obj if isinstance(obj, DataFrame) else DataFrame(obj.rename()) if + isinstance(obj, pandas.Series) and axis == 0 else DataFrame(obj) + for obj in objs] + df = objs[0] + objs = [obj._data_manager for obj in objs] + new_manager = df._data_manager.concat(axis, objs[1:], join=join, + join_axes=None, + ignore_index=False, + keys=None, + levels=None, + names=None, + verify_integrity=False, + copy=True) + return DataFrame(data_manager=new_manager) diff --git a/modin/pandas/dataframe.py b/modin/pandas/dataframe.py index 53feafa2d8d..e8d0623c57c 100644 --- a/modin/pandas/dataframe.py +++ b/modin/pandas/dataframe.py @@ -4,36 +4,24 @@ import pandas from pandas.api.types import is_scalar -from pandas.compat import lzip, to_str, string_types, cPickle as pkl +from pandas.compat import to_str, string_types, cPickle as pkl import pandas.core.common as com -from pandas.core.dtypes.cast import maybe_upcast_putmask from pandas.core.dtypes.common import (_get_dtype_from_object, is_bool_dtype, is_list_like, is_numeric_dtype, is_timedelta64_dtype) from pandas.core.index import _ensure_index_from_sequences -from pandas.core.indexing import check_bool_indexer -from pandas.errors import MergeError +from pandas.core.indexing import (check_bool_indexer, convert_to_index_sliceable) from pandas.util._validators import validate_bool_kwarg -from pandas._libs import lib import itertools -import io import functools import numpy as np -from numpy.testing import assert_equal import ray import re import sys import warnings -from .utils import (to_pandas, _blocks_to_col, _blocks_to_row, - _compile_remote_dtypes, _concat_index, _co_op_helper, - _create_block_partitions, _create_blocks_helper, - _deploy_func, _fix_blocks_dimensions, _inherit_docstrings, - _map_partitions, _match_partitioning, - _partition_pandas_dataframe, _reindex_helper) -from . import get_npartitions -from .index_metadata import _IndexMetadata +from .utils import (from_pandas, to_pandas, _inherit_docstrings) from .iterator import PartitionIterator @@ -46,12 +34,7 @@ def __init__(self, columns=None, dtype=None, copy=False, - col_partitions=None, - row_partitions=None, - block_partitions=None, - row_metadata=None, - col_metadata=None, - dtypes_cache=None): + data_manager=None): """Distributed DataFrame object backed by Pandas dataframes. Args: @@ -65,27 +48,15 @@ def __init__(self, dtype: Data type to force. Only a single dtype is allowed. If None, infer copy (boolean): Copy data from inputs. - Only affects DataFrame / 2d ndarray input - col_partitions ([ObjectID]): The list of ObjectIDs that contain - the column DataFrame partitions. - row_partitions ([ObjectID]): The list of ObjectIDs that contain the - row DataFrame partitions. - block_partitions: A 2D numpy array of block partitions. - row_metadata (_IndexMetadata): - Metadata for the new DataFrame's rows - col_metadata (_IndexMetadata): - Metadata for the new DataFrame's columns + Only affects DataFrame / 2d ndarray input. + data_manager: A manager object to manage distributed computation. """ if isinstance(data, DataFrame): - self._frame_data = data._frame_data + self._data_manager = data._data_manager return - self._dtypes_cache = dtypes_cache - # Check type of data and use appropriate constructor - if data is not None or (col_partitions is None - and row_partitions is None - and block_partitions is None): + if data is not None or data_manager is None: pandas_df = pandas.DataFrame( data=data, @@ -94,372 +65,63 @@ def __init__(self, dtype=dtype, copy=copy) - # Cache dtypes - self._dtypes_cache = pandas_df.dtypes - - # TODO convert _partition_pandas_dataframe to block partitioning. - row_partitions = \ - _partition_pandas_dataframe(pandas_df, - num_partitions=get_npartitions()) - - self._block_partitions = \ - _create_block_partitions(row_partitions, axis=0, - length=len(pandas_df.columns)) - - # Set in case we were only given a single row/column for below. - axis = 0 - columns = pandas_df.columns - index = pandas_df.index - else: - # created this invariant to make sure we never have to go into the - # partitions to get the columns - assert columns is not None or col_metadata is not None, \ - "Columns not defined, must define columns or col_metadata " \ - "for internal DataFrame creations" - - if block_partitions is not None: - axis = 0 - # put in numpy array here to make accesses easier since it's 2D - self._block_partitions = np.array(block_partitions) - self._block_partitions = \ - _fix_blocks_dimensions(self._block_partitions, axis) - - else: - if row_partitions is not None: - axis = 0 - partitions = row_partitions - axis_length = len(columns) if columns is not None else \ - len(col_metadata) - elif col_partitions is not None: - axis = 1 - partitions = col_partitions - axis_length = len(index) if index is not None else \ - len(row_metadata) - # All partitions will already have correct dtypes - self._dtypes_cache = [ - _deploy_func.remote(lambda df: df.dtypes, pandas_df) - for pandas_df in col_partitions - ] - - # TODO: write explicit tests for "short and wide" - # column partitions - self._block_partitions = \ - _create_block_partitions(partitions, axis=axis, - length=axis_length) - - assert self._block_partitions.ndim == 2, "Block Partitions must be 2D." - - # Create the row and column index objects for using our partitioning. - # If the objects haven't been inherited, then generate them - if row_metadata is not None: - self._row_metadata = row_metadata.copy() - if index is not None: - self.index = index - else: - self._row_metadata = _IndexMetadata( - self._block_partitions[:, 0], index=index, axis=0) - - if col_metadata is not None: - self._col_metadata = col_metadata.copy() - if columns is not None: - self.columns = columns + self._data_manager = from_pandas(pandas_df)._data_manager else: - self._col_metadata = _IndexMetadata( - self._block_partitions[0, :], index=columns, axis=1) - - if self._dtypes_cache is None: - self._get_remote_dtypes() - - def _get_frame_data(self): - data = {} - data['blocks'] = self._block_partitions - data['col_metadata'] = self._col_metadata - data['row_metadata'] = self._row_metadata - data['columns'] = self.columns - data['index'] = self.index - data['dtypes'] = self._dtypes_cache - - return data - - def _set_frame_data(self, data): - self._block_partitions = data['blocks'] - self._col_metadata = data['col_metadata'] - self._row_metadata = data['row_metadata'] - self.columns = data['columns'] - self.index = data['index'] - self._dtypes_cache = data['dtypes'] - - _frame_data = property(_get_frame_data, _set_frame_data) - - def _get_row_partitions(self): - empty_rows_mask = self._row_metadata._lengths > 0 - if any(empty_rows_mask): - self._row_metadata._lengths = \ - self._row_metadata._lengths[empty_rows_mask] - self._block_partitions = self._block_partitions[empty_rows_mask, :] - return [ - _blocks_to_row.remote(*part) - for i, part in enumerate(self._block_partitions) - ] - - def _set_row_partitions(self, new_row_partitions): - self._block_partitions = \ - _create_block_partitions(new_row_partitions, axis=0, - length=len(self.columns)) - - _row_partitions = property(_get_row_partitions, _set_row_partitions) - - def _get_col_partitions(self): - empty_cols_mask = self._col_metadata._lengths > 0 - if any(empty_cols_mask): - self._col_metadata._lengths = \ - self._col_metadata._lengths[empty_cols_mask] - self._block_partitions = self._block_partitions[:, empty_cols_mask] - return [ - _blocks_to_col.remote(*self._block_partitions[:, i]) - for i in range(self._block_partitions.shape[1]) - ] - - def _set_col_partitions(self, new_col_partitions): - self._block_partitions = \ - _create_block_partitions(new_col_partitions, axis=1, - length=len(self.index)) - - _col_partitions = property(_get_col_partitions, _set_col_partitions) + self._data_manager = data_manager def __str__(self): return repr(self) - def _repr_pandas_builder(self): - """Creates a pandas DataFrame of appropriate size from this DataFrame. + def _build_repr_df(self, num_rows, num_cols): + # Add one here so that pandas automatically adds the dots + # It turns out to be faster to extract 2 extra rows and columns than to + # build the dots ourselves. + num_rows_for_head = num_rows // 2 + 1 + num_cols_for_front = num_cols // 2 + 1 + + if len(self.index) <= num_rows: + head = self._data_manager + tail = None + else: + head = self._data_manager.head(num_rows_for_head) + tail = self._data_manager.tail(num_rows_for_head) - Note: Currently the values for the sizes are hard-coded, but eventually - we will need to have an options module for these to be changed. + if len(self.columns) <= num_cols: + head_front = head.to_pandas() + # Creating these empty to make the concat logic simpler + head_back = pandas.DataFrame() + tail_back = pandas.DataFrame() - Returns: - A new pandas DataFrame. repr() will be called on this DataFrame. - """ - - def front_block_builder(blocks, n, index): - """Get first n columns from the blocks provided. - - Note: This is called after we obtain the head/tail blocks. We do - not extract the n columns for each row, only for the head/tail. - - Args: - blocks: A numpy array of OIDs containing block partitions - n: The number of columns to extract - index: The pandas index to assign to the resulting DataFrame. - - Returns: - A pandas DataFrame containing the first n columns extracted - from the blocks provided. - """ - cum_col_lengths = self._col_metadata._lengths.cumsum() - idx = np.digitize(n, cum_col_lengths) - - if idx > 0: - # This value will be what we need to get from the last block - remaining = n - cum_col_lengths[idx - 1] - # These are the blocks that we will take (all the blocks before - # the cutoff n) - full_blocks = \ - pandas.concat([pandas.concat(ray.get(df.tolist()), - axis=1, copy=False) - for df in blocks[:, :idx]], - copy=False) - else: - remaining = n - full_blocks = pandas.DataFrame() - - if remaining == 0: - full_blocks.index = index - return full_blocks - - # These are the blocks that we need extract the remaining (not - # already extracted from full_blocks) columns from. - partial_blocks = \ - pandas.concat(ray.get([_deploy_func.remote( - lambda df: df.iloc[:, :remaining], df) - for df in blocks[:, idx]]), copy=False) - - all_n_columns = \ - pandas.concat([full_blocks, partial_blocks], - axis=1, copy=False) - all_n_columns.index = index - return all_n_columns - - def back_block_builder(blocks, n, index): - """Get last n columns from the blocks provided. - - Note: This is called after we obtain the head/tail blocks. We do - not extract the n columns for each row, only for the head/tail. - - Args: - blocks: A numpy array of OIDs containing block partitions - n: The number of columns to extract - index: The pandas index to assign to the resulting DataFrame. - - Returns: - A pandas DataFrame containing the last n columns extracted - from the blocks provided. - """ - # We use the number of partitions later to work backwards from the - # end of the columns. - nparts = len(self._col_metadata._lengths) - # We are cumulatively summing the lengths in reverse order because - # we'll build the last columns in reverse order - cum_col_lengths = self._col_metadata._lengths[::-1].cumsum() - idx = np.digitize(n, cum_col_lengths) - - if idx > 0: - # This value will be what we need to get from the last block - remaining = n - cum_col_lengths[idx - 1] - # These are the blocks that we will take (all the blocks before - # the cutoff n) - full_blocks = \ - pandas.concat([pandas.concat(ray.get(df.tolist()), - axis=1, copy=False) - for df in blocks[:, nparts - idx:]], - copy=False) + if tail is not None: + tail_front = tail.to_pandas() else: - remaining = n - full_blocks = pandas.DataFrame() - - if remaining == 0: - full_blocks.index = index - return full_blocks - - # These are the blocks that we need extract the remaining (not - # already extracted from full_blocks) columns from. - partial_blocks = \ - pandas.concat(ray.get([_deploy_func.remote( - lambda df: df.iloc[:, -remaining:], df) - for df in blocks[:, -idx - 1]]), copy=False) - - all_n_columns = \ - pandas.concat([partial_blocks, full_blocks], - axis=1, copy=False) - all_n_columns.index = index - return all_n_columns - - def row_dots_builder(full_head, full_tail): - """Inserts a row of dots between head and tail DataFrames - - Args: - full_head: The head pandas DataFrame for the repr. - full_tail: The tail pandas DataFrame for the repr. - - Returns: - A new DataFrame combining full_head and full_tail with a row - of dots inserted between. - """ - row_dots = \ - pandas.Series(["..." for _ in range(len(full_head.columns))]) - row_dots.index = full_head.columns - row_dots.name = "..." - - return full_head.append(row_dots).append(full_tail) - - def col_dots_builder(full_front, full_back): - """Inserts a column of dots between head and tail DataFrames. - - Args: - full_front: The front DataFrame for the repr. - full_back: The back DataFrame for the repr. - - Returns: - A new DataFrame combining front_blocks and back_blocks with a - column of dots inserted between. - """ - col_dots = pandas.Series(["..." for _ in range(len(full_front))]) - col_dots.index = index_of_head - col_dots.name = "..." - return pandas.concat([full_front, col_dots, full_back], - axis=1, - copy=False) - - # If we don't exceed the maximum number of values on either dimension - if len(self.index) <= 60 and len(self.columns) <= 20: - return to_pandas(self) - - if len(self.index) >= 60: - head_blocks = self._head_block_builder(30) - tail_blocks = self._tail_block_builder(30) - index_of_head = self.index[:30] - index_of_tail = self.index[-30:] + tail_front = pandas.DataFrame() else: - head_blocks = self._block_partitions - # We set this to None so we know - tail_blocks = None - index_of_head = self.index - - # Get first and last 10 columns if there are more than 20 columns - if len(self._col_metadata) >= 20: - # Building the front blocks from head_blocks - front_blocks = \ - front_block_builder(head_blocks, 10, index_of_head) - front_blocks.columns = self.columns[:10] - - # Building the back blocks from head_blocks - back_blocks = back_block_builder(head_blocks, 10, index_of_head) - back_blocks.columns = self.columns[-10:] - - full_head = col_dots_builder(front_blocks, back_blocks) - - # True only if we have >60 rows in the DataFrame - if tail_blocks is not None: - # Building the font blocks from tail_blocks - front_blocks = \ - front_block_builder(tail_blocks, 10, index_of_tail) - front_blocks.columns = self.columns[:10] - - # Building the back blocks from tail_blocks - back_blocks = \ - back_block_builder(tail_blocks, 10, index_of_tail) - back_blocks.columns = self.columns[-10:] - - full_tail = col_dots_builder(front_blocks, back_blocks) - - return row_dots_builder(full_head, full_tail) - else: - return full_head + head_front = head.front(num_cols_for_front).to_pandas() + head_back = head.back(num_cols_for_front).to_pandas() - else: - # Convert head_blocks into a pandas DataFrame - list_of_head_rows = [ - pandas.concat(ray.get(df.tolist()), axis=1) - for df in head_blocks - ] - full_head = pandas.concat(list_of_head_rows) - full_head.columns = self.columns - full_head.index = index_of_head - - # True only if we have >60 rows in the DataFrame - if tail_blocks is not None: - # Convert tail_blocks into a pandas DataFrame - list_of_tail_rows = \ - [pandas.concat(ray.get(df.tolist()), axis=1) - for df in tail_blocks] - full_tail = pandas.concat(list_of_tail_rows) - full_tail.columns = self.columns - full_tail.index = index_of_tail - - return row_dots_builder(full_head, full_tail) + if tail is not None: + tail_front = tail.front(num_cols_for_front).to_pandas() + tail_back = tail.back(num_cols_for_front).to_pandas() else: - return full_head + tail_front = tail_back = pandas.DataFrame() + + head_for_repr = pandas.concat([head_front, head_back], axis=1) + tail_for_repr = pandas.concat([tail_front, tail_back], axis=1) + + return pandas.concat([head_for_repr, tail_for_repr]) def __repr__(self): - # We use pandas repr so that we match them. - if len(self._row_metadata) <= 60 and \ - len(self._col_metadata) <= 20: - return repr(self._repr_pandas_builder()) - # The split here is so that we don't repr pandas row lengths. - result = self._repr_pandas_builder() - final_result = repr(result).rsplit("\n\n", 1)[0] + \ - "\n\n[{0} rows x {1} columns]".format(len(self.index), - len(self.columns)) - return final_result + # In the future, we can have this be configurable, just like Pandas. + num_rows = 60 + num_cols = 20 + + result = repr(self._build_repr_df(num_rows, num_cols)) + if len(self.index) > num_rows or len(self.columns) > num_cols: + # The split here is so that we don't repr pandas row lengths. + return result.rsplit("\n\n", 1)[0] + "\n\n[{0} rows x {1} columns]".format(len(self.index), len(self.columns)) + else: + return result def _repr_html_(self): """repr function for rendering in Jupyter Notebooks like Pandas @@ -468,16 +130,18 @@ def _repr_html_(self): Returns: The HTML representation of a Dataframe. """ + # In the future, we can have this be configurable, just like Pandas. + num_rows = 60 + num_cols = 20 + # We use pandas _repr_html_ to get a string of the HTML representation # of the dataframe. - if len(self._row_metadata) <= 60 and \ - len(self._col_metadata) <= 20: - return self._repr_pandas_builder()._repr_html_() - # We split so that we insert our correct dataframe dimensions. - result = self._repr_pandas_builder()._repr_html_() - return result.split("

")[0] + \ - "

{0} rows x {1} columns

\n".format(len(self.index), - len(self.columns)) + result = self._build_repr_df(num_rows, num_cols)._repr_html_() + if len(self.index) > num_rows or len(self.columns) > num_cols: + # We split so that we insert our correct dataframe dimensions. + return result.split("

")[0] + "

{0} rows x {1} columns

\n".format(len(self.index), len(self.columns)) + else: + return result def _get_index(self): """Get the index for this DataFrame. @@ -485,17 +149,7 @@ def _get_index(self): Returns: The union of all indexes across the partitions. """ - return self._row_metadata.index - - def _set_index(self, new_index): - """Set the index for this DataFrame. - - Args: - new_index: The new index to set this - """ - self._row_metadata.index = new_index - - index = property(_get_index, _set_index) + return self._data_manager.index def _get_columns(self): """Get the columns for this DataFrame. @@ -503,49 +157,29 @@ def _get_columns(self): Returns: The union of all indexes across the partitions. """ - return self._col_metadata.index + return self._data_manager.columns - def _set_columns(self, new_index): - """Set the columns for this DataFrame. + def _set_index(self, new_index): + """Set the index for this DataFrame. Args: new_index: The new index to set this """ - self._col_metadata.index = new_index + self._data_manager.index = new_index - columns = property(_get_columns, _set_columns) - - def _arithmetic_helper(self, remote_func, axis, level=None): - # TODO: We don't support `level` right now - if level is not None: - raise NotImplementedError("Level not yet supported.") + def _set_columns(self, new_columns): + """Set the columns for this DataFrame. - axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None \ - else 0 + Args: + new_index: The new index to set this + """ + self._data_manager.columns = new_columns - oid_series = ray.get( - _map_partitions( - remote_func, - self._col_partitions if axis == 0 else self._row_partitions)) + index = property(_get_index, _set_index) + columns = property(_get_columns, _set_columns) - if axis == 0: - # We use the index to get the internal index. - oid_series = [(oid_series[i], i) for i in range(len(oid_series))] - - if len(oid_series) > 0: - for df, partition in oid_series: - this_partition = \ - self._col_metadata.partition_series(partition) - df.index = \ - this_partition[this_partition.isin(df.index)].index - - result_series = pandas.concat([obj[0] for obj in oid_series], - axis=0, - copy=False) - else: - result_series = pandas.concat(oid_series, axis=0, copy=False) - result_series.index = self.index - return result_series + def _map_reduce(self, *args, **kwargs): + raise ValueError("Fix this implementation") def _validate_eval_query(self, expr, **kwargs): """Helper function to check the arguments to eval() and query() @@ -581,10 +215,8 @@ def ndim(self): Returns: The number of dimensions for this DataFrame. """ - # The number of dimensions is common across all partitions. - # The first partition will be enough. - return ray.get( - _deploy_func.remote(lambda df: df.ndim, self._row_partitions[0])) + # DataFrames have an invariant that requires they be 2 dimensions. + return 2 @property def ftypes(self): @@ -595,19 +227,12 @@ def ftypes(self): """ # The ftypes are common across all partitions. # The first partition will be enough. - result = ray.get( - _deploy_func.remote(lambda df: df.ftypes, self._row_partitions[0])) - result.index = self.columns + dtypes = self.dtypes.copy() + ftypes = ["{0}:dense".format(str(dtype)) + for dtype in dtypes.values] + result = pandas.Series(ftypes, index=self.columns) return result - def _get_remote_dtypes(self): - """Finds and caches ObjectIDs for the dtypes of each column partition. - """ - self._dtypes_cache = [ - _compile_remote_dtypes.remote(*column) - for column in self._block_partitions.T - ] - @property def dtypes(self): """Get the dtypes for this DataFrame. @@ -615,16 +240,7 @@ def dtypes(self): Returns: The dtypes for this DataFrame. """ - assert self._dtypes_cache is not None - - if isinstance(self._dtypes_cache, list) and \ - isinstance(self._dtypes_cache[0], - ray.ObjectID): - self._dtypes_cache = pandas.concat( - ray.get(self._dtypes_cache), copy=False) - self._dtypes_cache.index = self.columns - - return self._dtypes_cache + return self._data_manager.dtypes @property def empty(self): @@ -643,9 +259,7 @@ def values(self): Returns: The numpy representation of this DataFrame. """ - return np.concatenate( - ray.get( - _map_partitions(lambda df: df.values, self._row_partitions))) + return self.as_matrix() @property def axes(self): @@ -665,66 +279,15 @@ def shape(self): """ return len(self.index), len(self.columns) - def _update_inplace(self, - row_partitions=None, - col_partitions=None, - block_partitions=None, - columns=None, - index=None, - col_metadata=None, - row_metadata=None): - """updates the current DataFrame inplace. - - Behavior should be similar to the constructor, given the corresponding - arguments. Note that len(columns) and len(index) should match the - corresponding dimensions in the partition(s) passed in, otherwise this - function will complain. + def _update_inplace(self, new_manager): + """Updates the current DataFrame inplace. Args: - row_partitions ([ObjectID]): - The new partitions to replace self._row_partitions directly - col_partitions ([ObjectID]): - The new partitions to replace self._col_partitions directly - columns (pandas.Index): - Index of the column dimension to replace existing columns - index (pandas.Index): - Index of the row dimension to replace existing index - - Note: - If `columns` or `index` are not supplied, they will revert to - default columns or index respectively, as this function does - not have enough contextual info to rebuild the indexes - correctly based on the addition/subtraction of rows/columns. + new_manager: The new DataManager to use to manage the data """ - assert row_partitions is not None or col_partitions is not None\ - or block_partitions is not None, \ - "To update inplace, new column or row partitions must be set." - - if block_partitions is not None: - self._block_partitions = block_partitions - - elif row_partitions is not None: - self._row_partitions = row_partitions - - elif col_partitions is not None: - self._col_partitions = col_partitions - - if col_metadata is not None: - self._col_metadata = col_metadata - else: - assert columns is not None, \ - "If col_metadata is None, columns must be passed in" - self._col_metadata = _IndexMetadata( - self._block_partitions[0, :], index=columns, axis=1) - if row_metadata is not None: - self._row_metadata = row_metadata - else: - # Index can be None for default index, so we don't check - self._row_metadata = _IndexMetadata( - self._block_partitions[:, 0], index=index, axis=0) - - # Update dtypes - self._get_remote_dtypes() + old_manager = self._data_manager + self._data_manager = new_manager + old_manager.free() def add_prefix(self, prefix): """Add a prefix to each of the column names. @@ -732,13 +295,7 @@ def add_prefix(self, prefix): Returns: A new DataFrame containing the new column names. """ - new_cols = self.columns.map(lambda x: str(prefix) + str(x)) - return DataFrame( - block_partitions=self._block_partitions, - columns=new_cols, - col_metadata=self._col_metadata, - row_metadata=self._row_metadata, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.add_prefix(prefix)) def add_suffix(self, suffix): """Add a suffix to each of the column names. @@ -746,13 +303,7 @@ def add_suffix(self, suffix): Returns: A new DataFrame containing the new column names. """ - new_cols = self.columns.map(lambda x: str(x) + str(suffix)) - return DataFrame( - block_partitions=self._block_partitions, - columns=new_cols, - col_metadata=self._col_metadata, - row_metadata=self._row_metadata, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.add_suffix(suffix)) def applymap(self, func): """Apply a function to a DataFrame elementwise. @@ -764,15 +315,7 @@ def applymap(self, func): raise ValueError("\'{0}\' object is not callable".format( type(func))) - new_block_partitions = np.array([ - _map_partitions(lambda df: df.applymap(func), block) - for block in self._block_partitions - ]) - - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) + return DataFrame(data_manager=self._data_manager.applymap(func)) def copy(self, deep=True): """Creates a shallow copy of the DataFrame. @@ -780,11 +323,7 @@ def copy(self, deep=True): Returns: A new DataFrame pointing to the same partitions as this one. """ - return DataFrame( - block_partitions=self._block_partitions, - columns=self.columns, - index=self.index, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.copy()) def groupby(self, by=None, @@ -845,17 +384,15 @@ def sum(self, Returns: The sum of the DataFrame. """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.sum( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - min_count=min_count, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.sum( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + min_count=min_count, + **kwargs) def abs(self): """Apply an absolute value function to all numeric columns. @@ -868,16 +405,7 @@ def abs(self): # TODO Give a more accurate error to Pandas raise TypeError("bad operand type for abs():", "str") - new_block_partitions = np.array([ - _map_partitions(lambda df: df.abs(), block) - for block in self._block_partitions - ]) - - return DataFrame( - block_partitions=new_block_partitions, - columns=self.columns, - index=self.index, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.abs()) def isin(self, values): """Fill a DataFrame with booleans for cells contained in values. @@ -891,15 +419,7 @@ def isin(self, values): True: cell is contained in values. False: otherwise """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.isin(values), block) - for block in self._block_partitions - ]) - - return DataFrame( - block_partitions=new_block_partitions, - columns=self.columns, - index=self.index) + return DataFrame(data_manager=self._data_manager.isin(values=values)) def isna(self): """Fill a DataFrame with booleans for cells containing NA. @@ -910,19 +430,7 @@ def isna(self): True: cell contains NA. False: otherwise. """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.isna(), block) - for block in self._block_partitions - ]) - - new_dtypes = pandas.Series( - [np.dtype("bool")] * len(self.columns), index=self.columns) - - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata, - dtypes_cache=new_dtypes) + return DataFrame(data_manager=self._data_manager.isna()) def isnull(self): """Fill a DataFrame with booleans for cells containing a null value. @@ -933,19 +441,7 @@ def isnull(self): True: cell contains null. False: otherwise. """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.isnull(), block) - for block in self._block_partitions - ]) - - new_dtypes = pandas.Series( - [np.dtype("bool")] * len(self.columns), index=self.columns) - - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata, - dtypes_cache=new_dtypes) + return DataFrame(data_manager=self._data_manager.isnull()) def keys(self): """Get the info axis for the DataFrame. @@ -953,7 +449,6 @@ def keys(self): Returns: A pandas Index for this DataFrame. """ - # Each partition should have the same index, so we'll use 0's return self.columns def transpose(self, *args, **kwargs): @@ -962,15 +457,7 @@ def transpose(self, *args, **kwargs): Returns: A new DataFrame transposed from this DataFrame. """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.T, block) - for block in self._block_partitions - ]) - - return DataFrame( - block_partitions=new_block_partitions.T, - columns=self.index, - index=self.columns) + return DataFrame(data_manager=self._data_manager.transpose(*args, **kwargs)) T = property(transpose) @@ -1003,20 +490,15 @@ def dropna(self, axis = [pandas.DataFrame()._get_axis_number(ax) for ax in axis] result = self - # TODO(kunalgosar): this builds an intermediate dataframe, - # which does unnecessary computation + for ax in axis: result = result.dropna( axis=ax, how=how, thresh=thresh, subset=subset) if not inplace: return result - self._update_inplace( - block_partitions=result._block_partitions, - columns=result.columns, - index=result.index) - - return None + self._update_inplace(new_manager=result._data_manager) + return axis = pandas.DataFrame()._get_axis_number(axis) @@ -1025,7 +507,6 @@ def dropna(self, if how is None and thresh is None: raise TypeError('must specify how or thresh') - indices = None if subset is not None: if axis == 1: indices = self.index.get_indexer_for(subset) @@ -1038,69 +519,12 @@ def dropna(self, if check.any(): raise KeyError(list(np.compress(check, subset))) - def dropna_helper(df): - new_df = df.dropna( - axis=axis, - how=how, - thresh=thresh, - subset=indices, - inplace=False) - - if axis == 1: - new_index = new_df.columns - new_df.columns = pandas.RangeIndex(0, len(new_df.columns)) - else: - new_index = new_df.index - new_df.reset_index(drop=True, inplace=True) - - return new_df, new_index - - parts = self._col_partitions if axis == 1 else self._row_partitions - result = [ - _deploy_func._submit(args=(dropna_helper, df), num_return_vals=2) - for df in parts - ] - new_parts, new_vals = [list(t) for t in zip(*result)] - - if axis == 1: - new_vals = [ - self._col_metadata.get_global_indices(i, vals) - for i, vals in enumerate(ray.get(new_vals)) - ] - - # This flattens the 2d array to 1d - new_vals = [i for j in new_vals for i in j] - new_cols = self.columns[new_vals] - - if not inplace: - return DataFrame( - col_partitions=new_parts, - columns=new_cols, - index=self.index) - - self._update_inplace( - col_partitions=new_parts, columns=new_cols, index=self.index) + new_manager = self._data_manager.dropna(axis=axis, how=how, thresh=thresh, subset=subset) + if not inplace: + return DataFrame(data_manager=new_manager) else: - new_vals = [ - self._row_metadata.get_global_indices(i, vals) - for i, vals in enumerate(ray.get(new_vals)) - ] - - # This flattens the 2d array to 1d - new_vals = [i for j in new_vals for i in j] - new_rows = self.index[new_vals] - - if not inplace: - return DataFrame( - row_partitions=new_parts, - index=new_rows, - columns=self.columns) - - self._update_inplace( - row_partitions=new_parts, index=new_rows, columns=self.columns) - - return None + self._update_inplace(new_manager=new_manager) def add(self, other, axis='columns', level=None, fill_value=None): """Add this DataFrame to another or a scalar/list. @@ -1115,8 +539,16 @@ def add(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the applied addition. """ - return self._operator_helper(pandas.DataFrame.add, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.add(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def agg(self, func, axis=0, *args, **kwargs): return self.aggregate(func, axis, *args, **kwargs) @@ -1152,10 +584,8 @@ def _aggregate(self, arg, *args, **kwargs): raise NotImplementedError( "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - elif is_list_like(arg): + elif is_list_like(arg) or callable(arg): return self.apply(arg, axis=_axis, args=args, **kwargs) - elif callable(arg): - self._callable_function(arg, _axis, *args, **kwargs) else: # TODO Make pandas error raise ValueError("type {} is not callable".format(type(arg))) @@ -1181,106 +611,6 @@ def _string_function(self, func, *args, **kwargs): raise ValueError("{} is an unknown string function".format(func)) - def _callable_function(self, func, axis, *args, **kwargs): - kwargs['axis'] = axis - - def agg_helper(df, arg, index, columns, *args, **kwargs): - df.index = index - df.columns = columns - is_transform = kwargs.pop('is_transform', False) - new_df = df.agg(arg, *args, **kwargs) - - is_series = False - index = None - columns = None - - if isinstance(new_df, pandas.Series): - is_series = True - else: - columns = new_df.columns - index = new_df.index - new_df.columns = pandas.RangeIndex(0, len(new_df.columns)) - new_df.reset_index(drop=True, inplace=True) - - if is_transform: - if is_scalar(new_df) or len(new_df) != len(df): - raise ValueError("transforms cannot produce " - "aggregated results") - - return is_series, new_df, index, columns - - if axis == 0: - index = self.index - columns = [ - self._col_metadata.partition_series(i).index - for i in range(len(self._col_partitions)) - ] - - remote_result = \ - [_deploy_func._submit(args=( - lambda df: agg_helper(df, - func, - index, - cols, - *args, - **kwargs), - part), num_return_vals=4) - for cols, part in zip(columns, self._col_partitions)] - - if axis == 1: - indexes = [ - self._row_metadata.partition_series(i).index - for i in range(len(self._row_partitions)) - ] - columns = self.columns - - remote_result = \ - [_deploy_func._submit(args=( - lambda df: agg_helper(df, - func, - index, - columns, - *args, - **kwargs), - part), num_return_vals=4) - for index, part in zip(indexes, self._row_partitions)] - - # This magic transposes the list comprehension returned from remote - is_series, new_parts, index, columns = \ - [list(t) for t in zip(*remote_result)] - - # This part is because agg can allow returning a Series or a - # DataFrame, and we have to determine which here. Shouldn't add - # too much to latency in either case because the booleans can - # be returned immediately - is_series = ray.get(is_series) - if all(is_series): - new_series = pandas.concat(ray.get(new_parts), copy=False) - new_series.index = self.columns if axis == 0 else self.index - return new_series - # This error is thrown when some of the partitions return Series and - # others return DataFrames. We do not allow mixed returns. - elif any(is_series): - raise ValueError("no results.") - # The remaining logic executes when we have only DataFrames in the - # remote objects. We build a Ray DataFrame from the Pandas partitions. - elif axis == 0: - new_index = ray.get(index[0]) - # This does not handle the Multi Index case - new_columns = ray.get(columns) - new_columns = new_columns[0].append(new_columns[1:]) - - return DataFrame( - col_partitions=new_parts, columns=new_columns, index=new_index) - else: - new_columns = ray.get(columns[0]) - # This does not handle the Multi Index case - new_index = ray.get(index) - new_index = new_index[0].append(new_index[1:]) - - return DataFrame( - row_partitions=new_parts, columns=new_columns, index=new_index) - def align(self, other, join='outer', @@ -1304,16 +634,14 @@ def all(self, axis=None, bool_only=None, skipna=None, level=None, If axis=None or axis=0, this call applies df.all(axis=1) to the transpose of df. """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.all( - axis=axis, - bool_only=bool_only, - skipna=skipna, - level=level, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.all( + axis=axis, + bool_only=bool_only, + skipna=skipna, + level=level, + **kwargs) def any(self, axis=None, bool_only=None, skipna=None, level=None, **kwargs): @@ -1323,16 +651,14 @@ def any(self, axis=None, bool_only=None, skipna=None, level=None, If axis=None or axis=0, this call applies on the column partitions, otherwise operates on row partitions """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.any( - axis=axis, - bool_only=bool_only, - skipna=skipna, - level=level, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.any( + axis=axis, + bool_only=bool_only, + skipna=skipna, + level=level, + **kwargs) def append(self, other, ignore_index=False, verify_integrity=False): """Append another DataFrame/list/Series to this one. @@ -1359,29 +685,30 @@ def append(self, other, ignore_index=False, verify_integrity=False): # index name will be reset index = pandas.Index([other.name], name=self.index.name) - combined_columns = self.columns.tolist() + self.columns.union( - other.index).difference(self.columns).tolist() - other = other.reindex(combined_columns, copy=False) - other = pandas.DataFrame( - other.values.reshape((1, len(other))), - index=index, - columns=combined_columns) - other = other._convert(datetime=True, timedelta=True) - elif isinstance(other, list) and not isinstance(other[0], DataFrame): - other = pandas.DataFrame(other) - if (self.columns.get_indexer(other.columns) >= 0).all(): - other = other.loc[:, self.columns] - - from .concat import concat - if isinstance(other, (list, tuple)): - to_concat = [self] + other + # Create a Modin DataFrame from this Series for ease of development + other = DataFrame(pandas.DataFrame(other).T, index=index)._data_manager + elif isinstance(other, list): + if not isinstance(other[0], DataFrame): + other = pandas.DataFrame(other) + if (self.columns.get_indexer(other.columns) >= 0).all(): + other = DataFrame(other.loc[:, self.columns])._data_manager + else: + other = DataFrame(other)._data_manager + else: + other = [obj._data_manager for obj in other] else: - to_concat = [self, other] + other = other._data_manager - return concat( - to_concat, - ignore_index=ignore_index, - verify_integrity=verify_integrity) + # If ignore_index is False, by definition the Index will be correct. + # We also do this first to ensure that we don't waste compute/memory. + if verify_integrity and not ignore_index: + appended_index = self.index.append(other.index) + is_valid = next((False for idx in appended_index.duplicated() if idx), True) + if not is_valid: + raise ValueError("Indexes have overlapping values: {}".format(appended_index[appended_index.duplicated()])) + + data_manager = self._data_manager.concat(0, other, ignore_index=ignore_index) + return DataFrame(data_manager=data_manager) def apply(self, func, @@ -1419,50 +746,18 @@ def apply(self, 'duplicate column names not supported with apply().', FutureWarning, stacklevel=2) - has_list = list in map(type, func.values()) - part_ind_tuples = [(self._col_metadata[key], key) for key in func] - - if has_list: - # if input dict has a list, the function to apply must wrap - # single functions in lists as well to get the desired output - # format - result = [_deploy_func.remote( - lambda df: df.iloc[:, ind].apply( - func[key] if is_list_like(func[key]) - else [func[key]]), - self._col_partitions[part]) - for (part, ind), key in part_ind_tuples] - return pandas.concat(ray.get(result), axis=1, copy=False) - else: - result = [ - _deploy_func.remote( - lambda df: df.iloc[:, ind].apply(func[key]), - self._col_partitions[part]) - for (part, ind), key in part_ind_tuples - ] - return pandas.Series(ray.get(result), index=func.keys()) - elif is_list_like(func): if axis == 1: raise TypeError("(\"'list' object is not callable\", " "'occurred at index {0}'".format( self.index[0])) - # TODO: some checking on functions that return Series or Dataframe - new_cols = _map_partitions(lambda df: df.apply(func), - self._col_partitions) - - # resolve function names for the DataFrame index - new_index = [ - f_name if isinstance(f_name, string_types) else f_name.__name__ - for f_name in func - ] - return DataFrame( - col_partitions=new_cols, - columns=self.columns, - index=new_index, - col_metadata=self._col_metadata) - elif callable(func): - return self._callable_function(func, axis=axis, *args, **kwds) + elif not callable(func): + return + + data_manager = self._data_manager.apply(func, axis, *args, **kwds) + if isinstance(data_manager, pandas.Series): + return data_manager + return DataFrame(data_manager=data_manager) def as_blocks(self, copy=True): raise NotImplementedError( @@ -1503,42 +798,23 @@ def assign(self, **kwargs): "github.com/modin-project/modin.") def astype(self, dtype, copy=True, errors='raise', **kwargs): + col_dtypes = dict() if isinstance(dtype, dict): if (not set(dtype.keys()).issubset(set(self.columns)) and errors == 'raise'): raise KeyError("Only a column name can be used for the key in" "a dtype mappings argument.") - columns = list(dtype.keys()) - col_idx = [(self.columns.get_loc(columns[i]), - columns[i]) if columns[i] in self.columns else - (columns[i], columns[i]) for i in range(len(columns))] - new_dict = {} - for idx, key in col_idx: - new_dict[idx] = dtype[key] - new_rows = _map_partitions(lambda df, dt: df.astype(dtype=dt, - copy=True, - errors=errors, - **kwargs), - self._row_partitions, new_dict) - if copy: - return DataFrame( - row_partitions=new_rows, - columns=self.columns, - index=self.index) - self._row_partitions = new_rows + col_dtypes = dtype + + else: + for column in self.columns: + col_dtypes[column] = dtype + + new_data_manager = self._data_manager.astype(col_dtypes, errors, **kwargs) + if copy: + return DataFrame(data_manager=new_data_manager) else: - new_blocks = [_map_partitions(lambda d: d.astype(dtype=dtype, - copy=True, - errors=errors, - **kwargs), - block) - for block in self._block_partitions] - if copy: - return DataFrame( - block_partitions=new_blocks, - columns=self.columns, - index=self.index) - self._block_partitions = new_blocks + self._update_inplace(new_data_manager) def at_time(self, time, asof=False): raise NotImplementedError( @@ -1670,34 +946,14 @@ def count(self, axis=0, level=None, numeric_only=False): Returns: The count, in a Series (or DataFrame if level is specified). """ - - def remote_func(df): - return df.count(axis=axis, level=level, numeric_only=numeric_only) - - return self._arithmetic_helper(remote_func, axis, level) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return self._data_manager.count(axis=axis, level=level, numeric_only=numeric_only) def cov(self, min_periods=None): raise NotImplementedError( "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - def _cumulative_helper(self, func, axis): - axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None \ - else 0 - - if axis == 0: - new_cols = _map_partitions(func, self._col_partitions) - return DataFrame( - col_partitions=new_cols, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) - else: - new_rows = _map_partitions(func, self._row_partitions) - return DataFrame( - row_partitions=new_rows, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) - def cummax(self, axis=None, skipna=True, *args, **kwargs): """Perform a cumulative maximum across the DataFrame. @@ -1708,11 +964,8 @@ def cummax(self, axis=None, skipna=True, *args, **kwargs): Returns: The cumulative maximum of the DataFrame. """ - - def remote_func(df): - return df.cummax(axis=axis, skipna=skipna, *args, **kwargs) - - return self._cumulative_helper(remote_func, axis) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return DataFrame(data_manager=self._data_manager.cummax(axis=axis, skipna=skipna, **kwargs)) def cummin(self, axis=None, skipna=True, *args, **kwargs): """Perform a cumulative minimum across the DataFrame. @@ -1724,11 +977,8 @@ def cummin(self, axis=None, skipna=True, *args, **kwargs): Returns: The cumulative minimum of the DataFrame. """ - - def remote_func(df): - return df.cummin(axis=axis, skipna=skipna, *args, **kwargs) - - return self._cumulative_helper(remote_func, axis) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return DataFrame(data_manager=self._data_manager.cummin(axis=axis, skipna=skipna, **kwargs)) def cumprod(self, axis=None, skipna=True, *args, **kwargs): """Perform a cumulative product across the DataFrame. @@ -1740,11 +990,8 @@ def cumprod(self, axis=None, skipna=True, *args, **kwargs): Returns: The cumulative product of the DataFrame. """ - - def remote_func(df): - return df.cumprod(axis=axis, skipna=skipna, *args, **kwargs) - - return self._cumulative_helper(remote_func, axis) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return DataFrame(data_manager=self._data_manager.cumprod(axis=axis, skipna=skipna, **kwargs)) def cumsum(self, axis=None, skipna=True, *args, **kwargs): """Perform a cumulative sum across the DataFrame. @@ -1756,11 +1003,8 @@ def cumsum(self, axis=None, skipna=True, *args, **kwargs): Returns: The cumulative sum of the DataFrame. """ - - def remote_func(df): - return df.cumsum(axis=axis, skipna=skipna, *args, **kwargs) - - return self._cumulative_helper(remote_func, axis) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return DataFrame(data_manager=self._data_manager.cumsum(axis=axis, skipna=skipna, **kwargs)) def describe(self, percentiles=None, include=None, exclude=None): """ @@ -1775,29 +1019,20 @@ def describe(self, percentiles=None, include=None, exclude=None): Returns: Series/DataFrame of summary statistics """ + # This is important because we don't have communication between + # partitions. We need to communicate to the partitions if they should + # be operating on object data or not. + # TODO uncomment after dtypes is fixed + # if not all(t == np.dtype("O") for t in self.dtypes): + if exclude is None: + exclude = "object" + elif "object" not in include: + exclude = ([exclude] + "object") if isinstance(exclude, str) else list(exclude) + "object" - def describe_helper(df): - """This to ensure nothing goes on with non-numeric columns""" - try: - return df.select_dtypes(exclude='object').describe( - percentiles=percentiles, include=include, exclude=exclude) - # This exception is thrown when there are only non-numeric columns - # in this partition - except ValueError: - return pandas.DataFrame() - - # Begin fixing index based on the columns inside. - parts = ray.get(_map_partitions(describe_helper, self._col_partitions)) - # We use the index to get the internal index. - parts = [(parts[i], i) for i in range(len(parts))] - - for df, partition in parts: - this_partition = self._col_metadata.partition_series(partition) - df.columns = this_partition[this_partition.isin(df.columns)].index - - # Remove index from tuple - result = pandas.concat([obj[0] for obj in parts], axis=1, copy=False) - return result + if percentiles is not None: + pandas.DataFrame()._check_percentile(percentiles) + + return DataFrame(data_manager=self._data_manager.describe(percentiles=percentiles, include=include, exclude=exclude)) def diff(self, periods=1, axis=0): """Finds the difference between elements on the axis requested @@ -1809,19 +1044,7 @@ def diff(self, periods=1, axis=0): Returns: DataFrame with the diff applied """ - axis = pandas.DataFrame()._get_axis_number(axis) - partitions = (self._col_partitions - if axis == 0 else self._row_partitions) - - result = _map_partitions( - lambda df: df.diff(axis=axis, periods=periods), partitions) - - if (axis == 1): - return DataFrame( - row_partitions=result, columns=self.columns, index=self.index) - if (axis == 0): - return DataFrame( - col_partitions=result, columns=self.columns, index=self.index) + return DataFrame(data_manager=self._data_manager.diff(periods=periods, axis=axis)) def div(self, other, axis='columns', level=None, fill_value=None): """Divides this DataFrame against another DataFrame/Series/scalar. @@ -1835,8 +1058,16 @@ def div(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Divide applied. """ - return self._operator_helper(pandas.DataFrame.div, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.div(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def divide(self, other, axis='columns', level=None, fill_value=None): """Synonym for div. @@ -1896,87 +1127,44 @@ def drop(self, else: raise ValueError("Need to specify at least one of 'labels', " "'index' or 'columns'") - obj = self.copy() - - def drop_helper(obj, axis, label): - # TODO(patyang): If you drop from the index first, you can do it - # in batch by returning the dropped items. Likewise coords.drop - # leaves the coords df in an inconsistent state. - if axis == 'index': - try: - coords = obj._row_metadata[label] - object_partitions = obj._row_partitions - except KeyError: - return obj - else: - try: - coords = obj._col_metadata[label] - object_partitions = obj._col_partitions - except KeyError: - return obj - - if isinstance(coords, pandas.DataFrame): - drop_map = { - part: list(df['index_within_partition']) - for part, df in coords.copy().groupby('partition') - } - else: - partitions, indexes = coords - drop_map = {partitions: indexes} - - new_partitions = {} - - for part in drop_map: - index = drop_map[part] - new_partitions[part] = _deploy_func.remote( - lambda df: df.drop(labels=index, axis=axis, - errors='ignore'), - object_partitions[part]) - - if axis == 'index': - obj._row_partitions = \ - [object_partitions[i] if i not in new_partitions - else new_partitions[i] - for i in range(len(object_partitions))] - - obj._row_metadata.drop(labels=label) + # TODO Clean up this error checking + if "index" not in axes: + axes["index"] = None + elif axes["index"] is not None: + if not is_list_like(axes["index"]): + axes["index"] = [axes["index"]] + if errors == 'raise': + non_existant = [obj for obj in axes["index"] if obj not in self.index] + if len(non_existant): + raise ValueError("labels {} not contained in axis".format(non_existant)) + else: + axes["index"] = [obj for obj in axes["index"] if obj in self.index] + # If the length is zero, we will just do nothing + if not len(axes["index"]): + axes["index"] = None + + if "columns" not in axes: + axes["columns"] = None + elif axes["columns"] is not None: + if not is_list_like(axes["columns"]): + axes["columns"] = [axes["columns"]] + if errors == 'raise': + non_existant = [obj for obj in axes["columns"] if obj not in self.columns] + if len(non_existant): + raise ValueError("labels {} not contained in axis".format(non_existant)) else: - obj._col_partitions = \ - [object_partitions[i] if i not in new_partitions - else new_partitions[i] - for i in range(len(object_partitions))] + axes["columns"] = [obj for obj in axes["columns"] if obj in self.columns] + # If the length is zero, we will just do nothing + if not len(axes["columns"]): + axes["columns"] = None - obj._col_metadata.drop(labels=label) + new_manager = self._data_manager.drop(index=axes["index"], columns=axes["columns"]) - return obj + if inplace: + self._update_inplace(new_manager=new_manager) - for axis, labels in axes.items(): - if labels is None: - continue - - if is_list_like(labels): - for label in labels: - if errors != 'ignore' and label and \ - label not in getattr(self, axis): - raise ValueError("The label [{}] is not in the [{}]", - label, axis) - else: - obj = drop_helper(obj, axis, label) - else: - if errors != 'ignore' and labels and \ - labels not in getattr(self, axis): - raise ValueError("The label [{}] is not in the [{}]", - labels, axis) - else: - obj = drop_helper(obj, axis, labels) - - if not inplace: - return obj - else: - self._row_metadata = obj._row_metadata - self._col_metadata = obj._col_metadata - self._block_partitions = obj._block_partitions + return DataFrame(data_manager=new_manager) def drop_duplicates(self, subset=None, keep='first', inplace=False): raise NotImplementedError( @@ -1999,7 +1187,15 @@ def eq(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.eq, other, axis, level) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.eq(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def equals(self, other): """ @@ -2016,19 +1212,7 @@ def equals(self, other): self.columns.equals(other.columns): return False - # We copartition because we don't know what the DataFrames look like - # before this. Empty partitions can give problems with - # _match_partitioning (See _match_partitioning) - new_zipped_parts = self._copartition(other, self.index) - - equals_partitions = [ - _equals_helper.remote(left, right) - for left, right in new_zipped_parts - ] - - # To avoid getting all we use next notation. - return next((False for eq in equals_partitions if not ray.get(eq)), - True) + return all(self.eq(other).all()) def eval(self, expr, inplace=False, **kwargs): """Evaluate a Python expression as a string using various backends. @@ -2076,36 +1260,17 @@ def eval(self, expr, inplace=False, **kwargs): ndarray, numeric scalar, DataFrame, Series """ self._validate_eval_query(expr, **kwargs) - - columns = self.columns - - def eval_helper(df): - df.columns = columns - result = df.eval(expr, inplace=False, **kwargs) - # If result is a series, expr was not an assignment expression. - if not isinstance(result, pandas.Series): - result.columns = pandas.RangeIndex(0, len(result.columns)) - return result - inplace = validate_bool_kwarg(inplace, "inplace") - new_rows = _map_partitions(eval_helper, self._row_partitions) - result_type = ray.get( - _deploy_func.remote(lambda df: type(df), new_rows[0])) - if result_type is pandas.Series: - new_series = pandas.concat(ray.get(new_rows), axis=0, copy=False) - new_series.index = self.index - return new_series + result = self._data_manager.eval(expr, **kwargs) - columns_copy = self._col_metadata._coord_df.copy().T - columns_copy.eval(expr, inplace=True, **kwargs) - columns = columns_copy.columns - - if inplace: - self._update_inplace( - row_partitions=new_rows, columns=columns, index=self.index) + if isinstance(result, pandas.Series): + return result else: - return DataFrame(columns=columns, row_partitions=new_rows) + if inplace: + self._update_inplace(new_manager=result) + else: + return DataFrame(data_manager=result) def ewm(self, com=None, @@ -2202,70 +1367,15 @@ def fillna(self, .format(expecting=expecting, method=method) raise ValueError(msg) - if inplace: - new_obj = self - else: - new_obj = self.copy() - - parts, coords_obj = (new_obj._col_partitions, - new_obj._col_metadata) if axis == 0 else \ - (new_obj._row_partitions, - new_obj._row_metadata) - - if isinstance(value, (pandas.Series, dict)): - new_vals = {} - value = dict(value) - partition_dict = {} - for val in value: - # Get the local index for the partition - try: - part, index = coords_obj[val] - - if part not in partition_dict: - partition_dict[part] = {} - partition_dict[part][index] = value[val] - # Pandas ignores these errors so we will suppress them too. - except KeyError: - continue - - for part, value_map in partition_dict.items(): - new_vals[part] = _deploy_func.remote(lambda df: df.fillna( - value=value_map, - method=method, - axis=axis, - inplace=False, - limit=limit, - downcast=downcast, - **kwargs), parts[part]) - - # Not every partition was changed, so we put everything back that - # was not changed and update those that were. - new_parts = [ - parts[i] if i not in new_vals else new_vals[i] - for i in range(len(parts)) - ] - else: - new_parts = _map_partitions(lambda df: df.fillna( - value=value, - method=method, - axis=axis, - inplace=False, - limit=limit, - downcast=downcast, - **kwargs), parts) + if isinstance(value, pandas.Series): + raise NotImplementedError("value as a Series not yet supported.") - if axis == 0: - new_obj._update_inplace( - col_partitions=new_parts, - columns=self.columns, - index=self.index) + new_manager = self._data_manager.fillna(value=value, method=method, axis=axis, inplace=False, limit=limit, downcast=downcast, **kwargs) + + if inplace: + self._update_inplace(new_manager=new_manager) else: - new_obj._update_inplace( - row_partitions=new_parts, - columns=self.columns, - index=self.index) - if not inplace: - return new_obj + return DataFrame(data_manager=new_manager) def filter(self, items=None, like=None, regex=None, axis=None): """Subset rows or columns based on their labels @@ -2323,7 +1433,7 @@ def first_valid_index(self): Returns: scalar: type of index """ - return self._row_metadata.first_valid_index() + return self._data_manager.first_valid_index() def floordiv(self, other, axis='columns', level=None, fill_value=None): """Divides this DataFrame against another DataFrame/Series/scalar. @@ -2337,8 +1447,16 @@ def floordiv(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Divide applied. """ - return self._operator_helper(pandas.DataFrame.floordiv, other, axis, - level, fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.floordiv(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) @classmethod def from_csv(self, @@ -2389,7 +1507,15 @@ def ge(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.ge, other, axis, level) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.ge(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def get(self, key, default=None): """Get item from object for given key (DataFrame column, Panel @@ -2414,9 +1540,11 @@ def get_dtype_counts(self): Returns: The counts of dtypes in this object. """ - return ray.get( - _deploy_func.remote(lambda df: df.get_dtype_counts(), - self._row_partitions[0])) + result = self.dtypes.value_counts() + result.index = result.index.map(lambda x: str(x)) + result = result.sort_index() + result.index = result.index.map(lambda x: np.dtype(getattr(np, x))) + return result def get_ftype_counts(self): """Get the counts of ftypes in this object. @@ -2424,9 +1552,7 @@ def get_ftype_counts(self): Returns: The counts of ftypes in this object. """ - return ray.get( - _deploy_func.remote(lambda df: df.get_ftype_counts(), - self._row_partitions[0])) + return self.ftypes.value_counts().sort_index() def get_value(self, index, col, takeable=False): raise NotImplementedError( @@ -2449,23 +1575,15 @@ def gt(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.gt, other, axis, level) - - def _head_block_builder(self, n): - length_bins = np.cumsum(self._row_metadata._lengths) - idx = np.digitize(n, length_bins) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") - if idx > 0: - # This value will be what we need to get from the last block - remaining = n - length_bins[idx - 1] - else: - remaining = n - return np.array([ - self._block_partitions[i] if i != idx else [ - _deploy_func.remote(lambda df: df.head(remaining), blk) - for blk in self._block_partitions[i] - ] for i in range(idx + 1) - ]) + other = self._validate_other(other, axis) + new_manager = self._data_manager.gt(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def head(self, n=5): """Get the first n rows of the DataFrame. @@ -2476,18 +1594,10 @@ def head(self, n=5): Returns: A new DataFrame with the first n rows of the DataFrame. """ - if n >= len(self._row_metadata): + if n >= len(self.index): return self.copy() - new_blocks = self._head_block_builder(n) - - index = self._row_metadata.index[:n] - - return DataFrame( - block_partitions=new_blocks, - col_metadata=self._col_metadata, - index=index, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.head(n)) def hist(self, data, @@ -2524,12 +1634,7 @@ def idxmax(self, axis=0, skipna=True): raise TypeError( "reduction operation 'argmax' not allowed for this dtype") - def remote_func(df): - return df.idxmax(axis=axis, skipna=skipna) - - internal_indices = self._arithmetic_helper(remote_func, axis) - # do this to convert internal indices to correct index - return internal_indices.apply(lambda x: self.index[x]) + return self._data_manager.idxmax(axis=axis, skipna=skipna) def idxmin(self, axis=0, skipna=True): """Get the index of the first occurrence of the min value of the axis. @@ -2546,97 +1651,119 @@ def idxmin(self, axis=0, skipna=True): raise TypeError( "reduction operation 'argmax' not allowed for this dtype") - def remote_func(df): - return df.idxmin(axis=axis, skipna=skipna) - - internal_indices = self._arithmetic_helper(remote_func, axis) - # do this to convert internal indices to correct index - return internal_indices.apply(lambda x: self.index[x]) + return self._data_manager.idxmin(axis=axis, skipna=skipna) def infer_objects(self): raise NotImplementedError( "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - def info(self, - verbose=None, - buf=None, - max_cols=None, - memory_usage=None, - null_counts=None): - def info_helper(df): - output_buffer = io.StringIO() - df.info( - verbose=verbose, - buf=output_buffer, - max_cols=max_cols, - memory_usage=memory_usage, - null_counts=null_counts) - return output_buffer.getvalue() - - # Combine the per-partition info and split into lines - result = ''.join( - ray.get(_map_partitions(info_helper, self._col_partitions))) - lines = result.split('\n') + def info(self, + verbose=None, + buf=None, + max_cols=None, + memory_usage=None, + null_counts=None): + """Print a concise summary of a DataFrame, which includes the index + dtype and column dtypes, non-null values and memory usage. + + Args: + verbose (bool, optional): Whether to print the full summary. Defaults + to true + + buf (writable buffer): Where to send output. Defaults to sys.stdout + + max_cols (int, optional): When to switch from verbose to truncated + output. By defualt, this is 100. + + memory_usage (bool, str, optional): Specifies whether the total memory + usage of the DataFrame elements (including index) should be displayed. + True always show memory usage. False never shows memory usage. A value + of ‘deep’ is equivalent to “True with deep introspection”. Memory usage + is shown in human-readable units (base-2 representation). Without deep + introspection a memory estimation is made based in column dtype and number + of rows assuming values consume the same memory amount for corresponding + dtypes. With deep memory introspection, a real memory usage calculation is + performed at the cost of computational resources. Defaults to True. + + null_counts (bool, optional): Whetehr to show the non-null counts. By default, + this is shown only when the frame is smaller than 100 columns and 1690785 + rows. A value of True always shows the counts and False never shows the + counts. + + Returns: + Prints the summary of a DataFrame and returns None. + """ + index = self.index + columns = self.columns + dtypes = self.dtypes + + # Set up default values + verbose = True if verbose is None else verbose + buf = sys.stdout if not buf else buf + max_cols = 100 if not max_cols else max_cols + memory_usage = True if memory_usage is None else memory_usage + if not null_counts: + if len(columns) < 100 and len(index) < 1690785: + null_counts = True + else: + null_counts = False + + # Determine if actually verbose + actually_verbose = True if verbose and max_cols > len(columns) else False + if type(memory_usage) == str and memory_usage == 'deep': + memory_usage_deep = True + else: + memory_usage_deep = False + + # Start putting together output # Class denoted in info() output class_string = '\n' # Create the Index info() string by parsing self.index - index_string = self.index.summary() + '\n' - - # A column header is needed in the inf() output - col_header = 'Data columns (total {0} columns):\n' \ - .format(len(self.columns)) - - # Parse the per-partition values to get the per-column details - # Find all the lines in the output that start with integers - prog = re.compile('^[0-9]+.+') - col_lines = [prog.match(line) for line in lines] - cols = [c.group(0) for c in col_lines if c is not None] - # replace the partition columns names with real column names - columns = [ - "{0}\t{1}\n".format(self.columns[i], cols[i].split(" ", 1)[1]) - for i in range(len(cols)) - ] - col_string = ''.join(columns) + '\n' + index_string = index.summary() + '\n' + + if null_counts: + counts = self._data_manager.count() + if memory_usage: + memory_usage_data = self._data_manager.memory_usage(deep=memory_usage_deep, index=True) + + if actually_verbose: + # Create string for verbose output + col_string = 'Data columns (total {0} columns):\n' \ + .format(len(columns)) + for col, dtype in zip(columns, dtypes): + col_string += '{0}\t'.format(col) + if null_counts: + col_string += '{0} not-null '.format(counts[col]) + col_string += '{0}\n'.format(dtype) + else: + # Create string for not verbose output + col_string = 'Columns: {0} entries, {1} to {2}\n'\ + .format(len(columns), columns[0], columns[-1]) # A summary of the dtypes in the dataframe dtypes_string = "dtypes: " - for dtype, count in self.dtypes.value_counts().iteritems(): + for dtype, count in dtypes.value_counts().iteritems(): dtypes_string += "{0}({1}),".format(dtype, count) dtypes_string = dtypes_string[:-1] + '\n' - # Compute the memory usage by summing per-partitions return values - # Parse lines for memory usage number - prog = re.compile('^memory+.+') - mems = [prog.match(line) for line in lines] - mem_vals = [ - float(re.search(r'\d+', m.group(0)).group()) for m in mems - if m is not None - ] - - memory_string = "" - - if len(mem_vals) != 0: - # Sum memory usage from each partition - if memory_usage != 'deep': - memory_string = 'memory usage: {0}+ bytes' \ - .format(sum(mem_vals)) + # Create memory usage string + memory_string = '' + if memory_usage: + if memory_usage_deep: + memory_string = 'memory usage: {0} bytes'.format(memory_usage_data) else: - memory_string = 'memory usage: {0} bytes'.format(sum(mem_vals)) + memory_string = 'memory usage: {0}+ bytes'.format(memory_usage_data) # Combine all the components of the info() output result = ''.join([ - class_string, index_string, col_header, col_string, dtypes_string, - memory_string - ]) + class_string, index_string, col_string, dtypes_string, memory_string + ]) # Write to specified output buffer - if buf: - buf.write(result) - else: - sys.stdout.write(result) + buf.write(result) def insert(self, loc, column, value, allow_duplicates=False): """Insert column into DataFrame at specified location. @@ -2662,41 +1789,8 @@ def insert(self, loc, column, value, allow_duplicates=False): if loc < 0: raise ValueError("unbounded slice") - partition, index_within_partition = \ - self._col_metadata.insert(column, loc) - - index = self.index - - # Deploy insert function to specific column partition, and replace that - # column - def insert_col_part(df): - if isinstance(value, pandas.Series) and \ - isinstance(value.dtype, - pandas.core.dtypes.dtypes.DatetimeTZDtype): - # Need to set index to index of this dtype or inserted values - # become NaT - df.index = value - df.insert(index_within_partition, column, value, - allow_duplicates) - df.index = pandas.RangeIndex(0, len(df)) - else: - df.index = index - df.insert(index_within_partition, column, value, - allow_duplicates) - df.index = pandas.RangeIndex(0, len(df)) - return df - - new_obj = _deploy_func.remote(insert_col_part, - self._col_partitions[partition]) - - new_cols = [ - self._col_partitions[i] if i != partition else new_obj - for i in range(len(self._col_partitions)) - ] - new_col_names = self.columns.insert(loc, column) - - self._update_inplace( - col_partitions=new_cols, columns=new_col_names, index=self.index) + new_manager = self._data_manager.insert(loc, column, value) + self._update_inplace(new_manager=new_manager) def interpolate(self, method='linear', @@ -2721,17 +1815,14 @@ def iterrows(self): Returns: A generator that iterates over the rows of the frame. """ - index_iter = (self._row_metadata.partition_series(i).index - for i in range(len(self._row_partitions))) + index_iter = iter(self.index) - def iterrow_helper(part): - df = ray.get(part) + def iterrow_builder(df): df.columns = self.columns - df.index = next(index_iter) + df.index = [next(index_iter)] return df.iterrows() - partition_iterator = PartitionIterator(self._row_partitions, - iterrow_helper) + partition_iterator = PartitionIterator(self._data_manager, 0, iterrow_builder) for v in partition_iterator: yield v @@ -2747,17 +1838,14 @@ def items(self): Returns: A generator that iterates over the columns of the frame. """ - col_iter = (self._col_metadata.partition_series(i).index - for i in range(len(self._col_partitions))) + col_iter = iter(self.columns) - def items_helper(part): - df = ray.get(part) - df.columns = next(col_iter) + def items_builder(df): + df.columns = [next(col_iter)] df.index = self.index return df.items() - partition_iterator = PartitionIterator(self._col_partitions, - items_helper) + partition_iterator = PartitionIterator(self._data_manager, 1, items_builder) for v in partition_iterator: yield v @@ -2789,17 +1877,14 @@ def itertuples(self, index=True, name='Pandas'): Returns: A tuple representing row data. See args for varying tuples. """ - index_iter = (self._row_metadata.partition_series(i).index - for i in range(len(self._row_partitions))) + index_iter = iter(self.index) - def itertuples_helper(part): - df = ray.get(part) + def itertuples_builder(df): df.columns = self.columns - df.index = next(index_iter) + df.index = [next(index_iter)] return df.itertuples(index=index, name=name) - partition_iterator = PartitionIterator(self._row_partitions, - itertuples_helper) + partition_iterator = PartitionIterator(self._data_manager, 0, itertuples_builder) for v in partition_iterator: yield v @@ -2834,99 +1919,25 @@ def join(self, other = DataFrame({other.name: other}) if isinstance(other, DataFrame): - if on is not None: - index = self[on] - else: - index = self.index - - new_index = index.join(other.index, how=how, sort=sort) - - # Joining two empty DataFrames is fast, and error checks for us. - new_column_labels = pandas.DataFrame(columns=self.columns) \ - .join(pandas.DataFrame(columns=other.columns), - lsuffix=lsuffix, rsuffix=rsuffix).columns - - new_partition_num = max( - len(self._block_partitions.T), len(other._block_partitions.T)) - - # Join is a concat once we have shuffled the data internally. - # We shuffle the data by computing the correct order. - # Another important thing to note: We set the current self index - # to the index variable which may be 'on'. - new_self = np.array([ - _reindex_helper._submit( - args=tuple([index, new_index, 1, new_partition_num] + - block.tolist()), - num_return_vals=new_partition_num) - for block in self._block_partitions.T - ]) - new_other = np.array([ - _reindex_helper._submit( - args=tuple([other.index, new_index, 1, new_partition_num] + - block.tolist()), - num_return_vals=new_partition_num) - for block in other._block_partitions.T - ]) - - # Append the blocks together (i.e. concat) - new_block_parts = np.concatenate((new_self, new_other)).T - - # Default index in the case that on is set. - if on is not None: - new_index = None + # Joining the empty DataFrames with either index or columns is + # fast. It gives us proper error checking for the edge cases that + # would otherwise require a lot more logic. + pandas.DataFrame(columns=self.columns).join(pandas.DataFrame(columns=other.columns), lsuffix=lsuffix, rsuffix=rsuffix).columns - # TODO join the two metadata tables for performance. - return DataFrame( - block_partitions=new_block_parts, - index=new_index, - columns=new_column_labels) + return DataFrame(data_manager=self._data_manager.join(other._data_manager, how=how, lsuffix=lsuffix, rsuffix=rsuffix, sort=sort)) else: # This constraint carried over from Pandas. if on is not None: raise ValueError("Joining multiple DataFrames only supported" " for joining on index") - # Joining the empty DataFrames with either index or columns is - # fast. It gives us proper error checking for the edge cases that - # would otherwise require a lot more logic. - new_index = pandas.DataFrame(index=self.index).join( - [pandas.DataFrame(index=obj.index) for obj in other], - how=how, - sort=sort).index - - new_column_labels = pandas.DataFrame(columns=self.columns).join( + # See note above about error checking with an empty join. + pandas.DataFrame(columns=self.columns).join( [pandas.DataFrame(columns=obj.columns) for obj in other], lsuffix=lsuffix, rsuffix=rsuffix).columns - new_partition_num = max( - [len(self._block_partitions.T)] + - [len(obj._block_partitions.T) for obj in other]) - - new_self = np.array([ - _reindex_helper._submit( - args=tuple([self.index, new_index, 1, new_partition_num] + - block.tolist()), - num_return_vals=new_partition_num) - for block in self._block_partitions.T - ]) - - new_others = np.array([ - _reindex_helper._submit( - args=tuple([obj.index, new_index, 1, new_partition_num] + - block.tolist()), - num_return_vals=new_partition_num) for obj in other - for block in obj._block_partitions.T - ]) - - # Append the columns together (i.e. concat) - new_block_parts = np.concatenate((new_self, new_others)).T - - # TODO join the two metadata tables for performance. - return DataFrame( - block_partitions=new_block_parts, - index=new_index, - columns=new_column_labels) + return DataFrame(data_manager=self._data_manager.join([obj._data_manager for obj in other], how=how, lsuffix=lsuffix, rsuffix=rsuffix, sort=sort)) def kurt(self, axis=None, @@ -2959,7 +1970,7 @@ def last_valid_index(self): Returns: scalar: type of index """ - return self._row_metadata.last_valid_index() + return self._data_manager.last_valid_index() def le(self, other, axis='columns', level=None): """Checks element-wise that this is less than or equal to other. @@ -2972,7 +1983,15 @@ def le(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.le, other, axis, level) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.le(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def lookup(self, row_labels, col_labels): raise NotImplementedError( @@ -2990,7 +2009,15 @@ def lt(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.lt, other, axis, level) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.lt(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def mad(self, axis=None, skipna=None, level=None): raise NotImplementedError( @@ -3025,16 +2052,14 @@ def max(self, Returns: The max of the DataFrame. """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.max( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.max( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + **kwargs) def mean(self, axis=None, @@ -3051,16 +2076,14 @@ def mean(self, Returns: The mean of the DataFrame. (Pandas series) """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.mean( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.mean( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + **kwargs) def median(self, axis=None, @@ -3077,16 +2100,13 @@ def median(self, Returns: The median of the DataFrame. (Pandas series) """ - - def remote_func(df): - return df.median( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 + return self._data_manager.median( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + **kwargs) def melt(self, id_vars=None, @@ -3099,14 +2119,24 @@ def melt(self, "github.com/modin-project/modin.") def memory_usage(self, index=True, deep=False): - def remote_func(df): - return df.memory_usage(index=False, deep=deep) + """Returns the memory usage of each column in bytes - result = self._arithmetic_helper(remote_func, axis=0) + Args: + index (bool): Whether to include the memory usage of the DataFrame's + index in returned Series. Defaults to True + deep (bool): If True, introspect the data deeply by interrogating + objects dtypes for system-level memory consumption. Defaults to False + + Returns: + A Series where the index are the column names and the values are + the memory usage of each of the columns in bytes. If `index=true`, + then the first value of the Series will be 'Index' with its memory usage. + """ + result = self._data_manager.memory_usage(index=index, deep=deep) result.index = self.columns if index: - index_value = self._row_metadata.index.memory_usage(deep=deep) + index_value = self.index.memory_usage(deep=deep) return pandas.Series(index_value, index=['Index']).append(result) return result @@ -3156,118 +2186,8 @@ def merge(self, "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - args = (how, on, left_on, right_on, left_index, right_index, sort, - suffixes, False, indicator, validate) - - left_cols = ray.put(self.columns) - right_cols = ray.put(right.columns) - - # This can be put in a remote function because we don't need it until - # the end, and the columns can be built asynchronously. This takes the - # columns defining off the critical path and speeds up the overall - # merge. - new_columns = _merge_columns.remote(left_cols, right_cols, *args) - - if on is not None: - if left_on is not None or right_on is not None: - raise MergeError("Can only pass argument \"on\" OR \"left_on\"" - " and \"right_on\", not a combination of " - "both.") - if not is_list_like(on): - on = [on] - - if next((True for key in on if key not in self), False) or \ - next((True for key in on if key not in right), False): - - missing_key = \ - next((str(key) for key in on if key not in self), "") + \ - next((str(key) for key in on if key not in right), "") - raise KeyError(missing_key) - - elif right_on is not None or right_index is True: - if left_on is None and left_index is False: - # Note: This is not the same error as pandas, but pandas throws - # a ValueError NoneType has no len(), and I don't think that - # helps enough. - raise TypeError("left_on must be specified or left_index must " - "be true if right_on is specified.") - - elif left_on is not None or left_index is True: - if right_on is None and right_index is False: - # Note: See note above about TypeError. - raise TypeError("right_on must be specified or right_index " - "must be true if right_on is specified.") - - if left_on is not None: - if not is_list_like(left_on): - left_on = [left_on] - - if next((True for key in left_on if key not in self), False): - raise KeyError(next(key for key in left_on if key not in self)) - - if right_on is not None: - if not is_list_like(right_on): - right_on = [right_on] - - if next((True for key in right_on if key not in right), False): - raise KeyError( - next(key for key in right_on if key not in right)) - - # There's a small chance that our partitions are already perfect, but - # if it's not, we need to adjust them. We adjust the right against the - # left because the defaults of merge rely on the order of the left. We - # have to push the index down here, so if we're joining on the right's - # index we go ahead and push it down here too. - if not np.array_equal(self._row_metadata._lengths, - right._row_metadata._lengths) or right_index: - - repartitioned_right = np.array([ - _match_partitioning._submit( - args=(df, self._row_metadata._lengths, right.index), - num_return_vals=len(self._row_metadata._lengths)) - for df in right._col_partitions - ]).T - else: - repartitioned_right = right._block_partitions - - if not left_index and not right_index: - # Passing None to each call specifies that we don't care about the - # left's index for the join. - left_idx = itertools.repeat(None) - - # We only return the index if we need to update it, and that only - # happens when either left_index or right_index is True. We will - # use this value to add the return vals if we are getting an index - # back. - return_index = False - else: - # We build this to push the index down so that we can use it for - # the join. - left_idx = \ - (v.index for k, v in - self._row_metadata._coord_df.copy().groupby('partition')) - return_index = True - - new_blocks = \ - np.array([_co_op_helper._submit( - args=tuple([lambda x, y: x.merge(y, *args), - left_cols, right_cols, - len(self._block_partitions.T), next(left_idx)] + - np.concatenate(obj).tolist()), - num_return_vals=len(self._block_partitions.T) + return_index) - for obj in zip(self._block_partitions, - repartitioned_right)]) - - if not return_index: - # Default to RangeIndex if left_index and right_index both false. - new_index = None - else: - new_index_parts = new_blocks[:, -1] - new_index = _concat_index.remote(*new_index_parts) - new_blocks = new_blocks[:, :-1] - - return DataFrame( - block_partitions=new_blocks, columns=new_columns, index=new_index) + if left_index and right_index: + return self.join(right, how=how, lsuffix=suffixes[0], rsuffix=suffixes[1], sort=sort) def min(self, axis=None, @@ -3284,16 +2204,14 @@ def min(self, Returns: The min of the DataFrame. """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.min( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.min( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + **kwargs) def mod(self, other, axis='columns', level=None, fill_value=None): """Mods this DataFrame against another DataFrame/Series/scalar. @@ -3307,8 +2225,16 @@ def mod(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Mod applied. """ - return self._operator_helper(pandas.DataFrame.mod, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.mod(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def mode(self, axis=0, numeric_only=False): """Perform mode across the DataFrame. @@ -3322,34 +2248,7 @@ def mode(self, axis=0, numeric_only=False): """ axis = pandas.DataFrame()._get_axis_number(axis) - def mode_helper(df): - mode_df = df.mode(axis=axis, numeric_only=numeric_only) - return mode_df, mode_df.shape[axis] - - def fix_length(df, *lengths): - max_len = max(lengths[0]) - df = df.reindex(pandas.RangeIndex(max_len), axis=axis) - return df - - parts = self._col_partitions if axis == 0 else self._row_partitions - - result = [ - _deploy_func._submit( - args=(lambda df: mode_helper(df), part), num_return_vals=2) - for part in parts - ] - - parts, lengths = [list(t) for t in zip(*result)] - - parts = [ - _deploy_func.remote(lambda df, *l: fix_length(df, l), part, - *lengths) for part in parts - ] - - if axis == 0: - return DataFrame(col_partitions=parts, columns=self.columns) - else: - return DataFrame(row_partitions=parts, index=self.index) + return DataFrame(data_manager=self._data_manager.mode(axis=axis, numeric_only=numeric_only)) def mul(self, other, axis='columns', level=None, fill_value=None): """Multiplies this DataFrame against another DataFrame/Series/scalar. @@ -3363,8 +2262,16 @@ def mul(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Multiply applied. """ - return self._operator_helper(pandas.DataFrame.mul, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.mul(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def multiply(self, other, axis='columns', level=None, fill_value=None): """Synonym for mul. @@ -3391,7 +2298,15 @@ def ne(self, other, axis='columns', level=None): Returns: A new DataFrame filled with Booleans. """ - return self._operator_helper(pandas.DataFrame.ne, other, axis, level) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.ne(other=other, + axis=axis, + level=level) + return self._create_dataframe_from_manager(new_manager) def nlargest(self, n, columns, keep='first'): raise NotImplementedError( @@ -3401,50 +2316,20 @@ def nlargest(self, n, columns, keep='first'): def notna(self): """Perform notna across the DataFrame. - Args: - None - Returns: Boolean DataFrame where value is False if corresponding value is NaN, True otherwise """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.notna(), block) - for block in self._block_partitions - ]) - - new_dtypes = pandas.Series( - [np.dtype("bool")] * len(self.columns), index=self.columns) - - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata, - dtypes_cache=new_dtypes) + return DataFrame(data_manager=self._data_manager.notna()) def notnull(self): """Perform notnull across the DataFrame. - Args: - None - Returns: Boolean DataFrame where value is False if corresponding value is NaN, True otherwise """ - new_block_partitions = np.array([ - _map_partitions(lambda df: df.notnull(), block) - for block in self._block_partitions - ]) - - new_dtypes = pandas.Series( - [np.dtype("bool")] * len(self.columns), index=self.columns) - - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata, - dtypes_cache=new_dtypes) + return DataFrame(data_manager=self._data_manager.notnull()) def nsmallest(self, n, columns, keep='first'): raise NotImplementedError( @@ -3462,11 +2347,7 @@ def nunique(self, axis=0, dropna=True): Returns: nunique : Series """ - - def remote_func(df): - return df.nunique(axis=axis, dropna=dropna) - - return self._arithmetic_helper(remote_func, axis) + return self._data_manager.nunique(axis=axis, dropna=dropna) def pct_change(self, periods=1, @@ -3570,8 +2451,16 @@ def pow(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Pow applied. """ - return self._operator_helper(pandas.DataFrame.pow, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.pow(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def prod(self, axis=None, @@ -3592,17 +2481,15 @@ def prod(self, Returns: prod : Series or DataFrame (if level specified) """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.prod( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - min_count=min_count, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.prod( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + min_count=min_count, + **kwargs) def product(self, axis=None, @@ -3682,62 +2569,13 @@ def check_bad_dtype(t): # check that all qs are between 0 and 1 pandas.DataFrame()._check_percentile(q) - def quantile_helper(df, base_object): - """Quantile to be run inside each partitoin. - - Args: - df: The DataFrame composing the partition. - base_object: An empty pandas.Series or pandas.DataFrame - depending on q. - - Returns: - A new Series or DataFrame depending on q. - """ - # This if call prevents ValueErrors with object only partitions - if (numeric_only and all( - dtype == np.dtype('O') or is_timedelta64_dtype(dtype) - for dtype in df.dtypes)): - return base_object - else: - return df.quantile( - q=q, - axis=axis, - numeric_only=numeric_only, - interpolation=interpolation) - axis = pandas.DataFrame()._get_axis_number(axis) if isinstance(q, (pandas.Series, np.ndarray, pandas.Index, list)): - - q_index = pandas.Float64Index(q) - - if axis == 0: - new_partitions = _map_partitions( - lambda df: quantile_helper(df, pandas.DataFrame()), - self._col_partitions) - - # select only correct dtype columns - new_columns = self.dtypes[self.dtypes.apply( - lambda x: is_numeric_dtype(x))].index - - else: - new_partitions = _map_partitions( - lambda df: quantile_helper(df, pandas.DataFrame()), - self._row_partitions) - new_columns = self.index - - return DataFrame( - col_partitions=new_partitions, - index=q_index, - columns=new_columns) + return DataFrame(data_manager=self._data_manager.quantile_for_list_of_values(q=q, axis=axis, numeric_only=numeric_only, interpolation=interpolation)) else: - # When q is a single float, we return a Series, so using - # arithmetic_helper works well here. - result = self._arithmetic_helper( - lambda df: quantile_helper(df, pandas.Series()), axis) - result.name = q - return result + return self._data_manager.quantile_for_single_value(q=q, axis=axis, numeric_only=numeric_only, interpolation=interpolation) def query(self, expr, inplace=False, **kwargs): """Queries the Dataframe with a boolean expression @@ -3746,23 +2584,14 @@ def query(self, expr, inplace=False, **kwargs): A new DataFrame if inplace=False """ self._validate_eval_query(expr, **kwargs) + inplace = validate_bool_kwarg(inplace, "inplace") - columns = self.columns - - def query_helper(df): - df = df.copy() - df.columns = columns - df.query(expr, inplace=True, **kwargs) - df.columns = pandas.RangeIndex(0, len(df.columns)) - return df - - new_rows = _map_partitions(query_helper, self._row_partitions) + new_manager = self._data_manager.query(expr, **kwargs) if inplace: - self._update_inplace(row_partitions=new_rows, index=self.index) + self._update_inplace(new_manager=new_manager) else: - return DataFrame( - row_partitions=new_rows, col_metadata=self._col_metadata) + return DataFrame(data_manager=new_manager) def radd(self, other, axis='columns', level=None, fill_value=None): return self.add(other, axis, level, fill_value) @@ -3782,7 +2611,7 @@ def rank(self, Args: axis (int): 0 or 'index' for row-wise, 1 or 'columns' for column-wise - interpolation: {'average', 'min', 'max', 'first', 'dense'} + method: {'average', 'min', 'max', 'first', 'dense'} Specifies which method to use for equal vals numeric_only (boolean) Include only float, int, boolean data. @@ -3795,34 +2624,38 @@ def rank(self, Returns: A new DataFrame """ + axis = pandas.DataFrame()._get_axis_number(axis) - def rank_helper(df): - return df.rank( - axis=axis, - method=method, - numeric_only=numeric_only, - na_option=na_option, - ascending=ascending, - pct=pct) + return DataFrame(data_manager=self._data_manager.rank( + axis=axis, + method=method, + numeric_only=numeric_only, + na_option=na_option, + ascending=ascending, + pct=pct)) - axis = pandas.DataFrame()._get_axis_number(axis) + def rdiv(self, other, axis='columns', level=None, fill_value=None): + """Div this DataFrame against another DataFrame/Series/scalar. - if (axis == 1): - new_cols = self.dtypes[self.dtypes.apply( - lambda x: is_numeric_dtype(x))].index - result = _map_partitions(rank_helper, self._row_partitions) - return DataFrame( - row_partitions=result, columns=new_cols, index=self.index) + Args: + other: The object to use to apply the div against this. + axis: The axis to div over. + level: The Multilevel index level to apply div over. + fill_value: The value to fill NaNs with. - if (axis == 0): - result = _map_partitions(rank_helper, self._col_partitions) - return DataFrame( - col_partitions=result, columns=self.columns, index=self.index) + Returns: + A new DataFrame with the rdiv applied. + """ + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") - def rdiv(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rdiv(other, axis, level, fill_value), other, axis, - level) + other = self._validate_other(other, axis) + new_manager = self._data_manager.rdiv(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def reindex(self, labels=None, @@ -3848,38 +2681,20 @@ def reindex(self, elif labels is not None: columns = labels - new_blocks = self._block_partitions if index is not None: - old_index = self.index - new_blocks = np.array([ - reindex_helper._submit( - args=(old_index, index, 1, len(new_blocks), method, - fill_value, limit, tolerance) + tuple( - block.tolist()), - num_return_vals=len(new_blocks)) for block in new_blocks.T - ]).T + new_manager = self._data_manager.reindex(0, index, method=method, fill_value=fill_value, limit=limit, tolerance=tolerance) else: - index = self.index + new_manager = self._data_manager if columns is not None: - old_columns = self.columns - new_blocks = np.array([ - reindex_helper._submit( - args=(old_columns, columns, 0, new_blocks.shape[1], method, - fill_value, limit, tolerance) + tuple( - block.tolist()), - num_return_vals=new_blocks.shape[1]) - for block in new_blocks - ]) + final_manager = new_manager.reindex(1, columns, method=method, fill_value=fill_value, limit=limit, tolerance=tolerance) else: - columns = self.columns + final_manager = new_manager if copy: - return DataFrame( - block_partitions=new_blocks, index=index, columns=columns) + return DataFrame(data_manager=final_manager) - self._update_inplace( - block_partitions=new_blocks, index=index, columns=columns) + self._update_inplace(new_manager=final_manager) def reindex_axis(self, labels, @@ -4042,105 +2857,28 @@ def reset_index(self, Returns: A new DataFrame if inplace is False, None otherwise. """ - inplace = validate_bool_kwarg(inplace, 'inplace') - if inplace: - new_obj = self - else: - new_obj = self.copy() - - def _maybe_casted_values(index, labels=None): - if isinstance(index, pandas.PeriodIndex): - values = index.asobject.values - elif isinstance(index, pandas.DatetimeIndex) \ - and index.tz is not None: - values = index - else: - values = index.values - if values.dtype == np.object_: - values = lib.maybe_convert_objects(values) - - # if we have the labels, extract the values with a mask - if labels is not None: - mask = labels == -1 - - # we can have situations where the whole mask is -1, - # meaning there is nothing found in labels, so make all nan's - if mask.all(): - values = np.empty(len(mask)) - values.fill(np.nan) - else: - values = values.take(labels) - if mask.any(): - values, changed = maybe_upcast_putmask( - values, mask, np.nan) - return values - - # We're building a new default index dataframe for use later. - new_index = pandas.RangeIndex(len(self)) + # TODO Implement level if level is not None: - if not isinstance(level, (tuple, list)): - level = [level] - level = [self.index._get_level_number(lev) for lev in level] - if isinstance(self.index, pandas.MultiIndex): - if len(level) < self.index.nlevels: - new_index = self.index.droplevel(level) + raise NotImplementedError("Level not yet supported!") + inplace = validate_bool_kwarg(inplace, 'inplace') - if not drop: - if isinstance(self.index, pandas.MultiIndex): - names = [ - n if n is not None else ('level_%d' % i) - for (i, n) in enumerate(self.index.names) - ] - to_insert = lzip(self.index.levels, self.index.labels) - else: - default = 'index' - i = 0 - while default in self: - default = 'level_{}'.format(i) - i += 1 - - names = ([default] - if self.index.name is None else [self.index.name]) - to_insert = ((self.index, None), ) - - multi_col = isinstance(self.columns, pandas.MultiIndex) - for i, (lev, lab) in reversed(list(enumerate(to_insert))): - if not (level is None or i in level): - continue - name = names[i] - if multi_col: - col_name = (list(name) - if isinstance(name, tuple) else [name]) - if col_fill is None: - if len(col_name) not in (1, self.columns.nlevels): - raise ValueError("col_fill=None is incompatible " - "with incomplete column name " - "{}".format(name)) - col_fill = col_name[0] - - lev_num = self.columns._get_level_number(col_level) - name_lst = [col_fill] * lev_num + col_name - missing = self.columns.nlevels - len(name_lst) - name_lst += [col_fill] * missing - name = tuple(name_lst) - # to ndarray and maybe infer different dtype - level_values = _maybe_casted_values(lev, lab) - new_obj.insert(0, name, level_values) - - new_obj.index = new_index + # Error checking for matching Pandas. Pandas does not allow you to + # insert a dropped index into a DataFrame if these columns already + # exist. + if not drop and all(n in self.columns for n in ["level_0", "index"]): + raise ValueError("cannot insert level_0, already exists") - if not inplace: - return new_obj + new_manager = self._data_manager.reset_index(drop=drop, level=level) + if inplace: + self._update_inplace(new_manager=new_manager) + else: + return DataFrame(data_manager=new_manager) def rfloordiv(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rfloordiv(other, axis, level, fill_value), other, - axis, level) + return self.floordiv(other, axis, level, fill_value) def rmod(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rmod(other, axis, level, fill_value), other, axis, - level) + return self.mod(other, axis, level, fill_value) def rmul(self, other, axis='columns', level=None, fill_value=None): return self.mul(other, axis, level, fill_value) @@ -4159,31 +2897,66 @@ def rolling(self, "github.com/modin-project/modin.") def round(self, decimals=0, *args, **kwargs): - new_block_partitions = np.array([ - _map_partitions( - lambda df: df.round(decimals=decimals, *args, **kwargs), block) - for block in self._block_partitions - ]) + """Round each element in the DataFrame. + + Args: + decimals: The number of decimals to round to. - return DataFrame( - block_partitions=new_block_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) + Returns: + A new DataFrame. + """ + return DataFrame(data_manager=self._data_manager.round(decimals=decimals, **kwargs)) def rpow(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rpow(other, axis, level, fill_value), other, axis, - level) + """Pow this DataFrame against another DataFrame/Series/scalar. + + Args: + other: The object to use to apply the pow against this. + axis: The axis to pow over. + level: The Multilevel index level to apply pow over. + fill_value: The value to fill NaNs with. + + Returns: + A new DataFrame with the Pow applied. + """ + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.rpow(other=other, + axis=axis, + level=level, + fill_value=fill_value) + + return self._create_dataframe_from_manager(new_manager) + def rsub(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rsub(other, axis, level, fill_value), other, axis, - level) + """Subtract a DataFrame/Series/scalar from this DataFrame. + + Args: + other: The object to use to apply the subtraction to this. + axis: THe axis to apply the subtraction over. + level: Mutlilevel index level to subtract over. + fill_value: The value to fill NaNs with. + + Returns: + A new DataFrame with the subtraciont applied. + """ + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.rsub(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def rtruediv(self, other, axis='columns', level=None, fill_value=None): - return self._single_df_op_helper( - lambda df: df.rtruediv(other, axis, level, fill_value), other, - axis, level) + return self.truediv(other, axis, level, fill_value) def sample(self, n=None, @@ -4221,9 +2994,11 @@ def sample(self, else 0 if axis == 0: - axis_length = len(self._row_metadata) + axis_labels = self.index + axis_length = len(axis_labels) else: - axis_length = len(self._col_metadata) + axis_labels = self.column + axis_length = len(axis_labels) if weights is not None: @@ -4301,15 +3076,6 @@ def sample(self, columns=[] if axis == 1 else self.columns, index=self.index if axis == 1 else []) - if axis == 1: - axis_labels = self.columns - partition_metadata = self._col_metadata - partitions = self._col_partitions - else: - axis_labels = self.index - partition_metadata = self._row_metadata - partitions = self._row_partitions - if random_state is not None: # Get a random number generator depending on the type of # random_state that is passed in @@ -4324,36 +3090,20 @@ def sample(self, # choose random numbers and then get corresponding labels from # chosen axis - sample_indices = random_num_gen.randint( - low=0, high=len(partition_metadata), size=n) + sample_indices = random_num_gen.choice( + np.arange(0, axis_length), size=n, replace=replace) samples = axis_labels[sample_indices] else: # randomly select labels from chosen axis samples = np.random.choice( a=axis_labels, size=n, replace=replace, p=weights) - # create an array of (partition, index_within_partition) tuples for - # each sample - part_ind_tuples = [partition_metadata[sample] for sample in samples] - if axis == 1: - # tup[0] refers to the partition number and tup[1] is the index - # within that partition - new_cols = [ - _deploy_func.remote(lambda df: df.iloc[:, [tup[1]]], - partitions[tup[0]]) - for tup in part_ind_tuples - ] - return DataFrame( - col_partitions=new_cols, columns=samples, index=self.index) + data_manager = self._data_manager.getitem_col_array(samples) + return DataFrame(data_manager=data_manager) else: - new_rows = [ - _deploy_func.remote(lambda df: df.loc[[tup[1]]], - partitions[tup[0]]) - for tup in part_ind_tuples - ] - return DataFrame( - row_partitions=new_rows, columns=self.columns, index=samples) + data_manager = self._data_manager.getitem_row_array(samples) + return DataFrame(data_manager=data_manager) def select(self, crit, axis=0): raise NotImplementedError( @@ -4557,16 +3307,12 @@ def skew(self, Returns: skew : Series or DataFrame (if level specified) """ - - def remote_func(df): - return df.skew( - axis=axis, - skipna=skipna, - level=level, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.skew( + axis=axis, + skipna=skipna, + level=level, + numeric_only=numeric_only, + **kwargs) def slice_shift(self, periods=1, axis=0): raise NotImplementedError( @@ -4613,51 +3359,14 @@ def sort_index(self, axis = pandas.DataFrame()._get_axis_number(axis) - args = (axis, level, ascending, False, kind, na_position, - sort_remaining) - - def _sort_helper(df, index, axis, *args): - if axis == 0: - df.index = index - else: - df.columns = index - - result = df.sort_index(*args) - df.reset_index(drop=True, inplace=True) - df.columns = pandas.RangeIndex(len(df.columns)) - return result - - if axis == 0: - index = self.index - new_column_parts = _map_partitions( - lambda df: _sort_helper(df, index, axis, *args), - self._col_partitions) - - new_columns = self.columns + if not axis: new_index = self.index.sort_values(ascending=ascending) - new_row_parts = None + new_columns = None else: - columns = self.columns - new_row_parts = _map_partitions( - lambda df: _sort_helper(df, columns, axis, *args), - self._row_partitions) - + new_index = None new_columns = self.columns.sort_values(ascending=ascending) - new_index = self.index - new_column_parts = None - if not inplace: - return DataFrame( - col_partitions=new_column_parts, - row_partitions=new_row_parts, - index=new_index, - columns=new_columns) - else: - self._update_inplace( - row_partitions=new_row_parts, - col_partitions=new_column_parts, - columns=new_columns, - index=new_index) + return self.reindex(index=new_index, columns=new_columns) def sort_values(self, by, @@ -4685,86 +3394,26 @@ def sort_values(self, if not is_list_like(by): by = [by] + # Currently, sort_values will just reindex based on the sorted values. + # TODO create a more efficient way to sort if axis == 0: - broadcast_value_dict = {str(col): self[col] for col in by} - broadcast_values = pandas.DataFrame(broadcast_value_dict) + broadcast_value_dict = {col: self[col] for col in by} + broadcast_values = pandas.DataFrame(broadcast_value_dict, index=self.index) + + new_index = broadcast_values.sort_values(by=by, axis=axis, ascending=ascending, kind=kind).index + return self.reindex(index=new_index) else: - broadcast_value_list = [ - to_pandas(self[row::len(self.index)]) for row in by - ] + broadcast_value_list = [to_pandas(self[row::len(self.index)]) for row in by] index_builder = list(zip(broadcast_value_list, by)) - for row, idx in index_builder: - row.index = [str(idx)] - broadcast_values = \ pandas.concat([row for row, idx in index_builder], copy=False) - # We are converting the by to string here so that we don't have a - # collision with the RangeIndex on the inner frame. It is cheap and - # gaurantees that we sort by the correct column. - by = [str(col) for col in by] - - args = (by, axis, ascending, False, kind, na_position) - - def _sort_helper(df, broadcast_values, axis, *args): - """Sorts the data on a partition. - - Args: - df: The DataFrame to sort. - broadcast_values: The by DataFrame to use for the sort. - axis: The axis to sort over. - args: The args for the sort. - - Returns: - A new sorted DataFrame. - """ - if axis == 0: - broadcast_values.index = df.index - names = broadcast_values.columns - else: - broadcast_values.columns = df.columns - names = broadcast_values.index - - return pandas.concat([df, broadcast_values], axis=axis ^ 1, - copy=False).sort_values(*args) \ - .drop(names, axis=axis ^ 1) - - if axis == 0: - new_column_partitions = _map_partitions( - lambda df: _sort_helper(df, broadcast_values, axis, *args), - self._col_partitions) - - new_row_partitions = None - new_columns = self.columns - - # This is important because it allows us to get the axis that we - # aren't sorting over. We need the order of the columns/rows and - # this will provide that in the return value. - new_index = broadcast_values.sort_values(*args).index - else: - new_row_partitions = _map_partitions( - lambda df: _sort_helper(df, broadcast_values, axis, *args), - self._row_partitions) - - new_column_partitions = None - new_columns = broadcast_values.sort_values(*args).columns - new_index = self.index + broadcast_values.columns = self.columns + new_columns = broadcast_values.sort_values(by=by, axis=axis, ascending=ascending, kind=kind).columns - if inplace: - self._update_inplace( - row_partitions=new_row_partitions, - col_partitions=new_column_partitions, - columns=new_columns, - index=new_index) - else: - return DataFrame( - row_partitions=new_row_partitions, - col_partitions=new_column_partitions, - columns=new_columns, - index=new_index, - dtypes_cache=self._dtypes_cache) + return self.reindex(columns=new_columns) def sortlevel(self, level=0, @@ -4803,17 +3452,15 @@ def std(self, Returns: The std of the DataFrame (Pandas Series) """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.std( - axis=axis, - skipna=skipna, - level=level, - ddof=ddof, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.std( + axis=axis, + skipna=skipna, + level=level, + ddof=ddof, + numeric_only=numeric_only, + **kwargs) def sub(self, other, axis='columns', level=None, fill_value=None): """Subtract a DataFrame/Series/scalar from this DataFrame. @@ -4827,8 +3474,16 @@ def sub(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the subtraciont applied. """ - return self._operator_helper(pandas.DataFrame.sub, other, axis, level, - fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.sub(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def subtract(self, other, axis='columns', level=None, fill_value=None): """Alias for sub. @@ -4854,27 +3509,6 @@ def swaplevel(self, i=-2, j=-1, axis=0): "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - def _tail_block_builder(self, n): - npartitions = len(self._row_metadata._lengths) - 1 - length_bins = np.cumsum(self._row_metadata._lengths[::-1]) - - idx = np.digitize(n, length_bins) - - if idx > 0: - # This value will be what we need to get from the last block - remaining = n - length_bins[idx - 1] - else: - remaining = n - - # We are building the blocks in reverse order, then reversing the - # numpy array order - return np.array([ - self._block_partitions[npartitions - i] if i != idx else [ - _deploy_func.remote(lambda df: df.tail(remaining), blk) - for blk in self._block_partitions[npartitions - i] - ] for i in range(idx + 1) - ])[::-1] - def tail(self, n=5): """Get the last n rows of the DataFrame. @@ -4884,17 +3518,10 @@ def tail(self, n=5): Returns: A new DataFrame with the last n rows of this DataFrame. """ - if n >= len(self): + if n >= len(self.index): return self.copy() - new_blocks = self._tail_block_builder(n) - index = self._row_metadata.index[-n:] - - return DataFrame( - block_partitions=new_blocks, - col_metadata=self._col_metadata, - index=index, - dtypes_cache=self._dtypes_cache) + return DataFrame(data_manager=self._data_manager.tail(n)) def take(self, indices, axis=0, convert=None, is_copy=True, **kwargs): raise NotImplementedError( @@ -4954,58 +3581,9 @@ def to_csv(self, 'decimal': decimal } - if compression is not None: - warnings.warn("Defaulting to Pandas implementation", - PendingDeprecationWarning) - return to_pandas(self).to_csv(**kwargs) - - if tupleize_cols is not None: - warnings.warn( - "The 'tupleize_cols' parameter is deprecated and " - "will be removed in a future version", - FutureWarning, - stacklevel=2) - else: - tupleize_cols = False - - remote_kwargs_id = ray.put(dict(kwargs, path_or_buf=None)) - columns_id = ray.put(self.columns) - - def get_csv_str(df, index, columns, header, kwargs): - df.index = index - df.columns = columns - kwargs["header"] = header - return df.to_csv(**kwargs) - - idxs = [0] + np.cumsum(self._row_metadata._lengths).tolist() - idx_args = [ - self.index[idxs[i]:idxs[i + 1]] - for i in range(len(self._row_partitions)) - ] - csv_str_ids = _map_partitions( - get_csv_str, self._row_partitions, idx_args, - [columns_id] * len(self._row_partitions), - [header] + [False] * (len(self._row_partitions) - 1), - [remote_kwargs_id] * len(self._row_partitions)) - - if path_or_buf is None: - buf = io.StringIO() - elif isinstance(path_or_buf, str): - buf = open(path_or_buf, mode) - else: - buf = path_or_buf - - for csv_str_id in csv_str_ids: - buf.write(ray.get(csv_str_id)) - buf.flush() - - result = None - if path_or_buf is None: - result = buf.getvalue() - buf.close() - elif isinstance(path_or_buf, str): - buf.close() - return result + warnings.warn("Defaulting to Pandas implementation", + PendingDeprecationWarning) + return to_pandas(self).to_csv(**kwargs) def to_dense(self): raise NotImplementedError( @@ -5281,8 +3859,16 @@ def truediv(self, other, axis='columns', level=None, fill_value=None): Returns: A new DataFrame with the Divide applied. """ - return self._operator_helper(pandas.DataFrame.truediv, other, axis, - level, fill_value) + if level is not None: + raise NotImplementedError("Mutlilevel index not yet supported " + "in Pandas on Ray") + + other = self._validate_other(other, axis) + new_manager = self._data_manager.truediv(other=other, + axis=axis, + level=level, + fill_value=fill_value) + return self._create_dataframe_from_manager(new_manager) def truncate(self, before=None, after=None, axis=None, copy=True): raise NotImplementedError( @@ -5338,12 +3924,8 @@ def update(self, if not isinstance(other, DataFrame): other = DataFrame(other) - def update_helper(x, y): - x.update(y, join, overwrite, filter_func, False) - return x - - self._inter_df_op_helper( - update_helper, other, join, 0, None, inplace=True) + data_manager = self._data_manager.update(other._data_manager, join=join, overwrite=overwrite, filter_func=filter_func, raise_conflict=raise_conflict) + self._update_inplace(new_manager=data_manager) def var(self, axis=None, @@ -5362,17 +3944,15 @@ def var(self, Returns: The variance of the DataFrame. """ + axis = pandas.DataFrame()._get_axis_number(axis) if axis is not None else 0 - def remote_func(df): - return df.var( - axis=axis, - skipna=skipna, - level=level, - ddof=ddof, - numeric_only=numeric_only, - **kwargs) - - return self._arithmetic_helper(remote_func, axis, level) + return self._data_manager.var( + axis=axis, + skipna=skipna, + level=level, + ddof=ddof, + numeric_only=numeric_only, + **kwargs) def where(self, cond, @@ -5423,68 +4003,20 @@ def where(self, "self") cond = DataFrame(cond, index=self.index, columns=self.columns) - zipped_partitions = self._copartition(cond, self.index) - args = (False, axis, level, errors, try_cast, raise_on_error) - if isinstance(other, DataFrame): - other_zipped = (v for k, v in self._copartition(other, self.index)) - - new_partitions = [ - _where_helper.remote(k, v, next(other_zipped), self.columns, - cond.columns, other.columns, *args) - for k, v in zipped_partitions - ] + other = other._data_manager - # Series has to be treated specially because we're operating on row - # partitions from here on. elif isinstance(other, pandas.Series): - if axis == 0: - # Pandas determines which index to use based on axis. - other = other.reindex(self.index) - other.index = pandas.RangeIndex(len(other)) - - # Since we're working on row partitions, we have to partition - # the Series based on the partitioning of self (since both - # self and cond are co-partitioned by self. - other_builder = [] - for length in self._row_metadata._lengths: - other_builder.append(other[:length]) - other = other[length:] - # Resetting the index here ensures that we apply each part - # to the correct row within the partitions. - other.index = pandas.RangeIndex(len(other)) - - other = (obj for obj in other_builder) - - new_partitions = [ - _where_helper.remote(k, v, next(other, pandas.Series()), - self.columns, cond.columns, None, - *args) for k, v in zipped_partitions - ] - else: - other = other.reindex(self.columns) - new_partitions = [ - _where_helper.remote(k, v, other, self.columns, - cond.columns, None, *args) - for k, v in zipped_partitions - ] - + other = other.reindex(self.index if not axis else self.columns) else: - new_partitions = [ - _where_helper.remote(k, v, other, self.columns, cond.columns, - None, *args) for k, v in zipped_partitions - ] + index = self.index if not axis else self.columns + other = pandas.Series(other, index=index) + data_manager = self._data_manager.where(cond._data_manager, other, axis=axis, level=level) if inplace: - self._update_inplace( - row_partitions=new_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) + self._update_inplace(new_manager=data_manager) else: - return DataFrame( - row_partitions=new_partitions, - row_metadata=self._row_metadata, - col_metadata=self._col_metadata) + return DataFrame(data_manager=data_manager) def xs(self, key, axis=0, level=None, drop_level=True): raise NotImplementedError( @@ -5502,7 +4034,7 @@ def __getitem__(self, key): """ key = com._apply_if_callable(key, self) - # shortcut if we are an actual column + # Shortcut if key is an actual column is_mi_columns = isinstance(self.columns, pandas.MultiIndex) try: if key in self.columns and not is_mi_columns: @@ -5511,7 +4043,8 @@ def __getitem__(self, key): pass # see if we can slice the rows - indexer = self._row_metadata.convert_to_index_sliceable(key) + # This lets us reuse code in Pandas to error check + indexer = convert_to_index_sliceable(pandas.DataFrame(index=self.index), key) if indexer is not None: return self._getitem_slice(indexer) @@ -5529,12 +4062,7 @@ def __getitem__(self, key): return self._getitem_column(key) def _getitem_column(self, key): - # may result in multiple columns? - partition = self._col_metadata[key, 'partition'] - result = ray.get(self._getitem_indiv_col(key, partition)) - result.name = key - result.index = self.index - return result + return self._data_manager.getitem_single_key(key) def _getitem_array(self, key): if com.is_bool_indexer(key): @@ -5550,51 +4078,18 @@ def _getitem_array(self, key): len(key), len(self.index))) key = check_bool_indexer(self.index, key) - new_parts = _map_partitions(lambda df: df[key], - self._col_partitions) - columns = self.columns - index = self.index[key] - - return DataFrame( - col_partitions=new_parts, columns=columns, index=index) - else: - columns = self._col_metadata[key].index - column_indices = {item: i for i, item in enumerate(self.columns)} - indices_for_rows = [column_indices[column] for column in columns] - - def get_columns_partition(df): - result = df.__getitem__(indices_for_rows), - result.columns = pandas.RangeIndex(0, len(result.columns)) - return result - - new_parts = [ - _deploy_func.remote( - lambda df: df.__getitem__(indices_for_rows), part) - for part in self._row_partitions - ] - - index = self.index - - return DataFrame( - row_partitions=new_parts, columns=columns, index=index) - - def _getitem_indiv_col(self, key, part): - loc = self._col_metadata[key] - if isinstance(loc, pandas.Series): - index = loc[loc['partition'] == part] + # We convert here because the data_manager assumes it is a list of + # indices. This greatly decreases the complexity of the code. + key = self.index[key] + return DataFrame(data_manager=self._data_manager.getitem_row_array(key)) else: - index = loc[loc['partition'] == part]['index_within_partition'] - return _deploy_func.remote(lambda df: df.__getitem__(index), - self._col_partitions[part]) + return DataFrame(data_manager=self._data_manager.getitem_column_array(key)) def _getitem_slice(self, key): - new_cols = _map_partitions(lambda df: df[key], self._col_partitions) - - index = self.index[key] - return DataFrame( - col_partitions=new_cols, - col_metadata=self._col_metadata, - index=index) + # We convert here because the data_manager assumes it is a list of + # indices. This greatly decreases the complexity of the code. + key = self.index[key] + return DataFrame(data_manager=self._data_manager.getitem_row_array(key)) def __getattr__(self, key): """After regular attribute access, looks up the name in the columns @@ -5630,7 +4125,7 @@ def __len__(self): Returns: Returns an integer length of the DataFrame object. """ - return len(self._row_metadata) + return len(self.index) def __unicode__(self): raise NotImplementedError( @@ -5716,47 +4211,10 @@ def __delitem__(self, key): Args: key: key to delete """ + if key not in self: + raise KeyError(key) - # Create helper method for deleting column(s) in row partition. - def del_helper(df, to_delete): - cols = df.columns[to_delete] # either int or an array of ints - - if not is_list_like(cols): - cols = [cols] - - for col in cols: - df.__delitem__(col) - - # Reset the column index to conserve space - df.columns = pandas.RangeIndex(0, len(df.columns)) - return df - - # This structure is used to get the correct index inside the partition. - del_df = self._col_metadata[key] - - # We need to standardize between multiple and single occurrences in the - # columns. Putting single occurrences in a pandas.DataFrame and - # transposing results in the same structure as multiple with 'loc'. - if isinstance(del_df, pandas.Series): - del_df = pandas.DataFrame(del_df).T - - # Cast cols as pandas.Series as duplicate columns mean result may be - # np.int64 or pandas.Series - col_parts_to_del = \ - pandas.Series(del_df['partition'].copy()).unique() - self._col_metadata.drop(key) - - for i in col_parts_to_del: - # Compute the correct index inside the partition to delete. - to_delete_in_partition = \ - del_df[del_df['partition'] == i]['index_within_partition'] - - for j in range(self._block_partitions.shape[0]): - self._block_partitions[j, i] = _deploy_func.remote( - del_helper, self._block_partitions[j, i], - to_delete_in_partition) - - self._col_metadata.reset_partition_coords(col_parts_to_del) + self._update_inplace(new_manager=self._data_manager.delitem(key)) def __finalize__(self, other, method=None, **kwargs): raise NotImplementedError( @@ -5902,15 +4360,7 @@ def __neg__(self): raise TypeError( "Unary negative expects numeric dtype, not {}".format(t)) - new_block_partitions = np.array([ - _map_partitions(lambda df: df.__neg__(), block) - for block in self._block_partitions - ]) - - return DataFrame( - block_partitions=new_block_partitions, - col_metadata=self._col_metadata, - row_metadata=self._row_metadata) + return DataFrame(data_manager=self._data_manager.negative()) def __sizeof__(self): raise NotImplementedError( @@ -5947,8 +4397,8 @@ def loc(self): We currently support: single label, list array, slice object We do not support: boolean array, callable """ - from .indexing import _Loc_Indexer - return _Loc_Indexer(self) + from .indexing import _LocIndexer + return _LocIndexer(self) @property def is_copy(self): @@ -5973,136 +4423,35 @@ def iloc(self): We currently support: single label, list array, slice object We do not support: boolean array, callable """ - from .indexing import _iLoc_Indexer - return _iLoc_Indexer(self) - - def _copartition(self, other, new_index): - """Colocates the values of other with this for certain operations. - - NOTE: This method uses the indexes of each DataFrame to order them the - same. This operation does an implicit shuffling of data and zips - the two DataFrames together to be operated on. - - Args: - other: The other DataFrame to copartition with. - - Returns: - Two new sets of partitions, copartitioned and zipped. - """ - # Put in the object store so they aren't serialized each iteration. - old_self_index = ray.put(self.index) - new_index = ray.put(new_index) - old_other_index = ray.put(other.index) - - new_num_partitions = max( - len(self._block_partitions.T), len(other._block_partitions.T)) - - new_partitions_self = \ - np.array([_reindex_helper._submit( - args=tuple([old_self_index, new_index, 1, - new_num_partitions] + block.tolist()), - num_return_vals=new_num_partitions) - for block in self._block_partitions.T]).T - - new_partitions_other = \ - np.array([_reindex_helper._submit( - args=tuple([old_other_index, new_index, 1, - new_num_partitions] + block.tolist()), - num_return_vals=new_num_partitions) - for block in other._block_partitions.T]).T - - return zip(new_partitions_self, new_partitions_other) - - def _operator_helper(self, func, other, axis, level, *args): - """Helper method for inter-DataFrame and scalar operations""" - if isinstance(other, DataFrame): - return self._inter_df_op_helper( - lambda x, y: func(x, y, axis, level, *args), other, "outer", - axis, level) - else: - return self._single_df_op_helper( - lambda df: func(df, other, axis, level, *args), other, axis, - level) - - def _inter_df_op_helper(self, func, other, how, axis, level, - inplace=False): - if level is not None: - raise NotImplementedError("Mutlilevel index not yet supported " - "in Pandas on Ray") - axis = pandas.DataFrame()._get_axis_number(axis) - - new_column_index = self.columns.join(other.columns, how=how) - new_index = self.index.join(other.index, how=how) - copartitions = self._copartition(other, new_index) - - new_blocks = \ - np.array([_co_op_helper._submit( - args=tuple([func, self.columns, other.columns, - len(part[0]), None] + - np.concatenate(part).tolist()), - num_return_vals=len(part[0])) - for part in copartitions]) + from .indexing import _iLocIndexer + return _iLocIndexer(self) + def _create_dataframe_from_manager(self, new_manager, inplace=False): + """Returns or updates a DataFrame given new data_manager""" if not inplace: - # TODO join the Index Metadata objects together for performance. - return DataFrame( - block_partitions=new_blocks, - columns=new_column_index, - index=new_index) + return DataFrame(data_manager=new_manager) else: - self._update_inplace( - block_partitions=new_blocks, - columns=new_column_index, - index=new_index) + self._update_inplace(new_manager=new_manager) - def _single_df_op_helper(self, func, other, axis, level): - if level is not None: - raise NotImplementedError("Multilevel index not yet supported " - "in Pandas on Ray") + def _validate_other(self, other, axis): + """Helper method to check validity of other in inter-df operations""" axis = pandas.DataFrame()._get_axis_number(axis) - if is_list_like(other): - new_index = self.index - new_column_index = self.columns - new_col_metadata = self._col_metadata - new_row_metadata = self._row_metadata - new_blocks = None - + if isinstance(other, DataFrame): + return other._data_manager + elif is_list_like(other): if axis == 0: if len(other) != len(self.index): raise ValueError( "Unable to coerce to Series, length must be {0}: " "given {1}".format(len(self.index), len(other))) - new_columns = _map_partitions(func, self._col_partitions) - new_rows = None else: if len(other) != len(self.columns): raise ValueError( "Unable to coerce to Series, length must be {0}: " "given {1}".format(len(self.columns), len(other))) - new_rows = _map_partitions(func, self._row_partitions) - new_columns = None + return other - else: - new_blocks = np.array([ - _map_partitions(func, block) - for block in self._block_partitions - ]) - new_columns = None - new_rows = None - new_index = self.index - new_column_index = self.columns - new_col_metadata = self._col_metadata - new_row_metadata = self._row_metadata - - return DataFrame( - col_partitions=new_columns, - row_partitions=new_rows, - block_partitions=new_blocks, - index=new_index, - columns=new_column_index, - col_metadata=new_col_metadata, - row_metadata=new_row_metadata) @ray.remote @@ -6120,61 +4469,3 @@ def _merge_columns(left_columns, right_columns, *args): return pandas.DataFrame(columns=left_columns, index=[0], dtype='uint8') \ .merge(pandas.DataFrame(columns=right_columns, index=[0], dtype='uint8'), *args).columns - - -@ray.remote -def _where_helper(left, cond, other, left_columns, cond_columns, other_columns, - *args): - - left = pandas.concat(ray.get(left.tolist()), axis=1, copy=False) - # We have to reset the index and columns here because we are coming - # from blocks and the axes are set according to the blocks. We have - # already correctly copartitioned everything, so there's no - # correctness problems with doing this. - left.reset_index(inplace=True, drop=True) - left.columns = left_columns - - cond = pandas.concat(ray.get(cond.tolist()), axis=1, copy=False) - cond.reset_index(inplace=True, drop=True) - cond.columns = cond_columns - - if isinstance(other, np.ndarray): - other = pandas.concat(ray.get(other.tolist()), axis=1, copy=False) - other.reset_index(inplace=True, drop=True) - other.columns = other_columns - - return left.where(cond, other, *args) - - -@ray.remote -def reindex_helper(old_index, new_index, axis, npartitions, method, fill_value, - limit, tolerance, *df): - df = pandas.concat(df, axis=axis ^ 1, copy=False) - if axis == 1: - df.index = old_index - else: - df.columns = old_index - - df = df.reindex( - new_index, - copy=False, - axis=axis ^ 1, - method=method, - fill_value=fill_value, - limit=limit, - tolerance=tolerance) - return _create_blocks_helper(df, npartitions, axis) - - -@ray.remote -def _equals_helper(left, right): - right = pandas.concat(ray.get(right.tolist()), axis=1, copy=False) - left = pandas.concat(ray.get(left.tolist()), axis=1, copy=False) - # Since we know that the index and columns match, we can just check the - # values. We can't use np.array_equal here because it doesn't recognize - # np.nan as equal to another np.nan - try: - assert_equal(left.values, right.values) - except AssertionError: - return False - return True diff --git a/modin/pandas/datetimes.py b/modin/pandas/datetimes.py index 77a03de607c..8fe2b38765b 100644 --- a/modin/pandas/datetimes.py +++ b/modin/pandas/datetimes.py @@ -4,10 +4,7 @@ import pandas -import ray - from .dataframe import DataFrame -from .utils import _map_partitions def to_datetime(arg, @@ -26,6 +23,7 @@ def to_datetime(arg, Args: errors ('raise' or 'ignore'): If 'ignore', errors are silenced. + Pandas blatantly ignores this argument so we will too. dayfirst (bool): Date format is passed in as day first. yearfirst (bool): Date format is passed in as year first. utc (bool): retuns a UTC DatetimeIndex if True. @@ -56,38 +54,20 @@ def to_datetime(arg, unit=unit, infer_datetime_format=infer_datetime_format, origin=origin) - if errors == 'raise': - pandas.to_datetime( - pandas.DataFrame(columns=arg.columns), - errors=errors, - dayfirst=dayfirst, - yearfirst=yearfirst, - utc=utc, - box=box, - format=format, - exact=exact, - unit=unit, - infer_datetime_format=infer_datetime_format, - origin=origin) - - def datetime_helper(df, cols): - df.columns = cols - return pandas.to_datetime( - df, - errors=errors, - dayfirst=dayfirst, - yearfirst=yearfirst, - utc=utc, - box=box, - format=format, - exact=exact, - unit=unit, - infer_datetime_format=infer_datetime_format, - origin=origin) - datetime_series = _map_partitions(datetime_helper, arg._row_partitions, - arg.columns) - result = pandas.concat(ray.get(datetime_series), copy=False) - result.index = arg.index + # Pandas seems to ignore this kwarg so we will too + #if errors == 'raise': + pandas.to_datetime( + pandas.DataFrame(columns=arg.columns), + errors=errors, + dayfirst=dayfirst, + yearfirst=yearfirst, + utc=utc, + box=box, + format=format, + exact=exact, + unit=unit, + infer_datetime_format=infer_datetime_format, + origin=origin) - return result + return arg._data_manager.to_datetime() diff --git a/modin/pandas/groupby.py b/modin/pandas/groupby.py index 1ce41c0fb10..4915e173737 100644 --- a/modin/pandas/groupby.py +++ b/modin/pandas/groupby.py @@ -7,12 +7,7 @@ from pandas.core.dtypes.common import is_list_like import pandas.core.common as com -import numpy as np -import ray - -from .concat import concat -from .index_metadata import _IndexMetadata -from .utils import _inherit_docstrings, _reindex_helper, post_task_gc +from .utils import _inherit_docstrings @_inherit_docstrings( @@ -25,25 +20,16 @@ class DataFrameGroupBy(object): def __init__(self, df, by, axis, level, as_index, sort, group_keys, squeeze, **kwargs): - self._columns = df.columns - self._index = df.index self._axis = axis - - self._df = df + self._data_manager = df._data_manager + self._index = self._data_manager.index + self._columns = self._data_manager.columns self._by = by self._level = level - self._as_index = as_index - self._sort = sort - self._group_keys = group_keys - self._squeeze = squeeze - - self._row_metadata = df._row_metadata - self._col_metadata = df._col_metadata - - if axis == 0: - self._partitions = df._block_partitions.T - else: - self._partitions = df._block_partitions + self._kwargs = {"sort": sort, + "as_index": as_index, + "group_keys": group_keys, + "squeeze": squeeze} def __getattr__(self, key): """Afer regular attribute access, looks up the name in the columns @@ -70,15 +56,9 @@ def __getattr__(self, key): def _index_grouped(self): if self._index_grouped_cache is None: if self._axis == 0: - self._index_grouped_cache = pandas.Series( - np.zeros(len(self._index), dtype=np.uint8), - index=self._index).groupby( - by=self._by, sort=self._sort) + self._index_grouped_cache = self._index.groupby(self._by) else: - self._index_grouped_cache = pandas.Series( - np.zeros(len(self._columns), dtype=np.uint8), - index=self._columns).groupby( - by=self._by, sort=self._sort) + self._index_grouped_cache = self._columns.groupby(self._by) return self._index_grouped_cache @@ -87,67 +67,32 @@ def _index_grouped(self): @property def _keys_and_values(self): if self._keys_and_values_cache is None: - self._keys_and_values_cache = \ - [(k, v) for k, v in self._index_grouped] + self._keys_and_values_cache = list(self._index_grouped.items()) + if self._sort: + self._keys_and_values_cache.sort() return self._keys_and_values_cache - @property - def _grouped_partitions(self): - - # It is expensive to put this multiple times, so let's just put it once - remote_by = ray.put(self._by) - remote_index = \ - [ray.put(v.index) for _, v in - self._df._col_metadata._coord_df.copy().groupby(by='partition')] \ - if self._axis == 0 \ - else [ray.put(v.index) for _, v in - self._df._row_metadata._coord_df.copy() - .groupby(by='partition')] - - if len(self._index_grouped) > 1: - return zip(*(groupby._submit( - args=(remote_index[i], remote_by, self._axis, self._level, - self._as_index, self._sort, self._group_keys, - self._squeeze) + tuple(part.tolist()), - num_return_vals=len(self._index_grouped)) - for i, part in enumerate(self._partitions))) - elif self._axis == 0: - return [self._df._col_partitions] - else: - return [self._df._row_partitions] - @property def _iter(self): from .dataframe import DataFrame if self._axis == 0: - return ((self._keys_and_values[i][0], - DataFrame( - col_partitions=part, - columns=self._columns, - index=self._keys_and_values[i][1].index, - col_metadata=self._col_metadata)) - for i, part in enumerate(self._grouped_partitions)) + return ((k, DataFrame(data_manager=self._data_manager.getitem_row_array(self._index_grouped[k]))) for k, _ in self._keys_and_values) else: - return ((self._keys_and_values[i][0], - DataFrame( - row_partitions=part, - columns=self._keys_and_values[i][1].index, - index=self._index, - row_metadata=self._row_metadata)) - for i, part in enumerate(self._grouped_partitions)) + return ((k, DataFrame(data_manager=self._data_manager.getitem_column_array(self._index_grouped[k]))) for k, _ in self._keys_and_values) @property def ngroups(self): return len(self) def skew(self, **kwargs): - return self._apply_agg_function( - lambda df: _skew_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.skew(**kwargs)) def ffill(self, limit=None): - return self._apply_df_function( - lambda df: df.ffill(axis=self._axis, limit=limit)) + raise NotImplementedError( + "To contribute to Pandas on Ray, please visit " + "github.com/modin-project/modin.") + return self._apply_agg_function(lambda df: df.ffill(limit=limit)) def sem(self, ddof=1): raise NotImplementedError( @@ -155,12 +100,10 @@ def sem(self, ddof=1): "github.com/modin-project/modin.") def mean(self, *args, **kwargs): - return self._apply_agg_function( - lambda df: _mean_remote.remote(df, self._axis, kwargs, *args)) + return self._apply_agg_function(lambda df: df.mean(*args, **kwargs)) def any(self): - return self._apply_agg_function( - lambda df: _any_remote.remote(df, self._axis)) + return self._apply_agg_function(lambda df: df.any()) @property def plot(self): @@ -186,11 +129,10 @@ def tshift(self): @property def groups(self): - return {k: pandas.Index(v) for k, v in self._keys_and_values} + return self._index_grouped def min(self, **kwargs): - return self._apply_agg_function( - lambda df: _min_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.min(**kwargs)) def idxmax(self): raise NotImplementedError( @@ -212,8 +154,7 @@ def nth(self, n, dropna=None): "github.com/modin-project/modin.") def cumsum(self, axis=0, *args, **kwargs): - return self._apply_df_function( - lambda df: df.cumsum(axis, *args, **kwargs)) + return self._apply_agg_function(lambda df: df.cumsum(axis, *args, **kwargs)) @property def indices(self): @@ -230,62 +171,16 @@ def filter(self, func, dropna=True, *args, **kwargs): "github.com/modin-project/modin.") def cummax(self, axis=0, **kwargs): - return self._apply_df_function(lambda df: df.cummax(axis, **kwargs)) + return self._apply_agg_function(lambda df: df.cummax(axis, **kwargs)) def apply(self, func, *args, **kwargs): - def apply_helper(df): - return df.apply(func, axis=self._axis, *args, **kwargs) - - result = [func(v) for k, v in self._iter] - if self._axis == 0: - if isinstance(result[0], pandas.Series): - # Applied an aggregation function - new_df = concat(result, axis=1).T - new_df.columns = self._columns - new_df.index = [k for k, v in self._iter] - else: - new_df = concat(result, axis=self._axis) - new_df._block_partitions = np.array([ - _reindex_helper._submit( - args=tuple([ - new_df.index, self._index, self._axis ^ 1, - len(new_df._block_partitions) - ] + block.tolist()), - num_return_vals=len(new_df._block_partitions)) - for block in new_df._block_partitions.T - ]).T - new_df.index = self._index - new_df._row_metadata = \ - _IndexMetadata(new_df._block_partitions[:, 0], - index=new_df.index, axis=0) - else: - if isinstance(result[0], pandas.Series): - # Applied an aggregation function - new_df = concat(result, axis=1) - new_df.columns = [k for k, v in self._iter] - new_df.index = self._index - else: - new_df = concat(result, axis=self._axis) - new_df._block_partitions = np.array([ - _reindex_helper._submit( - args=tuple([ - new_df.columns, self._columns, self._axis ^ 1, - new_df._block_partitions.shape[1] - ] + block.tolist()), - num_return_vals=new_df._block_partitions.shape[1]) - for block in new_df._block_partitions - ]) - new_df.columns = self._columns - new_df._col_metadata = \ - _IndexMetadata(new_df._block_partitions[0, :], - index=new_df.columns, axis=1) - return new_df + return self._apply_agg_function(lambda df: df.apply(func, *args, **kwargs)) @property def dtypes(self): if self._axis == 1: raise ValueError("Cannot call dtypes on groupby with axis=1") - return self._apply_agg_function(lambda df: _dtypes_remote.remote(df)) + return self._apply_agg_function(lambda df: df.dtypes) def first(self, **kwargs): raise NotImplementedError( @@ -302,12 +197,14 @@ def __getitem__(self, key): "github.com/modin-project/modin.") def cummin(self, axis=0, **kwargs): - return self._apply_df_function( + return self._apply_agg_function( lambda df: df.cummin(axis=axis, **kwargs)) def bfill(self, limit=None): - return self._apply_df_function( - lambda df: df.bfill(axis=self._axis, limit=limit)) + raise NotImplementedError( + "To contribute to Pandas on Ray, please visit " + "github.com/modin-project/modin.") + return self._apply_agg_function(lambda df: df.bfill(limit)) def idxmin(self): raise NotImplementedError( @@ -315,12 +212,10 @@ def idxmin(self): "github.com/modin-project/modin.") def prod(self, **kwargs): - return self._apply_agg_function( - lambda df: _prod_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.prod(**kwargs)) def std(self, ddof=1, *args, **kwargs): - return self._apply_agg_function( - lambda df: _std_remote.remote(df, self._axis, ddof, kwargs, *args)) + return self._apply_agg_function(lambda df: df.std(ddof, *args, **kwargs)) def aggregate(self, arg, *args, **kwargs): if self._axis != 0: @@ -333,8 +228,7 @@ def aggregate(self, arg, *args, **kwargs): "This requires Multi-level index to be implemented. " "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - return self._apply_agg_function( - lambda df: _agg_remote.remote(df, self._axis, arg, kwargs, *args)) + return self._apply_agg_function(lambda df: df.aggregate(arg, *args, **kwargs)) def last(self, **kwargs): raise NotImplementedError( @@ -347,7 +241,7 @@ def mad(self): "github.com/modin-project/modin.") def rank(self): - return self._apply_df_function(lambda df: df.rank(axis=self._axis)) + return self._apply_agg_function(lambda df: df.rank()) @property def corrwith(self): @@ -361,12 +255,10 @@ def pad(self, limit=None): "github.com/modin-project/modin.") def max(self, **kwargs): - return self._apply_agg_function( - lambda df: _max_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.max(**kwargs)) def var(self, ddof=1, *args, **kwargs): - return self._apply_agg_function( - lambda df: _var_remote.remote(df, self._axis, ddof, kwargs, *args)) + return self._apply_agg_function(lambda df: df.var(ddof, *args, **kwargs)) def get_group(self, name, obj=None): raise NotImplementedError( @@ -377,15 +269,13 @@ def __len__(self): return len(self._index_grouped) def all(self, **kwargs): - return self._apply_agg_function( - lambda df: _all_remote.remote(df, kwargs)) + return self._apply_agg_function(lambda df: df.all(**kwargs)) def size(self): - return self._apply_agg_function(lambda df: _size_remote.remote(df)) + return self._apply_agg_function(lambda df: df.size()) def sum(self, **kwargs): - return self._apply_agg_function( - lambda df: _sum_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.sum(**kwargs)) def __unicode__(self): raise NotImplementedError( @@ -413,11 +303,11 @@ def boxplot(self, "github.com/modin-project/modin.") def ngroup(self, ascending=True): - return self._index_grouped.ngroup(ascending) + index = self._index if not self._axis else self._columns + return pandas.Series(index=index).groupby(by=self._by, **self._kwargs).ngroup(ascending) def nunique(self, dropna=True): - return self._apply_agg_function( - lambda df: _nunique_remote.remote(df, self._axis, dropna)) + return self._apply_agg_function(lambda df: df.nunique(dropna)) def resample(self, rule, *args, **kwargs): raise NotImplementedError( @@ -425,8 +315,7 @@ def resample(self, rule, *args, **kwargs): "github.com/modin-project/modin.") def median(self, **kwargs): - return self._apply_agg_function( - lambda df: _median_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.median(**kwargs)) def head(self, n=5): raise NotImplementedError( @@ -434,8 +323,7 @@ def head(self, n=5): "github.com/modin-project/modin.") def cumprod(self, axis=0, *args, **kwargs): - return self._apply_df_function( - lambda df: df.cumprod(axis, *args, **kwargs)) + return self._apply_agg_function(lambda df: df.cumprod(axis, *args, **kwargs)) def __iter__(self): return self._iter.__iter__() @@ -449,7 +337,7 @@ def cov(self): "github.com/modin-project/modin.") def transform(self, func, *args, **kwargs): - return self._apply_df_function( + return self._apply_agg_function( lambda df: df.transform(func, *args, **kwargs)) def corr(self, **kwargs): @@ -458,12 +346,10 @@ def corr(self, **kwargs): "github.com/modin-project/modin.") def fillna(self, **kwargs): - return self._apply_df_function( - lambda df: df.fillna(axis=self._axis, **kwargs)) + return self._apply_agg_function(lambda df: df.fillna(**kwargs)) def count(self, **kwargs): - return self._apply_agg_function( - lambda df: _count_remote.remote(df, self._axis, kwargs)) + return self._apply_agg_function(lambda df: df.count(**kwargs)) def pipe(self, func, *args, **kwargs): return com._pipe(self, func, *args, **kwargs) @@ -502,8 +388,7 @@ def quantile(self, q=0.5, **kwargs): "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - return self._apply_agg_function( - lambda df: _quantile_remote.remote(df, self._axis, q, kwargs)) + return self._apply_agg_function(lambda df: df.quantile(q, **kwargs)) def diff(self): raise NotImplementedError( @@ -511,235 +396,20 @@ def diff(self): "github.com/modin-project/modin.") def take(self, **kwargs): - return self._apply_df_function(lambda df: df.take(**kwargs)) + raise NotImplementedError( + "To contribute to Pandas on Ray, please visit " + "github.com/modin-project/modin.") - def _apply_agg_function(self, f, index=None): + def _apply_agg_function(self, f, **kwargs): """Perform aggregation and combine stages based on a given function. Args: - f: The function to apply to each group. f must be a remote - function. + f: The function to apply to each group. Returns: A new combined DataFrame with the result of all groups. """ assert callable(f), "\'{0}\' object is not callable".format(type(f)) - - blocks = np.array([[f(part) for part in group_of_parts] - for group_of_parts in self._grouped_partitions]) - from .dataframe import DataFrame - if self._axis == 0: - return DataFrame( - block_partitions=blocks, - columns=self._columns, - index=index - if index is not None else [k for k, _ in self._index_grouped]) - else: - return DataFrame( - block_partitions=blocks.T, - index=self._index, - columns=index - if index is not None else [k for k, _ in self._index_grouped]) - - def _apply_df_function(self, f, concat_axis=None): - assert callable(f), "\'{0}\' object is not callable".format(type(f)) - - result = [f(v) for k, v in self._iter] - concat_axis = self._axis if concat_axis is None else concat_axis - - new_df = concat(result, axis=concat_axis) - - if self._axis == 0: - new_df._block_partitions = np.array([ - _reindex_helper._submit( - args=tuple([ - new_df.index, self._index, 1, - len(new_df._block_partitions) - ] + block.tolist()), - num_return_vals=len(new_df._block_partitions)) - for block in new_df._block_partitions.T - ]).T - new_df.index = self._index - new_df._row_metadata = \ - _IndexMetadata(new_df._block_partitions[:, 0], - index=new_df.index, axis=0) - else: - new_df._block_partitions = np.array([ - _reindex_helper._submit( - args=tuple([ - new_df.columns, self._columns, 0, new_df. - _block_partitions.shape[1] - ] + block.tolist()), - num_return_vals=new_df._block_partitions.shape[1]) - for block in new_df._block_partitions - ]) - new_df.columns = self._columns - new_df._col_metadata = \ - _IndexMetadata(new_df._block_partitions[0, :], - index=new_df.columns, axis=1) - - return new_df - - -@ray.remote -@post_task_gc -def groupby(index, by, axis, level, as_index, sort, group_keys, squeeze, *df): - - df = pandas.concat(df, axis=axis) - - if axis == 0: - df.columns = index - else: - df.index = index - return [ - v for k, v in df.groupby( - by=by, - axis=axis, - level=level, - as_index=as_index, - sort=sort, - group_keys=group_keys, - squeeze=squeeze) - ] - - -@ray.remote -def _sum_remote(df, axis, kwargs): - result = pandas.DataFrame(df.sum(axis=axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _skew_remote(df, axis, kwargs): - result = pandas.DataFrame(df.skew(axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _mean_remote(df, axis, kwargs, *args): - result = pandas.DataFrame(df.mean(axis, *args, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _any_remote(df, axis): - result = pandas.DataFrame(df.any(axis)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _min_remote(df, axis, kwargs): - result = pandas.DataFrame(df.min(axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _dtypes_remote(df): - return pandas.DataFrame(df.dtypes).T - - -@ray.remote -def _prod_remote(df, axis, kwargs): - result = pandas.DataFrame(df.prod(axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _std_remote(df, axis, ddof, kwargs, *args): - result = pandas.DataFrame(df.std(axis=axis, ddof=ddof, *args, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _max_remote(df, axis, kwargs): - result = pandas.DataFrame(df.max(axis=axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _var_remote(df, axis, ddof, kwargs, *args): - result = pandas.DataFrame(df.var(axis=axis, ddof=ddof, *args, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _all_remote(df, kwargs): - return pandas.DataFrame(df.all(**kwargs)).T - - -@ray.remote -def _size_remote(df): - return pandas.DataFrame(df.size).T - - -@ray.remote -def _nunique_remote(df, axis, dropna): - result = pandas.DataFrame(df.nunique(axis=axis, dropna=dropna)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _median_remote(df, axis, kwargs): - result = pandas.DataFrame(df.median(axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _count_remote(df, axis, kwargs): - result = pandas.DataFrame(df.count(axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _quantile_remote(df, axis, q, kwargs): - result = pandas.DataFrame(df.quantile(q=q, axis=axis, **kwargs)) - if axis == 0: - return result.T - else: - return result - - -@ray.remote -def _agg_remote(df, axis, arg, kwargs, *args): - result = pandas.DataFrame(df.agg(arg, axis=axis, *args, **kwargs)) - if axis == 0: - return result.T - else: - return result + new_manager = self._data_manager.groupby_agg(self._by, self._axis, f, self._kwargs, kwargs) + return DataFrame(data_manager=new_manager) diff --git a/modin/pandas/index/__init__.py b/modin/pandas/index/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/modin/pandas/index/partitioned_index.py b/modin/pandas/index/partitioned_index.py new file mode 100644 index 00000000000..211ed305268 --- /dev/null +++ b/modin/pandas/index/partitioned_index.py @@ -0,0 +1,29 @@ +from __future__ import absolute_import +from __future__ import division +from __future__ import print_function + + +class PartitionedIndex(object): + + _index_lengths_cache = None + + def _get_partition_lengths(self): + if self._index_lengths_cache is None: + self._index_lengths_cache = [obj.apply(len).get() for obj in self.index_partitions[: 0]] + return self._index_lengths_cache + + def _set_partition_lengths(self, new_value): + self._partition_length_cache = new_value + + index_lengths = property(_get_partition_lengths, _set_partition_lengths) + + def __getitem__(self, key): + cls = type(self) + return cls(self.index_partitions[key]) + + +class RayPartitionedIndex(PartitionedIndex): + + def __init__(self, index_partitions): + self.index_partitions = index_partitions + diff --git a/modin/pandas/index_metadata.py b/modin/pandas/index_metadata.py deleted file mode 100644 index ecdbdbcfab1..00000000000 --- a/modin/pandas/index_metadata.py +++ /dev/null @@ -1,436 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import pandas - -import numpy as np -import ray - -from .utils import (_build_row_lengths, _build_col_widths, _build_coord_df) - -from pandas.core.indexing import convert_to_index_sliceable - - -class _IndexMetadata(object): - """Wrapper for Pandas indexes in Ray DataFrames. Handles all of the - metadata specific to the axis of partition (setting indexes, - calculating the index within partition of a value, etc.). This - implementation assumes the underlying index lies across multiple - partitions. - - IMPORTANT NOTE: Currently all operations, as implemented, are inplace. - - WARNING: Currently, the `_lengths` item is the source of truth for an - _IndexMetadata object, since it is easy to manage, and that the coord_df - item may be deprecated in the future. As such, it is _very_ important that - any functions that mutate the coord_df splits in anyway first modify the - lengths. Otherwise bad things might happen! - """ - - def __init__(self, - dfs=None, - index=None, - axis=0, - lengths_oid=None, - coord_df_oid=None): - """Inits a IndexMetadata from Ray DataFrame partitions - - Args: - dfs ([ObjectID]): ObjectIDs of DataFrame partitions - index (pandas.Index): Index of the Ray DataFrame. - axis: Axis of partition (0=row partitions, 1=column partitions) - - Returns: - A IndexMetadata backed by the specified pandas.Index, partitioned - off specified partitions - """ - assert (lengths_oid is None) == (coord_df_oid is None), \ - "Must pass both or neither of lengths_oid and coord_df_oid" - - if dfs is not None and lengths_oid is None: - if axis == 0: - lengths_oid = _build_row_lengths.remote(dfs) - else: - lengths_oid = _build_col_widths.remote(dfs) - coord_df_oid = _build_coord_df.remote(lengths_oid, index) - - self._lengths = lengths_oid - self._coord_df = coord_df_oid - self._index_cache = index - self._cached_index = False - - def _get__lengths(self): - if isinstance(self._lengths_cache, ray.ObjectID) or \ - (isinstance(self._lengths_cache, list) and - isinstance(self._lengths_cache[0], ray.ObjectID)): - self._lengths_cache = ray.get(self._lengths_cache) - return self._lengths_cache - - def _set__lengths(self, lengths): - self._lengths_cache = lengths - - _lengths = property(_get__lengths, _set__lengths) - - def _get__coord_df(self): - """Get the coordinate DataFrame wrapped by this _IndexMetadata. - - Since we may have had an index set before our coord_df was - materialized, we'll have to apply it to the newly materialized df - """ - if isinstance(self._coord_df_cache, ray.ObjectID): - self._coord_df_cache = ray.get(self._coord_df_cache) - if self._cached_index: - self._coord_df_cache.index = self._index_cache - self._cached_index = False - return self._coord_df_cache - - def _set__coord_df(self, coord_df): - """Set the coordinate DataFrame wrapped by this _IndexMetadata. - - Sometimes we set the _IndexMetadata's coord_df outside of the - constructor, generally using fxns like drop(). This produces a modified - index, so we need to reflect the change on the index cache. - - If the set _IndexMetadata is an OID instead (due to a copy or whatever - reason), we fall back relying on `_index_cache`. - """ - if not isinstance(coord_df, ray.ObjectID): - self._index_cache = coord_df.index - self._coord_df_cache = coord_df - - _coord_df = property(_get__coord_df, _set__coord_df) - - def _get_index(self): - """Get the index wrapped by this _IndexMetadata. - - The only time `self._index_cache` would be None is in a newly created - _IndexMetadata object without a specified `index` parameter (See the - _IndexMetadata constructor for more details) - """ - if isinstance(self._coord_df_cache, ray.ObjectID): - return self._index_cache - else: - return self._coord_df_cache.index - - def _set_index(self, new_index): - """Set the index wrapped by this _IndexMetadata. - - It is important to always set `_index_cache` even if the coord_df is - materialized due to the possibility that it is set to an OID later on. - This design is more straightforward than caching indexes on setting the - coord_df to an OID due to the possibility of an OID-to-OID change. - """ - new_index = pandas.DataFrame(index=new_index).index - assert len(new_index) == len(self) - - self._index_cache = new_index - if isinstance(self._coord_df_cache, ray.ObjectID): - self._cached_index = True - else: - self._coord_df_cache.index = new_index - - index = property(_get_index, _set_index) - - def _get_index_cache(self): - """Get the cached Index object, which may sometimes be an OID. - - This will ray.get the Index object out of the Ray store lazily, such - that it is not grabbed until it is needed in the driver. This layer of - abstraction is important for allowing this object to be instantiated - with a remote Index object. - - Returns: - The Index object in _index_cache. - """ - if self._index_cache_validator is None: - self._index_cache_validator = pandas.RangeIndex(len(self)) - elif isinstance(self._index_cache_validator, ray.ObjectID): - self._index_cache_validator = ray.get(self._index_cache_validator) - - return self._index_cache_validator - - def _set_index_cache(self, new_index): - """Sets the new index cache. - - Args: - new_index: The Index to set the _index_cache to. - """ - self._index_cache_validator = new_index - - # _index_cache_validator is an extra layer of abstraction to allow the - # cache to accept ObjectIDs and ray.get them when needed. - _index_cache = property(_get_index_cache, _set_index_cache) - - def coords_of(self, key): - """Returns the coordinates (partition, index_within_partition) of the - provided key in the index. Can be called on its own or implicitly - through __getitem__ - - Args: - key: - item to get coordinates of. Can also be a tuple of item - and {"partition", "index_within_partition"} if caller only - needs one of the coordinates - - Returns: - Pandas object with the keys specified. If key is a single object - it will be a pandas.Series with items `partition` and - `index_within_partition`, and if key is a slice or if the key is - duplicate it will be a pandas.DataFrame with said items as columns. - """ - return self._coord_df.loc[key] - - def groupby(self, - by=None, - axis=0, - level=None, - as_index=True, - sort=True, - group_keys=True, - squeeze=False, - **kwargs): - # TODO: Find out what this does, and write a docstring - assignments_df = self._coord_df.groupby(by=by, axis=axis, level=level, - as_index=as_index, sort=sort, - group_keys=group_keys, - squeeze=squeeze, **kwargs)\ - .apply(lambda x: x[:]) - return assignments_df - - def partition_series(self, partition): - return self[self._coord_df['partition'] == - partition, 'index_within_partition'] - - def __len__(self): - return int(sum(self._lengths)) - - def reset_partition_coords(self, partitions=None): - partitions = np.array(partitions) - - for partition in partitions: - partition_mask = (self._coord_df['partition'] == partition) - # Since we are replacing columns with RangeIndex inside the - # partition, we have to make sure that our reference to it is - # upandasated as well. - try: - self._coord_df.loc[partition_mask, - 'index_within_partition'] = np.arange( - sum(partition_mask)).astype(int) - except ValueError: - # Copy the arrow sealed dataframe so we can mutate it. - # We only do this the first time we try to mutate the sealed. - self._coord_df = self._coord_df.copy() - self._coord_df.loc[partition_mask, - 'index_within_partition'] = np.arange( - sum(partition_mask)).astype(int) - - def insert(self, - key, - loc=None, - partition=None, - index_within_partition=None): - """Inserts a key at a certain location in the index, or a certain coord - in a partition. Called with either `loc` or `partition` and - `index_within_partition`. If called with both, `loc` will be used. - - Args: - key: item to insert into index - loc: location to insert into index - partition: partition to insert into - index_within_partition: index within partition to insert into - - Returns: - DataFrame with coordinates of insert - """ - # Perform insert on a specific partition - # Determine which partition to place it in, and where in that partition - if loc is not None: - cum_lens = np.cumsum(self._lengths) - if len(cum_lens) > 1: - partition = np.digitize(loc, cum_lens[:-1], right=True) - else: - partition = 0 - if partition >= len(cum_lens): - if loc > cum_lens[-1]: - raise IndexError("index {0} is out of bounds".format(loc)) - else: - index_within_partition = self._lengths[-1] - else: - first_in_partition = \ - np.asscalar(np.concatenate(([0], cum_lens))[partition]) - index_within_partition = loc - first_in_partition - - # TODO: Stop-gap solution until we begin passing IndexMetadatas - return partition, index_within_partition - - # Generate new index - new_index = self.index.insert(loc, key) - - # Shift indices in partition where we inserted column - idx_locs = (self._coord_df.partition == partition) & \ - (self._coord_df.index_within_partition == - index_within_partition) - # TODO: Determine why self._coord_df{,_cache} are read-only - _coord_df_copy = self._coord_df.copy() - _coord_df_copy.loc[idx_locs, 'index_within_partition'] += 1 - - # TODO: Determine if there's a better way to do a row-index insert in - # pandas, because this is very annoying/unsure of efficiency - # Create new coord entry to insert - coord_to_insert = pandas.DataFrame( - { - 'partition': partition, - 'index_within_partition': index_within_partition - }, - index=[key]) - - # Insert into cached RangeIndex, and order by new column index - self._coord_df = _coord_df_copy.append(coord_to_insert).loc[new_index] - - # Return inserted coordinate for callee - return coord_to_insert - - def get_global_indices(self, partition, index_within_partition_list): - total = 0 - for i in range(partition): - total += self._lengths[i] - - return [total + i for i in index_within_partition_list] - - def squeeze(self, partition, index_within_partition): - """Prepare a single coordinate for removal by "squeezing" the - subsequent coordinates "up" one index within that partition. To be used - with "_IndexMetadata.drop" for when all the "squeezed" coordinates are - dropped in batch. Note that this function doesn't actually mutate the - coord_df. - """ - self._coord_df = self._coord_df.copy() - - partition_mask = self._coord_df.partition == partition - index_within_partition_mask = \ - self._coord_df.index_within_partition > index_within_partition - self._coord_df.loc[partition_mask & index_within_partition_mask, - 'index_within_partition'] -= 1 - - def copy(self): - # TODO: Investigate copy-on-write wrapper for metadata objects - coord_df_copy = self._coord_df_cache - if not isinstance(self._coord_df_cache, ray.ObjectID): - coord_df_copy = self._coord_df_cache.copy() - - lengths_copy = self._lengths_cache - if not isinstance(self._lengths_cache, ray.ObjectID): - lengths_copy = self._lengths_cache.copy() - - index_copy = self._index_cache - if self._index_cache is not None: - index_copy = self._index_cache.copy() - - return _IndexMetadata( - index=index_copy, - coord_df_oid=coord_df_copy, - lengths_oid=lengths_copy) - - def __getitem__(self, key): - """Returns the coordinates (partition, index_within_partition) of the - provided key in the index. Essentially just an alias for - `_IndexMetadata.coords_of` that allows for slice passing, since - slices cannot be passed with slice notation other than through - `__getitem__` calls. - - Args: - key: - item to get coordinates of. Can also be a tuple of item - and {"partition", "index_within_partition"} if caller only - needs one of the coordinates - - Returns: - Pandas object with the keys specified. If key is a single object - it will be a pandas.Series with items `partition` and - `index_within_partition`, and if key is a slice or if the key is - duplicate it will be a pandas.DataFrame with said items as columns. - """ - return self.coords_of(key) - - def first_valid_index(self): - return self._coord_df.first_valid_index() - - def last_valid_index(self): - return self._coord_df.last_valid_index() - - def drop(self, labels, errors='raise'): - """Drop the specified labels from the IndexMetadata - - Args: - labels (scalar or list-like): - The labels to drop - errors ('raise' or 'ignore'): - If 'ignore', suppress errors for when labels don't exist - - Returns: - DataFrame with coordinates of dropped labels - """ - dropped = self.coords_of(labels) - - # Upandasate first lengths to prevent possible length inconsistencies - if isinstance(dropped, pandas.DataFrame): - try: - drop_per_part = dropped.groupby(["partition"]).size()\ - .reindex(index=pandas.RangeIndex(len(self._lengths)), - fill_value=0) - except ValueError: - # Copy the arrow sealed dataframe so we can mutate it. - dropped = dropped.copy() - drop_per_part = dropped.groupby(["partition"]).size()\ - .reindex(index=pandas.RangeIndex(len(self._lengths)), - fill_value=0) - elif isinstance(dropped, pandas.Series): - drop_per_part = np.zeros_like(self._lengths) - drop_per_part[dropped["partition"]] = 1 - else: - raise AssertionError("Unrecognized result from `coords_of`") - - self._lengths = self._lengths - np.array(drop_per_part) - - new_coord_df = self._coord_df.drop(labels, errors=errors) - - num_dropped = 0 - for i, length in enumerate(self._lengths): - if length == 0: - num_dropped += 1 - if num_dropped > 0: - new_coord_df['partition'][new_coord_df['partition'] == i] \ - -= num_dropped - - new_coord_df['index_within_partition'] = [ - i for l in self._lengths for i in range(l) - ] - - self._coord_df = new_coord_df - return dropped - - def rename_index(self, mapper): - """Rename the index. - - Args: - mapper: name to rename the index as - """ - self._coord_df = self._coord_df.rename_axis(mapper, axis=0) - - def convert_to_index_sliceable(self, key): - """Converts and performs error checking on the passed slice - - Args: - key: slice to convert and check - """ - return convert_to_index_sliceable(self._coord_df, key) - - def get_partition(self, partition_id): - """Return a view of coord_df where partition = partition_id - """ - return self._coord_df[self._coord_df.partition == partition_id] - - def sorted_index(self): - return (self._coord_df.sort_values( - ['partition', 'index_within_partition']).index) diff --git a/modin/pandas/indexing.py b/modin/pandas/indexing.py index a780619ca00..3793b3b5783 100644 --- a/modin/pandas/indexing.py +++ b/modin/pandas/indexing.py @@ -2,24 +2,19 @@ from __future__ import division from __future__ import print_function +import numpy as np import pandas from pandas.api.types import (is_scalar, is_list_like, is_bool) from pandas.core.dtypes.common import is_integer from pandas.core.indexing import IndexingError - -import numpy as np -import ray +from typing import Tuple from warnings import warn -from .utils import (_get_nan_block_id, extractor, _repartition_coord_df, - _generate_blocks, _mask_block_partitions, writer, - _blocks_to_series) -from .index_metadata import _IndexMetadata from .dataframe import DataFrame -from . import get_npartitions + """Indexing Helper Class works as follows: -_Location_Indexer_Base provide methods framework for __getitem__ +_LocationIndexerBase provide methods framework for __getitem__ and __setitem__ that work with Ray DataFrame's internal index. Base class's __{get,set}item__ takes in partitions & idx_in_partition data and perform lookup/item write. @@ -97,21 +92,23 @@ def _parse_tuple(tup): row_loc = tup ndim = _compute_ndim(row_loc, col_loc) - row_loc = [row_loc] if is_scalar(row_loc) else row_loc - col_loc = [col_loc] if is_scalar(col_loc) else col_loc + row_scaler = is_scalar(row_loc) + col_scaler = is_scalar(col_loc) + row_loc = [row_loc] if row_scaler else row_loc + col_loc = [col_loc] if col_scaler else col_loc - return row_loc, col_loc, ndim + return row_loc, col_loc, ndim, row_scaler, col_scaler -def _is_enlargement(locator, coord_df): - """Determine if a locator will enlarge the corrd_df. +def _is_enlargement(locator, global_index): + """Determine if a locator will enlarge the global index. Enlargement happens when you trying to locate using labels isn't in the original index. In other words, enlargement == adding NaNs ! """ if is_list_like(locator) and not is_slice( locator) and len(locator) > 0 and not is_boolean_array(locator): - n_diff_elems = len(pandas.Index(locator).difference(coord_df.index)) + n_diff_elems = len(pandas.Index(locator).difference(global_index)) is_enlargement_boolean = n_diff_elems > 0 return is_enlargement_boolean return False @@ -137,140 +134,38 @@ def _compute_ndim(row_loc, col_loc): return ndim -class _Location_Indexer_Base(object): +class _LocationIndexerBase(object): """Base class for location indexer like loc and iloc """ - def __init__(self, ray_df): - self.df = ray_df - self.col_coord_df = ray_df._col_metadata._coord_df - self.row_coord_df = ray_df._row_metadata._coord_df - self.block_oids = ray_df._block_partitions - - self.is_view = False - if isinstance(ray_df, DataFrameView): - self.block_oids = ray_df._block_partitions_data - self.is_view = True - - def __getitem__(self, row_lookup, col_lookup, ndim): - """ - Args: - row_lookup: A pandas DataFrame, a partial view from row_coord_df - col_lookup: A pandas DataFrame, a partial view from col_coord_df - ndim: the dimension of returned data - """ - if ndim == 2: - return self._generate_view_copy(row_lookup, col_lookup) - - extracted = self._retrive_items(row_lookup, col_lookup) - if ndim == 1: - result = ray.get(_blocks_to_series.remote(*extracted)).squeeze() - - if is_scalar(result): - result = pandas.Series(result) - - scaler_axis = row_lookup if len(row_lookup) == 1 else col_lookup - series_name = scaler_axis.iloc[0].name - result.name = series_name - - index_axis = row_lookup if len(col_lookup) == 1 else col_lookup - result.index = index_axis.index - - if ndim == 0: - result = ray.get(extracted[0]).squeeze() - - return result - - def _retrive_items(self, row_lookup, col_lookup): - """Given lookup dataframes, return a list of result oids - """ - result_oids = [] - - # We have to copy before we groupby because - # https://github.com/pandas-dev/pandas/issues/10043 - row_groups = row_lookup.copy().groupby('partition') - col_groups = col_lookup.copy().groupby('partition') - for row_blk, row_data in row_groups: - for col_blk, col_data in col_groups: - block_oid = self.block_oids[row_blk, col_blk] - row_idx = row_data['index_within_partition'] - col_idx = col_data['index_within_partition'] - - result_oid = extractor.remote(block_oid, row_idx, col_idx) - result_oids.append(result_oid) - return result_oids - - def _generate_view_copy(self, row_lookup, col_lookup): - """Generate a new DataFrame by making copies. - - Note (simon): - - This is a temporary replacement for _generate_view - function below. - """ - warn(_VIEW_IS_COPY_WARNING) - - row_lookup_new = _repartition_coord_df(row_lookup, get_npartitions()) - col_lookup_new = _repartition_coord_df(col_lookup, get_npartitions()) - - new_blocks = _generate_blocks(row_lookup, row_lookup_new, col_lookup, - col_lookup_new, self.block_oids) - - row_lengths_oid = ray.put(np.bincount(row_lookup_new['partition'])) - col_lengths_oid = ray.put(np.bincount(col_lookup_new['partition'])) - - new_row_metadata = _IndexMetadata( - coord_df_oid=row_lookup_new, lengths_oid=row_lengths_oid) - - new_col_metadata = _IndexMetadata( - coord_df_oid=col_lookup_new, lengths_oid=col_lengths_oid) - - df_view = DataFrame( - block_partitions=new_blocks, - row_metadata=new_row_metadata, - col_metadata=new_col_metadata, - index=row_lookup.index, - columns=col_lookup.index) - - return df_view - - def _generate_view(self, row_lookup, col_lookup): - """Generate a DataFrameView from lookup + def __init__(self, ray_df: DataFrame): + self.dm = ray_df._data_manager + self.is_view = hasattr(self.dm, "is_view") - Note (simon): - - This is not used because of index metadata was broken - """ - row_lengths = [0] * len(self.df._row_metadata._lengths) - for i in row_lookup["partition"]: - row_lengths[i] += 1 - col_lengths = [0] * len(self.df._col_metadata._lengths) - for i in col_lookup["partition"]: - col_lengths[i] += 1 - - row_lengths_oid = ray.put(np.array(row_lengths)) - col_lengths_oid = ray.put(np.array(col_lengths)) - - row_metadata_view = _IndexMetadata( - coord_df_oid=row_lookup, lengths_oid=row_lengths_oid) - - col_metadata_view = _IndexMetadata( - coord_df_oid=col_lookup, lengths_oid=col_lengths_oid) + self.row_scaler = False + self.col_scaler = False - df_view = DataFrameView( - block_partitions=self.block_oids, - row_metadata=row_metadata_view, - col_metadata=col_metadata_view, - index=row_metadata_view.index, - columns=col_metadata_view.index) + def __getitem__(self, row_lookup: pandas.Index, col_lookup: pandas.Index, ndim: int): + if self.is_view: + dm_view = self.dm.__constructor__(self.dm.data, row_lookup, col_lookup) + else: + dm_view = self.dm.view(row_lookup, col_lookup) - return df_view - - def __setitem__(self, row_lookup, col_lookup, item): + if ndim == 2: + return DataFrame(data_manager=dm_view) + elif ndim == 0: + return dm_view.squeeze(ndim=0) + else: + single_axis = 1 if self.col_scaler else 0 + return dm_view.squeeze(ndim=1, axis=single_axis) + + def __setitem__(self, row_lookup: pandas.Index, col_lookup: pandas.Index, item): """ Args: - row_lookup: A pandas DataFrame, a partial view from row_coord_df - col_lookup: A pandas DataFrame, a partial view from col_coord_df + row_lookup: the global row index to write item to + col_lookup: the global col index to write item to item: The new item needs to be set. It can be any shape that's - broadcastable to the product of the lookup tables. + broadcast-able to the product of the lookup tables. """ to_shape = (len(row_lookup), len(col_lookup)) item = self._broadcast_item(item, to_shape) @@ -280,7 +175,7 @@ def _broadcast_item(self, item, to_shape): """Use numpy to broadcast or reshape item. Notes: - - Numpy is memory efficent, there shouldn't be performance issue. + - Numpy is memory efficient, there shouldn't be performance issue. """ try: item = np.array(item) @@ -297,58 +192,27 @@ def _broadcast_item(self, item, to_shape): def _write_items(self, row_lookup, col_lookup, item): """Perform remote write and replace blocks. """ + row_numeric_idx = self.dm.global_idx_to_numeric_idx('row', row_lookup) + col_numeric_idx = self.dm.global_idx_to_numeric_idx('col', col_lookup) + self.dm.write_items(row_numeric_idx, col_numeric_idx, item) - # We have to copy before we groupby because - # https://github.com/pandas-dev/pandas/issues/10043 - row_groups = row_lookup.copy().groupby('partition') - col_groups = col_lookup.copy().groupby('partition') - - row_item_index = 0 - for row_blk, row_data in row_groups: - row_len = len(row_data) - - col_item_index = 0 - for col_blk, col_data in col_groups: - col_len = len(col_data) - block_oid = self.block_oids[row_blk, col_blk] - row_idx = row_data['index_within_partition'] - col_idx = col_data['index_within_partition'] - - item_to_write = item[row_item_index:row_item_index + - row_len, col_item_index:col_item_index + - col_len] - - result_oid = writer.remote(block_oid, row_idx, col_idx, - item_to_write) - - if self.is_view: - self.df._block_partitions_data[row_blk, - col_blk] = result_oid - else: - self.df._block_partitions[row_blk, col_blk] = result_oid - - col_item_index += col_len - row_item_index += row_len - - -class _Loc_Indexer(_Location_Indexer_Base): +class _LocIndexer(_LocationIndexerBase): """A indexer for ray_df.loc[] functionality""" def __getitem__(self, key): - row_loc, col_loc, ndim = _parse_tuple(key) + row_loc, col_loc, ndim, self.row_scaler, self.col_scaler = _parse_tuple(key) self._handle_enlargement(row_loc, col_loc) row_lookup, col_lookup = self._compute_lookup(row_loc, col_loc) ndim = self._expand_dim(row_lookup, col_lookup, ndim) - result = super(_Loc_Indexer, self).__getitem__(row_lookup, col_lookup, - ndim) + result = super(_LocIndexer, self).__getitem__(row_lookup, col_lookup, + ndim) return result def __setitem__(self, key, item): - row_loc, col_loc, _ = _parse_tuple(key) - self._handle_enlargement(row_loc, col_loc) + row_loc, col_loc, _, __, ___ = _parse_tuple(key) row_lookup, col_lookup = self._compute_lookup(row_loc, col_loc) - super(_Loc_Indexer, self).__setitem__(row_lookup, col_lookup, item) + super(_LocIndexer, self).__setitem__(row_lookup, col_lookup, item) def _handle_enlargement(self, row_loc, col_loc): """Handle Enlargement (if there is one). @@ -356,70 +220,12 @@ def _handle_enlargement(self, row_loc, col_loc): Returns: None """ - locators = [row_loc, col_loc] - coord_dfs = [self.row_coord_df, self.col_coord_df] - axis = ['row', 'col'] - metadata = {'row': self.df._row_metadata, 'col': self.df._col_metadata} - - for loc, coord, axis in zip(locators, coord_dfs, axis): - if _is_enlargement(loc, coord): - new_meta = self._enlarge_axis(loc, axis=axis) - _warn_enlargement() - metadata[axis] = new_meta - - self.row_coord_df = metadata['row']._coord_df - self.col_coord_df = metadata['col']._coord_df - - def _enlarge_axis(self, locator, axis): - """Add rows/columns to block partitions according to locator. - - Returns: - metadata (_IndexMetadata) - """ - # 1. Prepare variables - row_based_bool = axis == 'row' - # major == the axis of the locator - major_meta = self.df._row_metadata if row_based_bool \ - else self.df._col_metadata - minor_meta = self.df._col_metadata if row_based_bool \ - else self.df._row_metadata - - # 2. Compute the nan labels and add blocks - nan_labels = self._compute_enlarge_labels(locator, major_meta.index) - num_nan_labels = len(nan_labels) - blk_part_n_row, blk_part_n_col = self.block_oids.shape - - nan_blk_lens = minor_meta._lengths - nan_blks = np.array([[ - _get_nan_block_id( - num_nan_labels, n_cols, transpose=not row_based_bool) - for n_cols in nan_blk_lens - ]]) - nan_blks = nan_blks.T if not row_based_bool else nan_blks - - self.block_oids = np.concatenate([self.block_oids, nan_blks], - axis=0 if row_based_bool else 1) - - # 3. Prepare metadata to return - nan_coord_df = pandas.DataFrame(data=[ - { - '': name, - 'partition': - blk_part_n_row if row_based_bool else blk_part_n_col, - 'index_within_partition': i - } for name, i in zip(nan_labels, np.arange(num_nan_labels)) - ]).set_index('') - - coord_df = pandas.concat([major_meta._coord_df, nan_coord_df]) - coord_df = coord_df.loc[locator] # Re-index that allows duplicates - - lens = major_meta._lengths - lens = np.concatenate([lens, np.array([num_nan_labels])]) - lens_oid = ray.put(np.array(lens)) - - metadata_view = _IndexMetadata( - coord_df_oid=coord_df, lengths_oid=lens_oid) - return metadata_view + if _is_enlargement(row_loc, self.dm.index) or _is_enlargement(col_loc, self.dm.columns): + _warn_enlargement() + self.dm.enlarge_partitions( + new_row_labels=self._compute_enlarge_labels(row_loc, self.dm.index), + new_col_labels=self._compute_enlarge_labels(col_loc, self.dm.columns) + ) def _compute_enlarge_labels(self, locator, base_index): """Helper for _enlarge_axis, compute common labels and extra labels. @@ -457,39 +263,39 @@ def _expand_dim(self, row_lookup, col_lookup, ndim): return ndim - def _compute_lookup(self, row_loc, col_loc): - # We use reindex for list to avoid duplicates. - row_lookup = self.row_coord_df.loc[row_loc] - col_lookup = self.col_coord_df.loc[col_loc] + def _compute_lookup(self, row_loc, col_loc) -> Tuple[pandas.Index, pandas.Index]: + row_lookup = self.dm.index.to_series().loc[row_loc].index + col_lookup = self.dm.columns.to_series().loc[col_loc].index return row_lookup, col_lookup -class _iLoc_Indexer(_Location_Indexer_Base): +class _iLocIndexer(_LocationIndexerBase): """A indexer for ray_df.iloc[] functionality""" def __getitem__(self, key): - row_loc, col_loc, ndim = _parse_tuple(key) + row_loc, col_loc, ndim, self.row_scaler, self.col_scaler = _parse_tuple(key) self._check_dtypes(row_loc) self._check_dtypes(col_loc) row_lookup, col_lookup = self._compute_lookup(row_loc, col_loc) - result = super(_iLoc_Indexer, self).__getitem__( + result = super(_iLocIndexer, self).__getitem__( row_lookup, col_lookup, ndim) return result def __setitem__(self, key, item): - row_loc, col_loc, _ = _parse_tuple(key) + row_loc, col_loc, _, __, ___ = _parse_tuple(key) self._check_dtypes(row_loc) self._check_dtypes(col_loc) row_lookup, col_lookup = self._compute_lookup(row_loc, col_loc) - super(_iLoc_Indexer, self).__setitem__(row_lookup, col_lookup, item) + super(_iLocIndexer, self).__setitem__(row_lookup, col_lookup, item) - def _compute_lookup(self, row_loc, col_loc): - # We use reindex for list to avoid duplicates. - return self.row_coord_df.iloc[row_loc], self.col_coord_df.iloc[col_loc] + def _compute_lookup(self, row_loc, col_loc) -> Tuple[pandas.Index, pandas.Index]: + row_lookup = self.dm.index.to_series().iloc[row_loc].index + col_lookup = self.dm.columns.to_series().iloc[col_loc].index + return row_lookup, col_lookup def _check_dtypes(self, locator): is_int = is_integer(locator) @@ -499,29 +305,3 @@ def _check_dtypes(self, locator): if not any([is_int, is_int_slice, is_int_list, is_bool_arr]): raise ValueError(_ILOC_INT_ONLY_ERROR) - - -class DataFrameView(DataFrame): - """A subclass of DataFrame where the index can be smaller than blocks. - - Deprecated because _generate_view_copy is used instead of _generate_view - """ - - def __init__(self, block_partitions, row_metadata, col_metadata, index, - columns): - self._block_partitions = block_partitions - self._row_metadata = row_metadata - self._col_metadata = col_metadata - self.index = index - self.columns = columns - - def _get_block_partitions(self): - oid_arr = _mask_block_partitions(self._block_partitions_data, - self._row_metadata, - self._col_metadata) - return oid_arr - - def _set_block_partitions(self, new_block_partitions): - self._block_partitions_data = new_block_partitions - - _block_partitions = property(_get_block_partitions, _set_block_partitions) diff --git a/modin/pandas/io.py b/modin/pandas/io.py index 97406892af3..4413aba62db 100644 --- a/modin/pandas/io.py +++ b/modin/pandas/io.py @@ -5,18 +5,20 @@ import pandas from pandas.io.common import _infer_compression +import inspect from io import BytesIO import os import py -from pyarrow.parquet import ParquetFile -import pyarrow.parquet as pq import re import warnings import numpy as np from .dataframe import ray, DataFrame -from . import get_npartitions -from .utils import from_pandas, _partition_pandas_dataframe +from .utils import from_pandas +from ..data_management.partitioning.partition_collections import RayBlockPartitions +from ..data_management.partitioning.remote_partition import RayRemotePartition +from ..data_management.partitioning.axis_partition import split_result_of_axis_func_pandas +from ..data_management.data_manager import PandasDataManager PQ_INDEX_REGEX = re.compile('__index_level_\d+__') @@ -37,33 +39,42 @@ def read_parquet(path, engine='auto', columns=None, **kwargs): ParquetFile API is used. Please refer to the documentation here https://arrow.apache.org/docs/python/parquet.html """ + return _read_parquet_pandas_on_ray(path, engine, columns, **kwargs) + + +def _read_parquet_pandas_on_ray(path, engine, columns, **kwargs): + from pyarrow.parquet import ParquetFile + if not columns: pf = ParquetFile(path) columns = [ name for name in pf.metadata.schema.names if not PQ_INDEX_REGEX.match(name) ] - + num_splits = min(len(columns), RayBlockPartitions._compute_num_partitions()) # Each item in this list will be a column of original df # partitioned to smaller pieces along rows. # We need to transpose the oids array to fit our schema. - blk_partitions = [ - ray.get(_read_parquet_column.remote(path, col, kwargs)) - for col in columns - ] - blk_partitions = np.array(blk_partitions).T - - return DataFrame(block_partitions=blk_partitions, columns=columns) + blk_partitions = np.array( + [_read_parquet_column._submit(args=(path, col, num_splits, kwargs), + num_return_vals=num_splits + 1) + for col in columns]).T + remote_partitions = np.array([[RayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) + index_len = ray.get(blk_partitions[-1][0]) + index = pandas.RangeIndex(index_len) + new_manager = PandasDataManager(RayBlockPartitions(remote_partitions), index, columns) + df = DataFrame(data_manager=new_manager) + return df # CSV def _skip_header(f, kwargs={}): lines_read = 0 - comment = kwargs["comment"] - skiprows = kwargs["skiprows"] - encoding = kwargs["encoding"] - header = kwargs["header"] - names = kwargs["names"] + comment = kwargs.get("comment", None) + skiprows = kwargs.get("skiprows", None) + encoding = kwargs.get("encoding", None) + header = kwargs.get("header", "infer") + names = kwargs.get("names", None) if header is None: return lines_read @@ -106,7 +117,7 @@ def _skip_header(f, kwargs={}): return lines_read -def _read_csv_from_file(filepath, npartitions, kwargs={}): +def _read_csv_from_file_pandas_on_ray(filepath, kwargs={}): """Constructs a DataFrame from a CSV file. Args: @@ -119,17 +130,16 @@ def _read_csv_from_file(filepath, npartitions, kwargs={}): """ empty_pd_df = pandas.read_csv( filepath, **dict(kwargs, nrows=0, skipfooter=0, skip_footer=0)) - names = empty_pd_df.columns + column_names = empty_pd_df.columns - skipfooter = kwargs["skipfooter"] - skip_footer = kwargs["skip_footer"] + skipfooter = kwargs.get("skipfooter", None) or kwargs.get("skip_footer", None) partition_kwargs = dict( - kwargs, header=None, names=names, skipfooter=0, skip_footer=0) + kwargs, header=None, names=column_names, skipfooter=0, skip_footer=0) with open(filepath, "rb") as f: # Get the BOM if necessary prefix = b"" - if kwargs["encoding"] is not None: + if kwargs.get("encoding", None) is not None: prefix = f.readline() partition_kwargs["skiprows"] = 1 f.seek(0, os.SEEK_SET) # Return to beginning of file @@ -144,33 +154,31 @@ def _read_csv_from_file(filepath, npartitions, kwargs={}): partition_ids = [] index_ids = [] total_bytes = os.path.getsize(filepath) - chunk_size = max(1, (total_bytes - f.tell()) // npartitions) + num_splits = min(len(column_names), RayBlockPartitions._compute_num_partitions()) + chunk_size = max(1, (total_bytes - f.tell()) // num_splits) + while f.tell() < total_bytes: start = f.tell() f.seek(chunk_size, os.SEEK_CUR) f.readline() # Read a whole number of lines - if f.tell() >= total_bytes: - kwargs["skipfooter"] = skipfooter - kwargs["skip_footer"] = skip_footer + partition_id = _read_csv_with_offset_pandas_on_ray._submit(args=(filepath, num_splits, start, f.tell(), partition_kwargs_id, prefix_id), num_return_vals=num_splits + 1) + partition_ids.append([RayRemotePartition(obj) for obj in partition_id[:-1]]) + index_ids.append(partition_id[-1]) - partition_id, index_id = _read_csv_with_offset._submit( - args=(filepath, start, f.tell(), partition_kwargs_id, - prefix_id), - num_return_vals=2) - partition_ids.append(partition_id) - index_ids.append(index_id) + index_col = kwargs.get("index_col", None) + if index_col is None: + new_index = pandas.RangeIndex(sum(ray.get(index_ids))) + else: + new_index_ids = get_index.remote([empty_pd_df.index.name], *index_ids) + new_index = ray.get(new_index_ids) - # Construct index - index_id = get_index.remote([empty_pd_df.index.name], *index_ids) \ - if kwargs["index_col"] is not None else None + new_manager = PandasDataManager(RayBlockPartitions(np.array(partition_ids)), new_index, column_names) + df = DataFrame(data_manager=new_manager) - df = DataFrame(row_partitions=partition_ids, columns=names, index=index_id) - - skipfooter = kwargs["skipfooter"] or kwargs["skip_footer"] if skipfooter: df = df.drop(df.index[-skipfooter:]) - if kwargs["squeeze"] and len(df.columns) == 1: + if kwargs.get("squeeze", False) and len(df.columns) == 1: return df[df.columns[0]] return df @@ -180,14 +188,13 @@ def _read_csv_from_pandas(filepath_or_buffer, kwargs): pd_obj = pandas.read_csv(filepath_or_buffer, **kwargs) if isinstance(pd_obj, pandas.DataFrame): - return from_pandas(pd_obj, get_npartitions()) + return from_pandas(pd_obj) elif isinstance(pd_obj, pandas.io.parsers.TextFileReader): # Overwriting the read method should return a ray DataFrame for calls # to __next__ and get_chunk pd_read = pd_obj.read pd_obj.read = lambda *args, **kwargs: \ - from_pandas(pd_read(*args, **kwargs), get_npartitions()) - + from_pandas(pd_read(*args, **kwargs)) return pd_obj @@ -252,69 +259,23 @@ def read_csv(filepath_or_buffer, We only support local files for now. kwargs: Keyword arguments in pandas::from_csv """ - - kwargs = { - 'sep': sep, - 'delimiter': delimiter, - 'header': header, - 'names': names, - 'index_col': index_col, - 'usecols': usecols, - 'squeeze': squeeze, - 'prefix': prefix, - 'mangle_dupe_cols': mangle_dupe_cols, - 'dtype': dtype, - 'engine': engine, - 'converters': converters, - 'true_values': true_values, - 'false_values': false_values, - 'skipinitialspace': skipinitialspace, - 'skiprows': skiprows, - 'nrows': nrows, - 'na_values': na_values, - 'keep_default_na': keep_default_na, - 'na_filter': na_filter, - 'verbose': verbose, - 'skip_blank_lines': skip_blank_lines, - 'parse_dates': parse_dates, - 'infer_datetime_format': infer_datetime_format, - 'keep_date_col': keep_date_col, - 'date_parser': date_parser, - 'dayfirst': dayfirst, - 'iterator': iterator, - 'chunksize': chunksize, - 'compression': compression, - 'thousands': thousands, - 'decimal': decimal, - 'lineterminator': lineterminator, - 'quotechar': quotechar, - 'quoting': quoting, - 'escapechar': escapechar, - 'comment': comment, - 'encoding': encoding, - 'dialect': dialect, - 'tupleize_cols': tupleize_cols, - 'error_bad_lines': error_bad_lines, - 'warn_bad_lines': warn_bad_lines, - 'skipfooter': skipfooter, - 'skip_footer': skip_footer, - 'doublequote': doublequote, - 'delim_whitespace': delim_whitespace, - 'as_recarray': as_recarray, - 'compact_ints': compact_ints, - 'use_unsigned': use_unsigned, - 'low_memory': low_memory, - 'buffer_lines': buffer_lines, - 'memory_map': memory_map, - 'float_precision': float_precision, - } + # The intention of the inspection code is to reduce the amount of + # communication we have to do between processes and nodes. We take a quick + # pass over the arguments and remove those that are default values so we + # don't have to serialize and send them to the workers. Because the + # arguments list is so long, this does end up saving time based on the + # number of nodes in the cluster. + frame = inspect.currentframe() + _, _, _, kwargs = inspect.getargvalues(frame) + args, _, _, defaults, _, _, _ = inspect.getfullargspec(read_csv) + defaults = dict(zip(args[1:], defaults)) + kwargs = {kw: kwargs[kw] for kw in kwargs if kw in defaults and kwargs[kw] != defaults[kw]} if isinstance(filepath_or_buffer, str): if not os.path.exists(filepath_or_buffer): warnings.warn(("File not found on disk. " "Defaulting to Pandas implementation."), PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) elif not isinstance(filepath_or_buffer, py.path.local): read_from_pandas = True @@ -331,27 +292,23 @@ def read_csv(filepath_or_buffer, warnings.warn(("Reading from buffer. " "Defaulting to Pandas implementation."), PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) if _infer_compression(filepath_or_buffer, compression) is not None: warnings.warn(("Compression detected. " "Defaulting to Pandas implementation."), PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) if as_recarray: warnings.warn("Defaulting to Pandas implementation.", PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) if chunksize is not None: warnings.warn(("Reading chunks from a file. " "Defaulting to Pandas implementation."), PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) if skiprows is not None and not isinstance(skiprows, int): @@ -365,10 +322,9 @@ def read_csv(filepath_or_buffer, if nrows is not None: warnings.warn("Defaulting to Pandas implementation.", PendingDeprecationWarning) - return _read_csv_from_pandas(filepath_or_buffer, kwargs) - return _read_csv_from_file(filepath_or_buffer, get_npartitions(), kwargs) + return _read_csv_from_file_pandas_on_ray(filepath_or_buffer, kwargs) def read_json(path_or_buf=None, @@ -393,7 +349,7 @@ def read_json(path_or_buf=None, path_or_buf, orient, typ, dtype, convert_axes, convert_dates, keep_default_dates, numpy, precise_float, date_unit, encoding, lines, chunksize, compression) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -421,7 +377,7 @@ def read_html(io, skiprows, attrs, parse_dates, tupleize_cols, thousands, encoding, decimal, converters, na_values, keep_default_na) - ray_frame = from_pandas(port_frame[0], get_npartitions()) + ray_frame = from_pandas(port_frame[0]) return ray_frame @@ -432,7 +388,7 @@ def read_clipboard(sep=r'\s+'): PendingDeprecationWarning) port_frame = pandas.read_clipboard(sep) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -464,7 +420,7 @@ def read_excel(io, io, sheet_name, header, skiprows, skip_footer, index_col, names, usecols, parse_dates, date_parser, na_values, thousands, convert_float, converters, dtype, true_values, false_values, engine, squeeze) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -475,7 +431,7 @@ def read_hdf(path_or_buf, key=None, mode='r'): PendingDeprecationWarning) port_frame = pandas.read_hdf(path_or_buf, key, mode) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -486,7 +442,7 @@ def read_feather(path, nthreads=1): PendingDeprecationWarning) port_frame = pandas.read_feather(path) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -497,7 +453,7 @@ def read_msgpack(path_or_buf, encoding='utf-8', iterator=False): PendingDeprecationWarning) port_frame = pandas.read_msgpack(path_or_buf, encoding, iterator) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -521,7 +477,7 @@ def read_stata(filepath_or_buffer, convert_categoricals, encoding, index_col, convert_missing, preserve_dtypes, columns, order_categoricals, chunksize, iterator) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -538,7 +494,7 @@ def read_sas(filepath_or_buffer, port_frame = pandas.read_sas(filepath_or_buffer, format, index, encoding, chunksize, iterator) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -549,7 +505,7 @@ def read_pickle(path, compression='infer'): PendingDeprecationWarning) port_frame = pandas.read_pickle(path, compression) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -568,7 +524,7 @@ def read_sql(sql, port_frame = pandas.read_sql(sql, con, index_col, coerce_float, params, parse_dates, columns, chunksize) - ray_frame = from_pandas(port_frame, get_npartitions()) + ray_frame = from_pandas(port_frame) return ray_frame @@ -581,20 +537,27 @@ def get_index(index_name, *partition_indices): @ray.remote -def _read_csv_with_offset(fn, start, end, kwargs={}, header=b''): - bio = open(fn, 'rb') +def _read_csv_with_offset_pandas_on_ray(fname, num_splits, start, end, kwargs, header): + bio = open(fname, 'rb') bio.seek(start) to_read = header + bio.read(end - start) bio.close() pandas_df = pandas.read_csv(BytesIO(to_read), **kwargs) - index = pandas_df.index - # Partitions must have RangeIndex - pandas_df.index = pandas.RangeIndex(0, len(pandas_df)) - return pandas_df, index + if kwargs.get("index_col", None) is not None: + index = pandas_df.index + # Partitions must have RangeIndex + pandas_df.index = pandas.RangeIndex(0, len(pandas_df)) + else: + # We will use the lengths to build the index if we are not given an + # `index_col`. + index = len(pandas_df) + + return split_result_of_axis_func_pandas(1, num_splits, pandas_df) + [index] @ray.remote -def _read_parquet_column(path, column, kwargs={}): +def _read_parquet_column(path, column, num_splits, kwargs={}): + import pyarrow.parquet as pq df = pq.read_pandas(path, columns=[column], **kwargs).to_pandas() - oids = _partition_pandas_dataframe(df, num_partitions=get_npartitions()) - return oids + # Append the length of the index here to build it externally + return split_result_of_axis_func_pandas(0, num_splits, df) + [len(df.index)] diff --git a/modin/pandas/iterator.py b/modin/pandas/iterator.py index 78a2af5e6b1..3743d605e05 100644 --- a/modin/pandas/iterator.py +++ b/modin/pandas/iterator.py @@ -6,17 +6,19 @@ class PartitionIterator(Iterator): - def __init__(self, partitions, func): + def __init__(self, data_manager, axis, func): """PartitionIterator class to define a generator on partitioned data Args: - partitions ([ObjectID]): Partitions to iterate over + data_manager (DataManager): Data manager for the dataframe + axis (int): axis to iterate over func (callable): The function to get inner iterables from each partition """ - self.partitions = iter(partitions) + self.data_manager = data_manager + self.axis = axis + self.index_iter = iter(self.data_manager.columns) if axis else iter(self.data_manager.index) self.func = func - self.iter_cache = iter([]) def __iter__(self): return self @@ -25,9 +27,11 @@ def __next__(self): return self.next() def next(self): - try: - return next(self.iter_cache) - except StopIteration: - next_partition = next(self.partitions) - self.iter_cache = self.func(next_partition) - return self.next() + if self.axis: + key = next(self.index_iter) + df = self.data_manager.getitem_column_array([key]).to_pandas() + else: + key = next(self.index_iter) + df = self.data_manager.getitem_row_array([key]).to_pandas() + return next(self.func(df)) + diff --git a/modin/pandas/pandas_code_gen.py b/modin/pandas/pandas_code_gen.py deleted file mode 100644 index e5fdf91d573..00000000000 --- a/modin/pandas/pandas_code_gen.py +++ /dev/null @@ -1,93 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -import inspect - - -def code_gen(pandas_obj, ray_obj, path): - """Generate code skeleton for methods not in Ray - - Args: - pandas_obj: The pandas object to generate code from. - ray_obj: The ray object to diff against. - path: Path to output the file to. - """ - - with open(path, "w") as outfile: - funcs = pandas_ray_diff(pandas_obj, ray_obj) - - for func in funcs: - if func[0] == "_" and func[1] != "_": - continue - if "attr" in func: - # let's not mess with these - continue - try: - outfile.write("\ndef " + func + str( - inspect.signature(getattr(pandas_obj, func))) + ":\n") - - except TypeError: - outfile.write("\n@property") - outfile.write("\ndef " + func + "(self):\n") - except ValueError: - continue - outfile.write( - " raise NotImplementedError(\"Not Yet implemented.\")\n") - - -def code_gen_test(ray_obj, path, name): - """Generate tests for methods in Ray.""" - - with open(path, "a") as outfile: - funcs = dir(ray_obj) - - for func in funcs: - if func[0] == "_" and func[1] != "_": - continue - - outfile.write("\n\ndef test_" + func + "():\n") - outfile.write(" ray_" + name + " = create_test_" + name + - "()\n\n" + - " with pytest.raises(NotImplementedError):\n" + - " ray_" + name + "." + func) - try: - first = True - param_num = \ - len(inspect.signature(getattr(ray_obj, func)).parameters) - if param_num > 1: - param_num -= 1 - - for _ in range(param_num): - if first: - outfile.write("(None") - first = False - else: - outfile.write(", None") - except (TypeError, ValueError, NotImplementedError): - outfile.write("\n") - continue - - if first: - outfile.write("(") - outfile.write(")\n") - - -def pandas_ray_diff(pandas_obj, ray_obj): - """Gets the diff of the methods in the Pandas and Ray objects. - - Args: - pandas_obj: The Pandas object to diff. - ray_obj: The Ray object to diff. - - Returns: - A list of method names that are different between the two. - """ - pandas_funcs = dir(pandas_obj) - ray_funcs = dir(ray_obj) - - pandas_funcs = set( - filter(lambda f: f[0] != "_" or f[1] == "_", pandas_funcs)) - - diff = [x for x in pandas_funcs if x not in set(ray_funcs)] - return diff diff --git a/modin/pandas/reshape.py b/modin/pandas/reshape.py index 2fbd0d2f418..529f07fe875 100644 --- a/modin/pandas/reshape.py +++ b/modin/pandas/reshape.py @@ -3,15 +3,9 @@ from __future__ import print_function import pandas -from pandas import compat from pandas.core.dtypes.common import is_list_like -from itertools import cycle -import ray -import numpy as np - from .dataframe import DataFrame -from .utils import _deploy_func def get_dummies(data, @@ -36,106 +30,31 @@ def get_dummies(data, Returns: DataFrame or one-hot encoded data. """ - if not isinstance(data, DataFrame): - return pandas.get_dummies( - data, - prefix=prefix, - prefix_sep=prefix_sep, - dummy_na=dummy_na, - columns=columns, - sparse=sparse, - drop_first=drop_first) - if sparse: raise NotImplementedError( "SparseDataFrame is not implemented. " "To contribute to Pandas on Ray, please visit " "github.com/modin-project/modin.") - if columns is None: - columns_to_encode = data.dtypes.isin([np.dtype("O"), 'category']) - columns_to_encode = data.columns[columns_to_encode] - else: - columns_to_encode = columns - - def check_len(item, name): - len_msg = ("Length of '{name}' ({len_item}) did not match the " - "length of the columns being encoded ({len_enc}).") - - if is_list_like(item): - if not len(item) == len(columns_to_encode): - len_msg = len_msg.format( - name=name, - len_item=len(item), - len_enc=len(columns_to_encode)) - raise ValueError(len_msg) - - check_len(prefix, 'prefix') - check_len(prefix_sep, 'prefix_sep') - if isinstance(prefix, compat.string_types): - prefix = cycle([prefix]) - prefix = [next(prefix) for i in range(len(columns_to_encode))] - if isinstance(prefix, dict): - prefix = [prefix[col] for col in columns_to_encode] - - if prefix is None: - prefix = columns_to_encode - - # validate separators - if isinstance(prefix_sep, compat.string_types): - prefix_sep = cycle([prefix_sep]) - prefix_sep = [next(prefix_sep) for i in range(len(columns_to_encode))] - elif isinstance(prefix_sep, dict): - prefix_sep = [prefix_sep[col] for col in columns_to_encode] - - if set(columns_to_encode) == set(data.columns): - with_dummies = [] - dropped_columns = pandas.Index() - else: - with_dummies = data.drop(columns_to_encode, axis=1)._col_partitions - dropped_columns = data.columns.drop(columns_to_encode) - - def get_dummies_remote(df, to_drop, prefix, prefix_sep): - df = df.drop(to_drop, axis=1) - - if df.size == 0: - return df, df.columns - - df = pandas.get_dummies( - df, + if not isinstance(data, DataFrame): + return pandas.get_dummies( + data, prefix=prefix, prefix_sep=prefix_sep, dummy_na=dummy_na, - columns=None, + columns=columns, sparse=sparse, drop_first=drop_first) - columns = df.columns - df.columns = pandas.RangeIndex(0, len(df.columns)) - return df, columns - - total = 0 - columns = [] - for i, part in enumerate(data._col_partitions): - col_index = data._col_metadata.partition_series(i) - - # TODO(kunalgosar): Handle the case of duplicate columns here - to_encode = col_index.index.isin(columns_to_encode) - - to_encode = col_index[to_encode] - to_drop = col_index.drop(to_encode.index) - - result = _deploy_func._submit( - args=(get_dummies_remote, part, to_drop, - prefix[total:total + len(to_encode)], - prefix_sep[total:total + len(to_encode)]), - num_return_vals=2) - with_dummies.append(result[0]) - columns.append(result[1]) - total += len(to_encode) + if isinstance(data, DataFrame): + df = data + elif is_list_like(data): + df = DataFrame(data) - columns = ray.get(columns) - dropped_columns = dropped_columns.append(columns) + new_manager = df._data_manager.get_dummies(columns, + prefix=prefix, + prefix_sep=prefix_sep, + dummy_na=dummy_na, + drop_first=drop_first) - return DataFrame( - col_partitions=with_dummies, columns=dropped_columns, index=data.index) + return DataFrame(data_manager=new_manager) diff --git a/modin/pandas/test/test_concat.py b/modin/pandas/test/test_concat.py index 33d08ed43d9..dab13e8877d 100644 --- a/modin/pandas/test/test_concat.py +++ b/modin/pandas/test/test_concat.py @@ -5,7 +5,7 @@ import pytest import pandas import modin.pandas as pd -from modin.pandas.utils import (to_pandas, from_pandas) +from modin.pandas.utils import from_pandas, to_pandas @pytest.fixture @@ -63,15 +63,27 @@ def test_df_concat(): def test_ray_concat(): df, df2 = generate_dfs() - ray_df, ray_df2 = from_pandas(df, 2), from_pandas(df2, 2) + ray_df, ray_df2 = from_pandas(df), from_pandas(df2) assert ray_df_equals_pandas( pd.concat([ray_df, ray_df2]), pandas.concat([df, df2])) +def test_ray_concat_with_series(): + df, df2 = generate_dfs() + ray_df, ray_df2 = from_pandas(df), from_pandas(df2) + pandas_series = pandas.Series([1,2,3,4], name="new_col") + + assert ray_df_equals_pandas( + pd.concat([ray_df, ray_df2, pandas_series], axis=0), pandas.concat([df, + df2, pandas_series], axis=0)) + + assert ray_df_equals_pandas( + pd.concat([ray_df, ray_df2, pandas_series], axis=1), pandas.concat([df, + df2, pandas_series], axis=1)) def test_ray_concat_on_index(): df, df2 = generate_dfs() - ray_df, ray_df2 = from_pandas(df, 2), from_pandas(df2, 2) + ray_df, ray_df2 = from_pandas(df), from_pandas(df2) assert ray_df_equals_pandas( pd.concat([ray_df, ray_df2], axis='index'), @@ -87,7 +99,7 @@ def test_ray_concat_on_index(): def test_ray_concat_on_column(): df, df2 = generate_dfs() - ray_df, ray_df2 = from_pandas(df, 2), from_pandas(df2, 2) + ray_df, ray_df2 = from_pandas(df), from_pandas(df2) assert ray_df_equals_pandas( pd.concat([ray_df, ray_df2], axis=1), pandas.concat([df, df2], axis=1)) @@ -99,7 +111,7 @@ def test_ray_concat_on_column(): def test_invalid_axis_errors(): df, df2 = generate_dfs() - ray_df, ray_df2 = from_pandas(df, 2), from_pandas(df2, 2) + ray_df, ray_df2 = from_pandas(df), from_pandas(df2) with pytest.raises(ValueError): pd.concat([ray_df, ray_df2], axis=2) @@ -109,7 +121,7 @@ def test_mixed_concat(): df, df2 = generate_dfs() df3 = df.copy() - mixed_dfs = [from_pandas(df, 2), from_pandas(df2, 2), df3] + mixed_dfs = [from_pandas(df), from_pandas(df2), df3] assert (ray_df_equals_pandas( pd.concat(mixed_dfs), pandas.concat([df, df2, df3]))) @@ -119,7 +131,7 @@ def test_mixed_inner_concat(): df, df2 = generate_dfs() df3 = df.copy() - mixed_dfs = [from_pandas(df, 2), from_pandas(df2, 2), df3] + mixed_dfs = [from_pandas(df), from_pandas(df2), df3] assert (ray_df_equals_pandas( pd.concat(mixed_dfs, join='inner'), @@ -130,7 +142,7 @@ def test_mixed_none_concat(): df, df2 = generate_none_dfs() df3 = df.copy() - mixed_dfs = [from_pandas(df, 2), from_pandas(df2, 2), df3] + mixed_dfs = [from_pandas(df), from_pandas(df2), df3] assert (ray_df_equals_pandas( pd.concat(mixed_dfs), pandas.concat([df, df2, df3]))) diff --git a/modin/pandas/test/test_dataframe.py b/modin/pandas/test/test_dataframe.py index 87662ece325..8702f39c751 100644 --- a/modin/pandas/test/test_dataframe.py +++ b/modin/pandas/test/test_dataframe.py @@ -3,12 +3,14 @@ from __future__ import print_function import pytest +import io import numpy as np import pandas import pandas.util.testing as tm from pandas.tests.frame.common import TestData import modin.pandas as pd from modin.pandas.utils import to_pandas +from numpy.testing import assert_array_equal @pytest.fixture @@ -23,7 +25,8 @@ def ray_series_equals_pandas(ray_series, pandas_series): @pytest.fixture def ray_df_equals(ray_df1, ray_df2): - return ray_df1.equals(ray_df2) + # return ray_df1.equals(ray_df2) + return to_pandas(ray_df1).equals(to_pandas(ray_df2)) @pytest.fixture @@ -62,6 +65,7 @@ def test_int_dataframe(): filter_by = {'items': ['col1', 'col5'], 'regex': '4$|3$', 'like': 'col'} + test_sample(ray_df, pandas_df) test_filter(ray_df, pandas_df, filter_by) test_index(ray_df, pandas_df) test_size(ray_df, pandas_df) @@ -232,6 +236,7 @@ def test_float_dataframe(): filter_by = {'items': ['col1', 'col5'], 'regex': '4$|3$', 'like': 'col'} + test_sample(ray_df, pandas_df) test_filter(ray_df, pandas_df, filter_by) test_index(ray_df, pandas_df) test_size(ray_df, pandas_df) @@ -259,7 +264,7 @@ def test_float_dataframe(): test_mean(ray_df, pandas_df) # TODO Clear floating point error. - # test_var(ray_df, pandas_df) + test_var(ray_df, pandas_df) test_std(ray_df, pandas_df) test_median(ray_df, pandas_df) @@ -340,8 +345,7 @@ def test_float_dataframe(): test_insert(ray_df, pandas_df, 1, "New Column", ray_df[key]) test_insert(ray_df, pandas_df, 4, "New Column", ray_df[key]) - # TODO Nans are always not equal to each other, fix it - # test___array__(ray_df, pandas_df) + test___array__(ray_df, pandas_df) apply_agg_functions = [ 'sum', lambda df: df.sum(), ['sum', 'mean'], ['sum', 'sum'] @@ -401,6 +405,7 @@ def test_mixed_dtype_dataframe(): filter_by = {'items': ['col1', 'col5'], 'regex': '4$|3$', 'like': 'col'} + test_sample(ray_df, pandas_df) test_filter(ray_df, pandas_df, filter_by) test_index(ray_df, pandas_df) test_size(ray_df, pandas_df) @@ -430,7 +435,7 @@ def test_mixed_dtype_dataframe(): test_mean(ray_df, pandas_df) # TODO Clear floating point error. - # test_var(ray_df, pandas_df) + test_var(ray_df, pandas_df) test_std(ray_df, pandas_df) test_median(ray_df, pandas_df) @@ -442,7 +447,7 @@ def test_mixed_dtype_dataframe(): test_describe(ray_df, pandas_df) # TODO Reolve once Pandas-20962 is resolved. - # test_rank(ray_df, pandas_df) + test_rank(ray_df, pandas_df) test_all(ray_df, pandas_df) test_any(ray_df, pandas_df) @@ -568,6 +573,7 @@ def test_nan_dataframe(): filter_by = {'items': ['col1', 'col5'], 'regex': '4$|3$', 'like': 'col'} + test_sample(ray_df, pandas_df) test_filter(ray_df, pandas_df, filter_by) test_index(ray_df, pandas_df) test_size(ray_df, pandas_df) @@ -673,7 +679,7 @@ def test_nan_dataframe(): test_insert(ray_df, pandas_df, 4, "New Column", ray_df[key]) # TODO Nans are always not equal to each other, fix it - # test___array__(ray_df, pandas_df) + test___array__(ray_df, pandas_df) apply_agg_functions = [ 'sum', lambda df: df.sum(), ['sum', 'mean'], ['sum', 'sum'] @@ -937,8 +943,8 @@ def test_copy(ray_df): new_ray_df = ray_df.copy() assert new_ray_df is not ray_df - assert np.array_equal(new_ray_df._block_partitions, - ray_df._block_partitions) + assert np.array_equal(new_ray_df._data_manager.data.partitions, + ray_df._data_manager.data.partitions) @pytest.fixture @@ -1047,6 +1053,7 @@ def test_append(): @pytest.fixture def test_apply(ray_df, pandas_df, func, axis): + print(func) ray_result = ray_df.apply(func, axis) pandas_result = pandas_df.apply(func, axis) if isinstance(ray_result, pd.DataFrame): @@ -1113,30 +1120,28 @@ def test_assign(): def test_astype(): td = TestData() - ray_df = pd.DataFrame(td.frame) - our_df_casted = ray_df.astype(np.int32) - expected_df_casted = pandas.DataFrame( - td.frame.values.astype(np.int32), + ray_df = pd.DataFrame(td.frame.values, + index=td.frame.index, + columns=td.frame.columns) + expected_df = pandas.DataFrame( + td.frame.values, index=td.frame.index, columns=td.frame.columns) - assert ray_df_equals_pandas(our_df_casted, expected_df_casted) + ray_df_casted = ray_df.astype(np.int32) + expected_df_casted = expected_df.astype(np.int32) - our_df_casted = ray_df.astype(np.float64) - expected_df_casted = pandas.DataFrame( - td.frame.values.astype(np.float64), - index=td.frame.index, - columns=td.frame.columns) + assert ray_df_equals_pandas(ray_df_casted, expected_df_casted) - assert ray_df_equals_pandas(our_df_casted, expected_df_casted) + ray_df_casted = ray_df.astype(np.float64) + expected_df_casted = expected_df.astype(np.float64) - our_df_casted = ray_df.astype(str) - expected_df_casted = pandas.DataFrame( - td.frame.values.astype(str), - index=td.frame.index, - columns=td.frame.columns) + assert ray_df_equals_pandas(ray_df_casted, expected_df_casted) - assert ray_df_equals_pandas(our_df_casted, expected_df_casted) + ray_df_casted = ray_df.astype(str) + expected_df_casted = expected_df.astype(str) + + assert ray_df_equals_pandas(ray_df_casted, expected_df_casted) def test_at_time(): @@ -1286,7 +1291,7 @@ def test_cumsum(ray_df, pandas_df): @pytest.fixture def test_describe(ray_df, pandas_df): - assert ray_df.describe().equals(pandas_df.describe()) + assert ray_df_equals_pandas(ray_df.describe(), pandas_df.describe()) @pytest.fixture @@ -1528,6 +1533,32 @@ def test_eval_df_use_case(): frame_data = {'a': np.random.randn(10), 'b': np.random.randn(10)} df = pandas.DataFrame(frame_data) ray_df = pd.DataFrame(frame_data) + + # test eval for series results + tmp_pandas = df.eval( + "arctan2(sin(a), b)", + engine='python', + parser='pandas') + tmp_ray = ray_df.eval( + "arctan2(sin(a), b)", + engine='python', + parser='pandas') + + assert isinstance(tmp_ray, pandas.Series) + assert ray_series_equals_pandas(tmp_ray, tmp_pandas) + + # Test not inplace assignments + tmp_pandas = df.eval( + "e = arctan2(sin(a), b)", + engine='python', + parser='pandas') + tmp_ray = ray_df.eval( + "e = arctan2(sin(a), b)", + engine='python', + parser='pandas') + assert ray_df_equals_pandas(tmp_ray, tmp_pandas) + + # Test inplace assignments df.eval( "e = arctan2(sin(a), b)", engine='python', @@ -1590,9 +1621,9 @@ def test_fillna(): # test_frame_pad_backfill_limit() test_fillna_dtype_conversion() test_fillna_skip_certain_blocks() - test_fillna_dict_series() with pytest.raises(NotImplementedError): + test_fillna_dict_series() test_fillna_dataframe() test_fillna_columns() @@ -2055,12 +2086,26 @@ def test_infer_objects(): ray_df.infer_objects() -@pytest.fixture -def test_info(ray_df): - info_string = ray_df.info() - assert '\n' in info_string - info_string = ray_df.info(memory_usage=True) - assert 'memory_usage: ' in info_string +#@pytest.fixture +def test_info(): + ray_df = pd.DataFrame({ + 'col1': [1, 2, 3, np.nan], + 'col2': [4, 5, np.nan, 7], + 'col3': [8, np.nan, 10, 11], + 'col4': [np.nan, 13, 14, 15] + }) + ray_df.info(memory_usage='deep') + with io.StringIO() as buf: + ray_df.info(buf=buf) + info_string = buf.getvalue() + assert '\n' in info_string + assert 'memory usage: ' in info_string + assert 'Data columns (total 4 columns):' in info_string + with io.StringIO() as buf: + ray_df.info(buf=buf, verbose=False, memory_usage=False) + info_string = buf.getvalue() + assert 'memory usage: ' not in info_string + assert 'Columns: 4 entries, col1 to col4' in info_string @pytest.fixture @@ -2243,8 +2288,9 @@ def test_melt(): ray_df.melt() -@pytest.fixture -def test_memory_usage(ray_df): +#@pytest.fixture +def test_memory_usage(): + ray_df = create_test_dataframe() assert type(ray_df.memory_usage()) is pandas.core.series.Series assert ray_df.memory_usage(index=True).at['Index'] is not None assert ray_df.memory_usage(deep=True).sum() >= \ @@ -2864,10 +2910,19 @@ def test_rtruediv(): test_inter_df_math_right_ops("rtruediv") -def test_sample(): - ray_df = create_test_dataframe() - assert len(ray_df.sample(n=4)) == 4 - assert len(ray_df.sample(frac=0.5)) == 2 +@pytest.fixture +def test_sample(ray_df, pd_df): + with pytest.raises(ValueError): + ray_df.sample(n=3, frac=0.4) + + assert ray_df_equals_pandas( + ray_df.sample(frac=0.5, random_state=42), + pd_df.sample(frac=0.5, random_state=42) + ) + assert ray_df_equals_pandas( + ray_df.sample(n=2, random_state=42), + pd_df.sample(n=2, random_state=42) + ) def test_select(): @@ -3144,12 +3199,10 @@ def test_unstack(): def test_update(): df = pd.DataFrame([[1.5, np.nan, 3.], [1.5, np.nan, 3.], [1.5, np.nan, 3], [1.5, np.nan, 3]]) - - other = pd.DataFrame([[3.6, 2., np.nan], [np.nan, np.nan, 7]], - index=[1, 3]) + other = pd.DataFrame( + [[3.6, 2., np.nan], [np.nan, np.nan, 7]], index=[1, 3]) df.update(other) - expected = pd.DataFrame([[1.5, np.nan, 3], [3.6, 2, 3], [1.5, np.nan, 3], [1.5, np.nan, 7.]]) assert ray_df_equals(df, expected) @@ -3164,33 +3217,26 @@ def test_where(): frame_data = np.random.randn(100, 10) pandas_df = pandas.DataFrame(frame_data, columns=list('abcdefghij')) ray_df = pd.DataFrame(frame_data, columns=list('abcdefghij')) - pandas_cond_df = pandas_df % 5 < 2 ray_cond_df = ray_df % 5 < 2 pandas_result = pandas_df.where(pandas_cond_df, -pandas_df) ray_result = ray_df.where(ray_cond_df, -ray_df) - - assert ray_df_equals_pandas(ray_result, pandas_result) + assert all((to_pandas(ray_result) == pandas_result).all()) other = pandas_df.loc[3] - pandas_result = pandas_df.where(pandas_cond_df, other, axis=1) ray_result = ray_df.where(ray_cond_df, other, axis=1) - - assert ray_df_equals_pandas(ray_result, pandas_result) + assert all((to_pandas(ray_result) == pandas_result).all()) other = pandas_df['e'] - pandas_result = pandas_df.where(pandas_cond_df, other, axis=0) ray_result = ray_df.where(ray_cond_df, other, axis=0) - - assert ray_df_equals_pandas(ray_result, pandas_result) + assert all((to_pandas(ray_result) == pandas_result).all()) pandas_result = pandas_df.where(pandas_df < 2, True) ray_result = ray_df.where(ray_df < 2, True) - - assert ray_df_equals_pandas(ray_result, pandas_result) + assert all((to_pandas(ray_result) == pandas_result).all()) def test_xs(): @@ -3311,7 +3357,7 @@ def test___round__(): @pytest.fixture def test___array__(ray_df, pandas_df): - assert np.array_equal(ray_df.__array__(), pandas_df.__array__()) + assert_array_equal(ray_df.__array__(), pandas_df.__array__()) def test___getstate__(): @@ -3391,29 +3437,22 @@ def test___repr__(): frame_data = np.random.randint(0, 100, size=(1000, 100)) pandas_df = pandas.DataFrame(frame_data) ray_df = pd.DataFrame(frame_data) - assert repr(pandas_df) == repr(ray_df) frame_data = np.random.randint(0, 100, size=(1000, 99)) pandas_df = pandas.DataFrame(frame_data) ray_df = pd.DataFrame(frame_data) - assert repr(pandas_df) == repr(ray_df) - # These currently fails because the dots do not line up. - # For some reason only two dots are being added for our DataFrame + frame_data = np.random.randint(0, 100, size=(1000, 101)) + pandas_df = pandas.DataFrame(frame_data) + ray_df = pd.DataFrame(frame_data) + assert repr(pandas_df) == repr(ray_df) - # frame_data = np.random.randint(0, 100, size=(1000, 101)) - # pandas_df = pandas.DataFrame(frame_data) - # ray_df = pd.DataFrame(frame_data) - # - # assert repr(pandas_df) == repr(ray_df) - # - # frame_data = np.random.randint(0, 100, size=(1000, 102)) - # pandas_df = pandas.DataFrame(frame_data) - # ray_df = pd.DataFrame(frame_data) - # - # assert repr(pandas_df) == repr(ray_df) + frame_data = np.random.randint(0, 100, size=(1000, 102)) + pandas_df = pandas.DataFrame(frame_data) + ray_df = pd.DataFrame(frame_data) + assert repr(pandas_df) == repr(ray_df) # ___repr___ method has a different code path depending on # whether the number of rows is >60; and a different code path @@ -3455,11 +3494,19 @@ def test_loc(ray_df, pd_df): # DataFrame assert ray_df_equals_pandas(ray_df.loc[[1, 2]], pd_df.loc[[1, 2]]) - assert ray_df_equals_pandas(ray_df.loc[[1, 2], ['col1']], - pd_df.loc[[1, 2], ['col1']]) + + # See issue #80 + # assert ray_df_equals_pandas(ray_df.loc[[1, 2], ['col1']], + # pd_df.loc[[1, 2], ['col1']]) assert ray_df_equals_pandas(ray_df.loc[1:2, 'col1':'col2'], pd_df.loc[1:2, 'col1':'col2']) + # Write Item + ray_df_copy = ray_df.copy() + pd_df_copy = pd_df.copy() + ray_df_copy.loc[[1, 2]] = 42 + pd_df_copy.loc[[1,2]] = 42 + assert ray_df_equals_pandas(ray_df_copy, pd_df_copy) def test_is_copy(): ray_df = create_test_dataframe() @@ -3498,13 +3545,21 @@ def test_iloc(ray_df, pd_df): # DataFrame assert ray_df_equals_pandas(ray_df.iloc[[1, 2]], pd_df.iloc[[1, 2]]) - assert ray_df_equals_pandas(ray_df.iloc[[1, 2], [1, 0]], - pd_df.iloc[[1, 2], [1, 0]]) + # See issue #80 + # assert ray_df_equals_pandas(ray_df.iloc[[1, 2], [1, 0]], + # pd_df.iloc[[1, 2], [1, 0]]) assert ray_df_equals_pandas(ray_df.iloc[1:2, 0:2], pd_df.iloc[1:2, 0:2]) # Issue #43 ray_df.iloc[0:3, :] + # Write Item + ray_df_copy = ray_df.copy() + pd_df_copy = pd_df.copy() + ray_df_copy.iloc[[1, 2]] = 42 + pd_df_copy.iloc[[1,2]] = 42 + assert ray_df_equals_pandas(ray_df_copy, pd_df_copy) + def test__doc__(): assert pd.DataFrame.__doc__ != pandas.DataFrame.__doc__ @@ -3529,6 +3584,17 @@ def test_get_dummies(): frame_data = {'A': ['a', 'b', 'a'], 'B': ['b', 'a', 'c'], 'C': [1, 2, 3]} ray_df = pd.DataFrame(frame_data) pd_df = pandas.DataFrame(frame_data) + assert ray_df_equals_pandas( + pd.get_dummies(ray_df), pandas.get_dummies(pd_df)) + frame_data = {'A': ['a'], 'B': ['b']} + ray_df = pd.DataFrame(frame_data) + pd_df = pandas.DataFrame(frame_data) + assert ray_df_equals_pandas( + pd.get_dummies(ray_df), pandas.get_dummies(pd_df)) + + frame_data = {'A': [1, 2, 3], 'B': [4, 5, 6], 'C': [1, 2, 3]} + ray_df = pd.DataFrame(frame_data) + pd_df = pandas.DataFrame(frame_data) assert ray_df_equals_pandas( pd.get_dummies(ray_df), pandas.get_dummies(pd_df)) diff --git a/modin/pandas/test/test_groupby.py b/modin/pandas/test/test_groupby.py index ed5653b60a3..3e136fbb218 100644 --- a/modin/pandas/test/test_groupby.py +++ b/modin/pandas/test/test_groupby.py @@ -7,7 +7,7 @@ import pandas import numpy as np import modin.pandas as pd -from modin.pandas.utils import (from_pandas, to_pandas) +from modin.pandas.utils import from_pandas, to_pandas PY2 = False if sys.version_info.major < 3: @@ -17,7 +17,7 @@ @pytest.fixture def ray_df_equals_pandas(ray_df, pandas_df): assert isinstance(ray_df, pd.DataFrame) - assert to_pandas(ray_df).equals(pandas_df) + assert to_pandas(ray_df).equals(pandas_df) or (all(ray_df.isna().all()) and all(pandas_df.isna().all())) @pytest.fixture @@ -25,7 +25,7 @@ def ray_df_almost_equals_pandas(ray_df, pandas_df): assert isinstance(ray_df, pd.DataFrame) difference = to_pandas(ray_df) - pandas_df diff_max = difference.max().max() - assert to_pandas(ray_df).equals(pandas_df) or diff_max < 0.0001 + assert to_pandas(ray_df).equals(pandas_df) or diff_max < 0.0001 or (all(ray_df.isna().all()) and all(pandas_df.isna().all())) @pytest.fixture @@ -54,7 +54,7 @@ def test_simple_row_groupby(): 'col5': [-4, -5, -6, -7] }) - ray_df = from_pandas(pandas_df, 2) + ray_df = from_pandas(pandas_df) by = [1, 2, 1, 2] n = 1 @@ -133,7 +133,7 @@ def test_single_group_row_groupby(): 'col5': [-4, 5, -6, -7] }) - ray_df = from_pandas(pandas_df, 2) + ray_df = from_pandas(pandas_df) by = [1, 1, 1, 1] n = 6 @@ -208,7 +208,7 @@ def test_large_row_groupby(): pandas_df = pandas.DataFrame( np.random.randint(0, 8, size=(100, 4)), columns=list('ABCD')) - ray_df = from_pandas(pandas_df, 2) + ray_df = from_pandas(pandas_df) by = [str(i) for i in pandas_df['A'].tolist()] n = 4 @@ -287,7 +287,7 @@ def test_simple_col_groupby(): 'col5': [-4, 5, 6, -7] }) - ray_df = from_pandas(pandas_df, 2) + ray_df = from_pandas(pandas_df) by = [1, 2, 3, 2, 1] @@ -366,7 +366,8 @@ def test_skew(ray_groupby, pandas_groupby): @pytest.fixture def test_ffill(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.ffill(), pandas_groupby.ffill()) + with pytest.raises(NotImplementedError): + ray_groupby.ffill() @pytest.fixture @@ -402,10 +403,9 @@ def test_ndim(ray_groupby, pandas_groupby): @pytest.fixture -def test_cumsum(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.cumsum(), pandas_groupby.cumsum()) +def test_cumsum(ray_groupby, pandas_groupby, axis=0): ray_df_equals_pandas( - ray_groupby.cumsum(axis=1), pandas_groupby.cumsum(axis=1)) + ray_groupby.cumsum(axis=axis), pandas_groupby.cumsum(axis=axis)) @pytest.fixture @@ -415,10 +415,9 @@ def test_pct_change(ray_groupby, pandas_groupby): @pytest.fixture -def test_cummax(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.cummax(), pandas_groupby.cummax()) +def test_cummax(ray_groupby, pandas_groupby, axis=0): ray_df_equals_pandas( - ray_groupby.cummax(axis=1), pandas_groupby.cummax(axis=1)) + ray_groupby.cummax(axis=axis), pandas_groupby.cummax(axis=axis)) @pytest.fixture @@ -439,19 +438,20 @@ def test_first(ray_groupby, pandas_groupby): @pytest.fixture def test_backfill(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.backfill(), pandas_groupby.backfill()) + with pytest.raises(NotImplementedError): + ray_groupby.backfill() @pytest.fixture -def test_cummin(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.cummin(), pandas_groupby.cummin()) +def test_cummin(ray_groupby, pandas_groupby, axis=0): ray_df_equals_pandas( - ray_groupby.cummin(axis=1), pandas_groupby.cummin(axis=1)) + ray_groupby.cummin(axis=axis), pandas_groupby.cummin(axis=axis)) @pytest.fixture def test_bfill(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.bfill(), pandas_groupby.bfill()) + with pytest.raises(NotImplementedError): + ray_groupby.bfill() @pytest.fixture @@ -530,7 +530,7 @@ def test_nunique(ray_groupby, pandas_groupby): @pytest.fixture def test_median(ray_groupby, pandas_groupby): - ray_df_equals_pandas(ray_groupby.median(), pandas_groupby.median()) + ray_df_almost_equals_pandas(ray_groupby.median(), pandas_groupby.median()) @pytest.fixture @@ -540,10 +540,10 @@ def test_head(ray_groupby, pandas_groupby, n): @pytest.fixture -def test_cumprod(ray_groupby, pandas_groupby): +def test_cumprod(ray_groupby, pandas_groupby, axis=0): ray_df_equals_pandas(ray_groupby.cumprod(), pandas_groupby.cumprod()) ray_df_equals_pandas( - ray_groupby.cumprod(axis=1), pandas_groupby.cumprod(axis=1)) + ray_groupby.cumprod(axis=axis), pandas_groupby.cumprod(axis=axis)) @pytest.fixture diff --git a/modin/pandas/utils.py b/modin/pandas/utils.py index efd233603d2..e43b29020c0 100644 --- a/modin/pandas/utils.py +++ b/modin/pandas/utils.py @@ -4,250 +4,20 @@ import pandas -import collections -import numpy as np -import ray -import time -import gc +from ..data_management.factories import BaseFactory -from . import get_npartitions -_NAN_BLOCKS = {} -_MEMOIZER_CAPACITY = 1000 # Capacity per function - - -class LRUCache(object): - """A LRUCache implemented with collections.OrderedDict - - Notes: - - OrderedDict will record the order each item is inserted. - - The head of the queue will be LRU items. - """ - - def __init__(self, capacity): - self.capacity = capacity - self.cache = collections.OrderedDict() - - def __contains__(self, key): - return key in self.cache - - def __getitem__(self, key): - """Retrieve item from cache and re-insert it to the back of the queue - """ - value = self.cache.pop(key) - self.cache[key] = value - return value - - def __setitem__(self, key, value): - if key in self.cache: - self.cache.pop(key) - - if len(self.cache) >= self.capacity: - # Pop oldest items at the beginning of the queue - self.cache.popitem(last=False) - - self.cache[key] = value - - -class memoize(object): - """A basic memoizer that cache the input and output of the remote function - - Notes: - - How is this implemented? - This meoizer is implemented by adding a caching layer to the remote - function's remote attribute. When user call f.remote(*args), we will - first check against the cache, and then call the ray remote function - if we can't find the return value in the cache. - - When should this be used? - This should be used when we anticipate temporal locality for the - function. For example, we can reasonally assume users will perform - columnar operation repetitively over time (like sum() or loc[]). - - Caveat - Don't use this decorator if the any argument to the remote function - will mutate. Following snippet will fail - ```py - @memoize - @ray.remote - def f(obj): - ... - - mutable_obj = [1] - oid_1 = f.remote(mutable_obj) # will be cached - - mutable_obj.append(3) - oid_2 = f.remote(mutable_obj) # cache hit! - - oid_1 == oid_2 # True! - ``` - In short, use this function sparingly. The ideal case is that all - inputs are ray ObjectIDs because they are immutable objects. - - Future Development - - Fix the mutability bug - - Dynamic cache size (Fixed as 1000 for now) - """ - - def __init__(self, f): - # Save of remote function - self.old_remote_func = f.remote - self.cache = LRUCache(capacity=_MEMOIZER_CAPACITY) - - def remote(self, *args): - """Return cached result if the arguments are cached - """ - args = tuple(args) - - if args in self.cache: - cached_result = self.cache[args] - return cached_result - - result = self.old_remote_func(*args) - self.cache[args] = result - return result - - -def post_task_gc(func): - """Perform garbage collection after the task is executed. - - Usage: - ``` - @ray.remote - @post_task_gc - def memory_hungry_op(): - ... - ``` - Note: - - This will invoke the GC for the entire process. Expect - About 100ms latency. - - We have a basic herustic in place to balance of tradeoff between - speed and memory. If the task takes more than 500ms to run, we - will do the GC. - """ - - def wrapped(*args): - start_time = time.time() - - result = func(*args) - - duration_s = time.time() - start_time - duration_ms = duration_s * 1000 - if duration_ms > 500: - gc.collect() - - return result - - return wrapped - - -def _get_nan_block_id(n_row=1, n_col=1, transpose=False): - """A memory efficent way to get a block of NaNs. - - Args: - n_rows(int): number of rows - n_col(int): number of columns - transpose(bool): if true, swap rows and columns - Returns: - ObjectID of the NaN block - """ - global _NAN_BLOCKS - if transpose: - n_row, n_col = n_col, n_row - shape = (n_row, n_col) - if shape not in _NAN_BLOCKS: - arr = np.tile(np.array(np.NaN), shape) - _NAN_BLOCKS[shape] = ray.put(pandas.DataFrame(data=arr)) - return _NAN_BLOCKS[shape] - - -def _get_lengths(df): - """Gets the length of the DataFrame. - Args: - df: A remote pandas.DataFrame object. - Returns: - Returns an integer length of the DataFrame object. If the attempt - fails, returns 0 as the length. - """ - try: - return len(df) - # Because we sometimes have cases where we have summary statistics in our - # DataFrames - except TypeError: - return 0 - - -def _get_widths(df): - """Gets the width (number of columns) of the DataFrame. - Args: - df: A remote pandas.DataFrame object. - Returns: - Returns an integer width of the DataFrame object. If the attempt - fails, returns 0 as the length. - """ - try: - return len(df.columns) - # Because we sometimes have cases where we have summary statistics in our - # DataFrames - except TypeError: - return 0 - - -def _partition_pandas_dataframe(df, num_partitions=None, row_chunksize=None): - """Partitions a Pandas DataFrame object. - Args: - df (pandas.DataFrame): The pandas DataFrame to convert. - npartitions (int): The number of partitions to split the DataFrame - into. Has priority over chunksize. - row_chunksize (int): The number of rows to put in each partition. - Returns: - [ObjectID]: A list of object IDs corresponding to the DataFrame - partitions - """ - if num_partitions is not None: - row_chunksize = len(df) // num_partitions \ - if len(df) % num_partitions == 0 \ - else len(df) // num_partitions + 1 - else: - assert row_chunksize is not None - - temp_df = df - - row_partitions = [] - while len(temp_df) > row_chunksize: - t_df = temp_df[:row_chunksize] - # reset_index here because we want a pandas.RangeIndex - # within the partitions. It is smaller and sometimes faster. - t_df.reset_index(drop=True, inplace=True) - t_df.columns = pandas.RangeIndex(0, len(t_df.columns)) - top = ray.put(t_df) - row_partitions.append(top) - temp_df = temp_df[row_chunksize:] - else: - # Handle the last chunk correctly. - # This call is necessary to prevent modifying original df - temp_df = temp_df[:] - temp_df.reset_index(drop=True, inplace=True) - temp_df.columns = pandas.RangeIndex(0, len(temp_df.columns)) - row_partitions.append(ray.put(temp_df)) - - return row_partitions - - -def from_pandas(df, num_partitions=None, chunksize=None): +def from_pandas(df): """Converts a pandas DataFrame to a Ray DataFrame. Args: df (pandas.DataFrame): The pandas DataFrame to convert. - num_partitions (int): The number of partitions to split the DataFrame - into. Has priority over chunksize. - chunksize (int): The number of rows to put in each partition. + Returns: A new Ray DataFrame object. """ from .dataframe import DataFrame - row_partitions = \ - _partition_pandas_dataframe(df, num_partitions, chunksize) - - return DataFrame( - row_partitions=row_partitions, columns=df.columns, index=df.index) + return DataFrame(data_manager=BaseFactory.from_pandas(df)) def to_pandas(df): @@ -257,216 +27,7 @@ def to_pandas(df): Returns: A new pandas DataFrame. """ - pandas_df = pandas.concat(ray.get(df._row_partitions), copy=False) - pandas_df.index = df.index - pandas_df.columns = df.columns - return pandas_df - - -""" -Indexing Section - Generate View Copy Helpers - Function list: - - `extract_block` (ray.remote function, move to EOF) - - `_generate_block` - - `_repartition_coord_df` - Call Dependency: - - _generate_block calls extract_block remote - Pipeline: - - Repartition the dataframe by npartition - - Use case: - The dataframe is a DataFrameView, the two coord_dfs only - describe the subset of the block partition data. We want - to create a new copy of this subset and re-partition - the new dataframe. -""" - - -def _repartition_coord_df(old_coord_df, npartition): - """Repartition the (view of) coord_df by npartition - - This function is best used when old_coord_df is not contigous. - For example, it turns: - - partition index_within_partition - i0 0 0 - i6 3 2 - - into - - partition index_within_partition - i0 0 0 - i6 0 1 - - Note(simon): - The resulting npartition will be <= npartition - passed in. - """ - length = len(old_coord_df) - chunksize = (len(old_coord_df) // npartition if len(old_coord_df) % - npartition == 0 else len(old_coord_df) // npartition + 1) - - # genereate array([0, 0, 0, 1, 1, 1, 2]) - partitions = np.repeat(np.arange(npartition), chunksize)[:length] - - # generate array([0, 1, 2, 0, 1, 2, 0]) - final_n_partition = np.max(partitions) - idx_in_part = np.tile(np.arange(chunksize), final_n_partition + 1)[:length] - - final_df = pandas.DataFrame({ - 'partition': partitions, - 'index_within_partition': idx_in_part - }, - index=old_coord_df.index) - - return final_df - - -def _generate_blocks(old_row, new_row, old_col, new_col, - block_partition_2d_oid_arr): - """ - Given the four coord_dfs: - - Old Row Coord df - - New Row Coord df - - Old Col Coord df - - New Col Coord df - and the block partition array, this function will generate the new - block partition array. - """ - - # We join the old and new coord_df to find out which chunk in the old - # partition belongs to the chunk in the new partition. The new coord df - # should have the same index as the old coord df in order to align the - # row/column. This is guaranteed by _repartition_coord_df. - def join(old, new): - return new.merge( - old, left_index=True, right_index=True, suffixes=('_new', '_old')) - - row_grouped = join(old_row, new_row).groupby('partition_new') - col_grouped = join(old_col, new_col).groupby('partition_new') - - oid_lst = [] - for row_idx, row_lookup in row_grouped: - for col_idx, col_lookup in col_grouped: - oid = extract_block.remote( - block_partition_2d_oid_arr, - row_lookup, - col_lookup, - col_name_suffix='_old') - oid_lst.append(oid) - return np.array(oid_lst).reshape(len(row_grouped), len(col_grouped)) - - -# Indexing -# Generate View Copy Helpers -# END - - -def _mask_block_partitions(blk_partitions, row_metadata, col_metadata): - """Return the squeezed/expanded block partitions as defined by - row_metadata and col_metadata. - - Note: - Very naive implementation. Extract one scaler at a time in a double - for loop. - """ - col_df = col_metadata._coord_df - row_df = row_metadata._coord_df - - result_oids = [] - shape = (len(row_df.index), len(col_df.index)) - - for _, row_partition_data in row_df.iterrows(): - for _, col_partition_data in col_df.iterrows(): - row_part = row_partition_data.partition - col_part = col_partition_data.partition - block_oid = blk_partitions[row_part, col_part] - - row_idx = row_partition_data['index_within_partition'] - col_idx = col_partition_data['index_within_partition'] - - result_oid = extractor.remote(block_oid, [row_idx], [col_idx]) - result_oids.append(result_oid) - return np.array(result_oids).reshape(shape) - - -def _map_partitions(func, partitions, *argslists): - """Apply a function across the specified axis - - Args: - func (callable): The function to apply - partitions ([ObjectID]): The list of partitions to map func on. - - Returns: - A list of partitions ([ObjectID]) with the result of the function - """ - if partitions is None: - return None - - assert (callable(func)) - if len(argslists) == 0: - return [_deploy_func.remote(func, part) for part in partitions] - elif len(argslists) == 1: - return [ - _deploy_func.remote(func, part, argslists[0]) - for part in partitions - ] - else: - assert (all(len(args) == len(partitions) for args in argslists)) - return [ - _deploy_func.remote(func, *args) - for args in zip(partitions, *argslists) - ] - - -def _create_block_partitions(partitions, axis=0, length=None): - - if length is not None and length != 0 and get_npartitions() > length: - npartitions = length - elif length == 0: - npartitions = 1 - else: - npartitions = get_npartitions() - - x = [ - create_blocks._submit( - args=(partition, npartitions, axis), num_return_vals=npartitions) - for partition in partitions - ] - - # In the case that axis is 1 we have to transpose because we build the - # columns into rows. Fortunately numpy is efficient at this. - blocks = np.array(x) if axis == 0 else np.array(x).T - - # Sometimes we only get a single column or row, which is - # problematic for building blocks from the partitions, so we - # add whatever dimension we're missing from the input. - return _fix_blocks_dimensions(blocks, axis) - - -def _create_blocks_helper(df, npartitions, axis): - # Single partition dataframes don't need to be repartitioned - if npartitions == 1: - return df - # In the case that the size is not a multiple of the number of partitions, - # we need to add one to each partition to avoid losing data off the end - block_size = df.shape[axis ^ 1] // npartitions \ - if df.shape[axis ^ 1] % npartitions == 0 \ - else df.shape[axis ^ 1] // npartitions + 1 - - # if not isinstance(df.columns, pandas.RangeIndex): - # df.columns = pandas.RangeIndex(0, len(df.columns)) - - blocks = [ - df.iloc[:, i * block_size:(i + 1) * block_size] - if axis == 0 else df.iloc[i * block_size:(i + 1) * block_size, :] - for i in range(npartitions) - ] - - for block in blocks: - block.columns = pandas.RangeIndex(0, len(block.columns)) - block.reset_index(inplace=True, drop=True) - return blocks + return df._data_manager.to_pandas() def _inherit_docstrings(parent, excluded=[]): @@ -503,274 +64,3 @@ def decorator(cls): return cls return decorator - - -def _fix_blocks_dimensions(blocks, axis): - """Checks that blocks is 2D, and adds a dimension if not. - """ - if blocks.ndim < 2: - return np.expand_dims(blocks, axis=axis ^ 1) - return blocks - - -@ray.remote -def _deploy_func(func, dataframe, *args): - """Deploys a function for the _map_partitions call. - Args: - dataframe (pandas.DataFrame): The pandas DataFrame for this partition. - Returns: - A futures object representing the return value of the function - provided. - """ - if len(args) == 0: - return func(dataframe) - else: - return func(dataframe, *args) - - -@ray.remote -def extractor(df_chunk, row_loc, col_loc): - """Retrieve an item from remote block - """ - # We currently have to do the writable flag trick because a pandas bug - # https://github.com/pandas-dev/pandas/issues/17192 - try: - row_loc.flags.writeable = True - col_loc.flags.writeable = True - except AttributeError: - # Locators might be scaler or python list - pass - # Python2 doesn't allow writable flag to be set on this object. Copying - # into a list allows it to be used by iloc. - except ValueError: - row_loc = list(row_loc) - col_loc = list(col_loc) - return df_chunk.iloc[row_loc, col_loc] - - -@ray.remote -def writer(df_chunk, row_loc, col_loc, item): - """Make a copy of the block and write new item to it - """ - df_chunk = df_chunk.copy() - df_chunk.iloc[row_loc, col_loc] = item - return df_chunk - - -@ray.remote -def _build_col_widths(df_col): - """Compute widths (# of columns) for each partition.""" - widths = np.array( - ray.get([_deploy_func.remote(_get_widths, d) for d in df_col])) - - return widths - - -@ray.remote -def _build_row_lengths(df_row): - """Compute lengths (# of rows) for each partition.""" - lengths = np.array( - ray.get([_deploy_func.remote(_get_lengths, d) for d in df_row])) - - return lengths - - -@ray.remote -def _build_coord_df(lengths, index): - """Build the coordinate DataFrame over all partitions.""" - filtered_lengths = [x for x in lengths if x > 0] - coords = None - if len(filtered_lengths) > 0: - coords = np.vstack([ - np.column_stack((np.full(l, i), np.arange(l))) - for i, l in enumerate(filtered_lengths) - ]) - col_names = ("partition", "index_within_partition") - return pandas.DataFrame(coords, index=index, columns=col_names) - - -@ray.remote -def create_blocks(df, npartitions, axis): - return _create_blocks_helper(df, npartitions, axis) - - -@memoize -@ray.remote -def _blocks_to_series(*partition): - """Used in indexing, concatenating blocks in a flexible way - """ - if len(partition) == 0: - return pandas.Series() - - partition = [pandas.Series(p.squeeze()) for p in partition] - series = pandas.concat(partition) - return series - - -@memoize -@ray.remote -def _blocks_to_col(*partition): - if len(partition): - return pandas.concat(partition, axis=0, copy=False)\ - .reset_index(drop=True) - else: - return pandas.DataFrame() - - -@memoize -@ray.remote -def _blocks_to_row(*partition): - if len(partition): - row_part = pandas.concat(partition, axis=1, copy=False)\ - .reset_index(drop=True) - # Because our block partitions contain different indices (for the - # columns), this change is needed to ensure correctness. - row_part.columns = pandas.RangeIndex(0, len(row_part.columns)) - return row_part - else: - return pandas.DataFrame() - - -@ray.remote -def _reindex_helper(old_index, new_index, axis, npartitions, *df): - """Reindexes a DataFrame to prepare for join/concat. - - Args: - df: The DataFrame partition - old_index: The index/column for this partition. - new_index: The new index/column to assign. - axis: Which axis to reindex over. - - Returns: - A new set of blocks made up of DataFrames. - """ - df = pandas.concat(df, axis=axis ^ 1) - if axis == 1: - df.index = old_index - elif axis == 0: - df.columns = old_index - - df = df.reindex(new_index, copy=False, axis=axis ^ 1) - return _create_blocks_helper(df, npartitions, axis) - - -@ray.remote -def _co_op_helper(func, left_columns, right_columns, left_df_len, left_idx, - *zipped): - """Copartition operation where two DataFrames must have aligned indexes. - - NOTE: This function assumes things are already copartitioned. Requires that - row partitions are passed in as blocks. - - Args: - func: The operation to conduct between two DataFrames. - left_columns: The column names for the left DataFrame. - right_columns: The column names for the right DataFrame. - left_df_len: The length of the left. This is used so we can split up - the zipped partitions. - zipped: The DataFrame partitions (in blocks). - - Returns: - A new set of blocks for the partitioned DataFrame. - """ - left = pandas.concat(zipped[:left_df_len], axis=1, copy=False).copy() - left.columns = left_columns - if left_idx is not None: - left.index = left_idx - - right = pandas.concat(zipped[left_df_len:], axis=1, copy=False).copy() - right.columns = right_columns - - new_rows = func(left, right) - - new_blocks = _create_blocks_helper(new_rows, left_df_len, 0) - - if left_idx is not None: - new_blocks.append(new_rows.index) - - return new_blocks - - -@ray.remote -def _match_partitioning(column_partition, lengths, index): - """Match the number of rows on each partition. Used in df.merge(). - - NOTE: This function can cause problems when there are empty column - partitions. - - The way this function is intended to be used is as follows: Align the - right partitioning with the left. The left will remain unchanged. Then, - you are free to perform actions on a per-partition basis with the - partitioning. - - The index objects must already be identical for this to work correctly. - - Args: - column_partition: The column partition to change. - lengths: The lengths of each row partition to match to. - index: The index index of the column_partition. This is used to push - down to the inner frame for correctness in the merge. - - Returns: - A list of blocks created from this column partition. - """ - partitioned_list = [] - - columns = column_partition.columns - # We set this because this is the only place we can guarantee correct - # placement. We use it in the case the user wants to join on the index. - column_partition.index = index - for length in lengths: - if len(column_partition) == 0: - partitioned_list.append(pandas.DataFrame(columns=columns)) - continue - - partitioned_list.append(column_partition.iloc[:length, :]) - column_partition = column_partition.iloc[length:, :] - return partitioned_list - - -@ray.remote -def _concat_index(*index_parts): - return index_parts[0].append(index_parts[1:]) - - -@ray.remote -def _compile_remote_dtypes(*column_of_blocks): - small_dfs = [df.loc[0:0] for df in column_of_blocks] - return pandas.concat(small_dfs, copy=False).dtypes - - -@ray.remote -def extract_block(blk_partitions, row_lookup, col_lookup, col_name_suffix): - """ - This function extracts a single block from blk_partitions using - the row_lookup and col_lookup. - - Pass in col_name_suffix='_old' when operate on a joined df. - """ - - def apply_suffix(s): - return s + col_name_suffix - - # Address Arrow Error: - # Buffer source array is read-only - row_lookup = row_lookup.copy() - col_lookup = col_lookup.copy() - - df_columns = [] - for row_idx, row_df in row_lookup.groupby(apply_suffix('partition')): - this_column = [] - for col_idx, col_df in col_lookup.groupby(apply_suffix('partition')): - block_df_oid = blk_partitions[row_idx, col_idx] - block_df = ray.get(block_df_oid) - chunk = block_df.iloc[row_df[apply_suffix( - 'index_within_partition' - )], col_df[apply_suffix('index_within_partition')]] - this_column.append(chunk) - df_columns.append(pandas.concat(this_column, axis=1)) - final_df = pandas.concat(df_columns) - final_df.index = pandas.RangeIndex(0, final_df.shape[0]) - final_df.columns = pandas.RangeIndex(0, final_df.shape[1]) - - return final_df