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/11/23 00:01:47 UTC

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

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

   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)**](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] chamikaramj commented on pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325394418

   Run Java PreCommit


-- 
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] pabloem commented on a diff in pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {
+        PCollectionTuple pcTuple = PCollectionTuple.empty(input.getPipeline());
+        for (String key : output.getAll().keySet()) {
+          pcTuple = pcTuple.and(key, output.get(key));
+        }
+        return pcTuple;
+      } else if (output.getAll().size() == 1) {
+        return output.getAll().values().iterator().next();
+      } else {
+        return PDone.in(input.getPipeline());
+      }
+    }
+  }
+
+  @Override
+  public PTransform getTransform(FunctionSpec spec) {
+    SchemaTransformPayload payload;
+    try {
+      payload = SchemaTransformPayload.parseFrom(spec.getPayload());
+      String identifier = payload.getIdentifier();
+      if (!schemaTransformProviders.containsKey(identifier)) {
+        throw new RuntimeException(
+            "Did not find a SchemaTransformProvider with the identifier " + identifier);
+      }
+
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Invalid payload type for URN " + getUrn(ExpansionMethods.Enum.SCHEMATRANSFORM), e);
+    }
+
+    String identifier = payload.getIdentifier();
+    org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider provider =
+        schemaTransformProviders.get(identifier);
+    if (provider == null) {
+      throw new IllegalArgumentException(
+          "Could not find a SchemaTransform with identifier " + identifier);
+    }
+
+    Schema configSchemaFromRequest =
+        SchemaTranslation.schemaFromProto((payload.getConfigurationSchema()));
+    Schema configSchemaFromProvider = provider.configurationSchema();
+
+    if (!configSchemaFromRequest.assignableTo(configSchemaFromProvider)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Config schema provided with the expansion request %s is not compatible with the "
+                  + "config of the Schema transform %s.",
+              configSchemaFromRequest, configSchemaFromProvider));
+    }
+
+    Row configRow;
+    try {
+      configRow =
+          RowCoder.of(provider.configurationSchema())
+              .decode(payload.getConfigurationRow().newInput());
+    } catch (IOException e) {
+      throw new RuntimeException("Error decoding payload", e);
+    }
+
+    return new RowTransform(provider.from(configRow).buildTransform());
+  }
+
+  Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
+    return schemaTransformProviders.values();

Review Comment:
   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?



