From 752a00be495ee06be54455f71a943430d539d1de Mon Sep 17 00:00:00 2001 From: Devin Petersohn Date: Thu, 19 Jul 2018 22:01:09 -0700 Subject: [PATCH 1/2] Fix bug where fillna assumes each column is its own partition --- modin/pandas/dataframe.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/modin/pandas/dataframe.py b/modin/pandas/dataframe.py index 04800afc94d..58f38914da1 100644 --- a/modin/pandas/dataframe.py +++ b/modin/pandas/dataframe.py @@ -2076,27 +2076,35 @@ def fillna(self, value=None, method=None, axis=None, inplace=False, 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 not part in partition_dict: + partition_dict[part] = {} + partition_dict[part][index] = value[val] + print(part, index) # Pandas ignores these errors so we will suppress them too. except KeyError: continue - new_vals[val] = _deploy_func.remote(lambda df: df.fillna( - value={index: value[val]}, + print(partition_dict) + 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]) + print(ray.get(new_vals[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 coords_obj.index[i] not in new_vals - else new_vals[coords_obj.index[i]] + 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( From cfe32ed9b52d90257f6fdbf2468d2eeb9cabb823 Mon Sep 17 00:00:00 2001 From: Devin Petersohn Date: Thu, 19 Jul 2018 22:06:43 -0700 Subject: [PATCH 2/2] Fix lint and remove debug code --- modin/pandas/dataframe.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/modin/pandas/dataframe.py b/modin/pandas/dataframe.py index 58f38914da1..1e8da7b2fa9 100644 --- a/modin/pandas/dataframe.py +++ b/modin/pandas/dataframe.py @@ -2082,15 +2082,13 @@ def fillna(self, value=None, method=None, axis=None, inplace=False, try: part, index = coords_obj[val] - if not part in partition_dict: + if part not in partition_dict: partition_dict[part] = {} partition_dict[part][index] = value[val] - print(part, index) # Pandas ignores these errors so we will suppress them too. except KeyError: continue - print(partition_dict) for part, value_map in partition_dict.items(): new_vals[part] = _deploy_func.remote(lambda df: df.fillna( value=value_map, @@ -2100,7 +2098,6 @@ def fillna(self, value=None, method=None, axis=None, inplace=False, limit=limit, downcast=downcast, **kwargs), parts[part]) - print(ray.get(new_vals[part])) # Not every partition was changed, so we put everything back that # was not changed and update those that were.