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

Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms #23413

Merged
merged 5 commits into from
Nov 24, 2022

Conversation

chamikaramj
Copy link
Contributor

This adds:

Support for dynamically discovering and registering SchemaTransforms in the Java expansion service.
Support for dynamically discovering registered SchemaTransforms from the Python side.
Support for using SchemaTransforms in Python pipelines.

This addresses #23412


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

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • 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.

@chamikaramj
Copy link
Contributor Author

cc: @pabloem @johnjcasey @ahmedabu98

@codecov
Copy link

codecov bot commented Sep 29, 2022

Codecov Report

Merging #23413 (aa6e6c7) into master (226bc97) will increase coverage by 0.23%.
The diff coverage is 77.58%.

@@            Coverage Diff             @@
##           master   #23413      +/-   ##
==========================================
+ Coverage   73.08%   73.31%   +0.23%     
==========================================
  Files         725      711      -14     
  Lines       96664    96195     -469     
==========================================
- Hits        70647    70530     -117     
+ Misses      24705    24353     -352     
  Partials     1312     1312              
Flag Coverage Δ
python 83.13% <77.58%> (+0.40%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
sdks/python/apache_beam/transforms/external.py 78.88% <77.19%> (-0.86%) ⬇️
sdks/python/apache_beam/portability/common_urns.py 100.00% <100.00%> (ø)
...python/apache_beam/examples/complete/distribopt.py 0.00% <0.00%> (-98.57%) ⬇️
...hon/apache_beam/examples/cookbook/mergecontacts.py 23.21% <0.00%> (-73.22%) ⬇️
...he_beam/examples/cookbook/multiple_output_pardo.py 37.50% <0.00%> (-58.34%) ⬇️
...python/apache_beam/examples/wordcount_debugging.py 38.77% <0.00%> (-57.15%) ⬇️
...python/apache_beam/examples/dataframe/wordcount.py 38.46% <0.00%> (-53.85%) ⬇️
...s/python/apache_beam/examples/wordcount_minimal.py 40.74% <0.00%> (-51.86%) ⬇️
sdks/python/apache_beam/examples/wordcount.py 48.27% <0.00%> (-44.83%) ⬇️
...apache_beam/examples/cookbook/custom_ptransform.py 53.65% <0.00%> (-41.58%) ⬇️
... and 76 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@github-actions
Copy link
Contributor

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

R: @AnandInguva for label python.
R: @lukecwik for label java.

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).

@chamikaramj
Copy link
Contributor Author

R: @robertwb

@github-actions
Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@TheNeuralBit TheNeuralBit self-requested a review October 4, 2022 21:08
return transformProvider;
}

static class RowTransform extends PTransform {
Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps at least a 1-line docstring as to why this class exists?

Copy link
Contributor

Choose a reason for hiding this comment

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

Looking at this a bit more, inputs always come in (and leave) as dicts. Could we just preserve them in this case rather than adding another level of indirection?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added a docstring explaining why this is needed. May be we could get rid of this in the future if PCollectionRowTuple becomes a portable type that is understood by all SDKs.

Copy link
Contributor

Choose a reason for hiding this comment

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

So this class basically exists to undo the logic at https://github.com/apache/beam/blob/release-2.42.0/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L348 .

I think we should instead override the createInput (and possibly extractOutputs) methods in ExpansionServiceSchemaTransformProvider.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, that worked! Updated. Thanks.

}

@Override
public POutput expand(PInput input) {
Copy link
Contributor

Choose a reason for hiding this comment

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

For UI and uniqueness considerations, should we inherit the stage name of its underlying transform?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't think we can use the exact same name due to uniqueness requirement of runners but changed the name of this transform to a derivative of the underlying transform.

Copy link
Contributor

Choose a reason for hiding this comment

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

One can nest transforms of the same name. (In fact, nesting with distinct prefixes is how we get around users having to specify names everywhere.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.

}
PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);

