Skip to content

Commit a1bcbf4

Browse files
devin-petersohnkunalgosar
authored andcommitted
Various fixes (ray-project#10)
1 parent fa840f7 commit a1bcbf4

File tree

2 files changed

+118
-22
lines changed

2 files changed

+118
-22
lines changed

python/ray/dataframe/dataframe.py

Lines changed: 110 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -9,10 +9,12 @@
99
from pandas._libs import lib
1010
from pandas.core.dtypes.cast import maybe_upcast_putmask
1111
from pandas.compat import lzip
12+
import pandas.core.common as com
1213
from pandas.core.dtypes.common import (
1314
is_bool_dtype,
1415
is_numeric_dtype,
1516
is_timedelta64_dtype)
17+
from pandas.core.indexing import convert_to_index_sliceable
1618
import warnings
1719
import numpy as np
1820
import ray
@@ -2874,12 +2876,72 @@ def __getitem__(self, key):
28742876
Returns:
28752877
A Pandas Series representing the value for the column.
28762878
"""
2877-
partition_id = self.get_col_partition(key)
2878-
index = self.get_col_index_within_partition(key)
2879-
res = ray.get(_deploy_func.remote(lambda df: df.__getitem__(index),
2880-
self._col_partitions[partition_id]))
2881-
res.name = key
2882-
return res
2879+
key = com._apply_if_callable(key, self)
2880+
2881+
# shortcut if we are an actual column
2882+
is_mi_columns = isinstance(self.columns, pd.MultiIndex)
2883+
try:
2884+
if key in self.columns and not is_mi_columns:
2885+
return self._getitem_column(key)
2886+
except:
2887+
pass
2888+
2889+
# see if we can slice the rows
2890+
indexer = convert_to_index_sliceable(self._row_index, key)
2891+
if indexer is not None:
2892+
raise NotImplementedError("To contribute to Pandas on Ray, please"
2893+
"visit github.com/ray-project/ray.")
2894+
# return self._getitem_slice(indexer)
2895+
2896+
if isinstance(key, (pd.Series, np.ndarray, pd.Index, list)):
2897+
return self._getitem_array(key)
2898+
elif isinstance(key, DataFrame):
2899+
raise NotImplementedError("To contribute to Pandas on Ray, please"
2900+
"visit github.com/ray-project/ray.")
2901+
# return self._getitem_frame(key)
2902+
elif is_mi_columns:
2903+
raise NotImplementedError("To contribute to Pandas on Ray, please"
2904+
"visit github.com/ray-project/ray.")
2905+
# return self._getitem_multilevel(key)
2906+
else:
2907+
return self._getitem_column(key)
2908+
2909+
def _getitem_column(self, key):
2910+
partition = self._get_col_locations(key).loc['partition']
2911+
result = ray.get(self._getitem_indiv_col(key, partition))
2912+
result.name = key
2913+
result.index = self.index
2914+
return result
2915+
2916+
def _getitem_array(self, array_key):
2917+
partitions = \
2918+
self._get_col_locations(array_key)['partition'].unique()
2919+
2920+
new_col_parts = [self._getitem_indiv_col(array_key, part)
2921+
for part in partitions]
2922+
2923+
# Pandas doesn't allow Index.get_loc for lists, so we have to do this.
2924+
isin = self.columns.isin(array_key)
2925+
indices_for_rows = [i for i in range(len(isin)) if isin[i]]
2926+
2927+
new_row_parts = [_deploy_func.remote(
2928+
lambda df: df.__getitem__(indices_for_rows),
2929+
part) for part in self._row_partitions]
2930+
2931+
return DataFrame(col_partitions=new_col_parts,
2932+
row_partitions=new_row_parts,
2933+
columns=array_key,
2934+
index=self.index)
2935+
2936+
def _getitem_indiv_col(self, key, partition):
2937+
loc = self._col_index.loc[key]
2938+
if isinstance(loc, pd.Series):
2939+
index = loc[loc['partition'] == partition]
2940+
else:
2941+
index = loc[loc['partition'] == partition]['index_within_partition']
2942+
return _deploy_func.remote(
2943+
lambda df: df.__getitem__(index),
2944+
self._col_partitions[partition])
28832945

28842946
def __setitem__(self, key, value):
28852947
raise NotImplementedError(
@@ -2981,9 +3043,7 @@ def __delitem__(self, key):
29813043
key: key to delete
29823044
"""
29833045
# Create helper method for deleting column(s) in row partition.
2984-
to_delete = self.columns.get_loc(key)
2985-
2986-
def del_helper(df):
3046+
def del_helper(df, to_delete):
29873047
cols = df.columns[to_delete] # either int or an array of ints
29883048

29893049
if isinstance(cols, int):
@@ -2992,23 +3052,41 @@ def del_helper(df):
29923052
for col in cols:
29933053
df.__delitem__(col)
29943054

3055+
# Reset the column index to conserve space
29953056
df.columns = pd.RangeIndex(0, len(df.columns))
29963057
return df
29973058

3059+
to_delete = self.columns.get_loc(key)
29983060
self._row_partitions = _map_partitions(
2999-
del_helper, self._row_partitions)
3061+
del_helper, self._row_partitions, to_delete)
3062+
3063+
# This structure is used to get the correct index inside the partition.
3064+
del_df = self._col_index.loc[key]
3065+
3066+
# We need to standardize between multiple and single occurrences in the
3067+
# columns. Putting single occurrences in a pd.DataFrame and transposing
3068+
# results in the same structure as multiple with 'loc'.
3069+
if isinstance(del_df, pd.Series):
3070+
del_df = pd.DataFrame(del_df).T
30003071

