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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to