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
[BEAM-12769] Adds support for expanding a Java cross-language transform using the class name and builder methods #15343
Conversation
R: @robertwb |
Codecov Report
@@ Coverage Diff @@
## master #15343 +/- ##
==========================================
- Coverage 83.77% 83.75% -0.03%
==========================================
Files 442 443 +1
Lines 60050 60077 +27
==========================================
+ Hits 50308 50318 +10
- Misses 9742 9759 +17
Continue to review full report at Codecov.
|
cc: @lukecwik |
Added following based on the discussions in the doc and the mailing list.
PTAL |
Friendly ping :) All tests pass now. |
R: @ihji as well |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does this work with complex types for method parameters like List and/or MyCustomUserType that can be loaded via Schema?
.../expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
Outdated
Show resolved
Hide resolved
.../expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
Outdated
Show resolved
Hide resolved
...ion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
Outdated
Show resolved
Hide resolved
...ce/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
Outdated
Show resolved
Hide resolved
...ce/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
Outdated
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Outdated
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. PTAL.
...ce/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
Outdated
Show resolved
Hide resolved
.../expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
Outdated
Show resolved
Hide resolved
.../expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
Outdated
Show resolved
Hide resolved
...ion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
Outdated
Show resolved
Hide resolved
...ce/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
Outdated
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Outdated
Show resolved
Hide resolved
...rc/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
Show resolved
Hide resolved
Regarding the "complex types" question. Yeah, this is expected to work for all types that can be represented by a Beam Row+schema. |
...ion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
Outdated
Show resolved
Hide resolved
...ion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
Outdated
Show resolved
Hide resolved
import org.apache.beam.sdk.options.Description; | ||
import org.apache.beam.sdk.options.PipelineOptions; | ||
|
||
public interface ExpansionServiceOptions extends PipelineOptions { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
interface comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
|
||
void setJavaClassLookupAllowlistFile(String file); | ||
|
||
class JavaClassLookupAllowListFactory implements DefaultValueFactory<AllowList> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Class comment, e.g.:
Loads the allow list from {@link #getJavaClassLookupAllowlistFile}, defaulting to an empty AllowList.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check that strField1, strField2 and intField1 was set.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check that strField1, strField2 and intField1 was set.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check that strField1, strField2 and intField1 was set.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Check that strField1, abc and xyz was set.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
|
||
public static class DummyTransform extends PTransform<PBegin, PCollection<String>> { | ||
|
||
String strField1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add tests for a wrapper type (e.g. Double field), and a complex type (non string, primitive, wrapper), and a list of simple type and list of complex type
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. I realized that we do not support some types yet. I added support for arrays.
Now we support:
- Java primitive types and Strings.
- Java types that can be represented by a Beam schema (if a schema has not
been registered service will try to generate a schema using
'JavaFieldSchema'). - Arrays of supported types.
We do not support collections as top level parameters yet. This can be added in the future.
I clarified what's supported in the spec and added tests for complex types and arrays.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added support for lists as well.
…class name and builder methods
Co-authored-by: Lukasz Cwik <lcwik@google.com>
58a192b
to
8ae5690
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. PTAL.
import org.apache.beam.sdk.options.Description; | ||
import org.apache.beam.sdk.options.PipelineOptions; | ||
|
||
public interface ExpansionServiceOptions extends PipelineOptions { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
|
||
void setJavaClassLookupAllowlistFile(String file); | ||
|
||
class JavaClassLookupAllowListFactory implements DefaultValueFactory<AllowList> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
public static final String ALLOW_LIST_VERSION = "v1"; | ||
|
||
public JavaClassLookupTransformProvider(AllowList allowList) { | ||
if (!allowList.getVersion().equals(ALLOW_LIST_VERSION)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will allow us to modify the format of the allow-list in the future while supporting old versions.
|
||
if (matchingMethods.size() != 1) { | ||
throw new RuntimeException( | ||
"Expected to find exact one matching method in transform " |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
.build(), | ||
"strField1")); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
"intField1")); | ||
payloadBuilder.addBuilderMethods(builderMethodBuilder); | ||
|
||
testClassLookupExpansionRequestConstruction(payloadBuilder.build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
|
||
public static class DummyTransform extends PTransform<PBegin, PCollection<String>> { | ||
|
||
String strField1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. I realized that we do not support some types yet. I added support for arrays.
Now we support:
- Java primitive types and Strings.
- Java types that can be represented by a Beam schema (if a schema has not
been registered service will try to generate a schema using
'JavaFieldSchema'). - Arrays of supported types.
We do not support collections as top level parameters yet. This can be added in the future.
I clarified what's supported in the spec and added tests for complex types and arrays.
…and each builder method. Updated the implementation accordingly and added additional tests.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As discussed offline, I simplified the proto and the implementation by removing the "Parameter" message and defining a single schema/payload for the constructor and each builder method.
This allows us to support more types in a generic way.
Added tests for complex types, arrays and lists.
PTAL. Thanks.
|
||
public static class DummyTransform extends PTransform<PBegin, PCollection<String>> { | ||
|
||
String strField1; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added support for lists as well.
Run Java PreCommit |
1 similar comment
Run Java PreCommit |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM,
mostly comment changes and nits
great work on expanding the set of types and simplifying down to one schema + row per method
// Name of the builder method | ||
string name = 1; | ||
|
||
// A schema that describes the parameter of the constructor or the constructor |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
// A schema that describes the parameter of the constructor or the constructor | |
// A schema that describes the parameters of the constructor or the constructor's |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Outdated now.
// The top level fields of the schema represents the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you disambiguate when we choose names over field ordering?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We check the number of fields as well as the names of the fields in order.
// Top level field names map to parameter names to use. | ||
Schema schema = 2; | ||
|
||
// A payload that maps to the provided builder method schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should state that we expect beam:coder:row:v1
encoding of the schema.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// Top level field names map to parameter names to use. | ||
Schema constructor_schema = 4; | ||
|
||
// A payload that maps to the provided constructor schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should state that we expect beam:coder:row:v1
encoding of the schema.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// transform object is constructed. | ||
// Given builder methods will be invoked in order when constructing the | ||
// transform objects. | ||
repeated BuilderMethod builder_methods = 3; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: re-order to place builder methods at the bottom so that constructor fields are grouped together.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// Top level field names map to parameter names to use. | ||
Schema schema = 2; | ||
|
||
// A payload that maps to the provided builder method schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should state that we expect beam:coder:row:v1
encoding of the schema.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// The top level fields of the schema represent the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
// The top level fields of the schema represent the parameters in order. | |
// Top level field names map to parameter names to use. | |
// The top level fields of the schema represent the method parameters in order. | |
// If able, top level field names are also verified against the method parameters for a match. |
Mapping names sounds like we will use the names to map parameters and not the field order. We seem to only use the names for validation that the correct parameter is being used.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// The top level fields of the schema represents the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
// The top level fields of the schema represents the parameters in order. | |
// Top level field names map to parameter names to use. | |
// The top level fields of the schema represent the method parameters in order. | |
// If able, top level field names are also verified against the method parameters for a match. |
Ditto here as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks!
// transform object is constructed. | ||
// Given builder methods will be invoked in order when constructing the | ||
// transform objects. | ||
repeated BuilderMethod builder_methods = 3; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// The top level fields of the schema represent the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// Top level field names map to parameter names to use. | ||
Schema constructor_schema = 4; | ||
|
||
// A payload that maps to the provided constructor schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// The top level fields of the schema represents the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// Top level field names map to parameter names to use. | ||
Schema schema = 2; | ||
|
||
// A payload that maps to the provided builder method schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// Name of the builder method | ||
string name = 1; | ||
|
||
// A schema that describes the parameter of the constructor or the constructor |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Outdated now.
// Top level field names map to parameter names to use. | ||
Schema schema = 2; | ||
|
||
// A payload that maps to the provided builder method schema. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
// The top level fields of the schema represents the parameters in order. | ||
// Top level field names map to parameter names to use. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We check the number of fields as well as the names of the fields in order.
Run XVR_Dataflow PostCommit |
Run XVR_Flink PostCommit |
Run Java PreCommit |
Run Java_Examples_Dataflow PreCommit |
Run Java PreCommit |
PreCommit failures are unrelated but trying again. |
Run Java PreCommit |
…rm using the class name and builder methods (apache#15343) * Adds support for expanding a Java cross-language transform using the class name and builder methods * Adds an allowlist and adds support for annotations * Fix tests * Address CheckerFramework errors * Adds license * Addresses reviewer comments. * Apply suggestions from code review Co-authored-by: Lukasz Cwik <lcwik@google.com> * Addresses reviewer comments. * Updated the proto to include a single schema/payload for constructor and each builder method. Updated the implementation accordingly and added additional tests. * Some doc updates and few other minor updates. * Addressing reviewer comments Co-authored-by: Lukasz Cwik <lcwik@google.com>
…rm using the class name and builder methods (apache#15343) * Adds support for expanding a Java cross-language transform using the class name and builder methods * Adds an allowlist and adds support for annotations * Fix tests * Address CheckerFramework errors * Adds license * Addresses reviewer comments. * Apply suggestions from code review Co-authored-by: Lukasz Cwik <lcwik@google.com> * Addresses reviewer comments. * Updated the proto to include a single schema/payload for constructor and each builder method. Updated the implementation accordingly and added additional tests. * Some doc updates and few other minor updates. * Addressing reviewer comments Co-authored-by: Lukasz Cwik <lcwik@google.com>
This adds proto and Java updates.
Python updates to follow.
Please see here for the design: https://docs.google.com/document/d/1ECXSWicE31K-vSxdb4qL6UcmovOAWvE-ZHFT3NTM654/edit?usp=sharing
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username
).[BEAM-XXX] Fixes bug in ApproximateQuantiles
, where you replaceBEAM-XXX
with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
ValidatesRunner
compliance status (on master branch)Examples testing status on various runners
Post-Commit SDK/Transform Integration Tests Status (on master branch)
Pre-Commit Tests Status (on master branch)
See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.