if (output.getAll().size() > 1) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there any advantage to doing this? (In particular, the name of the single output gets lost.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This just converts the PCollectionRowTuple to a PCollectionTuple. I think currently we need this conversion.

Copy link
Contributor

Choose a reason for hiding this comment

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

I meant why can't we just return the PCollectionRowTuple itself? (Possibly we need to update the expansion service code to handle this type, but that should be done anyway.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.

@@ -104,6 +105,43 @@ def payload(self):
"""
return self.build().SerializeToString()

def _get_schema_proto_and_payload(self, ignored_arg_format, *args, **kwargs):
Copy link
Contributor

Choose a reason for hiding this comment

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

The ignored_arg_format param feels a bit awkward, and it'd be easy to accidentally forget it and grab the first arg as well (and I'm not sure the JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT logic should be in this superclass as well).

Instead, could we have the java-class calling one just make a dict using JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT for the positional args, and then call shared code to go from a dict to a schema and payload?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

self._expansion_service)

@staticmethod
def discover(expansion_service, regex=None):
Copy link
Contributor

Choose a reason for hiding this comment

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

For simplicity I think we can skip the regex filtering until we know we need it.

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.

sdks/python/apache_beam/transforms/external.py Outdated Show resolved Hide resolved
}

Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
return schemaTransformProviders.values();
Copy link
Member

Choose a reason for hiding this comment

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

so if a provider is not (duh) provided, even if the SchemaTransform implementation exists, it would not show up in the Discover call. Is that fine?

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 hope that we can eventually replace writing a Provider with a decorator on the (suitably configured) PTransform class itself, but that's future work.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ack. Resolving.

@chamikaramj
Copy link
Contributor Author

@pabloem I don't think a SchemaTransform is useful without the provider. Is that not the case ?

BTW I was held up due to some other work but hope to address comments here and push a change soon.

Copy link
Contributor Author

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

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

Thanks. PTAL.

return transformProvider;
}

static class RowTransform extends PTransform {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added a docstring explaining why this is needed. May be we could get rid of this in the future if PCollectionRowTuple becomes a portable type that is understood by all SDKs.

}

@Override
public POutput expand(PInput input) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't think we can use the exact same name due to uniqueness requirement of runners but changed the name of this transform to a derivative of the underlying transform.

@@ -104,6 +105,43 @@ def payload(self):
"""
return self.build().SerializeToString()

def _get_schema_proto_and_payload(self, ignored_arg_format, *args, **kwargs):
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

self._expansion_service)

@staticmethod
def discover(expansion_service, regex=None):
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.

}
PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);

if (output.getAll().size() > 1) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This just converts the PCollectionRowTuple to a PCollectionTuple. I think currently we need this conversion.

@chamikaramj
Copy link
Contributor Author

Retest this please

@chamikaramj
Copy link
Contributor Author

Run Java PreCommit

@chamikaramj
Copy link
Contributor Author

Run Python PreCommit

@chamikaramj
Copy link
Contributor Author

Friendly ping :)


@Override
public String getName() {
return "RowTransform_of_" + this.rowTuplePTransform.getName();
Copy link
Contributor

Choose a reason for hiding this comment

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

Does the "RowTransform_of_" prefix add value? Maybe simply drop it, as this will be the name the user sees before drilling down.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.

}

@Override
public POutput expand(PInput input) {
Copy link
Contributor

Choose a reason for hiding this comment

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

One can nest transforms of the same name. (In fact, nesting with distinct prefixes is how we get around users having to specify names everywhere.)

}
PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);

if (output.getAll().size() > 1) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I meant why can't we just return the PCollectionRowTuple itself? (Possibly we need to update the expansion service code to handle this type, but that should be done anyway.)

return transformProvider;
}

static class RowTransform extends PTransform {
Copy link
Contributor

Choose a reason for hiding this comment

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

So this class basically exists to undo the logic at https://github.com/apache/beam/blob/release-2.42.0/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L348 .

I think we should instead override the createInput (and possibly extractOutputs) methods in ExpansionServiceSchemaTransformProvider.

}

Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
return schemaTransformProviders.values();
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 hope that we can eventually replace writing a Provider with a decorator on the (suitably configured) PTransform class itself, but that's future work.

