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 = '