-
Notifications
You must be signed in to change notification settings - Fork 7.1k
[DataFrame] Implement where #1989
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
340ef1b
d17a66b
7f869a9
6e33862
c93ebf8
dc16257
047edc6
ac9d60f
c427a91
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -4464,9 +4464,121 @@ def remote_func(df): | |
|
|
||
| def where(self, cond, other=np.nan, inplace=False, axis=None, level=None, | ||
| errors='raise', try_cast=False, raise_on_error=None): | ||
| raise NotImplementedError( | ||
| "To contribute to Pandas on Ray, please visit " | ||
| "github.com/ray-project/ray.") | ||
| """Replaces values not meeting condition with values in other. | ||
|
|
||
| Args: | ||
| cond: A condition to be met, can be callable, array-like or a | ||
| DataFrame. | ||
| other: A value or DataFrame of values to use for setting this. | ||
| inplace: Whether or not to operate inplace. | ||
| axis: The axis to apply over. Only valid when a Series is passed | ||
| as other. | ||
| level: The MultiLevel index level to apply over. | ||
| errors: Whether or not to raise errors. Does nothing in Pandas. | ||
| try_cast: Try to cast the result back to the input type. | ||
| raise_on_error: Whether to raise invalid datatypes (deprecated). | ||
|
|
||
| Returns: | ||
| A new DataFrame with the replaced values. | ||
| """ | ||
|
|
||
| inplace = validate_bool_kwarg(inplace, 'inplace') | ||
|
|
||
| if isinstance(other, pd.Series) and axis is None: | ||
| raise ValueError("Must specify axis=0 or 1") | ||
|
|
||
| if level is not None: | ||
| raise NotImplementedError("Multilevel Index not yet supported on " | ||
| "Pandas on Ray.") | ||
|
|
||
| axis = pd.DataFrame()._get_axis_number(axis) if axis is not None else 0 | ||
|
|
||
| cond = cond(self) if callable(cond) else cond | ||
|
|
||
| if not isinstance(cond, DataFrame): | ||
| if not hasattr(cond, 'shape'): | ||
| cond = np.asanyarray(cond) | ||
| if cond.shape != self.shape: | ||
| raise ValueError("Array conditional must be same shape as " | ||
| "self") | ||
| cond = DataFrame(cond, index=self.index, columns=self.columns) | ||
|
|
||
| zipped_partitions = self._copartition(cond, self.index) | ||
| args = (False, axis, level, errors, try_cast, raise_on_error) | ||
|
|
||
| @ray.remote | ||
| def where_helper(left, cond, other, *args): | ||
|
|
||
| left = pd.concat(ray.get(left.tolist()), axis=1) | ||
| # We have to reset the index and columns here because we are coming | ||
| # from blocks and the axes are set according to the blocks. We have | ||
| # already correctly copartitioned everything, so there's no | ||
| # correctness problems with doing this. | ||
| left.reset_index(inplace=True, drop=True) | ||
| left.columns = pd.RangeIndex(len(left.columns)) | ||
|
|
||
| cond = pd.concat(ray.get(cond.tolist()), axis=1) | ||
| cond.reset_index(inplace=True, drop=True) | ||
| cond.columns = pd.RangeIndex(len(cond.columns)) | ||
|
|
||
| if isinstance(other, np.ndarray): | ||
| other = pd.concat(ray.get(other.tolist()), axis=1) | ||
| other.reset_index(inplace=True, drop=True) | ||
| other.columns = pd.RangeIndex(len(other.columns)) | ||
|
|
||
| return left.where(cond, other, *args) | ||
|
|
||
| if isinstance(other, DataFrame): | ||
| other_zipped = (v for k, v in self._copartition(other, | ||
| self.index)) | ||
|
|
||
| new_partitions = [where_helper.remote(k, v, next(other_zipped), | ||
|
||
| *args) | ||
| for k, v in zipped_partitions] | ||
|
|
||
| # Series has to be treated specially because we're operating on row | ||
| # partitions from here on. | ||
| elif isinstance(other, pd.Series): | ||
| if axis == 0: | ||
| # Pandas determines which index to use based on axis. | ||
| other = other.reindex(self.index) | ||
| other.index = pd.RangeIndex(len(other)) | ||
|
|
||
| # Since we're working on row partitions, we have to partition | ||
| # the Series based on the partitioning of self (since both | ||
| # self and cond are co-partitioned by self. | ||
| other_builder = [] | ||
| for length in self._row_metadata._lengths: | ||
| other_builder.append(other[:length]) | ||
| other = other[length:] | ||
| # Resetting the index here ensures that we apply each part | ||
| # to the correct row within the partitions. | ||
| other.index = pd.RangeIndex(len(other)) | ||
|
|
||
| other = (obj for obj in other_builder) | ||
|
|
||
| new_partitions = [where_helper.remote(k, v, next(other, | ||
| pd.Series()), | ||
| *args) | ||
| for k, v in zipped_partitions] | ||
| else: | ||
| other = other.reindex(self.columns) | ||
| other.index = pd.RangeIndex(len(other)) | ||
| new_partitions = [where_helper.remote(k, v, other, *args) | ||
| for k, v in zipped_partitions] | ||
|
|
||
| else: | ||
| new_partitions = [where_helper.remote(k, v, other, *args) | ||
| for k, v in zipped_partitions] | ||
|
|
||
| if inplace: | ||
| self._update_inplace(row_partitions=new_partitions, | ||
| row_metadata=self._row_metadata, | ||
| col_metadata=self._col_metadata) | ||
| else: | ||
| return DataFrame(row_partitions=new_partitions, | ||
| row_metadata=self._row_metadata, | ||
| col_metadata=self._col_metadata) | ||
|
|
||
| def xs(self, key, axis=0, level=None, drop_level=True): | ||
| raise NotImplementedError( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -108,10 +108,11 @@ def to_pandas(df): | |
| A new pandas DataFrame. | ||
| """ | ||
| if df._row_partitions is not None: | ||
| pd_df = pd.concat(ray.get(df._row_partitions)) | ||
| print("Yes") | ||
|
||
| pd_df = pd.concat(ray.get(df._row_partitions), copy=False) | ||
| else: | ||
| pd_df = pd.concat(ray.get(df._col_partitions), | ||
| axis=1) | ||
| axis=1, copy=False) | ||
| pd_df.index = df.index | ||
| pd_df.columns = df.columns | ||
| return pd_df | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since everything is concatenated into row partitions, can you only reset the column index?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have to reset the index here because that's what
otheris relying on.