Skip to content

Commit

Permalink
efficiency
Browse files Browse the repository at this point in the history
  • Loading branch information
colin-ho committed Feb 15, 2024
1 parent e8ab252 commit d2093f6
Showing 1 changed file with 13 additions and 30 deletions.
43 changes: 13 additions & 30 deletions daft/dataframe/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -235,12 +235,18 @@ def _populate_preview(self) -> None:
self._preview.preview_partition is None or len(self._preview.preview_partition) < self._num_preview_rows
)
if preview_partition_invalid:
preview_df = self
if self._num_preview_rows < len(self):
preview_df = preview_df.limit(self._num_preview_rows)
preview_df._materialize_results()
preview_results = preview_df._result
assert preview_results is not None
need = self._num_preview_rows
preview_parts = []
for part in self._result.values():
part_len = len(part)
if part_len >= need: # if this part has enough rows, take what we need and break
preview_parts.append(part.slice(0, need))
break
else: # otherwise, take the whole part and keep going
need -= part_len
preview_parts.append(part)

preview_results = LocalPartitionSet({i: part for i, part in enumerate(preview_parts)})

preview_partition = preview_results._get_merged_vpartition()
self._preview = DataFramePreview(
Expand Down Expand Up @@ -330,30 +336,7 @@ def _from_tables(cls, *parts: MicroPartition) -> "DataFrame":
df._result_cache = cache_entry

# build preview
num_preview_rows = context.daft_execution_config.num_preview_rows
dataframe_num_rows = len(df)
if dataframe_num_rows > num_preview_rows:
need = num_preview_rows
preview_parts = []
for part in parts:
part_len = len(part)
if part_len >= need: # if this part has enough rows, take what we need and break
preview_parts.append(part.slice(0, need))
break
else: # otherwise, take the whole part and keep going
need -= part_len
preview_parts.append(part)

preview_results = LocalPartitionSet({i: part for i, part in enumerate(preview_parts)})
else:
preview_results = result_pset

# set preview
preview_partition = preview_results._get_merged_vpartition()
df._preview = DataFramePreview(
preview_partition=preview_partition,
dataframe_num_rows=dataframe_num_rows,
)
df._populate_preview()
return df

###
Expand Down

0 comments on commit d2093f6

Please sign in to comment.