30013072
# Cast cols as pd.Series as duplicate columns mean result may be
30023073
# np.int64 or pd.Series
30033074
col_parts_to_del = pd.Series(
30043075
self._col_index.loc[key, 'partition']).unique()
3005-
self._col_index = self._col_index.drop(key)
3076+
self._col_index.drop(key, inplace=True)
30063077
for i in col_parts_to_del:
3078+
# Compute the correct index inside the partition to delete.
3079+
to_delete_in_partition = \
3080+
del_df[del_df['partition'] == i]['index_within_partition']
3081+
30073082
self._col_partitions[i] = _deploy_func.remote(
3008-
del_helper, self._col_partitions[i])
3083+
del_helper, self._col_partitions[i], to_delete_in_partition)
30093084

30103085
partition_mask = (self._col_index['partition'] == i)
30113086

3087+
# Since we are replacing columns with RangeIndex inside the
3088+
# partition, we have to make sure that our reference to it is
3089+
# updated as well.
30123090
try:
30133091
self._col_index.loc[partition_mask,
30143092
'index_within_partition'] = [
@@ -3258,14 +3336,26 @@ def iloc(self):
32583336
from .indexing import _iLoc_Indexer
32593337
return _iLoc_Indexer(self)
32603338

3261-
def get_col_partition(self, col):
3262-
return self._col_index['partition'][col]
3339+
def _get_col_locations(self, col):
3340+
"""Gets the location(s) from the column index DataFrame.
32633341
3264-
def get_col_index_within_partition(self, col):
3265-
return self._col_index['index_within_partition'][col]
3342+
Args:
3343+
col: The column name.
32663344
3267-
def get_row_partition(self, row):
3268-
return self._row_index['partition'][row]
3345+
Returns:
3346+
The index(es) of _col_partitions and the local index(es) where
3347+
columns with this name exist.
3348+
"""
3349+
return self._col_index.loc[col]
3350+
3351+
def _get_row_locations(self, row):
3352+
"""Gets the location(s) from the row index DataFrame.
3353+
3354+
Args:
3355+
row: The index name.
32693356
3270-
def get_row_index_within_partition(self, row):
3271-
return self._row_index['index_within_partition'][row]
3357+
Returns:
3358+
The index(es) of _row_partitions and the local index(es) where rows
3359+
with this name exist.
3360+
"""
3361+
return self._row_index.loc[row]

python/ray/dataframe/utils.py

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -226,8 +226,11 @@ def _rebuild_rows(col_partitions, index, columns):
226226
"""
227227
n_rows = min(max(get_npartitions(), len(col_partitions)), len(index))
228228
partition_assignments = assign_partitions.remote(index, n_rows)
229-
shufflers = [ShuffleActor.remote(x, partition_axis=1, shuffle_axis=0)
230-
for x in col_partitions]
229+
shufflers = [ShuffleActor.remote(
230+
col_partitions[i] if i < len(col_partitions) else pd.DataFrame(),
231+
partition_axis=1,
232+
shuffle_axis=0)
233+
for i in range(n_rows)]
231234

232235
shufflers_done = \
233236
[shufflers[i].shuffle.remote(
@@ -291,6 +294,9 @@ def _map_partitions(func, partitions, *argslists):
291294
assert(callable(func))
292295
if argslists is None:
293296
return [_deploy_func.remote(func, part) for part in partitions]
297+
elif len(argslists) == 1:
298+
return [_deploy_func.remote(func, part, argslists[0])
299+
for part in partitions]
294300
else:
295301
assert(all([len(args) == len(partitions) for args in argslists]))
296302
return [_deploy_func.remote(func, part, *args) for part, *args in zip(partitions, *argslists)]

0 commit comments

Comments
 (0)