You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/08/19 22:25:41 UTC

[GitHub] [beam] chamikaramj opened a new pull request, #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

chamikaramj opened a new pull request, #22802:
URL: https://github.com/apache/beam/pull/22802

   This is a prototype implementation of https://s.apache.org/easy-multi-language
   
   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.
   
   End-to-end tested this by running a pipeline that writes to Kafka using a SchemaTransform-based Kafka Write implementation. Also update kafka.py to dynamically discover the identifier of the Kafka Write transform using the new transform discovery API.
   
   
   WIP (do not merge)
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) 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](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](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](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] robertwb commented on a diff in pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
robertwb commented on code in PR #22802:
URL: https://github.com/apache/beam/pull/22802#discussion_r950617150


##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,44 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def get_schema_proto_and_payload(self, *args, **kwargs):
+    named_fields = []
+    fields_to_values = OrderedDict()
+    next_field_id = 0
+    for value in args:
+      if value is None:
+        raise ValueError(
+            'Received value None. None values are currently not supported')
+      named_fields.append(
+          ((JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT % next_field_id),

Review Comment:
   This code isn't specific to JavaClassLookupPayload anymore, is it?



##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -106,4 +109,13 @@ message BuilderMethod {
   bytes payload = 3;
 }
 
+message SchemaTransformPayload {
+  // Identifier of the SchemaTransform
+  string identifier = 1;
 
+  // The config of the SchemaTransform.
+  // Should be decodable via beam:coder:row:v1.
+  // The schema of the Row should be compatible with the schema of the
+  // SchemaTransform denoted by the identifier.
+  bytes config_row = 2;

Review Comment:
   Should we also provide the schema that this is encoded against, in case the schema evolves (in a compatible way of course) between when the discovery happens and when this request is made. 



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,44 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def get_schema_proto_and_payload(self, *args, **kwargs):
+    named_fields = []
+    fields_to_values = OrderedDict()
+    next_field_id = 0
+    for value in args:
+      if value is None:
+        raise ValueError(
+            'Received value None. None values are currently not supported')
+      named_fields.append(
+          ((JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT % next_field_id),
+           convert_to_typing_type(instance_to_type(value))))
+      fields_to_values[(
+          JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT %
+          next_field_id)] = value
+      next_field_id += 1
+    for key, value in kwargs.items():
+      if not key:
+        raise ValueError('Parameter name cannot be empty')
+      if value is None:
+        raise ValueError(
+            'Received value None for key %s. None values are currently not '
+            'supported' % key)
+      named_fields.append(
+          (key, convert_to_typing_type(instance_to_type(value))))
+      fields_to_values[key] = value
+
+    schema_proto = named_fields_to_schema(named_fields)
+    row = named_tuple_from_schema(schema_proto)(**fields_to_values)
+
+    logging.error('********* xyz123 kwargs: %r', kwargs)

Review Comment:
   Extra logging.



##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -106,4 +109,13 @@ message BuilderMethod {
   bytes payload = 3;
 }
 
+message SchemaTransformPayload {
+  // Identifier of the SchemaTransform
+  string identifier = 1;

Review Comment:
   Should this typically be a urn as well? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on a diff in pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on code in PR #22802:
URL: https://github.com/apache/beam/pull/22802#discussion_r982698371


##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -106,4 +109,13 @@ message BuilderMethod {
   bytes payload = 3;
 }
 
+message SchemaTransformPayload {
+  // Identifier of the SchemaTransform
+  string identifier = 1;

Review Comment:
   I think so (though it's not specifically mentioned in the Schema-Aware Transform design doc)



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,44 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def get_schema_proto_and_payload(self, *args, **kwargs):
+    named_fields = []
+    fields_to_values = OrderedDict()
+    next_field_id = 0
+    for value in args:
+      if value is None:
+        raise ValueError(
+            'Received value None. None values are currently not supported')
+      named_fields.append(
+          ((JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT % next_field_id),
+           convert_to_typing_type(instance_to_type(value))))
+      fields_to_values[(
+          JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT %
+          next_field_id)] = value
+      next_field_id += 1
+    for key, value in kwargs.items():
+      if not key:
+        raise ValueError('Parameter name cannot be empty')
+      if value is None:
+        raise ValueError(
+            'Received value None for key %s. None values are currently not '
+            'supported' % key)
+      named_fields.append(
+          (key, convert_to_typing_type(instance_to_type(value))))
+      fields_to_values[key] = value
+
+    schema_proto = named_fields_to_schema(named_fields)
+    row = named_tuple_from_schema(schema_proto)(**fields_to_values)
+
+    logging.error('********* xyz123 kwargs: %r', kwargs)

Review Comment:
   Removed.



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,44 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def get_schema_proto_and_payload(self, *args, **kwargs):
+    named_fields = []
+    fields_to_values = OrderedDict()
+    next_field_id = 0
+    for value in args:
+      if value is None:
+        raise ValueError(
+            'Received value None. None values are currently not supported')
+      named_fields.append(
+          ((JavaClassLookupPayloadBuilder.IGNORED_ARG_FORMAT % next_field_id),

Review Comment:
   Updated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #22802:
URL: https://github.com/apache/beam/pull/22802#issuecomment-1221146919

   # [Codecov](https://codecov.io/gh/apache/beam/pull/22802?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#22802](https://codecov.io/gh/apache/beam/pull/22802?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (048da1d) into [master](https://codecov.io/gh/apache/beam/commit/720ee14e9c56e55750c1fed4b5c47d4486da5c80?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (720ee14) will **decrease** coverage by `0.08%`.
   > The diff coverage is `48.62%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #22802      +/-   ##
   ==========================================
   - Coverage   74.23%   74.14%   -0.09%     
   ==========================================
     Files         707      709       +2     
     Lines       93295    93579     +284     
   ==========================================
   + Hits        69257    69385     +128     
   - Misses      22770    22926     +156     
     Partials     1268     1268              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.45% <48.62%> (-0.17%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/22802?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ython/apache\_beam/examples/kafkataxi/kafka\_taxi.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMva2Fma2F0YXhpL2thZmthX3RheGkucHk=) | `0.00% <0.00%> (ø)` | |
   | [sdks/python/apache\_beam/io/kafka.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8va2Fma2EucHk=) | `54.34% <26.08%> (-25.66%)` | :arrow_down: |
   | [sdks/python/apache\_beam/transforms/external.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9leHRlcm5hbC5weQ==) | `75.40% <58.22%> (-4.34%)` | :arrow_down: |
   | [sdks/python/apache\_beam/portability/common\_urns.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcG9ydGFiaWxpdHkvY29tbW9uX3VybnMucHk=) | `100.00% <100.00%> (ø)` | |
   | [...l/job\_management/v1/beam\_expansion\_api\_pb2\_grpc.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcG9ydGFiaWxpdHkvYXBpL29yZy9hcGFjaGUvYmVhbS9tb2RlbC9qb2JfbWFuYWdlbWVudC92MS9iZWFtX2V4cGFuc2lvbl9hcGlfcGIyX2dycGMucHk=) | `51.85% <0.00%> (-6.05%)` | :arrow_down: |
   | [...examples/inference/sklearn\_mnist\_classification.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZXhhbXBsZXMvaW5mZXJlbmNlL3NrbGVhcm5fbW5pc3RfY2xhc3NpZmljYXRpb24ucHk=) | `43.75% <0.00%> (-3.75%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `95.12% <0.00%> (-2.44%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/subprocess\_server.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvc3VicHJvY2Vzc19zZXJ2ZXIucHk=) | `56.54% <0.00%> (-2.20%)` | :arrow_down: |
   | [...n/apache\_beam/ml/gcp/recommendations\_ai\_test\_it.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vbWwvZ2NwL3JlY29tbWVuZGF0aW9uc19haV90ZXN0X2l0LnB5) | `73.46% <0.00%> (-2.05%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | ... and [28 more](https://codecov.io/gh/apache/beam/pull/22802/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj closed pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj closed pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms
URL: https://github.com/apache/beam/pull/22802


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #22802: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #22802:
URL: https://github.com/apache/beam/pull/22802#issuecomment-1221183417

   Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment `assign set of reviewers`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org