@@ -32,10 +32,9 @@ def assign_partitions(index, num_partitions):
3232 raise TypeError ("Unexpected value of type {0} assigned to ShuffleActor"
3333 .format (type (index ).__name__ ))
3434
35- if len (uniques ) % num_partitions == 0 :
36- chunksize = int (len (uniques ) / num_partitions )
37- else :
38- chunksize = int (len (uniques ) / num_partitions ) + 1
35+ chunksize = len (uniques ) // num_partitions \
36+ if len (uniques ) % num_partitions == 0 \
37+ else len (uniques ) // num_partitions + 1
3938
4039 assignments = []
4140
@@ -81,55 +80,83 @@ def _get_widths(df):
8180 return 0
8281
8382
84- def _partition_pandas_dataframe (df , npartitions = None , chunksize = None ):
83+ def _partition_pandas_dataframe (df , num_partitions = None , row_chunksize = None ,
84+ col_chunksize = None ):
8585 """Partitions a Pandas DataFrame object.
8686 Args:
8787 df (pandas.DataFrame): The pandas DataFrame to convert.
8888 npartitions (int): The number of partitions to split the DataFrame
8989 into. Has priority over chunksize.
90- chunksize (int): The number of rows to put in each partition.
90+ row_chunksize (int): The number of rows to put in each partition.
9191 Returns:
9292 [ObjectID]: A list of object IDs corresponding to the dataframe
9393 partitions
9494 """
95- if npartitions is not None :
96- chunksize = len (df ) // npartitions + 1
97- elif chunksize is None :
98- raise ValueError ("The number of partitions or chunksize must be set." )
95+ if num_partitions is not None :
96+ row_chunksize = len (df ) // num_partitions \
97+ if len (df ) % num_partitions == 0 \
98+ else len (df ) // num_partitions + 1
99+
100+ col_chunksize = len (df .columns ) // num_partitions \
101+ if len (df .columns ) % num_partitions == 0 \
102+ else len (df .columns ) // num_partitions + 1
103+ else :
104+ assert row_chunksize is not None and col_chunksize is not None
105+
106+ temp_df = df
107+
108+ row_partitions = []
109+ while len (temp_df ) > row_chunksize :
110+ t_df = temp_df [:row_chunksize ]
111+ # reset_index here because we want a pd.RangeIndex
112+ # within the partitions. It is smaller and sometimes faster.
113+ t_df .reset_index (drop = True , inplace = True )
114+ t_df .columns = pd .RangeIndex (0 , len (t_df .columns ))
115+ top = ray .put (t_df )
116+ row_partitions .append (top )
117+ temp_df = temp_df [row_chunksize :]
118+ else :
119+ temp_df .reset_index (drop = True , inplace = True )
120+ temp_df .columns = pd .RangeIndex (0 , len (temp_df .columns ))
121+ row_partitions .append (ray .put (temp_df ))
99122
100123 temp_df = df
101124
102- dataframes = []
103- while len (temp_df ) > chunksize :
104- t_df = temp_df [: chunksize ]
125+ col_partitions = []
126+ while len (temp_df . columns ) > col_chunksize :
127+ t_df = temp_df . iloc [:, 0 : col_chunksize ]
105128 # reset_index here because we want a pd.RangeIndex
106129 # within the partitions. It is smaller and sometimes faster.
107- t_df = t_df .reset_index (drop = True )
130+ t_df .reset_index (drop = True , inplace = True )
131+ t_df .columns = pd .RangeIndex (0 , len (t_df .columns ))
108132 top = ray .put (t_df )
109- dataframes .append (top )
110- temp_df = temp_df [ chunksize :]
133+ col_partitions .append (top )
134+ temp_df = temp_df . iloc [:, col_chunksize :]
111135 else :
112- temp_df = temp_df .reset_index (drop = True )
113- dataframes .append (ray .put (temp_df ))
136+ temp_df .reset_index (drop = True , inplace = True )
137+ temp_df .columns = pd .RangeIndex (0 , len (temp_df .columns ))
138+ col_partitions .append (ray .put (temp_df ))
114139
115- return dataframes
140+ return row_partitions , col_partitions
116141
117142
118- def from_pandas (df , npartitions = None , chunksize = None ):
143+ def from_pandas (df , num_partitions = None , chunksize = None ):
119144 """Converts a pandas DataFrame to a Ray DataFrame.
120145 Args:
121146 df (pandas.DataFrame): The pandas DataFrame to convert.
122- npartitions (int): The number of partitions to split the DataFrame
147+ num_partitions (int): The number of partitions to split the DataFrame
123148 into. Has priority over chunksize.
124149 chunksize (int): The number of rows to put in each partition.
125150 Returns:
126151 A new Ray DataFrame object.
127152 """
128153 from .dataframe import DataFrame
129154
130- dataframes = _partition_pandas_dataframe (df , npartitions , chunksize )
155+ row_partitions , col_partitions = \
156+ _partition_pandas_dataframe (df , num_partitions , chunksize )
131157
132- return DataFrame (row_partitions = dataframes ,
158+ return DataFrame (row_partitions = row_partitions ,
159+ col_partitions = col_partitions ,
133160 columns = df .columns ,
134161 index = df .index )
135162
@@ -158,10 +185,13 @@ def _rebuild_cols(row_partitions, index, columns):
158185 [ObjectID]: List of new column partitions.
159186 """
160187 # NOTE: Reexamine if this is the correct number of columns solution
161- n_cols = min (get_npartitions (), len (row_partitions ), len (columns ))
188+ n_cols = min (max ( get_npartitions (), len (row_partitions ) ), len (columns ))
162189 partition_assignments = assign_partitions .remote (columns , n_cols )
163- shufflers = [ShuffleActor .remote (x , partition_axis = 0 , shuffle_axis = 1 )
164- for x in row_partitions ]
190+ shufflers = [ShuffleActor .remote (
191+ row_partitions [i ] if i < len (row_partitions ) else pd .DataFrame (),
192+ partition_axis = 0 ,
193+ shuffle_axis = 1 )
194+ for i in range (n_cols )]
165195
166196 shufflers_done = \
167197 [shufflers [i ].shuffle .remote (
@@ -176,11 +206,12 @@ def _rebuild_cols(row_partitions, index, columns):
176206
177207 # TODO: Determine if this is the right place to reset the index
178208 def fix_indexes (df ):
179- df .index = index
180- df .columns = np . arange ( len ( df . columns ) )
209+ df .columns = pd . RangeIndex ( 0 , len ( df . columns ))
210+ df .reset_index ( drop = True , inplace = True )
181211 return df
182212
183- return [shuffler .apply_func .remote (fix_indexes ) for shuffler in shufflers [:n_cols ]]
213+ return [shuffler .apply_func .remote (fix_indexes )
214+ for shuffler in shufflers [:n_cols ]]
184215
185216
186217@ray .remote
@@ -212,8 +243,9 @@ def _rebuild_rows(col_partitions, index, columns):
212243 # TODO: Determine if this is the right place to reset the index
213244 # TODO: Determine if this needs the same changes as above
214245 def fix_indexes (df ):
215- df .columns = columns
216- return df .reset_index (drop = True )
246+ df .columns = pd .RangeIndex (0 , len (df .columns ))
247+ df .reset_index (drop = True , inplace = True )
248+ return df
217249
218250 return [shuffler .apply_func .remote (fix_indexes ) for shuffler in shufflers [:n_rows ]]
219251
@@ -265,7 +297,7 @@ def _map_partitions(func, partitions, *argslists):
265297
266298
267299@ray .remote (num_return_vals = 2 )
268- def _compute_length_and_index (dfs ):
300+ def _compute_length_and_index (dfs , index ):
269301 """Create a default index, which is a RangeIndex
270302 Returns:
271303 The pd.RangeIndex object that represents this DataFrame.
@@ -278,11 +310,12 @@ def _compute_length_and_index(dfs):
278310
279311 idx_df_col_names = ("partition" , "index_within_partition" )
280312
281- return lengths , pd .DataFrame (dest_indices , columns = idx_df_col_names )
313+ return lengths , pd .DataFrame (dest_indices , index = index ,
314+ columns = idx_df_col_names )
282315
283316
284317@ray .remote (num_return_vals = 2 )
285- def _compute_width_and_index (dfs ):
318+ def _compute_width_and_index (dfs , columns ):
286319 """Create a default index, which is a RangeIndex
287320 Returns:
288321 The pd.RangeIndex object that represents this DataFrame.
@@ -295,4 +328,5 @@ def _compute_width_and_index(dfs):
295328
296329 idx_df_col_names = ("partition" , "index_within_partition" )
297330
298- return widths , pd .DataFrame (dest_indices , columns = idx_df_col_names )
331+ return widths , pd .DataFrame (dest_indices , index = columns ,
332+ columns = idx_df_col_names )
0 commit comments