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

Remove CoGBK in MLTransform's TFTProcessHandler #30146

Merged
merged 9 commits into from
Feb 13, 2024
Merged

Conversation

AnandInguva
Copy link
Contributor

@AnandInguva AnandInguva commented Jan 29, 2024

This PR attempts to remove CoGBK from MLTransform's TFTProcessHandler.

Instead of using CoGBK, encode the dict of columns not specified by the user into bytes and pass them in the original dict with a temporary key specified in the schema for TFT. In the TFT AnalyzeDataset or TransformDataset, this temp column would be a no-op. Once TFT is done processing data, decode the bytes into their actual format.

This only affects the data processing operations specified at apache_beam/ml/transforms/tft.py. The operations support float and string data. We have tests for them already.

For encoding and decoding purposes, I used apache_beam/pickler for now.

Fixes: #29593


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.

@AnandInguva AnandInguva marked this pull request as ready for review January 29, 2024 21:21
@AnandInguva AnandInguva requested a review from tvalentyn January 29, 2024 21:21
Copy link
Contributor

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

R: @damccorm 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).

"""
def process(self, element):
element.update(pickler.loads(element[_TEMP_KEY].item()))
del element[_TEMP_KEY]
Copy link
Contributor

Choose a reason for hiding this comment

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

let's not mutate elements. emit a copy.

Copy link
Contributor

Choose a reason for hiding this comment

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

sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved
if key in data_to_encode:
del data_to_encode[key]

bytes = pickler.dumps(data_to_encode)
Copy link
Contributor

Choose a reason for hiding this comment

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

It might be more efficient to use type-aware cythonized coder. Performance difference can be tested in a microbenchmark.

We might be able to convert elements to Beam Row, and use RowCoder. Seeing something similar in:

schema_proto = named_fields_to_schema(named_fields)
row = named_tuple_from_schema(schema_proto)(**fields_to_values)
schema = named_tuple_to_schema(type(row))
payload = RowCoder(schema).encode(row)
return (schema_proto, payload)

Can we expect the schema of the elements in the dict the same ? do we have type information from the elements or would need to infer them?

cc: @robertwb who might have a better idea

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For the unused elements, we won't know what the schema of the elements.

Copy link
Contributor

Choose a reason for hiding this comment

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

if we have to use pickling to serialize elements, I would use native pickler rather than Beam's, perhaps by way of PickleCoder.

Copy link
Contributor

@tvalentyn tvalentyn Jan 31, 2024

Choose a reason for hiding this comment

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

For the unused elements, we won't know what the schema of the elements.

According to MLTransform docs, elements end up being Rows:

To define a data processing transformation by using MLTransform, create instances of data processing transforms with columns as input parameters. The data in the specified columns is transformed and outputted to the beam.Row object.

do we infer the types later then?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Only the data for the. columns that are provided for transformation. For these columns using TFT, we infer the schema

@AnandInguva
Copy link
Contributor Author

I will make few changes and will request a review soon. Thanks

@AnandInguva AnandInguva requested a review from tvalentyn February 2, 2024 23:35
Copy link
Contributor

@tvalentyn tvalentyn left a comment

Choose a reason for hiding this comment

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

Have we run existing tests on this change?

@@ -447,22 +409,22 @@ def expand(
raw_data_metadata = metadata_io.read_metadata(
os.path.join(self.artifact_location, RAW_DATA_METADATA_DIR))

keyed_raw_data = (raw_data | beam.ParDo(_ComputeAndAttachUniqueID()))
keyed_raw_data = (raw_data) # | beam.ParDo(_ComputeAndAttachUniqueID()))
Copy link
Contributor

Choose a reason for hiding this comment

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

leftover comment, also we no longer add keys , so keyed_ might not be the best name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed the keyed_ from variable names

sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved

# To maintain consistency by outputting numpy array all the time,
# whether a scalar value or list or np array is passed as input,
# we will convert scalar values to list values and TFT will ouput
# numpy array all the time.
raw_data_list = (
Copy link
Contributor

Choose a reason for hiding this comment

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

for my understanding, why is this called raw_data_list? it's modified, so not raw i think, and what's here about _list?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, it is modified. I removed raw from the variable name.

_list: we convert the scalar element to list (len:1) to maintain uniformity. Users can pass list/np arrays to TFT ops and TFT outputs numpy arrays. Users when pass scalars, TFT outputs scalars. to maintain consistent output format, we convert scalar to list.

sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved
self.exclude_columns = exclude_columns

def encode(self, element):
if not self.exclude_columns:
Copy link
Contributor

Choose a reason for hiding this comment

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

interesting. Is it possible for exclude_columns be emtpy? I'd imagine it could rather be the opposite, where all columns are being processed, so there is nothing to encode/decode.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, that is right but it errors because we are adding the temp id column name to the schema during construction so TFT errors out if the pcoll doesn't have the temp id column. So when the unused columns are none, we have to encode the empty dict and pass it to the PColl.

Copy link
Contributor

@tvalentyn tvalentyn left a comment

Choose a reason for hiding this comment

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

Have we run existing tests on this change?

tvalentyn

This comment was marked as duplicate.

@AnandInguva
Copy link
Contributor Author

Have we run existing tests on this change?

yes, they should be covered in the py38coverage.

sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved

def decode(self, element):
clone = copy.copy(element)
clone.update(self.coder.decode(clone[_TEMP_KEY].item()))
Copy link
Contributor

Choose a reason for hiding this comment

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

what is the function of .item() here? what is the type of clone[_TEMP_KEY]? are the elements in given that we call .item() here - will elements in clone have consistent type after decoding?

Copy link
Contributor Author

@AnandInguva AnandInguva Feb 12, 2024

Choose a reason for hiding this comment

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

Type of clone[_TEMP_KEY] is a numpy array and .item() returns underlying element of the numpy array.

will elements in clone have consistent type after decoding.

It should be. depending on the Coder.

sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved
sdks/python/apache_beam/ml/transforms/handlers.py Outdated Show resolved Hide resolved
@AnandInguva AnandInguva self-assigned this Feb 9, 2024
@AnandInguva AnandInguva merged commit c004cc7 into master Feb 13, 2024
71 checks passed
@AnandInguva AnandInguva deleted the remove_cogbk branch February 13, 2024 19:19
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.

[Feature Request]: Evaluate MLTransform implementation without CoGroupByKey step
2 participants