Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Bug] fix fillna function on a single column fail #32594

Merged
merged 9 commits into from
Oct 9, 2024
Merged

Conversation

DKER2
Copy link
Contributor

@DKER2 DKER2 commented Sep 30, 2024

fixes #31855
In fillna to expression fuction in here, it add ConstantExpression as args to fillna() if value is constant. This would cause problem in expression to compute stage function as in this line all args will be added as inputs to compute stage. This constant expression will latter be computed by this function, but as it is a constant expression, it do not have any input that will make the program fail. We only should add ComputedExpression and PlaceHolderExpression


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

@DKER2 DKER2 changed the title [BUG] fillna function on a single col fail [BUG] fillna function on a single column fail Sep 30, 2024
@DKER2 DKER2 marked this pull request as draft September 30, 2024 03:31
@DKER2 DKER2 changed the title [BUG] fillna function on a single column fail [BUG] fix fillna function on a single column fail Sep 30, 2024
@DKER2 DKER2 marked this pull request as ready for review September 30, 2024 05:46
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @shunping for label python.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@DKER2 DKER2 changed the title [BUG] fix fillna function on a single column fail [Bug] fix fillna function on a single column fail Sep 30, 2024
@shunping
Copy link
Contributor

Thanks for fixing this bug, @DKER2 . Could you also add a unit test (something the following) in to your PR?

import tempfile

import apache_beam as beam
from apache_beam.dataframe import convert
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to

with tempfile.NamedTemporaryFile(mode="w") as fp:
  fp.writelines(("col1,col2\n", "1,1\n", "NaN,1\n", "-1,1\n"))
  fp.flush()

  pipeline = beam.Pipeline(None)
  beam_df = pipeline | 'Read CSV' >> beam.dataframe.io.read_csv(fp.name)
  beam_df['col1'] = beam_df['col1'].fillna(0)
  assert_that(
    convert.to_pcollection(beam_df),
    equal_to([(1,1), (0, 1),(-1, 1)]))

@shunping
Copy link
Contributor

@tvalentyn

@DKER2
Copy link
Contributor Author

DKER2 commented Sep 30, 2024

Could you please check the name of the test and confirm if the file where I've placed it is correct?

@@ -354,6 +354,16 @@ def test_rename(self):
0: 2, 2: 0
}, errors='raise'))

def test_dataframe_column_fillna_constant_as_value(self):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would place it together with other fillna tests:

def test_fillna_columns(self):

Copy link
Contributor Author

@DKER2 DKER2 Oct 1, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have tried to put in that file like this commit , but seem that this function

def expr_to_stages(expr):

is not triggered, the test triggered different flow. And the test pass even with the old version of code.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please help to advise

Copy link
Contributor

@shunping shunping Oct 8, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the late reply.

The bug is only triggered in the transform of DeferredDataFrame, so you have to put that in a pipeline so that when the pipeline runs the corresponding transform code path will run.

Could you put a test like the following into frames_test.py?

import numpy as np

import apache_beam as beam
from apache_beam.dataframe import convert
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to

with beam.Pipeline(None) as p:
  pcoll = (p | beam.Create([1.0, np.nan, -1.0]) | beam.Select(x=lambda x: x))
  df = convert.to_dataframe(pcoll)
  df_new = df['x'].fillna(0)
  assert_that(
    convert.to_pcollection(df_new),
    equal_to([1.0, 0.0, -1.0]))

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @shunping for your suggestion. That works as well

Copy link
Contributor

github-actions bot commented Oct 8, 2024

Reminder, please take a look at this pr: @shunping

Copy link
Contributor

@shunping shunping left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please add the test? Thanks!

@DKER2
Copy link
Contributor Author

DKER2 commented Oct 9, 2024

Added Tests

Copy link
Contributor

@shunping shunping left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

github-actions bot commented Oct 9, 2024

R: @damccorm for final approval

Copy link
Contributor

@damccorm damccorm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks!

@damccorm damccorm merged commit b781b82 into apache:master Oct 9, 2024
93 checks passed
reeba212 pushed a commit to reeba212/beam that referenced this pull request Dec 4, 2024
* fix bug all arg add as inputs

* fix bug for fillna

* Revert "fix bug for fillna"

This reverts commit 2a5736c.

* fix bug for fillna

* add test for fillna a column

* add test for fillna a column

* add test for fillna a column

* revert add test to frames_test

* Move test from transforms to frames
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Bug]: In Beam Dataframe api, the fillna operation doesn't work when applied on individual columns
4 participants