-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider
+    implements TransformProvider<PCollectionRowTuple, PCollectionRowTuple> {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  @Override
+  public PCollectionRowTuple createInput(Pipeline p, Map<String, PCollection<?>> inputs) {
+    if (inputs.size() == 0) {
+      return PCollectionRowTuple.empty(p);
+    }
+    if (inputs.size() == 1) {
+      return PCollectionRowTuple.of(
+          DEFAULT_INPUT_TAG, (PCollection<Row>) inputs.values().iterator().next());
+    } else {
+      PCollectionRowTuple inputRowTuple = PCollectionRowTuple.empty(p);

Review Comment:
   Updated.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider
+    implements TransformProvider<PCollectionRowTuple, PCollectionRowTuple> {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  @Override
+  public PCollectionRowTuple createInput(Pipeline p, Map<String, PCollection<?>> inputs) {
+    if (inputs.size() == 0) {
+      return PCollectionRowTuple.empty(p);
+    }
+    if (inputs.size() == 1) {
+      return PCollectionRowTuple.of(
+          DEFAULT_INPUT_TAG, (PCollection<Row>) inputs.values().iterator().next());
+    } else {
+      PCollectionRowTuple inputRowTuple = PCollectionRowTuple.empty(p);
+      for (Map.Entry<String, PCollection<?>> entry : inputs.entrySet()) {
+        inputRowTuple = inputRowTuple.and(entry.getKey(), (PCollection<Row>) entry.getValue());
+      }
+      return inputRowTuple;
+    }
+  }
+
+  @Override
+  public Map<String, PCollection<?>> extractOutputs(PCollectionRowTuple output) {
+    if (output.getAll().size() == 0) {
+      return Collections.emptyMap();
+    } else if (output.getAll().size() == 1) {
+      return ImmutableMap.of("output", output.getAll().values().iterator().next());
+    } else {
+      ImmutableMap.Builder<String, PCollection<?>> pCollectionMap = ImmutableMap.builder();

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] chamikaramj commented on pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1324564824

   Run GoPortable PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325723619

   Run Java_Examples_Dataflow PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1290950672

   Retest this please


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325790330

   Run Java_Examples_Dataflow_Java11 PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325723395

   Run GoPortable PreCommit


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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

   # [Codecov](https://codecov.io/gh/apache/beam/pull/23413?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 [#23413](https://codecov.io/gh/apache/beam/pull/23413?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6ec0d64) into [master](https://codecov.io/gh/apache/beam/commit/b59df6c7614f3234d5140892a108779aaa807e79?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (b59df6c) will **increase** coverage by `0.00%`.
   > The diff coverage is `64.55%`.
   
   ```diff
   @@           Coverage Diff            @@
   ##           master   #23413    +/-   ##
   ========================================
     Coverage   73.41%   73.42%            
   ========================================
     Files         718      718            
     Lines       95652    95766   +114     
   ========================================
   + Hits        70222    70313    +91     
   - Misses      24119    24142    +23     
     Partials     1311     1311            
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.17% <64.55%> (-0.01%)` | :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/23413?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/transforms/external.py](https://codecov.io/gh/apache/beam/pull/23413/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==) | `76.51% <64.10%> (-3.23%)` | :arrow_down: |
   | [sdks/python/apache\_beam/portability/common\_urns.py](https://codecov.io/gh/apache/beam/pull/23413/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/23413/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: |
   | [.../python/apache\_beam/testing/test\_stream\_service.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy90ZXN0X3N0cmVhbV9zZXJ2aWNlLnB5) | `88.09% <0.00%> (-4.77%)` | :arrow_down: |
   | [.../apache\_beam/runners/interactive/dataproc/types.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kYXRhcHJvYy90eXBlcy5weQ==) | `93.10% <0.00%> (-3.45%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/data\_plane.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvZGF0YV9wbGFuZS5weQ==) | `87.57% <0.00%> (-1.70%)` | :arrow_down: |
   | [.../python/apache\_beam/transforms/periodicsequence.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9wZXJpb2RpY3NlcXVlbmNlLnB5) | `98.38% <0.00%> (-1.62%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `90.50% <0.00%> (-1.27%)` | :arrow_down: |
   | [...eam/runners/portability/fn\_api\_runner/execution.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL2V4ZWN1dGlvbi5weQ==) | `92.44% <0.00%> (-0.65%)` | :arrow_down: |
   | [sdks/python/apache\_beam/metrics/execution.py](https://codecov.io/gh/apache/beam/pull/23413/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vbWV0cmljcy9leGVjdXRpb24ucHk=) | `87.96% <0.00%> (ø)` | |
   | ... and [6 more](https://codecov.io/gh/apache/beam/pull/23413/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 commented on pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1297515499

   Friendly ping :)


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1281401912

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


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -51,6 +51,11 @@ message ExpansionMethods {
     // Transform payload will be of type JavaClassLookupPayload.
     JAVA_CLASS_LOOKUP = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) =
       "beam:expansion:payload:java_class_lookup:v1"];
+
+    // Expanding a SchemaTransform identified by the expansion service.
+    // Transform payload will be of type  SchemaTransformPayload.
+    SCHEMATRANSFORM = 1 [(org.apache.beam.model.pipeline.v1.beam_urn) =

Review Comment:
   Done.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {

Review Comment:
   RowTransform class was removed. So this is obsolete now.



##########
sdks/python/apache_beam/transforms/external_test.py:
##########
@@ -445,6 +447,35 @@ class DataclassTransform(beam.ExternalTransform):
     return get_payload(DataclassTransform(**values))
 
 
+class SchemaTransformPayloadBuilderTest(unittest.TestCase):
+  def test_build_payload(self):
+    ComplexType = typing.NamedTuple(
+        "ComplexType", [
+            ("str_sub_field", str),
+            ("int_sub_field", np.int32),

Review Comment:
   Yeah just "int" works. Updated.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {
+        PCollectionTuple pcTuple = PCollectionTuple.empty(input.getPipeline());
+        for (String key : output.getAll().keySet()) {
+          pcTuple = pcTuple.and(key, output.get(key));
+        }
+        return pcTuple;
+      } else if (output.getAll().size() == 1) {
+        return output.getAll().values().iterator().next();
+      } else {
+        return PDone.in(input.getPipeline());
+      }
+    }
+  }
+
+  @Override
+  public PTransform getTransform(FunctionSpec spec) {
+    SchemaTransformPayload payload;
+    try {
+      payload = SchemaTransformPayload.parseFrom(spec.getPayload());
+      String identifier = payload.getIdentifier();
+      if (!schemaTransformProviders.containsKey(identifier)) {
+        throw new RuntimeException(
+            "Did not find a SchemaTransformProvider with the identifier " + identifier);
+      }
+
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Invalid payload type for URN " + getUrn(ExpansionMethods.Enum.SCHEMATRANSFORM), e);
+    }
+
+    String identifier = payload.getIdentifier();
+    org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider provider =
+        schemaTransformProviders.get(identifier);
+    if (provider == null) {
+      throw new IllegalArgumentException(
+          "Could not find a SchemaTransform with identifier " + identifier);
+    }
+
+    Schema configSchemaFromRequest =
+        SchemaTranslation.schemaFromProto((payload.getConfigurationSchema()));
+    Schema configSchemaFromProvider = provider.configurationSchema();
+
+    if (!configSchemaFromRequest.assignableTo(configSchemaFromProvider)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Config schema provided with the expansion request %s is not compatible with the "
+                  + "config of the Schema transform %s.",
+              configSchemaFromRequest, configSchemaFromProvider));
+    }
+
+    Row configRow;
+    try {
+      configRow =
+          RowCoder.of(provider.configurationSchema())
+              .decode(payload.getConfigurationRow().newInput());
+    } catch (IOException e) {
+      throw new RuntimeException("Error decoding payload", e);
+    }
+
+    return new RowTransform(provider.from(configRow).buildTransform());
+  }
+
+  Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
+    return schemaTransformProviders.values();

Review Comment:
   Ack. Resolving.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {

Review Comment:
   RowTransform class was removed. So this is obsolete now.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {

Review Comment:
   Yeah, that worked! Updated. Thanks.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -118,6 +123,11 @@ public POutput expand(PInput input) {
         return PDone.in(input.getPipeline());
       }
     }
+
+    @Override
+    public String getName() {
+      return "RowTransform_of_" + this.rowTuplePTransform.getName();

Review Comment:
   RowTransform class was removed. So this is obsolete now.



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -289,6 +310,70 @@ def _has_constructor(self):
         self._constructor_param_kwargs)
 
 
+# Information regarding a SchemaTransform available in an external SDK.
+SchemaTransformsConfig = namedtuple(
+    'SchemaTransformsConfig',
+    ['identifier', 'configuration_schema', 'inputs', 'outputs'])
+
+
+class SchemaAwareExternalTransform(ptransform.PTransform):
+  """A proxy transform for SchemaTransforms implemented in external SDKs.
+
+  This allows Python pipelines to directly use existing SchemaTransforms
+  available to the expansion service without adding additional code in external
+  SDKs.
+
+  :param identifier: unique identifier of the SchemaTransform.
+  :param expansion_service: (Optional) an expansion service to use.  If none is
+      provided, a default expansion service will be started.
+  :param classpath: (Optional) A list paths to additional jars to place on the
+      expansion service classpath.
+  :kwargs: field name to value mapping for configuring the schema transform.
+      keys map to the field names of the schema of the SchemaTransform
+      (in-order).
+  """
+  def __init__(
+      self, identifier, expansion_service=None, classpath=None, **kwargs):
+    self._expansion_service = expansion_service
+    self._payload_builder = SchemaTransformPayloadBuilder(identifier, **kwargs)
+    self._classpath = classpath
+
+  def expand(self, pcolls):
+    # Register transform with the expansion service and the identifier.
+    # 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',

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



-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {

Review Comment:
   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.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {

Review Comment:
   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.)



##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -51,6 +51,11 @@ message ExpansionMethods {
     // Transform payload will be of type JavaClassLookupPayload.
     JAVA_CLASS_LOOKUP = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) =
       "beam:expansion:payload:java_class_lookup:v1"];
+
+    // Expanding a SchemaTransform identified by the expansion service.
+    // Transform payload will be of type  SchemaTransformPayload.
+    SCHEMATRANSFORM = 1 [(org.apache.beam.model.pipeline.v1.beam_urn) =

Review Comment:
   Another ping on this.



##########
sdks/python/apache_beam/transforms/external_test.py:
##########
@@ -445,6 +447,35 @@ class DataclassTransform(beam.ExternalTransform):
     return get_payload(DataclassTransform(**values))
 
 
+class SchemaTransformPayloadBuilderTest(unittest.TestCase):
+  def test_build_payload(self):
+    ComplexType = typing.NamedTuple(
+        "ComplexType", [
+            ("str_sub_field", str),
+            ("int_sub_field", np.int32),

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



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -118,6 +123,11 @@ public POutput expand(PInput input) {
         return PDone.in(input.getPipeline());
       }
     }
+
+    @Override
+    public String getName() {
+      return "RowTransform_of_" + this.rowTuplePTransform.getName();

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



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {
+        PCollectionTuple pcTuple = PCollectionTuple.empty(input.getPipeline());
+        for (String key : output.getAll().keySet()) {
+          pcTuple = pcTuple.and(key, output.get(key));
+        }
+        return pcTuple;
+      } else if (output.getAll().size() == 1) {
+        return output.getAll().values().iterator().next();
+      } else {
+        return PDone.in(input.getPipeline());
+      }
+    }
+  }
+
+  @Override
+  public PTransform getTransform(FunctionSpec spec) {
+    SchemaTransformPayload payload;
+    try {
+      payload = SchemaTransformPayload.parseFrom(spec.getPayload());
+      String identifier = payload.getIdentifier();
+      if (!schemaTransformProviders.containsKey(identifier)) {
+        throw new RuntimeException(
+            "Did not find a SchemaTransformProvider with the identifier " + identifier);
+      }
+
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Invalid payload type for URN " + getUrn(ExpansionMethods.Enum.SCHEMATRANSFORM), e);
+    }
+
+    String identifier = payload.getIdentifier();
+    org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider provider =
+        schemaTransformProviders.get(identifier);
+    if (provider == null) {
+      throw new IllegalArgumentException(
+          "Could not find a SchemaTransform with identifier " + identifier);
+    }
+
+    Schema configSchemaFromRequest =
+        SchemaTranslation.schemaFromProto((payload.getConfigurationSchema()));
+    Schema configSchemaFromProvider = provider.configurationSchema();
+
+    if (!configSchemaFromRequest.assignableTo(configSchemaFromProvider)) {
+      throw new IllegalArgumentException(
+          String.format(
+              "Config schema provided with the expansion request %s is not compatible with the "
+                  + "config of the Schema transform %s.",
+              configSchemaFromRequest, configSchemaFromProvider));
+    }
+
+    Row configRow;
+    try {
+      configRow =
+          RowCoder.of(provider.configurationSchema())
+              .decode(payload.getConfigurationRow().newInput());
+    } catch (IOException e) {
+      throw new RuntimeException("Error decoding payload", e);
+    }
+
+    return new RowTransform(provider.from(configRow).buildTransform());
+  }
+
+  Iterable<org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider> getAllProviders() {
+    return schemaTransformProviders.values();

Review Comment:
   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.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {

Review Comment:
   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*.)



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -289,6 +310,70 @@ def _has_constructor(self):
         self._constructor_param_kwargs)
 
 
+# Information regarding a SchemaTransform available in an external SDK.
+SchemaTransformsConfig = namedtuple(
+    'SchemaTransformsConfig',
+    ['identifier', 'configuration_schema', 'inputs', 'outputs'])
+
+
+class SchemaAwareExternalTransform(ptransform.PTransform):
+  """A proxy transform for SchemaTransforms implemented in external SDKs.
+
+  This allows Python pipelines to directly use existing SchemaTransforms
+  available to the expansion service without adding additional code in external
+  SDKs.
+
+  :param identifier: unique identifier of the SchemaTransform.
+  :param expansion_service: (Optional) an expansion service to use.  If none is
+      provided, a default expansion service will be started.
+  :param classpath: (Optional) A list paths to additional jars to place on the
+      expansion service classpath.
+  :kwargs: field name to value mapping for configuring the schema transform.
+      keys map to the field names of the schema of the SchemaTransform
+      (in-order).
+  """
+  def __init__(
+      self, identifier, expansion_service=None, classpath=None, **kwargs):
+    self._expansion_service = expansion_service
+    self._payload_builder = SchemaTransformPayloadBuilder(identifier, **kwargs)
+    self._classpath = classpath
+
+  def expand(self, pcolls):
+    # Register transform with the expansion service and the identifier.
+    # 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',

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



-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1294261002

   Run Java PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1294261171

   Run Python PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1262731179

   R: @robertwb 


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);

Review Comment:
   Cast to PCollection<Row>?



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {

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



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {

Review Comment:
   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? 



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {

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



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {

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



##########
model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto:
##########
@@ -51,6 +51,11 @@ message ExpansionMethods {
     // Transform payload will be of type JavaClassLookupPayload.
     JAVA_CLASS_LOOKUP = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) =
       "beam:expansion:payload:java_class_lookup:v1"];
+
+    // Expanding a SchemaTransform identified by the expansion service.
+    // Transform payload will be of type  SchemaTransformPayload.
+    SCHEMATRANSFORM = 1 [(org.apache.beam.model.pipeline.v1.beam_urn) =

Review Comment:
   SCHEMA_TRANSFORM



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,43 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def _get_schema_proto_and_payload(self, ignored_arg_format, *args, **kwargs):

Review Comment:
   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?



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -289,6 +311,96 @@ def _has_constructor(self):
         self._constructor_param_kwargs)
 
 
+class SchemaTransformsConfig(object):

Review Comment:
   Should this be a named tuple? Any reason to not return the proto itself? 



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -289,6 +311,96 @@ def _has_constructor(self):
         self._constructor_param_kwargs)
 
 
+class SchemaTransformsConfig(object):
+  """
+  Information regarding a SchemaTransform available in an external SDK.
+  """
+  def __init__(self, identifier, schema, named_inputs, named_outputs):
+    self._identifier = identifier
+    self._configuration_schema = schema
+    self._named_inputs = named_inputs
+    self._named_outputs = named_outputs
+
+  @property
+  def identifier(self):
+    return self._identifier
+
+  @property
+  def configuration_schema(self):
+    return self._configuration_schema
+
+  @property
+  def named_inputs(self):
+    return self._named_inputs
+
+  @property
+  def named_outputs(self):
+    return self._named_outputs
+
+
+class SchemaAwareExternalTransform(ptransform.PTransform):
+  """A proxy transform for SchemaTransforms implemented in external SDKs.
+
+  This allows Python pipelines to directly use existing SchemaTransforms
+  available to the expansion service without adding additional code in external
+  SDKs.
+
+  :param identifier: unique identifier of the SchemaTransform.
+  :param expansion_service: (Optional) an expansion service to use.  If none is
+      provided, a default expansion service will be started.
+  :param classpath: (Optional) A list paths to additional jars to place on the
+      expansion service classpath.
+  :kwargs: field name to value mapping for configuring the schema transform.
+      keys map to the field names of the schema of the SchemaTransform
+      (in-order).
+  """
+  def __init__(
+      self, identifier, expansion_service=None, classpath=None, **kwargs):
+    self._expansion_service = expansion_service
+    self._payload_builder = SchemaTransformPayloadBuilder(identifier, **kwargs)
+    self._classpath = classpath
+
+  def expand(self, pcolls):
+    # Register transform with the expansion service and the identifier.
+    # 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',
+          extra_args=['{{PORT}}'],
+          classpath=self._classpath)
+    return pcolls | ExternalTransform(
+        common_urns.schematransform_based_expand.urn,
+        self._payload_builder,
+        self._expansion_service)
+
+  @staticmethod
+  def discover(expansion_service, regex=None):

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



-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1324564691

   Run Java PreCommit


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider
+    implements TransformProvider<PCollectionRowTuple, PCollectionRowTuple> {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  @Override
+  public PCollectionRowTuple createInput(Pipeline p, Map<String, PCollection<?>> inputs) {
+    if (inputs.size() == 0) {
+      return PCollectionRowTuple.empty(p);
+    }
+    if (inputs.size() == 1) {
+      return PCollectionRowTuple.of(
+          DEFAULT_INPUT_TAG, (PCollection<Row>) inputs.values().iterator().next());
+    } else {
+      PCollectionRowTuple inputRowTuple = PCollectionRowTuple.empty(p);
+      for (Map.Entry<String, PCollection<?>> entry : inputs.entrySet()) {
+        inputRowTuple = inputRowTuple.and(entry.getKey(), (PCollection<Row>) entry.getValue());
+      }
+      return inputRowTuple;
+    }
+  }
+
+  @Override
+  public Map<String, PCollection<?>> extractOutputs(PCollectionRowTuple output) {
+    if (output.getAll().size() == 0) {
+      return Collections.emptyMap();
+    } else if (output.getAll().size() == 1) {
+      return ImmutableMap.of("output", output.getAll().values().iterator().next());
+    } else {
+      ImmutableMap.Builder<String, PCollection<?>> pCollectionMap = ImmutableMap.builder();

Review Comment:
   Similarly. 



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider
+    implements TransformProvider<PCollectionRowTuple, PCollectionRowTuple> {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  @Override
+  public PCollectionRowTuple createInput(Pipeline p, Map<String, PCollection<?>> inputs) {
+    if (inputs.size() == 0) {
+      return PCollectionRowTuple.empty(p);
+    }
+    if (inputs.size() == 1) {
+      return PCollectionRowTuple.of(
+          DEFAULT_INPUT_TAG, (PCollection<Row>) inputs.values().iterator().next());
+    } else {
+      PCollectionRowTuple inputRowTuple = PCollectionRowTuple.empty(p);

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



-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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

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


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1261637260

   cc: @pabloem @johnjcasey @ahmedabu98 


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325506573

   Run Java PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1324385239

   Thanks!


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1307714258

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


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325773407

   Run GoPortable PreCommit


-- 
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 pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #23413:
URL: https://github.com/apache/beam/pull/23413#issuecomment-1325790446

   Run Java_Examples_Dataflow_Java17 PreCommit


-- 
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 merged pull request #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

Posted by GitBox <gi...@apache.org>.
chamikaramj merged PR #23413:
URL: https://github.com/apache/beam/pull/23413


-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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


##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {

Review Comment:
   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.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {

Review Comment:
   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.



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -289,6 +311,96 @@ def _has_constructor(self):
         self._constructor_param_kwargs)
 
 
+class SchemaTransformsConfig(object):
+  """
+  Information regarding a SchemaTransform available in an external SDK.
+  """
+  def __init__(self, identifier, schema, named_inputs, named_outputs):
+    self._identifier = identifier
+    self._configuration_schema = schema
+    self._named_inputs = named_inputs
+    self._named_outputs = named_outputs
+
+  @property
+  def identifier(self):
+    return self._identifier
+
+  @property
+  def configuration_schema(self):
+    return self._configuration_schema
+
+  @property
+  def named_inputs(self):
+    return self._named_inputs
+
+  @property
+  def named_outputs(self):
+    return self._named_outputs
+
+
+class SchemaAwareExternalTransform(ptransform.PTransform):
+  """A proxy transform for SchemaTransforms implemented in external SDKs.
+
+  This allows Python pipelines to directly use existing SchemaTransforms
+  available to the expansion service without adding additional code in external
+  SDKs.
+
+  :param identifier: unique identifier of the SchemaTransform.
+  :param expansion_service: (Optional) an expansion service to use.  If none is
+      provided, a default expansion service will be started.
+  :param classpath: (Optional) A list paths to additional jars to place on the
+      expansion service classpath.
+  :kwargs: field name to value mapping for configuring the schema transform.
+      keys map to the field names of the schema of the SchemaTransform
+      (in-order).
+  """
+  def __init__(
+      self, identifier, expansion_service=None, classpath=None, **kwargs):
+    self._expansion_service = expansion_service
+    self._payload_builder = SchemaTransformPayloadBuilder(identifier, **kwargs)
+    self._classpath = classpath
+
+  def expand(self, pcolls):
+    # Register transform with the expansion service and the identifier.
+    # 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',
+          extra_args=['{{PORT}}'],
+          classpath=self._classpath)
+    return pcolls | ExternalTransform(
+        common_urns.schematransform_based_expand.urn,
+        self._payload_builder,
+        self._expansion_service)
+
+  @staticmethod
+  def discover(expansion_service, regex=None):

Review Comment:
   Removed.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);
+      } else if (input instanceof PBegin) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+      } else if (input instanceof PCollectionTuple) {
+        inputRowTuple = PCollectionRowTuple.empty(input.getPipeline());
+        PCollectionTuple inputTuple = (PCollectionTuple) input;
+        for (TupleTag<?> tag : inputTuple.getAll().keySet()) {
+          inputRowTuple = inputRowTuple.and(tag.getId(), (PCollection<Row>) inputTuple.get(tag));
+        }
+      } else {
+        throw new RuntimeException(String.format("Unsupported input type: %s", input));
+      }
+      PCollectionRowTuple output = inputRowTuple.apply(this.rowTuplePTransform);
+
+      if (output.getAll().size() > 1) {

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



##########
sdks/python/apache_beam/transforms/external.py:
##########
@@ -104,6 +105,43 @@ def payload(self):
     """
     return self.build().SerializeToString()
 
+  def _get_schema_proto_and_payload(self, ignored_arg_format, *args, **kwargs):

Review Comment:
   Done.



##########
sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceSchemaTransformProvider.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@SuppressWarnings({"rawtypes"})
+public class ExpansionServiceSchemaTransformProvider implements TransformProvider {
+
+  static final String DEFAULT_INPUT_TAG = "INPUT";
+
+  private Map<String, org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider>
+      schemaTransformProviders = new HashMap<>();
+  private static @Nullable ExpansionServiceSchemaTransformProvider transformProvider = null;
+
+  private ExpansionServiceSchemaTransformProvider() {
+    try {
+      for (org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider schemaTransformProvider :
+          ServiceLoader.load(
+              org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider.class)) {
+        if (schemaTransformProviders.containsKey(schemaTransformProvider.identifier())) {
+          throw new IllegalArgumentException(
+              "Found multiple SchemaTransformProvider implementations with the same identifier "
+                  + schemaTransformProvider.identifier());
+        }
+        schemaTransformProviders.put(schemaTransformProvider.identifier(), schemaTransformProvider);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage());
+    }
+  }
+
+  public static ExpansionServiceSchemaTransformProvider of() {
+    if (transformProvider == null) {
+      transformProvider = new ExpansionServiceSchemaTransformProvider();
+    }
+
+    return transformProvider;
+  }
+
+  static class RowTransform extends PTransform {
+
+    private PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform;
+
+    public RowTransform(PTransform<PCollectionRowTuple, PCollectionRowTuple> rowTuplePTransform) {
+      this.rowTuplePTransform = rowTuplePTransform;
+    }
+
+    @Override
+    public POutput expand(PInput input) {
+      PCollectionRowTuple inputRowTuple;
+
+      if (input instanceof PCollectionRowTuple) {
+        inputRowTuple = (PCollectionRowTuple) input;
+      } else if (input instanceof PCollection) {
+        inputRowTuple = PCollectionRowTuple.of(DEFAULT_INPUT_TAG, (PCollection) input);

Review Comment:
   Not sure I understand, this cast was needed to create a PCollectionRowTuple here.



-- 
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 #23413: Updates ExpansionService to support dynamically discovering and expanding SchemaTransforms

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

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


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