# Expand the transform using the expansion service and the config_row.
if self._expansion_service is None:
self._expansion_service = BeamJarExpansionService(
':sdks:java:expansion-service:app:shadowJar',
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we expect many schema transforms to live in this jar, or should we make identifying the expansion service mandatory?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't expect many (or any) SchemaTransforms to live in this jar. So I believe we should make specifying an expansion service mandatory.

ComplexType = typing.NamedTuple(
"ComplexType", [
("str_sub_field", str),
("int_sub_field", np.int32),
Copy link
Contributor

Choose a reason for hiding this comment

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

Can't we just use int here? (Does that give us an int64?)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah just "int" works. Updated.

@chamikaramj
Copy link
Contributor Author

Thanks. Hope to respond by the end of the week (have been bit busy due to the release).

Copy link
Contributor Author

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

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

Thanks Robert. PTAL.

return transformProvider;
}

static class RowTransform extends PTransform {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, that worked! Updated. Thanks.

}

@Override
public POutput expand(PInput input) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.

}
PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);

if (output.getAll().size() > 1) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.


@Override
public String getName() {
return "RowTransform_of_" + this.rowTuplePTransform.getName();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

RowTransform class was removed. So this is obsolete now.

}

Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
return schemaTransformProviders.values();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ack. Resolving.

# Expand the transform using the expansion service and the config_row.
if self._expansion_service is None:
self._expansion_service = BeamJarExpansionService(
':sdks:java:expansion-service:app:shadowJar',
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't expect many (or any) SchemaTransforms to live in this jar. So I believe we should make specifying an expansion service mandatory.

ComplexType = typing.NamedTuple(
"ComplexType", [
("str_sub_field", str),
("int_sub_field", np.int32),
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah just "int" works. Updated.

return PCollectionRowTuple.of(
DEFAULT_INPUT_TAG, (PCollection<Row>) inputs.values().iterator().next());
} else {
PCollectionRowTuple inputRowTuple = PCollectionRowTuple.empty(p);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can't we use this for all arities, rather than special-casing the 0 and 1 sizes above?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated.

} else if (output.getAll().size() == 1) {
return ImmutableMap.of("output", output.getAll().values().iterator().next());
} else {
ImmutableMap.Builder<String, PCollection<?>> pCollectionMap = ImmutableMap.builder();
Copy link
Contributor

Choose a reason for hiding this comment

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

Similarly.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated.

@chamikaramj chamikaramj deleted the easy_x-lang_final branch November 23, 2022 00:00
@chamikaramj chamikaramj restored the easy_x-lang_final branch November 23, 2022 00:01
@chamikaramj chamikaramj reopened this Nov 23, 2022
@chamikaramj
Copy link
Contributor Author

Thanks!

@chamikaramj
Copy link
Contributor Author

Run Java PreCommit

@chamikaramj
Copy link
Contributor Author

Run GoPortable PreCommit

@chamikaramj
Copy link
Contributor Author

Run Java PreCommit

1 similar comment
@chamikaramj
Copy link
Contributor Author

Run Java PreCommit

@chamikaramj
Copy link
Contributor Author

Run GoPortable PreCommit

@chamikaramj
Copy link
Contributor Author

Run Java_Examples_Dataflow PreCommit

@chamikaramj
Copy link
Contributor Author

Run GoPortable PreCommit

@chamikaramj
Copy link
Contributor Author

Run Java_Examples_Dataflow_Java11 PreCommit

@chamikaramj
Copy link
Contributor Author

Run Java_Examples_Dataflow_Java17 PreCommit

@chamikaramj chamikaramj merged commit 23e59af into apache:master Nov 24, 2022
ruslan-ikhsan pushed a commit to ruslan-ikhsan/beam that referenced this pull request Nov 30, 2022
…ding SchemaTransforms (apache#23413)

* Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

* Fixing checker framework errors.

* Address reviewer comments

* Addressing reviewer comments

* Addressing reviewer comments
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.

3 participants