dask/dask-expr

dask-expr planning causes incorrect merge behavior at compute time after dropna

zmbc opened this issue · 0 comments

zmbc commented

Describe the issue: With query planning turned on, a dropna-then-merge that results in overlapping columns will correctly determine what columns it should create but they don't actually exist when that merge is computed.

Minimal Complete Verifiable Example:

import dask
# If you uncomment this, it works
# dask.config.set({"dataframe.query-planning": False})

import dask.dataframe as dd
import pandas as pd

sample_df = dd.from_pandas(pd.DataFrame({'foo': ["1", "2", "3"], 'bar': ["4", "5", "6"]}), npartitions=1)
sample_df_dropna = sample_df.dropna(subset=["foo"], how="any")

print(sample_df.merge(
    sample_df,
    on=["foo"], how="left"
).columns)

print(sample_df.merge(
    sample_df,
    on=["foo"], how="left"
).compute().columns)

print(sample_df_dropna.merge(
    sample_df_dropna,
    on=["foo"], how="left"
).columns)

print(sample_df_dropna.merge(
    sample_df_dropna,
    on=["foo"], how="left"
).compute().columns)

The last print statement shows that when the expression is actually computed, bar_x and bar_y aren't created by the merge as they should be.

Anything else we need to know?:

Environment:

  • Dask version: 2024.5.0
  • Python version: 3.10.14
  • Operating System: Linux
  • Install method (conda, pip, source): pip